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/07/20 03:57:22 UTC

[01/53] [abbrv] WIP fragmentation, physical plan, byte compiling, some vector work

Updated Branches:
  refs/heads/master 5052b64d9 -> 8186b5a1a


http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/MockRecordReader.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/MockRecordReader.java b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/MockRecordReader.java
new file mode 100644
index 0000000..e1f56bd
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/MockRecordReader.java
@@ -0,0 +1,108 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.store;
+
+import org.apache.drill.common.expression.types.DataType;
+import org.apache.drill.exec.exception.ExecutionSetupException;
+import org.apache.drill.exec.exception.SchemaChangeException;
+import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.ops.OutputMutator;
+import org.apache.drill.exec.record.BatchSchema;
+import org.apache.drill.exec.record.vector.Int16Vector;
+import org.apache.drill.exec.record.vector.Int32Vector;
+import org.apache.drill.exec.record.vector.ValueVector;
+
+public class MockRecordReader implements RecordReader {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(MockRecordReader.class);
+
+  private BatchSchema expectedSchema;
+  private OutputMutator output;
+  private MockRecordConfig config;
+  private FragmentContext context;
+  private ValueVector<?>[] valueVectors;
+  private int recordsRead;
+
+  public MockRecordReader(FragmentContext context, MockRecordConfig config) {
+    this.config = config;
+  }
+
+  private int getEstimatedRecordSize(DataType[] types) {
+    int x = 0;
+    for (int i = 0; i < types.length; i++) {
+      x += getEstimatedColumnSize(i);
+    }
+    return x;
+  }
+
+  private int getEstimatedColumnSize(int fieldId) {
+    return 4;
+  }
+
+  private ValueVector<?> getVector(int fieldId, DataType dt, int length) {
+    ValueVector<?> v;
+    if (dt == DataType.INT16) {
+      v = new Int16Vector(fieldId, context.getAllocator());
+    } else if (dt == DataType.INT32) {
+      v = new Int32Vector(fieldId, context.getAllocator());
+    } else {
+      throw new UnsupportedOperationException();
+    }
+    v.allocateNew(length);
+    return v;
+
+  }
+
+  @Override
+  public void setup(BatchSchema expectedSchema, OutputMutator output) throws ExecutionSetupException {
+    try {
+      this.expectedSchema = expectedSchema;
+      this.output = output;
+      int estimateRowSize = getEstimatedRecordSize(config.getTypes());
+      valueVectors = new ValueVector<?>[config.getTypes().length];
+      int batchRecordCount = 250000 / estimateRowSize;
+
+      for (int i = 0; i < config.getTypes().length; i++) {
+        valueVectors[i] = getVector(i, config.getTypes()[i], batchRecordCount);
+        output.addField(i, valueVectors[i]);
+      }
+    } catch (SchemaChangeException e) {
+      throw new ExecutionSetupException("Failure while setting up fields", e);
+    }
+
+  }
+
+  @Override
+  public int next() {
+    int recordSetSize = Math.min(valueVectors[0].size(), this.config.getRecordCount()- recordsRead);
+    recordsRead += recordSetSize;
+    return recordSetSize;
+  }
+
+  @Override
+  public void cleanup() {
+    for (int i = 0; i < valueVectors.length; i++) {
+      try {
+        output.removeField(valueVectors[i].getField().getFieldId());
+      } catch (SchemaChangeException e) {
+        logger.warn("Failure while trying tremove field.", e);
+      }
+      valueVectors[i].close();
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/MockStorageEngine.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/MockStorageEngine.java b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/MockStorageEngine.java
new file mode 100644
index 0000000..cc82540
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/MockStorageEngine.java
@@ -0,0 +1,54 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.store;
+
+import java.io.IOException;
+import java.util.Collection;
+
+import org.apache.drill.common.logical.data.Scan;
+import org.apache.drill.common.proto.CoordinationProtos.DrillbitEndpoint;
+import org.apache.drill.exec.ops.FragmentContext;
+
+import com.google.common.collect.ListMultimap;
+
+public class MockStorageEngine extends AbstractStorageEngine{
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(MockStorageEngine.class);
+
+  @Override
+  public boolean supportsRead() {
+    return true;
+  }
+
+  @Override
+  public Collection<ReadEntry> getReadEntries(Scan scan) throws IOException {
+    return null;
+  }
+
+  @Override
+  public ListMultimap<ReadEntry, DrillbitEndpoint> getReadLocations(Collection<ReadEntry> entries) {
+    return null;
+  }
+
+  @Override
+  public RecordReader getReader(FragmentContext context, ReadEntry readEntry) throws IOException {
+    return null;
+  }
+
+  
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/exec/java-exec/src/test/resources/drill-module.conf
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/resources/drill-module.conf b/sandbox/prototype/exec/java-exec/src/test/resources/drill-module.conf
index 7c97f66..8785736 100644
--- a/sandbox/prototype/exec/java-exec/src/test/resources/drill-module.conf
+++ b/sandbox/prototype/exec/java-exec/src/test/resources/drill-module.conf
@@ -10,7 +10,12 @@ drill.exec: {
   optimizer: {
     implementation: "org.apache.drill.exec.opt.IdentityOptimizer"
   },
-  
+  storage: {
+	packages += "org.apache.drill.exec.store"  
+  }
+  metrics : { 
+  	context: "drillbit"
+  },
   zk: {
 	connect: "localhost:2181",
 	root: "/drill",

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/exec/java-exec/src/test/resources/physical_screen.json
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/resources/physical_screen.json b/sandbox/prototype/exec/java-exec/src/test/resources/physical_screen.json
new file mode 100644
index 0000000..c63aef1
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/test/resources/physical_screen.json
@@ -0,0 +1,25 @@
+{
+    head:{
+        type:"APACHE_DRILL_PHYSICAL",
+        version:"1",
+        generator:{
+            type:"manual"
+        }
+    },
+    graph:[
+        {
+            @id:1,
+            pop:"mock-scan",
+            url: "http://apache.org",
+            entries:[
+            	{id:1}
+            ],
+            cost: { disk: 1.0, memory: 1.0, cpu: 1.0, network: 1.0}
+        },
+        {
+            @id:2,
+            pop: "screen",
+            child: 1
+        }
+    ]
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/exec/java-exec/src/test/resources/physical_simpleexchange.json
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/resources/physical_simpleexchange.json b/sandbox/prototype/exec/java-exec/src/test/resources/physical_simpleexchange.json
new file mode 100644
index 0000000..e332785
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/test/resources/physical_simpleexchange.json
@@ -0,0 +1,41 @@
+{
+    head:{
+        type:"APACHE_DRILL_PHYSICAL",
+        version:"1",
+        generator:{
+            type:"manual"
+        }
+    },
+    graph:[
+        {
+            @id:1,
+            pop:"mock-scan",
+            url: "http://apache.org",
+            entries:[
+            	{id:1}
+            ],
+            cost: { disk: 1.0, memory: 1.0, cpu: 1.0, network: 1.0}
+        },
+        {
+            @id:2,
+            pop: "partition-to-random-exchange",
+            child: 1,
+            partition: {
+              mode: "DUPLICATE"
+            }
+        },
+        {
+            @id:3,
+            child: 2,
+            pop:"filter",
+            expr: "b > 5",
+            cost: { disk: 1.0, memory: 1.0, cpu: 1.0, network: 1.0}
+        },
+        {
+            @id: 4,
+            child: 3,
+            pop: "mock-store",
+            cost: { disk: 1.0, memory: 1.0, cpu: 1.0, network: 1.0}
+        }
+    ]
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/exec/java-exec/src/test/sh/runbit
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/sh/runbit b/sandbox/prototype/exec/java-exec/src/test/sh/runbit
index 10fc1d5..2885f7f 100755
--- a/sandbox/prototype/exec/java-exec/src/test/sh/runbit
+++ b/sandbox/prototype/exec/java-exec/src/test/sh/runbit
@@ -5,5 +5,5 @@ PROJECT_ROOT=../../../
 mvn dependency:build-classpath -f=$PROJECT_ROOT/pom.xml -Dmdep.outputFile=target/sh/cp.txt
 CP=`cat $PROJECT_ROOT/target/sh/cp.txt`
 CP=$CP:$PROJECT_ROOT/target/classes:$PROJECT_ROOT/target/test-classes
-java -cp $CP org.apache.drill.exec.server.Drillbit
+java -javaagent:/src/jrebel/jrebel.jar -cp $CP org.apache.drill.exec.server.Drillbit
 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/pom.xml
----------------------------------------------------------------------
diff --git a/sandbox/prototype/pom.xml b/sandbox/prototype/pom.xml
index a4abf8b..fa89417 100644
--- a/sandbox/prototype/pom.xml
+++ b/sandbox/prototype/pom.xml
@@ -16,6 +16,20 @@
 
 	<name>prototype-parent</name>
 
+        <repositories>
+                <repository>
+                        <releases>
+                                <enabled>true</enabled>
+                                <updatePolicy>always</updatePolicy>
+                                <checksumPolicy>warn</checksumPolicy>
+                        </releases>
+                        <id>conjars</id>
+                        <name>Conjars</name>
+                        <url>http://conjars.org/repo</url>
+                        <layout>default</layout>
+                </repository>
+        </repositories>
+
 	<properties>
 		<dep.junit.version>4.11</dep.junit.version>
 		<dep.slf4j.version>1.7.2</dep.slf4j.version>
@@ -144,4 +158,4 @@
 		<module>planner</module>
 		<module>sqlparser</module>
 	</modules>
-</project>
\ No newline at end of file
+</project>


Re: [46/53] [abbrv] git commit: Working project operator. Update to ASM 4.1

Posted by Ted Dunning <te...@gmail.com>.
On Fri, Jul 19, 2013 at 6:58 PM, <ja...@apache.org> wrote:

> Working project operator.  Update to ASM 4.1
>
>
> Project: http://git-wip-us.apache.org/repos/asf/incubator-drill/repo
> Commit:
> http://git-wip-us.apache.org/repos/asf/incubator-drill/commit/80b1d24f
> Tree: http://git-wip-us.apache.org/repos/asf/incubator-drill/tree/80b1d24f
> Diff: http://git-wip-us.apache.org/repos/asf/incubator-drill/diff/80b1d24f
>
> Branch: refs/heads/master
> Commit: 80b1d24f427f23feddd9c93ef0130c4657569f7e
> Parents: ce0da88
> Author: Jacques Nadeau <ja...@apache.org>
> Authored: Tue Jul 9 16:31:25 2013 -0700
> Committer: Jacques Nadeau <ja...@apache.org>
> Committed: Fri Jul 19 14:53:24 2013 -0700
>
> ----------------------------------------------------------------------
>  sandbox/prototype/common/pom.xml                |   7 +
>  sandbox/prototype/exec/java-exec/pom.xml        |   8 +-
>  .../drill/exec/compile/ClassTransformer.java    | 224 ++++++++++++++-----
>  .../drill/exec/compile/JaninoClassCompiler.java |   7 +-
>  .../exec/compile/TemplateClassDefinition.java   |  23 +-
>  .../apache/drill/exec/expr/CodeGenerator.java   |  32 ++-
>  .../apache/drill/exec/expr/DrillBatchFunc.java  |  10 +
>  .../drill/exec/expr/EvaluationVisitor.java      |  76 ++++---
>  .../exec/expr/ExpressionTreeMaterializer.java   |   2 +-
>  .../drill/exec/expr/fn/FunctionConverter.java   |   9 +-
>  .../drill/exec/expr/fn/FunctionHolder.java      |  32 ++-
>  .../drill/exec/expr/fn/impl/MathFunctions.java  |   4 +-
>  .../drill/exec/expr/holders/BooleanHolder.java  |  10 +
>  .../drill/exec/expr/holders/IntHolder.java      |  10 +
>  .../drill/exec/expr/holders/LongHolder.java     |  10 +
>  .../expr/holders/NullableBooleanHolder.java     |  11 +
>  .../exec/expr/holders/NullableIntHolder.java    |  11 +
>  .../exec/expr/holders/NullableLongHolder.java   |  11 +
>  .../holders/ValueHolderImplmenetations.java     |  44 ----
>  .../apache/drill/exec/ops/FragmentContext.java  |   9 +-
>  .../exec/physical/impl/FilterRecordBatch.java   |   4 +-
>  .../drill/exec/physical/impl/ImplCreator.java   |   9 +-
>  .../drill/exec/physical/impl/ScanBatch.java     |   2 +-
>  .../drill/exec/physical/impl/ScreenCreator.java |   7 +-
>  .../exec/physical/impl/WireRecordBatch.java     |   2 +-
>  .../physical/impl/filter/ExampleFilter.java     |   2 +-
>  .../physical/impl/filter/FilterTemplate.java    |  48 ++++
>  .../SelectionVectorPopulationExpression.java    |  39 ++++
>  .../impl/project/ProjectBatchCreator.java       |  23 ++
>  .../physical/impl/project/ProjectEvaluator.java |   2 +-
>  .../impl/project/ProjectRecordBatch.java        |  40 ++--
>  .../exec/physical/impl/project/Projector.java   |   8 +-
>  .../impl/project/ProjectorTemplate.java         |  38 +---
>  .../apache/drill/exec/record/RecordBatch.java   |   4 +-
>  .../apache/drill/exec/record/SchemaBuilder.java |   3 +-
>  .../exec/record/vector/NullableFixed8.java      |   7 +
>  .../exec/record/vector/SelectionVector2.java    |   3 +
>  .../exec/compile/TestClassTransformation.java   |  13 +-
>  .../apache/drill/exec/expr/ExpressionTest.java  |   7 +-
>  .../exec/physical/impl/SimpleRootExec.java      |  40 ++++
>  .../impl/project/TestSimpleProjection.java      |  69 ++++++
>  .../record/ExpressionTreeMaterializerTest.java  |   8 +-
>  .../exec/test/generated/TestGeneratedClass.java |  15 ++
>  .../src/test/resources/project/test1.json       |  37 +++
>  sandbox/prototype/pom.xml                       |  17 ++
>  45 files changed, 750 insertions(+), 247 deletions(-)
> ----------------------------------------------------------------------
>
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/80b1d24f/sandbox/prototype/common/pom.xml
> ----------------------------------------------------------------------
> diff --git a/sandbox/prototype/common/pom.xml
> b/sandbox/prototype/common/pom.xml
> index 4891191..a33ced9 100644
> --- a/sandbox/prototype/common/pom.xml
> +++ b/sandbox/prototype/common/pom.xml
> @@ -74,6 +74,13 @@
>    <build>
>      <plugins>
>        <plugin>
> +          <artifactId>maven-surefire-plugin</artifactId>
> +          <version>2.15</version>
> +          <configuration>
> +            <useSystemClassLoader>false</useSystemClassLoader>
> +          </configuration>
> +      </plugin>
> +      <plugin>
>          <artifactId>maven-jar-plugin</artifactId>
>          <executions>
>            <execution>
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/80b1d24f/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 e348bc7..f2f0bff 100644
> --- a/sandbox/prototype/exec/java-exec/pom.xml
> +++ b/sandbox/prototype/exec/java-exec/pom.xml
> @@ -13,14 +13,14 @@
>
>    <dependencies>
>      <dependency>
> -      <groupId>asm</groupId>
> +      <groupId>org.ow2.asm</groupId>
>        <artifactId>asm-util</artifactId>
> -      <version>3.3.1</version>
> +      <version>4.1</version>
>      </dependency>
>      <dependency>
> -      <groupId>asm</groupId>
> +      <groupId>org.ow2.asm</groupId>
>        <artifactId>asm-commons</artifactId>
> -      <version>3.3.1</version>
> +      <version>4.1</version>
>      </dependency>
>      <dependency>
>        <groupId>com.sun.codemodel</groupId>
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/80b1d24f/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/compile/ClassTransformer.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/compile/ClassTransformer.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/compile/ClassTransformer.java
> index 079ab6c..4bf6e7e 100644
> ---
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/compile/ClassTransformer.java
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/compile/ClassTransformer.java
> @@ -19,33 +19,37 @@ package org.apache.drill.exec.compile;
>
>  import java.io.File;
>  import java.io.IOException;
> -import java.io.PrintWriter;
>  import java.lang.reflect.Modifier;
>  import java.net.URL;
>  import java.util.Iterator;
> +import java.util.Random;
> +import java.util.Set;
>  import java.util.concurrent.ExecutionException;
>  import java.util.concurrent.TimeUnit;
>  import java.util.concurrent.atomic.AtomicLong;
>
>  import org.apache.drill.exec.exception.ClassTransformationException;
>  import org.codehaus.commons.compiler.CompileException;
> -import org.objectweb.asm.ClassAdapter;
>  import org.objectweb.asm.ClassReader;
>  import org.objectweb.asm.ClassVisitor;
>  import org.objectweb.asm.ClassWriter;
> +import org.objectweb.asm.FieldVisitor;
>  import org.objectweb.asm.MethodVisitor;
> -import org.objectweb.asm.commons.EmptyVisitor;
> +import org.objectweb.asm.Opcodes;
> +import org.objectweb.asm.commons.Remapper;
>  import org.objectweb.asm.commons.RemappingClassAdapter;
>  import org.objectweb.asm.commons.RemappingMethodAdapter;
>  import org.objectweb.asm.commons.SimpleRemapper;
>  import org.objectweb.asm.tree.ClassNode;
>  import org.objectweb.asm.tree.FieldNode;
>  import org.objectweb.asm.tree.MethodNode;
> -import org.objectweb.asm.util.TraceClassVisitor;
>
> +import com.beust.jcommander.internal.Sets;
> +import com.google.common.base.Preconditions;
>  import com.google.common.cache.CacheBuilder;
>  import com.google.common.cache.CacheLoader;
>  import com.google.common.cache.LoadingCache;
> +import com.google.common.io.Files;
>  import com.google.common.io.Resources;
>
>  public class ClassTransformer {
> @@ -53,76 +57,113 @@ public class ClassTransformer {
>
>    private AtomicLong index = new AtomicLong(0);
>    private AtomicLong interfaceIndex = new AtomicLong(0);
> -  private LoadingCache<String, byte[]> byteCode =
> CacheBuilder.newBuilder()
> -      .maximumSize(10000)
> -      .expireAfterWrite(10, TimeUnit.MINUTES)
> -      .build(new ClassBytesCacheLoader());
> -
> -
> -  private class ClassBytesCacheLoader extends CacheLoader<String, byte[]>{
> +  private LoadingCache<String, byte[]> byteCode =
> CacheBuilder.newBuilder().maximumSize(10000)
> +      .expireAfterWrite(10, TimeUnit.MINUTES).build(new
> ClassBytesCacheLoader());
> +
> +  private class ClassBytesCacheLoader extends CacheLoader<String, byte[]>
> {
>      public byte[] load(String path) throws ClassTransformationException,
> IOException {
>        URL u = this.getClass().getResource(path);
> -      if (u == null) throw new
> ClassTransformationException(String.format("Unable to find TemplateClass at
> path %s",path));
> -      return Resources.toByteArray(u);
> +      if (u == null)
> +        throw new ClassTransformationException(String.format("Unable to
> find TemplateClass at path %s", path));
> +      return Resources.toByteArray(u);
>      }
>    };
> -
> +
>    private byte[] getClassByteCodeFromPath(String path) throws
> ClassTransformationException, IOException {
> -    try{
> +    try {
>        return byteCode.get(path);
>      } catch (ExecutionException e) {
>        Throwable c = e.getCause();
> -      if(c instanceof ClassTransformationException) throw
> (ClassTransformationException) c;
> -      if(c instanceof IOException) throw (IOException) c;
> +      if (c instanceof ClassTransformationException)
> +        throw (ClassTransformationException) c;
> +      if (c instanceof IOException)
> +        throw (IOException) c;
>        throw new ClassTransformationException(c);
>      }
>    }
>
> -
> -
> -  @SuppressWarnings("unchecked")
> -  public <T, I> T getImplementationClass(QueryClassLoader classLoader,
> -      TemplateClassDefinition<T, I> templateDefinition, String
> internalClassBody, I initObject)
> -      throws ClassTransformationException {
> -    final String implClassName =
> templateDefinition.getTemplateClassName() +
> interfaceIndex.getAndIncrement();
> +  public <T, I> T getImplementationClassByBody( //
> +      QueryClassLoader classLoader, //
> +      TemplateClassDefinition<T> templateDefinition, //
> +      String internalClassBody //
> +  ) throws ClassTransformationException {
>      final String materializedClassName = "org.apache.drill.generated."
> -        + templateDefinition.getExternalInterface().getSimpleName() +
> index.getAndIncrement();
> -    // final String materializedClassName =
> templateDefinition.getTemplateClassName();
> +        + "Gen" +
> templateDefinition.getExternalInterface().getSimpleName() //
> +        + index.getAndIncrement();
> +    // Get Implementation Class
>      try {
> -
> -      // Get Implementation Class
>        String classBody = ClassBodyBuilder.newBuilder() //
> -          .setClassName(implClassName) //
> +          .setClassName(materializedClassName) //
>
>  .setImplementedInterfaces(templateDefinition.getInternalInterface()) //
>            .setBody(internalClassBody) //
>            .build();
> -      final byte[] implementationClass =
> classLoader.getClassByteCode(implClassName, classBody);
> +      return getImplementationClass(classLoader, templateDefinition,
> classBody, materializedClassName);
> +    } catch (IOException | CompileException e) {
> +      throw new ClassTransformationException("Failure generating class
> body for runtime generated class.", e);
> +    }
> +
> +  }
> +
> +  @SuppressWarnings("unchecked")
> +  public <T, I> T getImplementationClass( //
> +      QueryClassLoader classLoader, //
> +      TemplateClassDefinition<T> templateDefinition, //
> +      String entireClass, //
> +      String materializedClassName) throws ClassTransformationException {
> +
> +    try {
> +
> +      final byte[] implementationClass =
> classLoader.getClassByteCode(materializedClassName, entireClass);
>
>        // Get Template Class
>        final String templateClassName =
> templateDefinition.getTemplateClassName().replaceAll("\\.", File.separator);
>        final String templateClassPath = File.separator + templateClassName
> + ".class";
>        final byte[] templateClass =
> getClassByteCodeFromPath(templateClassPath);
> -
> +      int fileNum = new Random().nextInt(100);
> +      Files.write(templateClass, new
> File(String.format("/tmp/%d-template.class", fileNum)));
>        // Generate Merge Class
> -      ClassNode impl = getClassNodeFromByteCode(implementationClass);
> -      // traceClassToSystemOut(implementationClass);
> -      // traceClassToSystemOut(templateClass);
> -      ClassWriter cw = new ClassWriter(0);
> -      MergeAdapter adapter = new MergeAdapter(cw, impl);
> -      ClassReader tReader = new ClassReader(templateClass);
> -      tReader.accept(adapter, 0);
> -
> -      byte[] outputClass = cw.toByteArray();
> -
> -      cw = new ClassWriter(0);
> -      RemappingClassAdapter r = new RemappingClassAdapter(cw, new
> SimpleRemapper(templateClassName,
> -          materializedClassName.replace('.', '/')));
> -      new ClassReader(outputClass).accept(r, 0);
> -      outputClass = cw.toByteArray();
> -      // traceClassToSystemOut(outputClass);
> -
> -      // Load the class
> -      classLoader.injectByteCode(materializedClassName, outputClass);
> +
> +      // Setup adapters for merging, remapping class names and class
> writing. This is done in reverse order of how they
> +      // will be evaluated.
> +      String oldTemplateSlashName =
> templateDefinition.getTemplateClassName().replace('.', '/');
> +      String materializedSlashName = materializedClassName.replace('.',
> '/');
> +      RemapClasses remapper = new RemapClasses(oldTemplateSlashName,
> materializedSlashName);
> +
> +      {
> +        ClassNode impl = getClassNodeFromByteCode(implementationClass);
> +        ClassWriter cw = new ClassWriter(ClassWriter.COMPUTE_FRAMES);
> +
> +        ClassVisitor remappingAdapter = new RemappingClassAdapter(cw,
> remapper);
> +        MergeAdapter mergingAdapter = new
> MergeAdapter(oldTemplateSlashName, materializedSlashName, remappingAdapter,
> +            impl);
> +        ClassReader tReader = new ClassReader(templateClass);
> +        tReader.accept(mergingAdapter, ClassReader.EXPAND_FRAMES);
> +        byte[] outputClass = cw.toByteArray();
> +        Files.write(outputClass, new
> File(String.format("/tmp/%d-output.class", fileNum)));
> +        outputClass = cw.toByteArray();
> +
> +        // Load the class
> +        classLoader.injectByteCode(materializedClassName, outputClass);
> +      }
> +
> +      int i = 0;
> +      for (String s : remapper.getSubclasses()) {
> +        logger.debug("Setting up sub class {}", s);
> +        // for each sub class, remap them into the new class.
> +        String subclassPath = File.separator + s + ".class";
> +        final byte[] bytecode = getClassByteCodeFromPath(subclassPath);
> +        RemapClasses localRemapper = new
> RemapClasses(oldTemplateSlashName, materializedSlashName);
> +
>  Preconditions.checkArgument(localRemapper.getSubclasses().isEmpty(),
> "Class transformations are only supported for classes that have a single
> level of inner classes.");
> +        ClassWriter subcw = new ClassWriter(ClassWriter.COMPUTE_FRAMES);
> +        ClassVisitor remap = new RemappingClassAdapter(subcw,
> localRemapper);
> +        ClassReader reader = new ClassReader(bytecode);
> +        reader.accept(remap, ClassReader.EXPAND_FRAMES);
> +        byte[] newByteCode = subcw.toByteArray();
> +        classLoader.injectByteCode(s.replace(oldTemplateSlashName,
> materializedSlashName).replace('/', '.'), newByteCode);
> +        Files.write(subcw.toByteArray(), new
> File(String.format("/tmp/%d-sub-%d.class", fileNum, i)));
> +        i++;
> +      }
> +
>        Class<?> c = classLoader.findClass(materializedClassName);
>        if (templateDefinition.getExternalInterface().isAssignableFrom(c)) {
>          return (T) c.newInstance();
> @@ -131,7 +172,8 @@ public class ClassTransformer {
>        }
>
>      } catch (CompileException | IOException | ClassNotFoundException |
> InstantiationException | IllegalAccessException e) {
> -      throw new ClassTransformationException("Failure generating
> transformation classes.", e);
> +      throw new ClassTransformationException(String.format(
> +          "Failure generating transformation classes for value: \n %s",
> entireClass), e);
>      }
>
>    }
> @@ -143,19 +185,37 @@ public class ClassTransformer {
>      return impl;
>    }
>
> -  private void traceClassToSystemOut(byte[] bytecode) {
> -    TraceClassVisitor tcv = new TraceClassVisitor(new EmptyVisitor(), new
> PrintWriter(System.out));
> -    ClassReader cr = new ClassReader(bytecode);
> -    cr.accept(tcv, 0);
> -  }
> +  // private void traceClassToSystemOut(byte[] bytecode) {
> +  // TraceClassVisitor tcv = new TraceClassVisitor(new EmptyVisitor(),
> new PrintWriter(System.out));
> +  // ClassReader cr = new ClassReader(bytecode);
> +  // cr.accept(tcv, 0);
> +  // }
>
> -  public class MergeAdapter extends ClassAdapter {
> +  public class MergeAdapter extends ClassVisitor {
>      private ClassNode classToMerge;
>      private String cname;
> +    private String templateName;
> +    private String newName;
>
> -    public MergeAdapter(ClassVisitor cv, ClassNode cn) {
> -      super(cv);
> +    public MergeAdapter(String templateName, String newName, ClassVisitor
> cv, ClassNode cn) {
> +      super(Opcodes.ASM4, cv);
>        this.classToMerge = cn;
> +      this.templateName = templateName;
> +      this.newName = newName.replace('.', '/');
> +      ;
> +
> +    }
> +
> +    @Override
> +    public void visitInnerClass(String name, String outerName, String
> innerName, int access) {
> +      logger.debug(String.format(
> +          "[Inner Class] Name: %s, outerName: %s, innerName: %s,
> templateName: %s, newName: %s.", name, outerName,
> +          innerName, templateName, newName));
> +      if (name.startsWith(templateName)) {
> +        super.visitInnerClass(name.replace(templateName, newName),
> newName, innerName, access);
> +      } else {
> +        super.visitInnerClass(name, outerName, innerName, access);
> +      }
>      }
>
>      // visit the class
> @@ -193,7 +253,8 @@ public class ClassTransformer {
>          MethodNode mn = (MethodNode) it.next();
>
>          // skip the init.
> -        if (mn.name.equals("<init>")) continue;
> +        if (mn.name.equals("<init>"))
> +          continue;
>
>          String[] exceptions = new String[mn.exceptions.size()];
>          mn.exceptions.toArray(exceptions);
> @@ -206,6 +267,47 @@ public class ClassTransformer {
>        }
>        super.visitEnd();
>      }
> +
> +    @Override
> +    public FieldVisitor visitField(int access, String name, String desc,
> String signature, Object value) {
> +      return super.visitField(access, name, desc, signature, value);
> +    }
> +
> +  }
> +
> +  static class RemapClasses extends Remapper {
> +
> +    final Set<String> subclasses = Sets.newHashSet();
> +
> +    String oldName;
> +    String newName;
> +
> +    public RemapClasses(String oldName, String newName) {
> +      super();
> +      Preconditions.checkArgument(!newName.startsWith(oldName), "The new
> name of a class cannot start with the old name of a class, otherwise class
> renaming will cause problems.");
> +      this.oldName = oldName;
> +      this.newName = newName;
> +    }
> +
> +    @Override
> +    public String map(String typeName) {
> +      // remap the names of all classes that start with the old class
> name.
> +      if (typeName.startsWith(oldName)) {
> +
> +        // write down all the sub classes.
> +        if (typeName.startsWith(oldName + "$")){
> +          subclasses.add(typeName);
> +        }
> +
> +        return typeName.replace(oldName, newName);
> +      }
> +      return typeName;
> +    }
> +
> +    public Set<String> getSubclasses() {
> +      return subclasses;
> +    }
> +
>    }
>
> -}
> +}
> \ No newline at end of file
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/80b1d24f/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/compile/JaninoClassCompiler.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/compile/JaninoClassCompiler.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/compile/JaninoClassCompiler.java
> index 86fe58b..abe2afe 100644
> ---
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/compile/JaninoClassCompiler.java
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/compile/JaninoClassCompiler.java
> @@ -35,15 +35,16 @@ public class JaninoClassCompiler implements
> ClassCompiler{
>
>    private IClassLoader compilationClassLoader;
>
> -  private boolean debugLines;
> -  private boolean debugVars;
> -  private boolean debugSource;
> +  private boolean debugLines = true;
> +  private boolean debugVars = true;
> +  private boolean debugSource = true;
>
>    public JaninoClassCompiler(ClassLoader parentClassLoader) {
>      this.compilationClassLoader = new
> ClassLoaderIClassLoader(parentClassLoader);
>    }
>
>    public byte[] getClassByteCode(final String className, final String
> code) throws CompileException, IOException, ClassNotFoundException,
> ClassTransformationException {
> +    logger.debug("Compiling:\n {}", code);
>      StringReader reader = new StringReader(code);
>      Scanner scanner = new Scanner((String) null, reader);
>      Java.CompilationUnit compilationUnit = new
> Parser(scanner).parseCompilationUnit();
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/80b1d24f/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 fee4c97..5a01dce 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
> @@ -20,21 +20,25 @@ package org.apache.drill.exec.compile;
>  import java.lang.reflect.Method;
>
>
> -public class TemplateClassDefinition<T, I>{
> +public class TemplateClassDefinition<T>{
>
>    private final Class<T> externalInterface;
>    private final String templateClassName;
>    private final Class<?> internalInterface;
> -  private final Class<I> constructorObject;
> +  private final String setupName;
> +  private final String evalName;
>
> -  public TemplateClassDefinition(Class<T> externalInterface, String
> templateClassName, Class<?> internalInterface, Class<I> constructorObject) {
> +
> +  public TemplateClassDefinition(Class<T> externalInterface, String
> templateClassName, Class<?> internalInterface, String setupName, String
> evalName) {
>      super();
>      this.externalInterface = externalInterface;
> -    this.templateClassName = templateClassName;
> +    this.templateClassName = templateClassName;
>      this.internalInterface = internalInterface;
> -    this.constructorObject = constructorObject;
> +    this.setupName = setupName;
> +    this.evalName = evalName;
>    }
>
> +
>    public Class<T> getExternalInterface() {
>      return externalInterface;
>    }
> @@ -48,8 +52,13 @@ public class TemplateClassDefinition<T, I>{
>      return templateClassName;
>    }
>
> -  public Class<I> getConstructorObject() {
> -    return constructorObject;
> +  public String getSetupName() {
> +    return setupName;
> +  }
> +
> +
> +  public String getEvalName() {
> +    return evalName;
>    }
>
>
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/80b1d24f/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 77ae77d..ed6bd9b 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
> @@ -4,13 +4,14 @@ import java.io.IOException;
>
>  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.expr.fn.FunctionImplementationRegistry;
> -import
> org.apache.drill.exec.expr.holders.ValueHolderImplmenetations.BooleanHolder;
> -import
> org.apache.drill.exec.expr.holders.ValueHolderImplmenetations.IntHolder;
> -import
> org.apache.drill.exec.expr.holders.ValueHolderImplmenetations.LongHolder;
> -import
> org.apache.drill.exec.expr.holders.ValueHolderImplmenetations.NullableBooleanHolder;
> -import
> org.apache.drill.exec.expr.holders.ValueHolderImplmenetations.NullableIntHolder;
> -import
> org.apache.drill.exec.expr.holders.ValueHolderImplmenetations.NullableLongHolder;
> +import org.apache.drill.exec.expr.holders.BooleanHolder;
> +import org.apache.drill.exec.expr.holders.IntHolder;
> +import org.apache.drill.exec.expr.holders.LongHolder;
> +import org.apache.drill.exec.expr.holders.NullableBooleanHolder;
> +import org.apache.drill.exec.expr.holders.NullableIntHolder;
> +import org.apache.drill.exec.expr.holders.NullableLongHolder;
>  import org.apache.drill.exec.ops.FragmentContext;
>  import org.apache.drill.exec.record.RecordBatch;
>
> @@ -26,7 +27,7 @@ import com.sun.codemodel.JMod;
>  import com.sun.codemodel.JType;
>  import com.sun.codemodel.JVar;
>
> -public class CodeGenerator {
> +public class CodeGenerator<T> {
>    static final org.slf4j.Logger logger =
> org.slf4j.LoggerFactory.getLogger(CodeGenerator.class);
>
>    public JDefinedClass clazz;
> @@ -37,17 +38,19 @@ public class CodeGenerator {
>    private final EvaluationVisitor evaluationVisitor;
>    private final String setupName;
>    private final String perRecordName;
> -
> +  private final TemplateClassDefinition<T> definition;
>    private JCodeModel model;
>    private int index = 0;
>
> -  public CodeGenerator(String setupName, String perRecordName,
> FunctionImplementationRegistry funcRegistry) {
> +  public CodeGenerator(TemplateClassDefinition<T> definition,
> FunctionImplementationRegistry funcRegistry) {
>      super();
>      try{
> -      this.setupName = setupName;
> -      this.perRecordName = perRecordName;
> +      this.definition = definition;
> +      this.setupName = definition.getSetupName();
> +      this.perRecordName = definition.getEvalName();
>        this.model = new JCodeModel();
>        this.clazz =
> model._package("org.apache.drill.exec.test.generated")._class("Test1");
> +      clazz._implements(definition.getInternalInterface());
>        this.parentEvalBlock = new JBlock();
>        this.parentSetupBlock = new JBlock();
>        this.evaluationVisitor = new EvaluationVisitor(funcRegistry);
> @@ -57,6 +60,7 @@ public class CodeGenerator {
>    }
>
>    public void addNextWrite(ValueVectorWriteExpression ex){
> +    logger.debug("Adding next write {}", ex);
>      currentEvalBlock = new JBlock();
>      parentEvalBlock.add(currentEvalBlock);
>      currentSetupBlock = new JBlock();
> @@ -68,6 +72,10 @@ public class CodeGenerator {
>      return currentEvalBlock;
>    }
>
> +  public String getMaterializedClassName(){
> +    return "org.apache.drill.exec.test.generated.Test1";
> +  }
> +
>    public JBlock getSetupBlock(){
>      return currentSetupBlock;
>    }
> @@ -134,7 +142,7 @@ public class CodeGenerator {
>    }
>
>
> -  public class HoldingContainer{
> +  public static class HoldingContainer{
>      private final JVar holder;
>      private final JFieldRef value;
>      private final JFieldRef isSet;
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/80b1d24f/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/DrillBatchFunc.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/DrillBatchFunc.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/DrillBatchFunc.java
> new file mode 100644
> index 0000000..5042708
> --- /dev/null
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/DrillBatchFunc.java
> @@ -0,0 +1,10 @@
> +package org.apache.drill.exec.expr;
> +
> +import org.apache.drill.exec.record.RecordBatch;
> +
> +public interface DrillBatchFunc {
> +  public void setup(RecordBatch incoming);
> +  public void eval();
> +  public void batchReset();
> +
> +}
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/80b1d24f/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 a5bc5fa..04f78d0 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
> @@ -10,20 +10,24 @@ 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.AbstractExprVisitor;
> +import org.apache.drill.common.types.Types;
>  import org.apache.drill.exec.expr.CodeGenerator.HoldingContainer;
>  import org.apache.drill.exec.expr.fn.FunctionHolder;
>  import org.apache.drill.exec.expr.fn.FunctionImplementationRegistry;
> +import
> org.apache.drill.exec.physical.impl.filter.SelectionVectorPopulationExpression;
> +import org.apache.drill.exec.record.vector.SelectionVector2;
>  import org.apache.drill.exec.record.vector.TypeHelper;
>
> +import com.google.common.base.Preconditions;
>  import com.sun.codemodel.JBlock;
>  import com.sun.codemodel.JClass;
>  import com.sun.codemodel.JConditional;
>  import com.sun.codemodel.JExpr;
> -import com.sun.codemodel.JInvocation;
> +import com.sun.codemodel.JExpression;
>  import com.sun.codemodel.JType;
>  import com.sun.codemodel.JVar;
>
> -public class EvaluationVisitor extends
> AbstractExprVisitor<HoldingContainer, CodeGenerator, RuntimeException> {
> +public class EvaluationVisitor extends
> AbstractExprVisitor<HoldingContainer, CodeGenerator<?>, RuntimeException> {
>
>    private FunctionImplementationRegistry registry;
>
> @@ -33,7 +37,7 @@ public class EvaluationVisitor extends
> AbstractExprVisitor<HoldingContainer, Cod
>    }
>
>    @Override
> -  public HoldingContainer visitFunctionCall(FunctionCall call,
> CodeGenerator generator) throws RuntimeException {
> +  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);
> @@ -43,7 +47,7 @@ public class EvaluationVisitor extends
> AbstractExprVisitor<HoldingContainer, Cod
>    }
>
>    @Override
> -  public HoldingContainer visitIfExpression(IfExpression ifExpr,
> CodeGenerator generator) throws RuntimeException {
> +  public HoldingContainer visitIfExpression(IfExpression ifExpr,
> CodeGenerator<?> generator) throws RuntimeException {
>      JBlock local = generator.getBlock();
>
>      HoldingContainer output = generator.declare(ifExpr.getMajorType());
> @@ -93,46 +97,46 @@ public class EvaluationVisitor extends
> AbstractExprVisitor<HoldingContainer, Cod
>    }
>
>    @Override
> -  public HoldingContainer visitSchemaPath(SchemaPath path, CodeGenerator
> generator) throws RuntimeException {
> +  public HoldingContainer visitSchemaPath(SchemaPath path,
> CodeGenerator<?> generator) throws RuntimeException {
>      throw new UnsupportedOperationException("All schema paths should have
> been replaced with ValueVectorExpressions.");
>    }
>
>    @Override
> -  public HoldingContainer visitLongConstant(LongExpression e,
> CodeGenerator generator) throws RuntimeException {
> +  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 visitDoubleConstant(DoubleExpression e,
> CodeGenerator generator) throws RuntimeException {
> +  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 {
> +  public HoldingContainer visitBooleanConstant(BooleanExpression e,
> CodeGenerator<?> generator) throws RuntimeException {
>      HoldingContainer out = generator.declare(e.getMajorType());
>      generator.getBlock().assign(out.getValue(),
> JExpr.lit(e.getBoolean()));
>      return out;
>    }
> -
> -
>
>    @Override
> -  public HoldingContainer visitUnknown(LogicalExpression e, CodeGenerator
> generator) throws RuntimeException {
> +  public HoldingContainer visitUnknown(LogicalExpression e,
> CodeGenerator<?> generator) throws RuntimeException {
>      if(e instanceof ValueVectorReadExpression){
>        return visitValueVectorExpression((ValueVectorReadExpression) e,
> generator);
>      }else if(e instanceof ValueVectorWriteExpression){
>        return visitValueVectorWriteExpression((ValueVectorWriteExpression)
> e, generator);
> +    }else if(e instanceof SelectionVectorPopulationExpression){
> +      return
> visitSelectionVectorExpression((SelectionVectorPopulationExpression) e,
> generator);
>      }else{
>        return super.visitUnknown(e, generator);
>      }
>
>    }
>
> -  private HoldingContainer
> visitValueVectorWriteExpression(ValueVectorWriteExpression e, CodeGenerator
> generator){
> +  private HoldingContainer
> visitValueVectorWriteExpression(ValueVectorWriteExpression e,
> CodeGenerator<?> generator){
>      LogicalExpression child = e.getChild();
>      HoldingContainer hc = child.accept(this, generator);
>      JBlock block = generator.getBlock();
> @@ -142,9 +146,11 @@ public class EvaluationVisitor extends
> AbstractExprVisitor<HoldingContainer, Cod
>      JVar vv = generator.declareClassField("vv", vvType);
>
>      // get value vector in setup block.
> -    generator.getSetupBlock().assign(vv,
> JExpr.direct("outgoing").invoke("getValueVector") //
> -      .arg(JExpr.lit(e.getFieldId())) //
> -      .arg( ((JClass)vvType).dotclass()));
> +    JVar obj = generator.getSetupBlock().decl( //
> +        generator.getModel()._ref(Object.class), //
> +        generator.getNextVar("obj"), //
> +
>  JExpr.direct("outgoing").invoke("getValueVectorById").arg(JExpr.lit(e.getFieldId())).arg(
> ((JClass)vvType).dotclass()));
> +    generator.getSetupBlock().assign(vv, JExpr.cast(vvType, obj));
>
>      if(hc.isOptional()){
>        vv.invoke("set").arg(JExpr.direct("outIndex"));
> @@ -156,18 +162,19 @@ public class EvaluationVisitor extends
> AbstractExprVisitor<HoldingContainer, Cod
>      return null;
>    }
>
> -  private HoldingContainer
> visitValueVectorExpression(ValueVectorReadExpression e, CodeGenerator
> generator) throws RuntimeException{
> +  private HoldingContainer
> visitValueVectorExpression(ValueVectorReadExpression e, CodeGenerator<?>
> generator) throws RuntimeException{
>      // declare value vector
>      Class<?> vvClass =
> TypeHelper.getValueVectorClass(e.getMajorType().getMinorType(),
> e.getMajorType().getMode());
>      JType vvType = generator.getModel()._ref(vvClass);
>      JVar vv1 = generator.declareClassField("vv", vvType);
>
>      // get value vector from incoming batch and
> -    JInvocation getValue = JExpr //
> -        .invoke(JExpr.direct("incoming"), "getValueVector") //
> -        .arg(JExpr.lit(e.getFieldId())) //
> -        .arg( ((JClass)vvType).dotclass());
> -    generator.getSetupBlock().assign(vv1, getValue);
> +    // get value vector in setup block.
> +    JVar obj = generator.getSetupBlock().decl( //
> +        generator.getModel()._ref(Object.class), //
> +        generator.getNextVar("obj"), //
> +
>  JExpr.direct("outgoing").invoke("getValueVectorById").arg(JExpr.lit(e.getFieldId())).arg(
> ((JClass)vvType).dotclass()));
> +    generator.getSetupBlock().assign(vv1, JExpr.cast(vvType, obj));
>
>      // evaluation work.
>      HoldingContainer out = generator.declare(e.getMajorType());
> @@ -175,25 +182,42 @@ public class EvaluationVisitor extends
> AbstractExprVisitor<HoldingContainer, Cod
>
>      if(out.isOptional()){
>        JBlock blk = generator.getBlock();
> -      blk.assign(out.getIsSet(),
> vv1.invoke("isSet").arg(JExpr.direct("index")));
> +      blk.assign(out.getIsSet(),
> vv1.invoke("isSet").arg(JExpr.direct("inIndex")));
>        JConditional jc = blk._if(out.getIsSet());
>        jc._then() //
> -        .assign(out.getValue(),
> vv1.invoke("get").arg(JExpr.direct("index"))); //
> +        .assign(out.getValue(),
> vv1.invoke("get").arg(JExpr.direct("inIndex"))); //
>          //.assign(out.getIsSet(), JExpr.lit(1));
>        //jc._else()
>          //.assign(out.getIsSet(), JExpr.lit(0));
>
>      }else{
> -      generator.getBlock().assign(out.getValue(),
> vv1.invoke("get").arg(JExpr.direct("index")));
> +      generator.getBlock().assign(out.getValue(),
> vv1.invoke("get").arg(JExpr.direct("inIndex")));
>      }
>      return out;
>    }
>
>
> +  private HoldingContainer
> visitSelectionVectorExpression(SelectionVectorPopulationExpression e,
> CodeGenerator<?> generator){
> +    JType svClass = generator.getModel()._ref(SelectionVector2.class);
> +    JVar sv = generator.declareClassField("sv", svClass);
> +    JVar index = generator.declareClassField("svIndex",
> generator.getModel().CHAR);
> +    LogicalExpression child = e.getChild();
> +
>  Preconditions.checkArgument(child.getMajorType().equals(Types.REQUIRED_BOOLEAN));
> +    HoldingContainer hc = child.accept(this, generator);
> +    generator.getBlock()._return(hc.getValue());
> +
> +//    JBlock blk = generator.getSetupBlock();
> +//    blk.assign(sv,
> JExpr.direct("outgoing").invoke("getSelectionVector2"));
> +//    JConditional jc = blk._if(hc.getValue());
> +//    JBlock body = jc._then();
> +//    body.add(sv.invoke("set").arg(index).arg(JExpr.direct("inIndex")));
> +//    body.assign(index, index.plus(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 use the valuevector classes and internal vectors.");
> +  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));
>    }
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/80b1d24f/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 bc44490..72e5c93 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
> @@ -97,7 +97,7 @@ public class ExpressionTreeMaterializer {
>      @Override
>      public LogicalExpression visitSchemaPath(SchemaPath path) {
>        logger.debug("Visiting schema path {}", path);
> -      TypedFieldId tfId = batch.getValueVector(path);
> +      TypedFieldId tfId = batch.getValueVectorId(path);
>        if (tfId == null) {
>          return NullExpression.INSTANCE;
>        } else {
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/80b1d24f/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 7276e7d..84f04f0 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
> @@ -16,6 +16,7 @@ import org.apache.drill.exec.expr.annotations.Output;
>  import org.apache.drill.exec.expr.annotations.Param;
>  import org.apache.drill.exec.expr.annotations.WorkSpace;
>  import org.apache.drill.exec.expr.fn.FunctionHolder.ValueReference;
> +import org.apache.drill.exec.expr.fn.FunctionHolder.WorkspaceReference;
>  import org.apache.drill.exec.expr.holders.ValueHolder;
>  import org.codehaus.commons.compiler.CompileException;
>  import org.codehaus.janino.Java;
> @@ -43,7 +44,8 @@ public class FunctionConverter {
>
>      // start by getting field information.
>      List<ValueReference> params = Lists.newArrayList();
> -    List<String> workspaceFields = Lists.newArrayList();
> +    List<WorkspaceReference> workspaceFields = Lists.newArrayList();
> +
>      ValueReference outputField = null;
>
>
> @@ -96,7 +98,7 @@ public class FunctionConverter {
>
>        }else{
>          // workspace work.
> -        workspaceFields.add(field.getName());
> +        workspaceFields.add(new WorkspaceReference(field.getType(),
> field.getName()));
>        }
>
>      }
> @@ -118,7 +120,8 @@ public class FunctionConverter {
>
>      // return holder
>      ValueReference[] ps = params.toArray(new
> ValueReference[params.size()]);
> -    FunctionHolder fh = new FunctionHolder(template.scope(),
> template.nulls(), template.isBinaryCommutative(), template.name(), ps,
> outputField, methods);
> +    WorkspaceReference[] works = workspaceFields.toArray(new
> WorkspaceReference[workspaceFields.size()]);
> +    FunctionHolder fh = new FunctionHolder(template.scope(),
> template.nulls(), template.isBinaryCommutative(), template.name(), ps,
> outputField, works, methods);
>      return fh;
>    }
>
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/80b1d24f/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 460169f..f7ecd02 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,5 +1,6 @@
>  package org.apache.drill.exec.expr.fn;
>
> +import java.lang.reflect.Type;
>  import java.util.Arrays;
>  import java.util.Map;
>
> @@ -32,13 +33,15 @@ public class FunctionHolder {
>    private String evalBody;
>    private String addBody;
>    private String setupBody;
> +  private WorkspaceReference[] workspaceVars;
>    private ValueReference[] parameters;
>    private ValueReference returnValue;
>
> -  public FunctionHolder(FunctionScope scope, NullHandling nullHandling,
> boolean isBinaryCommutative, String functionName, ValueReference[]
> parameters, ValueReference returnValue, Map<String, String> methods) {
> +  public FunctionHolder(FunctionScope scope, NullHandling nullHandling,
> boolean isBinaryCommutative, String functionName, ValueReference[]
> parameters, ValueReference returnValue, WorkspaceReference[] workspaceVars,
> Map<String, String> methods) {
>      super();
>      this.scope = scope;
>      this.nullHandling = nullHandling;
> +    this.workspaceVars = workspaceVars;
>      this.isBinaryCommutative = isBinaryCommutative;
>      this.functionName = functionName;
>      this.setupBody = methods.get("setup");
> @@ -48,9 +51,9 @@ public class FunctionHolder {
>      this.returnValue = returnValue;
>    }
>
> -  public HoldingContainer generateEvalBody(CodeGenerator g,
> HoldingContainer[] inputVariables){
> +  public HoldingContainer generateEvalBody(CodeGenerator<?> g,
> HoldingContainer[] inputVariables){
>
> -    g.getBlock().directStatement(String.format("//---- start of eval
> portion of %s function. ----//", functionName));
> +    //g.getBlock().directStatement(String.format("//---- start of eval
> portion of %s function. ----//", functionName));
>
>      JBlock sub = new JBlock(true, true);
>
> @@ -87,6 +90,16 @@ public class FunctionHolder {
>      // add the subblock after the out declaration.
>      g.getBlock().add(sub);
>
> +    JVar[] workspaceJVars = new JVar[workspaceVars.length];
> +    for(int i =0 ; i < workspaceVars.length; i++){
> +      workspaceJVars[i] = g.declareClassField("work",
> g.getModel()._ref(workspaceVars[i].type)));
> +    }
> +
> +    for(WorkspaceReference r : workspaceVars){
> +      g.declareClassField(, t)
> +    }
> +
> +    g.declareClassField(prefix, t)
>      // locally name external blocks.
>
>      // internal out value.
> @@ -99,12 +112,13 @@ public class FunctionHolder {
>        sub.decl(JMod.FINAL, inputVariable.getHolder().type(),
> parameter.name, inputVariable.getHolder());
>      }
>
> +
>      // add function body.
>      sub.directStatement(evalBody);
>
>      sub.assign(out.getHolder(), internalOutput);
>
> -    g.getBlock().directStatement(String.format("//---- end of eval
> portion of %s function. ----//\n", functionName));
> +    //g.getBlock().directStatement(String.format("//---- end of eval
> portion of %s function. ----//\n", functionName));
>      return out;
>    }
>
> @@ -144,6 +158,16 @@ public class FunctionHolder {
>
>    }
>
> +  public static class WorkspaceReference{
> +    Class<?> type;
> +    String name;
> +    public WorkspaceReference(Class<?> type, String name) {
> +      super();
> +      this.type = type;
> +      this.name = name;
> +    }
> +
> +  }
>    @Override
>    public String toString() {
>      final int maxLen = 10;
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/80b1d24f/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/MathFunctions.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/MathFunctions.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/MathFunctions.java
> index 702a0ad..939e997 100644
> ---
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/MathFunctions.java
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/MathFunctions.java
> @@ -6,8 +6,8 @@ import
> org.apache.drill.exec.expr.annotations.FunctionTemplate.FunctionScope;
>  import
> org.apache.drill.exec.expr.annotations.FunctionTemplate.NullHandling;
>  import org.apache.drill.exec.expr.annotations.Output;
>  import org.apache.drill.exec.expr.annotations.Param;
> -import
> org.apache.drill.exec.expr.holders.ValueHolderImplmenetations.IntHolder;
> -import
> org.apache.drill.exec.expr.holders.ValueHolderImplmenetations.LongHolder;
> +import org.apache.drill.exec.expr.holders.IntHolder;
> +import org.apache.drill.exec.expr.holders.LongHolder;
>  import org.apache.drill.exec.record.RecordBatch;
>
>  public class MathFunctions{
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/80b1d24f/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/holders/BooleanHolder.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/holders/BooleanHolder.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/holders/BooleanHolder.java
> new file mode 100644
> index 0000000..70bcd10
> --- /dev/null
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/holders/BooleanHolder.java
> @@ -0,0 +1,10 @@
> +package org.apache.drill.exec.expr.holders;
> +
> +import org.apache.drill.common.types.Types;
> +import org.apache.drill.common.types.TypeProtos.MajorType;
> +import org.apache.drill.common.types.TypeProtos.MinorType;
> +
> +public final class BooleanHolder implements ValueHolder{
> +  public static final MajorType TYPE = Types.required(MinorType.BOOLEAN);
> +  public int value;
> +}
> \ No newline at end of file
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/80b1d24f/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/holders/IntHolder.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/holders/IntHolder.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/holders/IntHolder.java
> new file mode 100644
> index 0000000..81c82b9
> --- /dev/null
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/holders/IntHolder.java
> @@ -0,0 +1,10 @@
> +package org.apache.drill.exec.expr.holders;
> +
> +import org.apache.drill.common.types.Types;
> +import org.apache.drill.common.types.TypeProtos.MajorType;
> +import org.apache.drill.common.types.TypeProtos.MinorType;
> +
> +public final class IntHolder implements ValueHolder{
> +  public static final MajorType TYPE = Types.required(MinorType.INT);
> +  public int value;
> +}
> \ No newline at end of file
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/80b1d24f/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/holders/LongHolder.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/holders/LongHolder.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/holders/LongHolder.java
> new file mode 100644
> index 0000000..c73f9ad
> --- /dev/null
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/holders/LongHolder.java
> @@ -0,0 +1,10 @@
> +package org.apache.drill.exec.expr.holders;
> +
> +import org.apache.drill.common.types.Types;
> +import org.apache.drill.common.types.TypeProtos.MajorType;
> +import org.apache.drill.common.types.TypeProtos.MinorType;
> +
> +public final class LongHolder implements ValueHolder {
> +  public static final MajorType TYPE = Types.required(MinorType.BIGINT);
> +  public long value;
> +}
> \ No newline at end of file
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/80b1d24f/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/holders/NullableBooleanHolder.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/holders/NullableBooleanHolder.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/holders/NullableBooleanHolder.java
> new file mode 100644
> index 0000000..f6d2f2e
> --- /dev/null
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/holders/NullableBooleanHolder.java
> @@ -0,0 +1,11 @@
> +package org.apache.drill.exec.expr.holders;
> +
> +import org.apache.drill.common.types.Types;
> +import org.apache.drill.common.types.TypeProtos.MajorType;
> +import org.apache.drill.common.types.TypeProtos.MinorType;
> +
> +public final class NullableBooleanHolder implements ValueHolder {
> +  public static final MajorType TYPE = Types.optional(MinorType.BOOLEAN);
> +  public int value;
> +  public int isSet;
> +}
> \ No newline at end of file
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/80b1d24f/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/holders/NullableIntHolder.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/holders/NullableIntHolder.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/holders/NullableIntHolder.java
> new file mode 100644
> index 0000000..7eca21e
> --- /dev/null
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/holders/NullableIntHolder.java
> @@ -0,0 +1,11 @@
> +package org.apache.drill.exec.expr.holders;
> +
> +import org.apache.drill.common.types.Types;
> +import org.apache.drill.common.types.TypeProtos.MajorType;
> +import org.apache.drill.common.types.TypeProtos.MinorType;
> +
> +public final class NullableIntHolder implements ValueHolder {
> +  public static final MajorType TYPE = Types.optional(MinorType.INT);
> +  public int value;
> +  public int isSet;
> +}
> \ No newline at end of file
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/80b1d24f/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/holders/NullableLongHolder.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/holders/NullableLongHolder.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/holders/NullableLongHolder.java
> new file mode 100644
> index 0000000..d1be8f1
> --- /dev/null
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/holders/NullableLongHolder.java
> @@ -0,0 +1,11 @@
> +package org.apache.drill.exec.expr.holders;
> +
> +import org.apache.drill.common.types.Types;
> +import org.apache.drill.common.types.TypeProtos.MajorType;
> +import org.apache.drill.common.types.TypeProtos.MinorType;
> +
> +public final class NullableLongHolder implements ValueHolder {
> +  public static final MajorType TYPE = Types.optional(MinorType.BIGINT);
> +  public long value;
> +  public int isSet;
> +}
> \ No newline at end of file
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/80b1d24f/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/holders/ValueHolderImplmenetations.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/holders/ValueHolderImplmenetations.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/holders/ValueHolderImplmenetations.java
> deleted file mode 100644
> index adf97fe..0000000
> ---
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/holders/ValueHolderImplmenetations.java
> +++ /dev/null
> @@ -1,44 +0,0 @@
> -package org.apache.drill.exec.expr.holders;
> -
> -import org.apache.drill.common.types.TypeProtos.MajorType;
> -import org.apache.drill.common.types.TypeProtos.MinorType;
> -import org.apache.drill.common.types.Types;
> -
> -public class ValueHolderImplmenetations {
> -
> -  private ValueHolderImplmenetations(){}
> -
> -  public final static class BooleanHolder implements ValueHolder{
> -    public static final MajorType TYPE =
> Types.required(MinorType.BOOLEAN);
> -    public int value;
> -  }
> -
> -  public final static class NullableBooleanHolder implements ValueHolder {
> -    public static final MajorType TYPE =
> Types.optional(MinorType.BOOLEAN);
> -    public int value;
> -    public int isSet;
> -  }
> -
> -  public final static class IntHolder implements ValueHolder{
> -    public static final MajorType TYPE = Types.required(MinorType.INT);
> -    public int value;
> -  }
> -
> -  public final static class NullableIntHolder implements ValueHolder {
> -    public static final MajorType TYPE = Types.optional(MinorType.INT);
> -    public int value;
> -    public int isSet;
> -  }
> -
> -  public final static class LongHolder implements ValueHolder {
> -    public static final MajorType TYPE = Types.required(MinorType.BIGINT);
> -    public long value;
> -  }
> -
> -  public final static class NullableLongHolder implements ValueHolder {
> -    public static final MajorType TYPE = Types.optional(MinorType.BIGINT);
> -    public long value;
> -    public int isSet;
> -  }
> -
> -}
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/80b1d24f/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 1dd7de8..876b873 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
> @@ -103,14 +103,11 @@ public class FragmentContext {
>      return context.getAllocator();
>    }
>
> -  public <T> T getImplementationClass(TemplateClassDefinition<T, Void>
> templateDefinition, CodeGenerator cg) throws ClassTransformationException,
> IOException{
> -    return transformer.getImplementationClass(this.loader,
> templateDefinition, cg.generate(), null);
> -  }
> -
> -  public FilteringRecordBatchTransformer
> getFilteringExpression(LogicalExpression expr){
> -    return null;
> +  public <T> T getImplementationClass(TemplateClassDefinition<T>
> templateDefinition, CodeGenerator<T> cg) throws
> ClassTransformationException, IOException{
> +    return transformer.getImplementationClass(this.loader,
> templateDefinition, cg.generate(), cg.getMaterializedClassName());
>    }
>
> +
>    public void addMetricsToStatus(FragmentStatus.Builder stats){
>      stats.setBatchesCompleted(batchesCompleted.get());
>      stats.setDataProcessed(dataProcessed.get());
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/80b1d24f/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/FilterRecordBatch.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/FilterRecordBatch.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/FilterRecordBatch.java
> index 80d48f4..1ddc3f9 100644
> ---
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/FilterRecordBatch.java
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/FilterRecordBatch.java
> @@ -78,7 +78,7 @@ public abstract class FilterRecordBatch implements
> RecordBatch {
>    }
>
>    @Override
> -  public TypedFieldId getValueVector(SchemaPath path) {
> +  public TypedFieldId getValueVectorId(SchemaPath path) {
>      return null;
>    }
>
> @@ -111,7 +111,7 @@ public abstract class FilterRecordBatch implements
> RecordBatch {
>        IterOutcome o = incoming.next();
>        switch (o) {
>        case OK_NEW_SCHEMA:
> -        transformer = incoming.getContext().getFilteringExpression(null);
> +        transformer = null;
>          schema = transformer.getSchema();
>          // fall through to ok.
>        case OK:
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/80b1d24f/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 d98c107..739c0d4 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
> @@ -28,9 +28,11 @@ import
> org.apache.drill.exec.physical.base.PhysicalOperator;
>  import org.apache.drill.exec.physical.base.Scan;
>  import org.apache.drill.exec.physical.config.MockScanBatchCreator;
>  import org.apache.drill.exec.physical.config.MockScanPOP;
> +import org.apache.drill.exec.physical.config.Project;
>  import org.apache.drill.exec.physical.config.RandomReceiver;
>  import org.apache.drill.exec.physical.config.Screen;
>  import org.apache.drill.exec.physical.config.SingleSender;
> +import org.apache.drill.exec.physical.impl.project.ProjectBatchCreator;
>  import org.apache.drill.exec.record.RecordBatch;
>
>  import com.google.common.base.Preconditions;
> @@ -43,6 +45,7 @@ public class ImplCreator extends
> AbstractPhysicalVisitor<RecordBatch, FragmentCo
>    private ScreenCreator sc = new ScreenCreator();
>    private RandomReceiverCreator rrc = new RandomReceiverCreator();
>    private SingleSenderCreator ssc = new SingleSenderCreator();
> +  private ProjectBatchCreator pbc = new ProjectBatchCreator();
>    private RootExec root = null;
>
>    private ImplCreator(){}
> @@ -51,7 +54,11 @@ public class ImplCreator extends
> AbstractPhysicalVisitor<RecordBatch, FragmentCo
>      return root;
>    }
>
> -
> +  @Override
> +  public RecordBatch visitProject(Project op, FragmentContext context)
> throws ExecutionSetupException {
> +    return pbc.getBatch(context, op, getChildren(op, context));
> +  }
> +
>    @Override
>    public RecordBatch visitScan(Scan<?> scan, FragmentContext context)
> throws ExecutionSetupException {
>      Preconditions.checkNotNull(scan);
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/80b1d24f/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScanBatch.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScanBatch.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScanBatch.java
> index 1e0c000..be80349 100644
> ---
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScanBatch.java
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScanBatch.java
> @@ -173,7 +173,7 @@ public class ScanBatch implements RecordBatch {
>    }
>
>    @Override
> -  public TypedFieldId getValueVector(SchemaPath path) {
> +  public TypedFieldId getValueVectorId(SchemaPath path) {
>      return holder.getValueVector(path);
>    }
>
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/80b1d24f/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 3819036..516b5af 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
> @@ -50,7 +50,7 @@ public class ScreenCreator implements
> RootCreator<Screen>{
>    }
>
>
> -  private static class ScreenRoot implements RootExec{
> +  static class ScreenRoot implements RootExec{
>      static final org.slf4j.Logger logger =
> org.slf4j.LoggerFactory.getLogger(ScreenRoot.class);
>      volatile boolean ok = true;
>
> @@ -138,6 +138,11 @@ public class ScreenCreator implements
> RootCreator<Screen>{
>        }
>
>      }
> +
> +    RecordBatch getIncoming() {
> +      return incoming;
> +    }
> +
>
>    }
>
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/80b1d24f/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/WireRecordBatch.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/WireRecordBatch.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/WireRecordBatch.java
> index f4921b5..7b2a533 100644
> ---
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/WireRecordBatch.java
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/WireRecordBatch.java
> @@ -85,7 +85,7 @@ public class WireRecordBatch implements RecordBatch{
>    }
>
>    @Override
> -  public TypedFieldId getValueVector(SchemaPath path) {
> +  public TypedFieldId getValueVectorId(SchemaPath path) {
>      return batchLoader.getValueVector(path);
>    }
>
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/80b1d24f/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/filter/ExampleFilter.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/filter/ExampleFilter.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/filter/ExampleFilter.java
> index 158350f..c4a4032 100644
> ---
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/filter/ExampleFilter.java
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/filter/ExampleFilter.java
> @@ -78,7 +78,7 @@ public class ExampleFilter implements RecordBatch {
>    }
>
>    @Override
> -  public TypedFieldId getValueVector(SchemaPath path) {
> +  public TypedFieldId getValueVectorId(SchemaPath path) {
>      return null;
>    }
>
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/80b1d24f/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/filter/FilterTemplate.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/filter/FilterTemplate.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/filter/FilterTemplate.java
> new file mode 100644
> index 0000000..d2983ad
> --- /dev/null
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/filter/FilterTemplate.java
> @@ -0,0 +1,48 @@
> +package org.apache.drill.exec.physical.impl.filter;
> +
> +import org.apache.drill.exec.record.RecordBatch;
> +import org.apache.drill.exec.record.vector.SelectionVector2;
> +
> +public abstract class FilterTemplate {
> +  static final org.slf4j.Logger logger =
> org.slf4j.LoggerFactory.getLogger(FilterTemplate.class);
> +
> +  SelectionVector2 outgoingSelectionVector;
> +  SelectionVector2 incomingSelectionVector;
> +
> +  public void setup(RecordBatch incoming, RecordBatch outgoing){
> +    outgoingSelectionVector = outgoing.getSelectionVector2();
> +
> +    switch(incoming.getSchema().getSelectionVector()){
> +    case NONE:
> +      break;
> +    case TWO_BYTE:
> +      this.incomingSelectionVector = incoming.getSelectionVector2();
> +      break;
> +    default:
> +      throw new UnsupportedOperationException();
> +    }
> +  }
> +
> +  public void filterBatchSV2(int recordCount){
> +    int svIndex = 0;
> +    for(char i =0; i < recordCount; i++){
> +      if(include(i)){
> +        outgoingSelectionVector.setIndex(svIndex, i);
> +        svIndex+=2;
> +      }
> +    }
> +  }
> +
> +  public void filterBatchNoSV(int recordCount){
> +    int svIndex = 0;
> +    for(char i =0; i < recordCount; i++){
> +
> +      if(include(i)){
> +        outgoingSelectionVector.setIndex(svIndex, i);
> +        svIndex+=2;
> +      }
> +    }
> +  }
> +
> +  protected abstract boolean include(int index);
> +}
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/80b1d24f/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/filter/SelectionVectorPopulationExpression.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/filter/SelectionVectorPopulationExpression.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/filter/SelectionVectorPopulationExpression.java
> new file mode 100644
> index 0000000..f253695
> --- /dev/null
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/filter/SelectionVectorPopulationExpression.java
> @@ -0,0 +1,39 @@
> +package org.apache.drill.exec.physical.impl.filter;
> +
> +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;
> +
> +public class SelectionVectorPopulationExpression implements
> LogicalExpression{
> +  static final org.slf4j.Logger logger =
> org.slf4j.LoggerFactory.getLogger(SelectionVectorPopulationExpression.class);
> +
> +  private LogicalExpression child;
> +
> +  public SelectionVectorPopulationExpression(LogicalExpression child) {
> +    this.child = child;
> +  }
> +
> +  public LogicalExpression getChild() {
> +    return child;
> +  }
> +
> +  @Override
> +  public MajorType getMajorType() {
> +    return Types.NULL;
> +  }
> +
> +  @Override
> +  public <T, V, E extends Exception> T accept(ExprVisitor<T, V, E>
> visitor, V value) throws E {
> +    return visitor.visitUnknown(this, value);
> +  }
> +
> +  @Override
> +  public ExpressionPosition getPosition() {
> +    return ExpressionPosition.UNKNOWN;
> +  }
> +
> +
> +
> +}
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/80b1d24f/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/ProjectBatchCreator.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/ProjectBatchCreator.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/ProjectBatchCreator.java
> new file mode 100644
> index 0000000..3538469
> --- /dev/null
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/ProjectBatchCreator.java
> @@ -0,0 +1,23 @@
> +package org.apache.drill.exec.physical.impl.project;
> +
> +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.Project;
> +import org.apache.drill.exec.physical.impl.BatchCreator;
> +import org.apache.drill.exec.record.RecordBatch;
> +
> +import com.google.common.base.Preconditions;
> +
> +public class ProjectBatchCreator implements BatchCreator<Project>{
> +  static final org.slf4j.Logger logger =
> org.slf4j.LoggerFactory.getLogger(ProjectBatchCreator.class);
> +
> +  @Override
> +  public RecordBatch getBatch(FragmentContext context, Project config,
> List<RecordBatch> children) throws ExecutionSetupException {
> +    Preconditions.checkArgument(children.size() == 1);
> +    return new ProjectRecordBatch(config, children.iterator().next(),
> context);
> +  }
> +
> +
> +}
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/80b1d24f/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 69daae0..86caf28 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
> @@ -7,6 +7,6 @@ import org.apache.drill.exec.record.RecordBatch;
>  public interface ProjectEvaluator {
>    static final org.slf4j.Logger logger =
> org.slf4j.LoggerFactory.getLogger(ProjectEvaluator.class);
>
> -  public abstract void setupEvaluators(FragmentContext context,
> RecordBatch incoming) throws SchemaChangeException;
> +  public abstract void setupEvaluators(FragmentContext context,
> RecordBatch incoming, RecordBatch outgoing) throws SchemaChangeException;
>    public abstract void doPerRecordWork(int inIndex, int outIndex);
>  }
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/80b1d24f/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/ProjectRecordBatch.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/ProjectRecordBatch.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/ProjectRecordBatch.java
> index cfdb7bc..7a3fec1 100644
> ---
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/ProjectRecordBatch.java
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/ProjectRecordBatch.java
> @@ -5,6 +5,7 @@ import java.util.List;
>
>  import org.apache.drill.common.expression.ErrorCollector;
>  import org.apache.drill.common.expression.ErrorCollectorImpl;
> +import org.apache.drill.common.expression.FieldReference;
>  import org.apache.drill.common.expression.LogicalExpression;
>  import org.apache.drill.common.expression.PathSegment;
>  import org.apache.drill.common.expression.SchemaPath;
> @@ -18,6 +19,7 @@ import
> org.apache.drill.exec.expr.ValueVectorReadExpression;
>  import org.apache.drill.exec.expr.ValueVectorWriteExpression;
>  import org.apache.drill.exec.ops.FragmentContext;
>  import org.apache.drill.exec.physical.config.Project;
> +import org.apache.drill.exec.physical.impl.VectorHolder;
>  import org.apache.drill.exec.proto.SchemaDefProtos.FieldDef;
>  import org.apache.drill.exec.proto.SchemaDefProtos.NamePart;
>  import org.apache.drill.exec.proto.SchemaDefProtos.NamePart.Type;
> @@ -32,6 +34,7 @@ import
> org.apache.drill.exec.record.vector.SelectionVector4;
>  import org.apache.drill.exec.record.vector.TypeHelper;
>  import org.apache.drill.exec.record.vector.ValueVector;
>
> +
>  import com.google.common.base.Preconditions;
>  import com.google.common.collect.Lists;
>
> @@ -45,6 +48,7 @@ public class ProjectRecordBatch implements RecordBatch{
>    private Projector projector;
>    private List<ValueVector<?>> allocationVectors;
>    private List<ValueVector<?>> outputVectors;
> +  private VectorHolder vh;
>
>
>    public ProjectRecordBatch(Project pop, RecordBatch incoming,
> FragmentContext context){
> @@ -86,19 +90,20 @@ public class ProjectRecordBatch implements RecordBatch{
>    }
>
>    @Override
> -  public TypedFieldId getValueVector(SchemaPath path) {
> -    return null;
> +  public TypedFieldId getValueVectorId(SchemaPath path) {
> +    return vh.getValueVector(path);
>    }
>
>    @Override
> -  public <T extends ValueVector<T>> T getValueVectorById(int fieldId,
> Class<?> vvClass) {
> -    return null;
> +  public <T extends ValueVector<T>> T getValueVectorById(int fieldId,
> Class<?> clazz) {
> +    return vh.getValueVector(fieldId, clazz);
>    }
>
>    @Override
>    public IterOutcome next() {
>
>      IterOutcome upstream = incoming.next();
> +    logger.debug("Upstream... {}", upstream);
>      switch(upstream){
>      case NONE:
>      case NOT_YET:
> @@ -109,6 +114,7 @@ public class ProjectRecordBatch implements RecordBatch{
>          projector = createNewProjector();
>        }catch(SchemaChangeException ex){
>          incoming.kill();
> +        logger.error("Failure during query", ex);
>          context.fail(ex);
>          return IterOutcome.STOP;
>        }
> @@ -119,6 +125,9 @@ public class ProjectRecordBatch implements RecordBatch{
>          v.allocateNew(recordCount);
>        }
>        projector.projectRecords(recordCount, 0);
> +      for(ValueVector<?> v : this.outputVectors){
> +        v.setRecordCount(recordCount);
> +      }
>        return upstream; // change if upstream changed, otherwise normal.
>      default:
>        throw new UnsupportedOperationException();
> @@ -134,24 +143,21 @@ public class ProjectRecordBatch implements
> RecordBatch{
>        }
>      }
>      this.outputVectors = Lists.newArrayList();
> -
> +    this.vh = new VectorHolder(outputVectors);
>      final List<NamedExpression> exprs = pop.getExprs();
>      final ErrorCollector collector = new ErrorCollectorImpl();
>      final List<TransferPairing<?>> transfers = Lists.newArrayList();
>
> -    final CodeGenerator cg = new CodeGenerator("setupEvaluators",
> "doPerRecordWork", context.getFunctionRegistry());
> +    final CodeGenerator<Projector> cg = new
> CodeGenerator<Projector>(Projector.TEMPLATE_DEFINITION,
> context.getFunctionRegistry());
>
>      for(int i =0; i < exprs.size(); i++){
>        final NamedExpression namedExpression = exprs.get(i);
> -      final MaterializedField outputField =
> getMaterializedField(namedExpression);
>        final LogicalExpression expr =
> ExpressionTreeMaterializer.materialize(namedExpression.getExpr(), incoming,
> collector);
> +      final MaterializedField outputField =
> getMaterializedField(namedExpression.getRef(), expr);
>        if(collector.hasErrors()){
>          throw new SchemaChangeException(String.format("Failure while
> trying to materialize incoming schema.  Errors:\n %s.",
> collector.toErrorString()));
>        }
>
> -
> -
> -
>        // add value vector to transfer if direct reference and this is
> allowed, otherwise, add to evaluation stack.
>        if(expr instanceof ValueVectorReadExpression &&
> incoming.getSchema().getSelectionVector() == SelectionVectorMode.NONE){
>          ValueVectorReadExpression vectorRead =
> (ValueVectorReadExpression) expr;
> @@ -179,21 +185,21 @@ public class ProjectRecordBatch implements
> RecordBatch{
>      this.outSchema = bldr.build();
>
>      try {
> -      return
> context.getImplementationClass(Projector.TEMPLATE_DEFINITION, cg);
> +      Projector projector =
> context.getImplementationClass(Projector.TEMPLATE_DEFINITION, cg);
> +      projector.setup(context, incoming, this, transfers);
> +      return projector;
>      } catch (ClassTransformationException | IOException e) {
>        throw new SchemaChangeException("Failure while attempting to load
> generated class", e);
>      }
>    }
>
> -
>    @Override
>    public WritableBatch getWritableBatch() {
> -    return null;
> +    return WritableBatch.get(incoming.getRecordCount(), outputVectors);
>    }
>
> -
> -  private MaterializedField getMaterializedField(NamedExpression ex){
> -    return new MaterializedField(getFieldDef(ex.getRef(),
> ex.getExpr().getMajorType()));
> +  private MaterializedField getMaterializedField(FieldReference
> reference, LogicalExpression expr){
> +    return new MaterializedField(getFieldDef(reference,
> expr.getMajorType()));
>    }
>
>    private FieldDef getFieldDef(SchemaPath path, MajorType type){
> @@ -212,7 +218,9 @@ public class ProjectRecordBatch implements RecordBatch{
>        }else{
>
>  parts.add(NamePart.newBuilder().setType(Type.NAME).setName(seg.getNameSegment().getPath().toString()).build());
>        }
> +      seg = seg.getChild();
>      }
>      return parts;
>    }
> +
>  }
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/80b1d24f/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 31c418c..8f22b40 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
> @@ -9,12 +9,12 @@ import org.apache.drill.exec.record.RecordBatch;
>
>  public interface Projector {
>
> -  public abstract void setup(FragmentContext context, RecordBatch
> incoming, List<TransferPairing<?>> transfers)  throws SchemaChangeException;
> +  public abstract void setup(FragmentContext context, RecordBatch
> incoming,  RecordBatch outgoing, List<TransferPairing<?>> transfers)
>  throws SchemaChangeException;
>
>
> -  public abstract void projectRecords(int recordCount, int
> firstOutputIndex);
> +  public abstract int projectRecords(int recordCount, int
> firstOutputIndex);
>
> -  public static TemplateClassDefinition<Projector, Void>
> TEMPLATE_DEFINITION = new TemplateClassDefinition<Projector, Void>( //
> -      Projector.class,
> "org.apache.drill.exec.physical.impl.project.ProjectTemplate",
> ProjectEvaluator.class, Void.class);
> +  public static TemplateClassDefinition<Projector> TEMPLATE_DEFINITION =
> new TemplateClassDefinition<Projector>( //
> +      Projector.class,
> "org.apache.drill.exec.physical.impl.project.ProjectorTemplate",
> ProjectEvaluator.class, "setupEvaluators", "doPerRecordWork");
>
>  }
> \ No newline at end of file
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/80b1d24f/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/ProjectorTemplate.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/ProjectorTemplate.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/ProjectorTemplate.java
> index 60af7d2..1163fef 100644
> ---
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/ProjectorTemplate.java
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/ProjectorTemplate.java
> @@ -2,50 +2,28 @@ package org.apache.drill.exec.physical.impl.project;
>
>  import java.util.List;
>
> -import org.apache.drill.common.expression.ErrorCollector;
> -import org.apache.drill.common.expression.ErrorCollectorImpl;
> -import org.apache.drill.common.expression.LogicalExpression;
> -import org.apache.drill.common.expression.PathSegment;
> -import org.apache.drill.common.expression.SchemaPath;
> -import org.apache.drill.common.logical.data.NamedExpression;
> -import org.apache.drill.common.types.TypeProtos.MajorType;
>  import org.apache.drill.exec.exception.SchemaChangeException;
> -import org.apache.drill.exec.expr.CodeGenerator;
> -import org.apache.drill.exec.expr.ExpressionTreeMaterializer;
> -import org.apache.drill.exec.expr.ValueVectorReadExpression;
> -import org.apache.drill.exec.expr.ValueVectorWriteExpression;
> -import org.apache.drill.exec.expr.fn.FunctionImplementationRegistry;
>  import org.apache.drill.exec.ops.FragmentContext;
> -import org.apache.drill.exec.physical.config.Project;
> -import org.apache.drill.exec.proto.SchemaDefProtos.FieldDef;
> -import org.apache.drill.exec.proto.SchemaDefProtos.NamePart;
> -import org.apache.drill.exec.proto.SchemaDefProtos.NamePart.Type;
>  import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
> -import org.apache.drill.exec.record.MaterializedField;
>  import org.apache.drill.exec.record.RecordBatch;
>  import org.apache.drill.exec.record.vector.SelectionVector2;
>  import org.apache.drill.exec.record.vector.SelectionVector4;
> -import org.apache.drill.exec.record.vector.TypeHelper;
> -import org.apache.drill.exec.record.vector.ValueVector;
>
> -import com.google.common.base.Preconditions;
>  import com.google.common.collect.ImmutableList;
> -import com.google.common.collect.Lists;
>
>  public abstract class ProjectorTemplate implements Projector {
>    static final org.slf4j.Logger logger =
> org.slf4j.LoggerFactory.getLogger(ProjectorTemplate.class);
> -
> +
>    private ImmutableList<TransferPairing<?>> transfers;
>    private SelectionVector2 vector2;
>    private SelectionVector4 vector4;
>    private SelectionVectorMode svMode;
>
> -  public ProjectorTemplate(final FragmentContext context, final
> RecordBatch incomingBatch, final Project pop,
> FunctionImplementationRegistry funcRegistry) throws SchemaChangeException{
> -    super();
> +  public ProjectorTemplate() throws SchemaChangeException{
>    }
>
>    @Override
> -  public final void projectRecords(final int recordCount, int
> firstOutputIndex) {
> +  public final int projectRecords(final int recordCount, int
> firstOutputIndex) {
>      switch(svMode){
>      case FOUR_BYTE:
>        throw new UnsupportedOperationException();
> @@ -56,7 +34,7 @@ public abstract class ProjectorTemplate implements
> Projector {
>        for(int i = 0; i < count; i+=2, firstOutputIndex++){
>          doPerRecordWork(vector2.getIndex(i), firstOutputIndex);
>        }
> -      return;
> +      return recordCount;
>
>
>      case NONE:
> @@ -68,7 +46,7 @@ public abstract class ProjectorTemplate implements
> Projector {
>        for (int i = 0; i < countN; i++, firstOutputIndex++) {
>          doPerRecordWork(i, firstOutputIndex);
>        }
> -      return;
> +      return recordCount;
>
>
>      default:
> @@ -77,7 +55,7 @@ public abstract class ProjectorTemplate implements
> Projector {
>    }
>
>    @Override
> -  public final void setup(FragmentContext context, RecordBatch incoming,
> List<TransferPairing<?>> transfers)  throws SchemaChangeException{
> +  public final void setup(FragmentContext context, RecordBatch incoming,
> RecordBatch outgoing, List<TransferPairing<?>> transfers)  throws
> SchemaChangeException{
>
>      this.svMode = incoming.getSchema().getSelectionVector();
>      switch(svMode){
> @@ -89,10 +67,10 @@ public abstract class ProjectorTemplate implements
> Projector {
>        break;
>      }
>      this.transfers = ImmutableList.copyOf(transfers);
> -    setupEvaluators(context, incoming);
> +    setupEvaluators(context, incoming, outgoing);
>    }
>
> -  protected abstract void setupEvaluators(FragmentContext context,
> RecordBatch incoming) throws SchemaChangeException;
> +  protected abstract void setupEvaluators(FragmentContext context,
> RecordBatch incoming, RecordBatch outgoing) throws SchemaChangeException;
>    protected abstract void doPerRecordWork(int inIndex, int outIndex);
>
>
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/80b1d24f/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatch.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatch.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatch.java
> index 40447ec..0edb93e 100644
> ---
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatch.java
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatch.java
> @@ -85,10 +85,10 @@ public interface RecordBatch {
>     * @param path The path where the vector should be located.
>     * @return The local field id associated with this vector.
>     */
> -  public abstract TypedFieldId getValueVector(SchemaPath path);
> +  public abstract TypedFieldId getValueVectorId(SchemaPath path);
>
>
> -  public abstract <T extends ValueVector<T>> T getValueVectorById(int
> fieldId, Class<?> vvClass);
> +  public abstract <T extends ValueVector<T>> T getValueVectorById(int
> fieldId, Class<?> clazz);
>
>    /**
>     * Update the data in each Field reading interface for the next range
> of records. Once a RecordBatch returns an
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/80b1d24f/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/SchemaBuilder.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/SchemaBuilder.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/SchemaBuilder.java
> index 20cc82c..0989c1d 100644
> ---
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/SchemaBuilder.java
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/SchemaBuilder.java
> @@ -21,6 +21,7 @@ import java.util.List;
>  import java.util.Set;
>
>  import org.apache.drill.exec.exception.SchemaChangeException;
> +import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
>
>  import com.beust.jcommander.internal.Sets;
>  import com.google.common.collect.Lists;
> @@ -32,7 +33,7 @@ import com.google.common.collect.Lists;
>  public class SchemaBuilder {
>    private Set<MaterializedField> fields = Sets.newHashSet();
>
> -  private BatchSchema.SelectionVectorMode selectionVectorMode;
> +  private BatchSchema.SelectionVectorMode selectionVectorMode =
> SelectionVectorMode.NONE;
>
>    SchemaBuilder() {
>    }
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/80b1d24f/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/NullableFixed8.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/NullableFixed8.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/NullableFixed8.java
> index 3fc39eb..48aa0c2 100644
> ---
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/NullableFixed8.java
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/NullableFixed8.java
> @@ -32,5 +32,12 @@ public final class NullableFixed8 extends
> NullableValueVector<NullableFixed8, Fi
>      return new Fixed8(this.field, allocator);
>    }
>
> +  public long get(int index){
> +    return 1l;
> +  }
> +
> +  public void set(int index, long value){
> +
> +  }
>
>  }
>
>

[46/53] [abbrv] git commit: Working project operator. Update to ASM 4.1

Posted by ja...@apache.org.
Working project operator.  Update to ASM 4.1


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

Branch: refs/heads/master
Commit: 80b1d24f427f23feddd9c93ef0130c4657569f7e
Parents: ce0da88
Author: Jacques Nadeau <ja...@apache.org>
Authored: Tue Jul 9 16:31:25 2013 -0700
Committer: Jacques Nadeau <ja...@apache.org>
Committed: Fri Jul 19 14:53:24 2013 -0700

----------------------------------------------------------------------
 sandbox/prototype/common/pom.xml                |   7 +
 sandbox/prototype/exec/java-exec/pom.xml        |   8 +-
 .../drill/exec/compile/ClassTransformer.java    | 224 ++++++++++++++-----
 .../drill/exec/compile/JaninoClassCompiler.java |   7 +-
 .../exec/compile/TemplateClassDefinition.java   |  23 +-
 .../apache/drill/exec/expr/CodeGenerator.java   |  32 ++-
 .../apache/drill/exec/expr/DrillBatchFunc.java  |  10 +
 .../drill/exec/expr/EvaluationVisitor.java      |  76 ++++---
 .../exec/expr/ExpressionTreeMaterializer.java   |   2 +-
 .../drill/exec/expr/fn/FunctionConverter.java   |   9 +-
 .../drill/exec/expr/fn/FunctionHolder.java      |  32 ++-
 .../drill/exec/expr/fn/impl/MathFunctions.java  |   4 +-
 .../drill/exec/expr/holders/BooleanHolder.java  |  10 +
 .../drill/exec/expr/holders/IntHolder.java      |  10 +
 .../drill/exec/expr/holders/LongHolder.java     |  10 +
 .../expr/holders/NullableBooleanHolder.java     |  11 +
 .../exec/expr/holders/NullableIntHolder.java    |  11 +
 .../exec/expr/holders/NullableLongHolder.java   |  11 +
 .../holders/ValueHolderImplmenetations.java     |  44 ----
 .../apache/drill/exec/ops/FragmentContext.java  |   9 +-
 .../exec/physical/impl/FilterRecordBatch.java   |   4 +-
 .../drill/exec/physical/impl/ImplCreator.java   |   9 +-
 .../drill/exec/physical/impl/ScanBatch.java     |   2 +-
 .../drill/exec/physical/impl/ScreenCreator.java |   7 +-
 .../exec/physical/impl/WireRecordBatch.java     |   2 +-
 .../physical/impl/filter/ExampleFilter.java     |   2 +-
 .../physical/impl/filter/FilterTemplate.java    |  48 ++++
 .../SelectionVectorPopulationExpression.java    |  39 ++++
 .../impl/project/ProjectBatchCreator.java       |  23 ++
 .../physical/impl/project/ProjectEvaluator.java |   2 +-
 .../impl/project/ProjectRecordBatch.java        |  40 ++--
 .../exec/physical/impl/project/Projector.java   |   8 +-
 .../impl/project/ProjectorTemplate.java         |  38 +---
 .../apache/drill/exec/record/RecordBatch.java   |   4 +-
 .../apache/drill/exec/record/SchemaBuilder.java |   3 +-
 .../exec/record/vector/NullableFixed8.java      |   7 +
 .../exec/record/vector/SelectionVector2.java    |   3 +
 .../exec/compile/TestClassTransformation.java   |  13 +-
 .../apache/drill/exec/expr/ExpressionTest.java  |   7 +-
 .../exec/physical/impl/SimpleRootExec.java      |  40 ++++
 .../impl/project/TestSimpleProjection.java      |  69 ++++++
 .../record/ExpressionTreeMaterializerTest.java  |   8 +-
 .../exec/test/generated/TestGeneratedClass.java |  15 ++
 .../src/test/resources/project/test1.json       |  37 +++
 sandbox/prototype/pom.xml                       |  17 ++
 45 files changed, 750 insertions(+), 247 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/80b1d24f/sandbox/prototype/common/pom.xml
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/pom.xml b/sandbox/prototype/common/pom.xml
index 4891191..a33ced9 100644
--- a/sandbox/prototype/common/pom.xml
+++ b/sandbox/prototype/common/pom.xml
@@ -74,6 +74,13 @@
   <build>
     <plugins>
       <plugin>
+          <artifactId>maven-surefire-plugin</artifactId>
+          <version>2.15</version>
+          <configuration>
+            <useSystemClassLoader>false</useSystemClassLoader>
+          </configuration>
+      </plugin>    
+      <plugin>
         <artifactId>maven-jar-plugin</artifactId>
         <executions>
           <execution>

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/80b1d24f/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 e348bc7..f2f0bff 100644
--- a/sandbox/prototype/exec/java-exec/pom.xml
+++ b/sandbox/prototype/exec/java-exec/pom.xml
@@ -13,14 +13,14 @@
 
   <dependencies>
     <dependency>
-      <groupId>asm</groupId>
+      <groupId>org.ow2.asm</groupId>
       <artifactId>asm-util</artifactId>
-      <version>3.3.1</version>
+      <version>4.1</version>
     </dependency>
     <dependency>
-      <groupId>asm</groupId>
+      <groupId>org.ow2.asm</groupId>
       <artifactId>asm-commons</artifactId>
-      <version>3.3.1</version>
+      <version>4.1</version>
     </dependency>
     <dependency>
       <groupId>com.sun.codemodel</groupId>

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/80b1d24f/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/compile/ClassTransformer.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/compile/ClassTransformer.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/compile/ClassTransformer.java
index 079ab6c..4bf6e7e 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/compile/ClassTransformer.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/compile/ClassTransformer.java
@@ -19,33 +19,37 @@ package org.apache.drill.exec.compile;
 
 import java.io.File;
 import java.io.IOException;
-import java.io.PrintWriter;
 import java.lang.reflect.Modifier;
 import java.net.URL;
 import java.util.Iterator;
+import java.util.Random;
+import java.util.Set;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicLong;
 
 import org.apache.drill.exec.exception.ClassTransformationException;
 import org.codehaus.commons.compiler.CompileException;
-import org.objectweb.asm.ClassAdapter;
 import org.objectweb.asm.ClassReader;
 import org.objectweb.asm.ClassVisitor;
 import org.objectweb.asm.ClassWriter;
+import org.objectweb.asm.FieldVisitor;
 import org.objectweb.asm.MethodVisitor;
-import org.objectweb.asm.commons.EmptyVisitor;
+import org.objectweb.asm.Opcodes;
+import org.objectweb.asm.commons.Remapper;
 import org.objectweb.asm.commons.RemappingClassAdapter;
 import org.objectweb.asm.commons.RemappingMethodAdapter;
 import org.objectweb.asm.commons.SimpleRemapper;
 import org.objectweb.asm.tree.ClassNode;
 import org.objectweb.asm.tree.FieldNode;
 import org.objectweb.asm.tree.MethodNode;
-import org.objectweb.asm.util.TraceClassVisitor;
 
+import com.beust.jcommander.internal.Sets;
+import com.google.common.base.Preconditions;
 import com.google.common.cache.CacheBuilder;
 import com.google.common.cache.CacheLoader;
 import com.google.common.cache.LoadingCache;
+import com.google.common.io.Files;
 import com.google.common.io.Resources;
 
 public class ClassTransformer {
@@ -53,76 +57,113 @@ public class ClassTransformer {
 
   private AtomicLong index = new AtomicLong(0);
   private AtomicLong interfaceIndex = new AtomicLong(0);
-  private LoadingCache<String, byte[]> byteCode = CacheBuilder.newBuilder()
-      .maximumSize(10000)
-      .expireAfterWrite(10, TimeUnit.MINUTES)
-      .build(new ClassBytesCacheLoader());
-  
-  
-  private class ClassBytesCacheLoader extends CacheLoader<String, byte[]>{
+  private LoadingCache<String, byte[]> byteCode = CacheBuilder.newBuilder().maximumSize(10000)
+      .expireAfterWrite(10, TimeUnit.MINUTES).build(new ClassBytesCacheLoader());
+
+  private class ClassBytesCacheLoader extends CacheLoader<String, byte[]> {
     public byte[] load(String path) throws ClassTransformationException, IOException {
       URL u = this.getClass().getResource(path);
-      if (u == null) throw new ClassTransformationException(String.format("Unable to find TemplateClass at path %s",path));
-      return Resources.toByteArray(u);              
+      if (u == null)
+        throw new ClassTransformationException(String.format("Unable to find TemplateClass at path %s", path));
+      return Resources.toByteArray(u);
     }
   };
-  
+
   private byte[] getClassByteCodeFromPath(String path) throws ClassTransformationException, IOException {
-    try{
+    try {
       return byteCode.get(path);
     } catch (ExecutionException e) {
       Throwable c = e.getCause();
-      if(c instanceof ClassTransformationException) throw (ClassTransformationException) c;
-      if(c instanceof IOException) throw (IOException) c;
+      if (c instanceof ClassTransformationException)
+        throw (ClassTransformationException) c;
+      if (c instanceof IOException)
+        throw (IOException) c;
       throw new ClassTransformationException(c);
     }
   }
 
-  
-  
-  @SuppressWarnings("unchecked")
-  public <T, I> T getImplementationClass(QueryClassLoader classLoader,
-      TemplateClassDefinition<T, I> templateDefinition, String internalClassBody, I initObject)
-      throws ClassTransformationException {
-    final String implClassName = templateDefinition.getTemplateClassName() + interfaceIndex.getAndIncrement();
+  public <T, I> T getImplementationClassByBody( //
+      QueryClassLoader classLoader, //
+      TemplateClassDefinition<T> templateDefinition, //
+      String internalClassBody //
+  ) throws ClassTransformationException {
     final String materializedClassName = "org.apache.drill.generated."
-        + templateDefinition.getExternalInterface().getSimpleName() + index.getAndIncrement();
-    // final String materializedClassName = templateDefinition.getTemplateClassName();
+        + "Gen" + templateDefinition.getExternalInterface().getSimpleName() //
+        + index.getAndIncrement();
+    // Get Implementation Class
     try {
-
-      // Get Implementation Class
       String classBody = ClassBodyBuilder.newBuilder() //
-          .setClassName(implClassName) //
+          .setClassName(materializedClassName) //
           .setImplementedInterfaces(templateDefinition.getInternalInterface()) //
           .setBody(internalClassBody) //
           .build();
-      final byte[] implementationClass = classLoader.getClassByteCode(implClassName, classBody);
+      return getImplementationClass(classLoader, templateDefinition, classBody, materializedClassName);
+    } catch (IOException | CompileException e) {
+      throw new ClassTransformationException("Failure generating class body for runtime generated class.", e);
+    }
+
+  }
+
+  @SuppressWarnings("unchecked")
+  public <T, I> T getImplementationClass( //
+      QueryClassLoader classLoader, //
+      TemplateClassDefinition<T> templateDefinition, //
+      String entireClass, //
+      String materializedClassName) throws ClassTransformationException {
+
+    try {
+
+      final byte[] implementationClass = classLoader.getClassByteCode(materializedClassName, entireClass);
 
       // Get Template Class
       final String templateClassName = templateDefinition.getTemplateClassName().replaceAll("\\.", File.separator);
       final String templateClassPath = File.separator + templateClassName + ".class";
       final byte[] templateClass = getClassByteCodeFromPath(templateClassPath);
-
+      int fileNum = new Random().nextInt(100);
+      Files.write(templateClass, new File(String.format("/tmp/%d-template.class", fileNum)));
       // Generate Merge Class
-      ClassNode impl = getClassNodeFromByteCode(implementationClass);
-      // traceClassToSystemOut(implementationClass);
-      // traceClassToSystemOut(templateClass);
-      ClassWriter cw = new ClassWriter(0);
-      MergeAdapter adapter = new MergeAdapter(cw, impl);
-      ClassReader tReader = new ClassReader(templateClass);
-      tReader.accept(adapter, 0);
-
-      byte[] outputClass = cw.toByteArray();
-
-      cw = new ClassWriter(0);
-      RemappingClassAdapter r = new RemappingClassAdapter(cw, new SimpleRemapper(templateClassName,
-          materializedClassName.replace('.', '/')));
-      new ClassReader(outputClass).accept(r, 0);
-      outputClass = cw.toByteArray();
-      // traceClassToSystemOut(outputClass);
-
-      // Load the class
-      classLoader.injectByteCode(materializedClassName, outputClass);
+
+      // Setup adapters for merging, remapping class names and class writing. This is done in reverse order of how they
+      // will be evaluated.
+      String oldTemplateSlashName = templateDefinition.getTemplateClassName().replace('.', '/');
+      String materializedSlashName = materializedClassName.replace('.', '/');
+      RemapClasses remapper = new RemapClasses(oldTemplateSlashName, materializedSlashName);
+      
+      {
+        ClassNode impl = getClassNodeFromByteCode(implementationClass);
+        ClassWriter cw = new ClassWriter(ClassWriter.COMPUTE_FRAMES);
+
+        ClassVisitor remappingAdapter = new RemappingClassAdapter(cw, remapper);
+        MergeAdapter mergingAdapter = new MergeAdapter(oldTemplateSlashName, materializedSlashName, remappingAdapter,
+            impl);
+        ClassReader tReader = new ClassReader(templateClass);
+        tReader.accept(mergingAdapter, ClassReader.EXPAND_FRAMES);
+        byte[] outputClass = cw.toByteArray();
+        Files.write(outputClass, new File(String.format("/tmp/%d-output.class", fileNum)));
+        outputClass = cw.toByteArray();
+
+        // Load the class
+        classLoader.injectByteCode(materializedClassName, outputClass);
+      }
+
+      int i = 0;
+      for (String s : remapper.getSubclasses()) {
+        logger.debug("Setting up sub class {}", s);
+        // for each sub class, remap them into the new class.
+        String subclassPath = File.separator + s + ".class";
+        final byte[] bytecode = getClassByteCodeFromPath(subclassPath);
+        RemapClasses localRemapper = new RemapClasses(oldTemplateSlashName, materializedSlashName);
+        Preconditions.checkArgument(localRemapper.getSubclasses().isEmpty(), "Class transformations are only supported for classes that have a single level of inner classes.");
+        ClassWriter subcw = new ClassWriter(ClassWriter.COMPUTE_FRAMES);
+        ClassVisitor remap = new RemappingClassAdapter(subcw, localRemapper);
+        ClassReader reader = new ClassReader(bytecode);
+        reader.accept(remap, ClassReader.EXPAND_FRAMES);
+        byte[] newByteCode = subcw.toByteArray();
+        classLoader.injectByteCode(s.replace(oldTemplateSlashName, materializedSlashName).replace('/', '.'), newByteCode);
+        Files.write(subcw.toByteArray(), new File(String.format("/tmp/%d-sub-%d.class", fileNum, i)));
+        i++;
+      }
+
       Class<?> c = classLoader.findClass(materializedClassName);
       if (templateDefinition.getExternalInterface().isAssignableFrom(c)) {
         return (T) c.newInstance();
@@ -131,7 +172,8 @@ public class ClassTransformer {
       }
 
     } catch (CompileException | IOException | ClassNotFoundException | InstantiationException | IllegalAccessException e) {
-      throw new ClassTransformationException("Failure generating transformation classes.", e);
+      throw new ClassTransformationException(String.format(
+          "Failure generating transformation classes for value: \n %s", entireClass), e);
     }
 
   }
@@ -143,19 +185,37 @@ public class ClassTransformer {
     return impl;
   }
 
-  private void traceClassToSystemOut(byte[] bytecode) {
-    TraceClassVisitor tcv = new TraceClassVisitor(new EmptyVisitor(), new PrintWriter(System.out));
-    ClassReader cr = new ClassReader(bytecode);
-    cr.accept(tcv, 0);
-  }
+  // private void traceClassToSystemOut(byte[] bytecode) {
+  // TraceClassVisitor tcv = new TraceClassVisitor(new EmptyVisitor(), new PrintWriter(System.out));
+  // ClassReader cr = new ClassReader(bytecode);
+  // cr.accept(tcv, 0);
+  // }
 
-  public class MergeAdapter extends ClassAdapter {
+  public class MergeAdapter extends ClassVisitor {
     private ClassNode classToMerge;
     private String cname;
+    private String templateName;
+    private String newName;
 
-    public MergeAdapter(ClassVisitor cv, ClassNode cn) {
-      super(cv);
+    public MergeAdapter(String templateName, String newName, ClassVisitor cv, ClassNode cn) {
+      super(Opcodes.ASM4, cv);
       this.classToMerge = cn;
+      this.templateName = templateName;
+      this.newName = newName.replace('.', '/');
+      ;
+
+    }
+
+    @Override
+    public void visitInnerClass(String name, String outerName, String innerName, int access) {
+      logger.debug(String.format(
+          "[Inner Class] Name: %s, outerName: %s, innerName: %s, templateName: %s, newName: %s.", name, outerName,
+          innerName, templateName, newName));
+      if (name.startsWith(templateName)) {
+        super.visitInnerClass(name.replace(templateName, newName), newName, innerName, access);
+      } else {
+        super.visitInnerClass(name, outerName, innerName, access);
+      }
     }
 
     // visit the class
@@ -193,7 +253,8 @@ public class ClassTransformer {
         MethodNode mn = (MethodNode) it.next();
 
         // skip the init.
-        if (mn.name.equals("<init>")) continue;
+        if (mn.name.equals("<init>"))
+          continue;
 
         String[] exceptions = new String[mn.exceptions.size()];
         mn.exceptions.toArray(exceptions);
@@ -206,6 +267,47 @@ public class ClassTransformer {
       }
       super.visitEnd();
     }
+
+    @Override
+    public FieldVisitor visitField(int access, String name, String desc, String signature, Object value) {
+      return super.visitField(access, name, desc, signature, value);
+    }
+
+  }
+
+  static class RemapClasses extends Remapper {
+
+    final Set<String> subclasses = Sets.newHashSet();
+
+    String oldName;
+    String newName;
+
+    public RemapClasses(String oldName, String newName) {
+      super();
+      Preconditions.checkArgument(!newName.startsWith(oldName), "The new name of a class cannot start with the old name of a class, otherwise class renaming will cause problems.");
+      this.oldName = oldName;
+      this.newName = newName;
+    }
+
+    @Override
+    public String map(String typeName) {
+      // remap the names of all classes that start with the old class name.
+      if (typeName.startsWith(oldName)) {
+        
+        // write down all the sub classes.
+        if (typeName.startsWith(oldName + "$")){
+          subclasses.add(typeName);
+        }
+          
+        return typeName.replace(oldName, newName);
+      }
+      return typeName;
+    }
+
+    public Set<String> getSubclasses() {
+      return subclasses;
+    }
+
   }
 
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/80b1d24f/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/compile/JaninoClassCompiler.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/compile/JaninoClassCompiler.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/compile/JaninoClassCompiler.java
index 86fe58b..abe2afe 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/compile/JaninoClassCompiler.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/compile/JaninoClassCompiler.java
@@ -35,15 +35,16 @@ public class JaninoClassCompiler implements ClassCompiler{
 
   private IClassLoader compilationClassLoader;
 
-  private boolean debugLines;
-  private boolean debugVars;
-  private boolean debugSource;
+  private boolean debugLines = true;
+  private boolean debugVars = true;
+  private boolean debugSource = true;
 
   public JaninoClassCompiler(ClassLoader parentClassLoader) {
     this.compilationClassLoader = new ClassLoaderIClassLoader(parentClassLoader);
   }
 
   public byte[] getClassByteCode(final String className, final String code) throws CompileException, IOException, ClassNotFoundException, ClassTransformationException {
+    logger.debug("Compiling:\n {}", code);
     StringReader reader = new StringReader(code);
     Scanner scanner = new Scanner((String) null, reader);
     Java.CompilationUnit compilationUnit = new Parser(scanner).parseCompilationUnit();

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/80b1d24f/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 fee4c97..5a01dce 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
@@ -20,21 +20,25 @@ package org.apache.drill.exec.compile;
 import java.lang.reflect.Method;
 
 
-public class TemplateClassDefinition<T, I>{
+public class TemplateClassDefinition<T>{
   
   private final Class<T> externalInterface;
   private final String templateClassName;
   private final Class<?> internalInterface;
-  private final Class<I> constructorObject;
+  private final String setupName;
+  private final String evalName;
   
-  public TemplateClassDefinition(Class<T> externalInterface, String templateClassName, Class<?> internalInterface, Class<I> constructorObject) {
+
+  public TemplateClassDefinition(Class<T> externalInterface, String templateClassName, Class<?> internalInterface, String setupName, String evalName) {
     super();
     this.externalInterface = externalInterface;
-    this.templateClassName = templateClassName; 
+    this.templateClassName = templateClassName;
     this.internalInterface = internalInterface;
-    this.constructorObject = constructorObject;
+    this.setupName = setupName;
+    this.evalName = evalName;
   }
 
+
   public Class<T> getExternalInterface() {
     return externalInterface;
   }
@@ -48,8 +52,13 @@ public class TemplateClassDefinition<T, I>{
     return templateClassName;
   }
 
-  public Class<I> getConstructorObject() {
-    return constructorObject;
+  public String getSetupName() {
+    return setupName;
+  }
+
+
+  public String getEvalName() {
+    return evalName;
   }
   
   

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/80b1d24f/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 77ae77d..ed6bd9b 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
@@ -4,13 +4,14 @@ import java.io.IOException;
 
 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.expr.fn.FunctionImplementationRegistry;
-import org.apache.drill.exec.expr.holders.ValueHolderImplmenetations.BooleanHolder;
-import org.apache.drill.exec.expr.holders.ValueHolderImplmenetations.IntHolder;
-import org.apache.drill.exec.expr.holders.ValueHolderImplmenetations.LongHolder;
-import org.apache.drill.exec.expr.holders.ValueHolderImplmenetations.NullableBooleanHolder;
-import org.apache.drill.exec.expr.holders.ValueHolderImplmenetations.NullableIntHolder;
-import org.apache.drill.exec.expr.holders.ValueHolderImplmenetations.NullableLongHolder;
+import org.apache.drill.exec.expr.holders.BooleanHolder;
+import org.apache.drill.exec.expr.holders.IntHolder;
+import org.apache.drill.exec.expr.holders.LongHolder;
+import org.apache.drill.exec.expr.holders.NullableBooleanHolder;
+import org.apache.drill.exec.expr.holders.NullableIntHolder;
+import org.apache.drill.exec.expr.holders.NullableLongHolder;
 import org.apache.drill.exec.ops.FragmentContext;
 import org.apache.drill.exec.record.RecordBatch;
 
@@ -26,7 +27,7 @@ import com.sun.codemodel.JMod;
 import com.sun.codemodel.JType;
 import com.sun.codemodel.JVar;
 
-public class CodeGenerator {
+public class CodeGenerator<T> {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(CodeGenerator.class);
   
   public JDefinedClass clazz;
@@ -37,17 +38,19 @@ public class CodeGenerator {
   private final EvaluationVisitor evaluationVisitor;
   private final String setupName;
   private final String perRecordName;
-  
+  private final TemplateClassDefinition<T> definition;
   private JCodeModel model;
   private int index = 0;
 
-  public CodeGenerator(String setupName, String perRecordName, FunctionImplementationRegistry funcRegistry) {
+  public CodeGenerator(TemplateClassDefinition<T> definition, FunctionImplementationRegistry funcRegistry) {
     super();
     try{
-      this.setupName = setupName;
-      this.perRecordName = perRecordName;
+      this.definition = definition;
+      this.setupName = definition.getSetupName();
+      this.perRecordName = definition.getEvalName();
       this.model = new JCodeModel();
       this.clazz = model._package("org.apache.drill.exec.test.generated")._class("Test1");
+      clazz._implements(definition.getInternalInterface());
       this.parentEvalBlock = new JBlock();
       this.parentSetupBlock = new JBlock();
       this.evaluationVisitor = new EvaluationVisitor(funcRegistry);
@@ -57,6 +60,7 @@ public class CodeGenerator {
   }
 
   public void addNextWrite(ValueVectorWriteExpression ex){
+    logger.debug("Adding next write {}", ex);
     currentEvalBlock = new JBlock();
     parentEvalBlock.add(currentEvalBlock);
     currentSetupBlock = new JBlock();
@@ -68,6 +72,10 @@ public class CodeGenerator {
     return currentEvalBlock;
   }
 
+  public String getMaterializedClassName(){
+    return "org.apache.drill.exec.test.generated.Test1";
+  }
+  
   public JBlock getSetupBlock(){
     return currentSetupBlock;
   }
@@ -134,7 +142,7 @@ public class CodeGenerator {
   }
   
   
-  public class HoldingContainer{
+  public static class HoldingContainer{
     private final JVar holder;
     private final JFieldRef value;
     private final JFieldRef isSet;

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/80b1d24f/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/DrillBatchFunc.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/DrillBatchFunc.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/DrillBatchFunc.java
new file mode 100644
index 0000000..5042708
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/DrillBatchFunc.java
@@ -0,0 +1,10 @@
+package org.apache.drill.exec.expr;
+
+import org.apache.drill.exec.record.RecordBatch;
+
+public interface DrillBatchFunc {
+  public void setup(RecordBatch incoming);
+  public void eval();
+  public void batchReset();
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/80b1d24f/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 a5bc5fa..04f78d0 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
@@ -10,20 +10,24 @@ 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.AbstractExprVisitor;
+import org.apache.drill.common.types.Types;
 import org.apache.drill.exec.expr.CodeGenerator.HoldingContainer;
 import org.apache.drill.exec.expr.fn.FunctionHolder;
 import org.apache.drill.exec.expr.fn.FunctionImplementationRegistry;
+import org.apache.drill.exec.physical.impl.filter.SelectionVectorPopulationExpression;
+import org.apache.drill.exec.record.vector.SelectionVector2;
 import org.apache.drill.exec.record.vector.TypeHelper;
 
+import com.google.common.base.Preconditions;
 import com.sun.codemodel.JBlock;
 import com.sun.codemodel.JClass;
 import com.sun.codemodel.JConditional;
 import com.sun.codemodel.JExpr;
-import com.sun.codemodel.JInvocation;
+import com.sun.codemodel.JExpression;
 import com.sun.codemodel.JType;
 import com.sun.codemodel.JVar;
 
-public class EvaluationVisitor extends AbstractExprVisitor<HoldingContainer, CodeGenerator, RuntimeException> {
+public class EvaluationVisitor extends AbstractExprVisitor<HoldingContainer, CodeGenerator<?>, RuntimeException> {
 
   private FunctionImplementationRegistry registry;
   
@@ -33,7 +37,7 @@ public class EvaluationVisitor extends AbstractExprVisitor<HoldingContainer, Cod
   }
 
   @Override
-  public HoldingContainer visitFunctionCall(FunctionCall call, CodeGenerator generator) throws RuntimeException {
+  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);
@@ -43,7 +47,7 @@ public class EvaluationVisitor extends AbstractExprVisitor<HoldingContainer, Cod
   }
   
   @Override
-  public HoldingContainer visitIfExpression(IfExpression ifExpr, CodeGenerator generator) throws RuntimeException {
+  public HoldingContainer visitIfExpression(IfExpression ifExpr, CodeGenerator<?> generator) throws RuntimeException {
     JBlock local = generator.getBlock();
     
     HoldingContainer output = generator.declare(ifExpr.getMajorType());
@@ -93,46 +97,46 @@ public class EvaluationVisitor extends AbstractExprVisitor<HoldingContainer, Cod
   }
 
   @Override
-  public HoldingContainer visitSchemaPath(SchemaPath path, CodeGenerator generator) throws RuntimeException {
+  public HoldingContainer visitSchemaPath(SchemaPath path, CodeGenerator<?> generator) throws RuntimeException {
     throw new UnsupportedOperationException("All schema paths should have been replaced with ValueVectorExpressions.");
   }
 
   @Override
-  public HoldingContainer visitLongConstant(LongExpression e, CodeGenerator generator) throws RuntimeException {
+  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 visitDoubleConstant(DoubleExpression e, CodeGenerator generator) throws RuntimeException {
+  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 {
+  public HoldingContainer visitBooleanConstant(BooleanExpression e, CodeGenerator<?> generator) throws RuntimeException {
     HoldingContainer out = generator.declare(e.getMajorType());
     generator.getBlock().assign(out.getValue(), JExpr.lit(e.getBoolean()));
     return out;
   }
-
-  
   
   @Override
-  public HoldingContainer visitUnknown(LogicalExpression e, CodeGenerator generator) throws RuntimeException {
+  public HoldingContainer visitUnknown(LogicalExpression e, CodeGenerator<?> generator) throws RuntimeException {
     if(e instanceof ValueVectorReadExpression){
       return visitValueVectorExpression((ValueVectorReadExpression) e, generator);
     }else if(e instanceof ValueVectorWriteExpression){
       return visitValueVectorWriteExpression((ValueVectorWriteExpression) e, generator);
+    }else if(e instanceof SelectionVectorPopulationExpression){
+      return visitSelectionVectorExpression((SelectionVectorPopulationExpression) e, generator);
     }else{
       return super.visitUnknown(e, generator);  
     }
     
   }
 
-  private HoldingContainer visitValueVectorWriteExpression(ValueVectorWriteExpression e, CodeGenerator generator){
+  private HoldingContainer visitValueVectorWriteExpression(ValueVectorWriteExpression e, CodeGenerator<?> generator){
     LogicalExpression child = e.getChild();
     HoldingContainer hc = child.accept(this, generator);
     JBlock block = generator.getBlock();
@@ -142,9 +146,11 @@ public class EvaluationVisitor extends AbstractExprVisitor<HoldingContainer, Cod
     JVar vv = generator.declareClassField("vv", vvType);
     
     // get value vector in setup block.
-    generator.getSetupBlock().assign(vv, JExpr.direct("outgoing").invoke("getValueVector") //
-      .arg(JExpr.lit(e.getFieldId())) //
-      .arg( ((JClass)vvType).dotclass()));
+    JVar obj = generator.getSetupBlock().decl( //
+        generator.getModel()._ref(Object.class), //
+        generator.getNextVar("obj"), // 
+        JExpr.direct("outgoing").invoke("getValueVectorById").arg(JExpr.lit(e.getFieldId())).arg( ((JClass)vvType).dotclass()));
+    generator.getSetupBlock().assign(vv, JExpr.cast(vvType, obj));
     
     if(hc.isOptional()){
       vv.invoke("set").arg(JExpr.direct("outIndex"));
@@ -156,18 +162,19 @@ public class EvaluationVisitor extends AbstractExprVisitor<HoldingContainer, Cod
     return null;
   }
   
-  private HoldingContainer visitValueVectorExpression(ValueVectorReadExpression e, CodeGenerator generator) throws RuntimeException{
+  private HoldingContainer visitValueVectorExpression(ValueVectorReadExpression e, CodeGenerator<?> generator) throws RuntimeException{
     // declare value vector
     Class<?> vvClass = TypeHelper.getValueVectorClass(e.getMajorType().getMinorType(), e.getMajorType().getMode());
     JType vvType = generator.getModel()._ref(vvClass);
     JVar vv1 = generator.declareClassField("vv", vvType);
     
     // get value vector from incoming batch and 
-    JInvocation getValue = JExpr //
-        .invoke(JExpr.direct("incoming"), "getValueVector") //
-        .arg(JExpr.lit(e.getFieldId())) //
-        .arg( ((JClass)vvType).dotclass());
-    generator.getSetupBlock().assign(vv1, getValue);
+    // get value vector in setup block.
+    JVar obj = generator.getSetupBlock().decl( //
+        generator.getModel()._ref(Object.class), //
+        generator.getNextVar("obj"), // 
+        JExpr.direct("outgoing").invoke("getValueVectorById").arg(JExpr.lit(e.getFieldId())).arg( ((JClass)vvType).dotclass()));
+    generator.getSetupBlock().assign(vv1, JExpr.cast(vvType, obj));
 
     // evaluation work.
     HoldingContainer out = generator.declare(e.getMajorType());
@@ -175,25 +182,42 @@ public class EvaluationVisitor extends AbstractExprVisitor<HoldingContainer, Cod
     
     if(out.isOptional()){
       JBlock blk = generator.getBlock();
-      blk.assign(out.getIsSet(), vv1.invoke("isSet").arg(JExpr.direct("index")));
+      blk.assign(out.getIsSet(), vv1.invoke("isSet").arg(JExpr.direct("inIndex")));
       JConditional jc = blk._if(out.getIsSet());
       jc._then() //
-        .assign(out.getValue(), vv1.invoke("get").arg(JExpr.direct("index"))); //
+        .assign(out.getValue(), vv1.invoke("get").arg(JExpr.direct("inIndex"))); //
         //.assign(out.getIsSet(), JExpr.lit(1));
       //jc._else()
         //.assign(out.getIsSet(), JExpr.lit(0));
       
     }else{
-      generator.getBlock().assign(out.getValue(), vv1.invoke("get").arg(JExpr.direct("index")));
+      generator.getBlock().assign(out.getValue(), vv1.invoke("get").arg(JExpr.direct("inIndex")));
     }
     return out;
   }
   
   
+  private HoldingContainer visitSelectionVectorExpression(SelectionVectorPopulationExpression e, CodeGenerator<?> generator){
+    JType svClass = generator.getModel()._ref(SelectionVector2.class);
+    JVar sv = generator.declareClassField("sv", svClass);
+    JVar index = generator.declareClassField("svIndex", generator.getModel().CHAR);
+    LogicalExpression child = e.getChild();
+    Preconditions.checkArgument(child.getMajorType().equals(Types.REQUIRED_BOOLEAN));
+    HoldingContainer hc = child.accept(this, generator);
+    generator.getBlock()._return(hc.getValue());
+    
+//    JBlock blk = generator.getSetupBlock();
+//    blk.assign(sv, JExpr.direct("outgoing").invoke("getSelectionVector2"));
+//    JConditional jc = blk._if(hc.getValue());
+//    JBlock body = jc._then();
+//    body.add(sv.invoke("set").arg(index).arg(JExpr.direct("inIndex")));
+//    body.assign(index, index.plus(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 use the valuevector classes and internal vectors.");
+  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));
   }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/80b1d24f/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 bc44490..72e5c93 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
@@ -97,7 +97,7 @@ public class ExpressionTreeMaterializer {
     @Override
     public LogicalExpression visitSchemaPath(SchemaPath path) {
       logger.debug("Visiting schema path {}", path);
-      TypedFieldId tfId = batch.getValueVector(path);
+      TypedFieldId tfId = batch.getValueVectorId(path);
       if (tfId == null) {
         return NullExpression.INSTANCE;
       } else {

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/80b1d24f/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 7276e7d..84f04f0 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
@@ -16,6 +16,7 @@ import org.apache.drill.exec.expr.annotations.Output;
 import org.apache.drill.exec.expr.annotations.Param;
 import org.apache.drill.exec.expr.annotations.WorkSpace;
 import org.apache.drill.exec.expr.fn.FunctionHolder.ValueReference;
+import org.apache.drill.exec.expr.fn.FunctionHolder.WorkspaceReference;
 import org.apache.drill.exec.expr.holders.ValueHolder;
 import org.codehaus.commons.compiler.CompileException;
 import org.codehaus.janino.Java;
@@ -43,7 +44,8 @@ public class FunctionConverter {
     
     // start by getting field information.
     List<ValueReference> params = Lists.newArrayList();
-    List<String> workspaceFields = Lists.newArrayList();
+    List<WorkspaceReference> workspaceFields = Lists.newArrayList();
+    
     ValueReference outputField = null;
     
     
@@ -96,7 +98,7 @@ public class FunctionConverter {
         
       }else{
         // workspace work.
-        workspaceFields.add(field.getName());
+        workspaceFields.add(new WorkspaceReference(field.getType(), field.getName()));
       }
       
     }
@@ -118,7 +120,8 @@ public class FunctionConverter {
 
     // return holder
     ValueReference[] ps = params.toArray(new ValueReference[params.size()]);
-    FunctionHolder fh = new FunctionHolder(template.scope(), template.nulls(), template.isBinaryCommutative(), template.name(), ps, outputField, methods);
+    WorkspaceReference[] works = workspaceFields.toArray(new WorkspaceReference[workspaceFields.size()]);
+    FunctionHolder fh = new FunctionHolder(template.scope(), template.nulls(), template.isBinaryCommutative(), template.name(), ps, outputField, works, methods);
     return fh;
   }
   

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/80b1d24f/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 460169f..f7ecd02 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,5 +1,6 @@
 package org.apache.drill.exec.expr.fn;
 
+import java.lang.reflect.Type;
 import java.util.Arrays;
 import java.util.Map;
 
@@ -32,13 +33,15 @@ public class FunctionHolder {
   private String evalBody;
   private String addBody;
   private String setupBody;
+  private WorkspaceReference[] workspaceVars;
   private ValueReference[] parameters;
   private ValueReference returnValue;
   
-  public FunctionHolder(FunctionScope scope, NullHandling nullHandling, boolean isBinaryCommutative, String functionName, ValueReference[] parameters, ValueReference returnValue, Map<String, String> methods) {
+  public FunctionHolder(FunctionScope scope, NullHandling nullHandling, boolean isBinaryCommutative, String functionName, ValueReference[] parameters, ValueReference returnValue, WorkspaceReference[] workspaceVars, Map<String, String> methods) {
     super();
     this.scope = scope;
     this.nullHandling = nullHandling;
+    this.workspaceVars = workspaceVars;
     this.isBinaryCommutative = isBinaryCommutative;
     this.functionName = functionName;
     this.setupBody = methods.get("setup");
@@ -48,9 +51,9 @@ public class FunctionHolder {
     this.returnValue = returnValue;
   }
 
-  public HoldingContainer generateEvalBody(CodeGenerator g, HoldingContainer[] inputVariables){
+  public HoldingContainer generateEvalBody(CodeGenerator<?> g, HoldingContainer[] inputVariables){
     
-    g.getBlock().directStatement(String.format("//---- start of eval portion of %s function. ----//", functionName));
+    //g.getBlock().directStatement(String.format("//---- start of eval portion of %s function. ----//", functionName));
     
     JBlock sub = new JBlock(true, true);
     
@@ -87,6 +90,16 @@ public class FunctionHolder {
     // add the subblock after the out declaration.
     g.getBlock().add(sub);
     
+    JVar[] workspaceJVars = new JVar[workspaceVars.length];
+    for(int i =0 ; i < workspaceVars.length; i++){
+      workspaceJVars[i] = g.declareClassField("work", g.getModel()._ref(workspaceVars[i].type)));
+    }
+    
+    for(WorkspaceReference r : workspaceVars){
+      g.declareClassField(, t)
+    }
+  
+    g.declareClassField(prefix, t)
     // locally name external blocks.
     
     // internal out value.
@@ -99,12 +112,13 @@ public class FunctionHolder {
       sub.decl(JMod.FINAL, inputVariable.getHolder().type(), parameter.name, inputVariable.getHolder());  
     }
     
+    
     // add function body.
     sub.directStatement(evalBody);
     
     sub.assign(out.getHolder(), internalOutput);
 
-    g.getBlock().directStatement(String.format("//---- end of eval portion of %s function. ----//\n", functionName));
+    //g.getBlock().directStatement(String.format("//---- end of eval portion of %s function. ----//\n", functionName));
     return out;
   }
   
@@ -144,6 +158,16 @@ public class FunctionHolder {
     
   }
 
+  public static class WorkspaceReference{
+    Class<?> type;
+    String name;
+    public WorkspaceReference(Class<?> type, String name) {
+      super();
+      this.type = type;
+      this.name = name;
+    }
+    
+  }
   @Override
   public String toString() {
     final int maxLen = 10;

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/80b1d24f/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/MathFunctions.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/MathFunctions.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/MathFunctions.java
index 702a0ad..939e997 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/MathFunctions.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/MathFunctions.java
@@ -6,8 +6,8 @@ import org.apache.drill.exec.expr.annotations.FunctionTemplate.FunctionScope;
 import org.apache.drill.exec.expr.annotations.FunctionTemplate.NullHandling;
 import org.apache.drill.exec.expr.annotations.Output;
 import org.apache.drill.exec.expr.annotations.Param;
-import org.apache.drill.exec.expr.holders.ValueHolderImplmenetations.IntHolder;
-import org.apache.drill.exec.expr.holders.ValueHolderImplmenetations.LongHolder;
+import org.apache.drill.exec.expr.holders.IntHolder;
+import org.apache.drill.exec.expr.holders.LongHolder;
 import org.apache.drill.exec.record.RecordBatch;
 
 public class MathFunctions{

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/80b1d24f/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/holders/BooleanHolder.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/holders/BooleanHolder.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/holders/BooleanHolder.java
new file mode 100644
index 0000000..70bcd10
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/holders/BooleanHolder.java
@@ -0,0 +1,10 @@
+package org.apache.drill.exec.expr.holders;
+
+import org.apache.drill.common.types.Types;
+import org.apache.drill.common.types.TypeProtos.MajorType;
+import org.apache.drill.common.types.TypeProtos.MinorType;
+
+public final class BooleanHolder implements ValueHolder{
+  public static final MajorType TYPE = Types.required(MinorType.BOOLEAN); 
+  public int value;
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/80b1d24f/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/holders/IntHolder.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/holders/IntHolder.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/holders/IntHolder.java
new file mode 100644
index 0000000..81c82b9
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/holders/IntHolder.java
@@ -0,0 +1,10 @@
+package org.apache.drill.exec.expr.holders;
+
+import org.apache.drill.common.types.Types;
+import org.apache.drill.common.types.TypeProtos.MajorType;
+import org.apache.drill.common.types.TypeProtos.MinorType;
+
+public final class IntHolder implements ValueHolder{
+  public static final MajorType TYPE = Types.required(MinorType.INT); 
+  public int value;
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/80b1d24f/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/holders/LongHolder.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/holders/LongHolder.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/holders/LongHolder.java
new file mode 100644
index 0000000..c73f9ad
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/holders/LongHolder.java
@@ -0,0 +1,10 @@
+package org.apache.drill.exec.expr.holders;
+
+import org.apache.drill.common.types.Types;
+import org.apache.drill.common.types.TypeProtos.MajorType;
+import org.apache.drill.common.types.TypeProtos.MinorType;
+
+public final class LongHolder implements ValueHolder {
+  public static final MajorType TYPE = Types.required(MinorType.BIGINT);
+  public long value;
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/80b1d24f/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/holders/NullableBooleanHolder.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/holders/NullableBooleanHolder.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/holders/NullableBooleanHolder.java
new file mode 100644
index 0000000..f6d2f2e
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/holders/NullableBooleanHolder.java
@@ -0,0 +1,11 @@
+package org.apache.drill.exec.expr.holders;
+
+import org.apache.drill.common.types.Types;
+import org.apache.drill.common.types.TypeProtos.MajorType;
+import org.apache.drill.common.types.TypeProtos.MinorType;
+
+public final class NullableBooleanHolder implements ValueHolder {
+  public static final MajorType TYPE = Types.optional(MinorType.BOOLEAN);
+  public int value;
+  public int isSet;
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/80b1d24f/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/holders/NullableIntHolder.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/holders/NullableIntHolder.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/holders/NullableIntHolder.java
new file mode 100644
index 0000000..7eca21e
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/holders/NullableIntHolder.java
@@ -0,0 +1,11 @@
+package org.apache.drill.exec.expr.holders;
+
+import org.apache.drill.common.types.Types;
+import org.apache.drill.common.types.TypeProtos.MajorType;
+import org.apache.drill.common.types.TypeProtos.MinorType;
+
+public final class NullableIntHolder implements ValueHolder {
+  public static final MajorType TYPE = Types.optional(MinorType.INT);
+  public int value;
+  public int isSet;
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/80b1d24f/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/holders/NullableLongHolder.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/holders/NullableLongHolder.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/holders/NullableLongHolder.java
new file mode 100644
index 0000000..d1be8f1
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/holders/NullableLongHolder.java
@@ -0,0 +1,11 @@
+package org.apache.drill.exec.expr.holders;
+
+import org.apache.drill.common.types.Types;
+import org.apache.drill.common.types.TypeProtos.MajorType;
+import org.apache.drill.common.types.TypeProtos.MinorType;
+
+public final class NullableLongHolder implements ValueHolder {
+  public static final MajorType TYPE = Types.optional(MinorType.BIGINT);
+  public long value;
+  public int isSet;
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/80b1d24f/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/holders/ValueHolderImplmenetations.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/holders/ValueHolderImplmenetations.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/holders/ValueHolderImplmenetations.java
deleted file mode 100644
index adf97fe..0000000
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/holders/ValueHolderImplmenetations.java
+++ /dev/null
@@ -1,44 +0,0 @@
-package org.apache.drill.exec.expr.holders;
-
-import org.apache.drill.common.types.TypeProtos.MajorType;
-import org.apache.drill.common.types.TypeProtos.MinorType;
-import org.apache.drill.common.types.Types;
-
-public class ValueHolderImplmenetations {
-  
-  private ValueHolderImplmenetations(){}
-  
-  public final static class BooleanHolder implements ValueHolder{
-    public static final MajorType TYPE = Types.required(MinorType.BOOLEAN); 
-    public int value;
-  }
-  
-  public final static class NullableBooleanHolder implements ValueHolder {
-    public static final MajorType TYPE = Types.optional(MinorType.BOOLEAN);
-    public int value;
-    public int isSet;
-  }
-  
-  public final static class IntHolder implements ValueHolder{
-    public static final MajorType TYPE = Types.required(MinorType.INT); 
-    public int value;
-  }
-  
-  public final static class NullableIntHolder implements ValueHolder {
-    public static final MajorType TYPE = Types.optional(MinorType.INT);
-    public int value;
-    public int isSet;
-  }
-
-  public final static class LongHolder implements ValueHolder {
-    public static final MajorType TYPE = Types.required(MinorType.BIGINT);
-    public long value;
-  }
-  
-  public final static class NullableLongHolder implements ValueHolder {
-    public static final MajorType TYPE = Types.optional(MinorType.BIGINT);
-    public long value;
-    public int isSet;
-  }
-  
-}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/80b1d24f/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 1dd7de8..876b873 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
@@ -103,14 +103,11 @@ public class FragmentContext {
     return context.getAllocator();
   }
 
-  public <T> T getImplementationClass(TemplateClassDefinition<T, Void> templateDefinition, CodeGenerator cg) throws ClassTransformationException, IOException{
-    return transformer.getImplementationClass(this.loader, templateDefinition, cg.generate(), null);
-  }
-  
-  public FilteringRecordBatchTransformer getFilteringExpression(LogicalExpression expr){
-    return null;
+  public <T> T getImplementationClass(TemplateClassDefinition<T> templateDefinition, CodeGenerator<T> cg) throws ClassTransformationException, IOException{
+    return transformer.getImplementationClass(this.loader, templateDefinition, cg.generate(), cg.getMaterializedClassName());
   }
   
+
   public void addMetricsToStatus(FragmentStatus.Builder stats){
     stats.setBatchesCompleted(batchesCompleted.get());
     stats.setDataProcessed(dataProcessed.get());

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/80b1d24f/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/FilterRecordBatch.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/FilterRecordBatch.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/FilterRecordBatch.java
index 80d48f4..1ddc3f9 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/FilterRecordBatch.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/FilterRecordBatch.java
@@ -78,7 +78,7 @@ public abstract class FilterRecordBatch implements RecordBatch {
   }
 
   @Override
-  public TypedFieldId getValueVector(SchemaPath path) {
+  public TypedFieldId getValueVectorId(SchemaPath path) {
     return null;
   }
 
@@ -111,7 +111,7 @@ public abstract class FilterRecordBatch implements RecordBatch {
       IterOutcome o = incoming.next();
       switch (o) {
       case OK_NEW_SCHEMA:
-        transformer = incoming.getContext().getFilteringExpression(null);
+        transformer = null;
         schema = transformer.getSchema();
         // fall through to ok.
       case OK:

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/80b1d24f/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 d98c107..739c0d4 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
@@ -28,9 +28,11 @@ import org.apache.drill.exec.physical.base.PhysicalOperator;
 import org.apache.drill.exec.physical.base.Scan;
 import org.apache.drill.exec.physical.config.MockScanBatchCreator;
 import org.apache.drill.exec.physical.config.MockScanPOP;
+import org.apache.drill.exec.physical.config.Project;
 import org.apache.drill.exec.physical.config.RandomReceiver;
 import org.apache.drill.exec.physical.config.Screen;
 import org.apache.drill.exec.physical.config.SingleSender;
+import org.apache.drill.exec.physical.impl.project.ProjectBatchCreator;
 import org.apache.drill.exec.record.RecordBatch;
 
 import com.google.common.base.Preconditions;
@@ -43,6 +45,7 @@ public class ImplCreator extends AbstractPhysicalVisitor<RecordBatch, FragmentCo
   private ScreenCreator sc = new ScreenCreator();
   private RandomReceiverCreator rrc = new RandomReceiverCreator();
   private SingleSenderCreator ssc = new SingleSenderCreator();
+  private ProjectBatchCreator pbc = new ProjectBatchCreator();
   private RootExec root = null;
   
   private ImplCreator(){}
@@ -51,7 +54,11 @@ public class ImplCreator extends AbstractPhysicalVisitor<RecordBatch, FragmentCo
     return root;
   }
   
-  
+  @Override
+  public RecordBatch visitProject(Project op, FragmentContext context) throws ExecutionSetupException {
+    return pbc.getBatch(context, op, getChildren(op, context));
+  }
+
   @Override
   public RecordBatch visitScan(Scan<?> scan, FragmentContext context) throws ExecutionSetupException {
     Preconditions.checkNotNull(scan);

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/80b1d24f/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScanBatch.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScanBatch.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScanBatch.java
index 1e0c000..be80349 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScanBatch.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScanBatch.java
@@ -173,7 +173,7 @@ public class ScanBatch implements RecordBatch {
   }
 
   @Override
-  public TypedFieldId getValueVector(SchemaPath path) {
+  public TypedFieldId getValueVectorId(SchemaPath path) {
     return holder.getValueVector(path);
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/80b1d24f/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 3819036..516b5af 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
@@ -50,7 +50,7 @@ public class ScreenCreator implements RootCreator<Screen>{
   }
   
   
-  private static class ScreenRoot implements RootExec{
+  static class ScreenRoot implements RootExec{
     static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ScreenRoot.class);
     volatile boolean ok = true;
     
@@ -138,6 +138,11 @@ public class ScreenCreator implements RootCreator<Screen>{
       }
       
     }
+
+    RecordBatch getIncoming() {
+      return incoming;
+    }
+    
     
   }
   

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/80b1d24f/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/WireRecordBatch.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/WireRecordBatch.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/WireRecordBatch.java
index f4921b5..7b2a533 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/WireRecordBatch.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/WireRecordBatch.java
@@ -85,7 +85,7 @@ public class WireRecordBatch implements RecordBatch{
   }
 
   @Override
-  public TypedFieldId getValueVector(SchemaPath path) {
+  public TypedFieldId getValueVectorId(SchemaPath path) {
     return batchLoader.getValueVector(path);
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/80b1d24f/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/filter/ExampleFilter.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/filter/ExampleFilter.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/filter/ExampleFilter.java
index 158350f..c4a4032 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/filter/ExampleFilter.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/filter/ExampleFilter.java
@@ -78,7 +78,7 @@ public class ExampleFilter implements RecordBatch {
   }
 
   @Override
-  public TypedFieldId getValueVector(SchemaPath path) {
+  public TypedFieldId getValueVectorId(SchemaPath path) {
     return null;
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/80b1d24f/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/filter/FilterTemplate.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/filter/FilterTemplate.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/filter/FilterTemplate.java
new file mode 100644
index 0000000..d2983ad
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/filter/FilterTemplate.java
@@ -0,0 +1,48 @@
+package org.apache.drill.exec.physical.impl.filter;
+
+import org.apache.drill.exec.record.RecordBatch;
+import org.apache.drill.exec.record.vector.SelectionVector2;
+
+public abstract class FilterTemplate {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(FilterTemplate.class);
+  
+  SelectionVector2 outgoingSelectionVector;
+  SelectionVector2 incomingSelectionVector;
+  
+  public void setup(RecordBatch incoming, RecordBatch outgoing){
+    outgoingSelectionVector = outgoing.getSelectionVector2();
+
+    switch(incoming.getSchema().getSelectionVector()){
+    case NONE:
+      break;
+    case TWO_BYTE:
+      this.incomingSelectionVector = incoming.getSelectionVector2();
+      break;
+    default:
+      throw new UnsupportedOperationException();
+    }
+  }
+  
+  public void filterBatchSV2(int recordCount){
+    int svIndex = 0;
+    for(char i =0; i < recordCount; i++){
+      if(include(i)){
+        outgoingSelectionVector.setIndex(svIndex, i);
+        svIndex+=2;
+      }
+    }
+  }
+  
+  public void filterBatchNoSV(int recordCount){
+    int svIndex = 0;
+    for(char i =0; i < recordCount; i++){
+      
+      if(include(i)){
+        outgoingSelectionVector.setIndex(svIndex, i);
+        svIndex+=2;
+      }
+    }
+  }
+  
+  protected abstract boolean include(int index);
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/80b1d24f/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/filter/SelectionVectorPopulationExpression.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/filter/SelectionVectorPopulationExpression.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/filter/SelectionVectorPopulationExpression.java
new file mode 100644
index 0000000..f253695
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/filter/SelectionVectorPopulationExpression.java
@@ -0,0 +1,39 @@
+package org.apache.drill.exec.physical.impl.filter;
+
+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;
+
+public class SelectionVectorPopulationExpression implements LogicalExpression{
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(SelectionVectorPopulationExpression.class);
+
+  private LogicalExpression child;
+  
+  public SelectionVectorPopulationExpression(LogicalExpression child) {
+    this.child = child;
+  }
+
+  public LogicalExpression getChild() {
+    return child;
+  }
+
+  @Override
+  public MajorType getMajorType() {
+    return Types.NULL;
+  }
+
+  @Override
+  public <T, V, E extends Exception> T accept(ExprVisitor<T, V, E> visitor, V value) throws E {
+    return visitor.visitUnknown(this, value);
+  }
+
+  @Override
+  public ExpressionPosition getPosition() {
+    return ExpressionPosition.UNKNOWN;
+  }
+  
+  
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/80b1d24f/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/ProjectBatchCreator.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/ProjectBatchCreator.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/ProjectBatchCreator.java
new file mode 100644
index 0000000..3538469
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/ProjectBatchCreator.java
@@ -0,0 +1,23 @@
+package org.apache.drill.exec.physical.impl.project;
+
+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.Project;
+import org.apache.drill.exec.physical.impl.BatchCreator;
+import org.apache.drill.exec.record.RecordBatch;
+
+import com.google.common.base.Preconditions;
+
+public class ProjectBatchCreator implements BatchCreator<Project>{
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ProjectBatchCreator.class);
+
+  @Override
+  public RecordBatch getBatch(FragmentContext context, Project config, List<RecordBatch> children) throws ExecutionSetupException {
+    Preconditions.checkArgument(children.size() == 1);
+    return new ProjectRecordBatch(config, children.iterator().next(), context);
+  }
+  
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/80b1d24f/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 69daae0..86caf28 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
@@ -7,6 +7,6 @@ import org.apache.drill.exec.record.RecordBatch;
 public interface ProjectEvaluator {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ProjectEvaluator.class);
   
-  public abstract void setupEvaluators(FragmentContext context, RecordBatch incoming) throws SchemaChangeException;
+  public abstract void setupEvaluators(FragmentContext context, RecordBatch incoming, RecordBatch outgoing) throws SchemaChangeException;
   public abstract void doPerRecordWork(int inIndex, int outIndex);
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/80b1d24f/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/ProjectRecordBatch.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/ProjectRecordBatch.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/ProjectRecordBatch.java
index cfdb7bc..7a3fec1 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/ProjectRecordBatch.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/ProjectRecordBatch.java
@@ -5,6 +5,7 @@ import java.util.List;
 
 import org.apache.drill.common.expression.ErrorCollector;
 import org.apache.drill.common.expression.ErrorCollectorImpl;
+import org.apache.drill.common.expression.FieldReference;
 import org.apache.drill.common.expression.LogicalExpression;
 import org.apache.drill.common.expression.PathSegment;
 import org.apache.drill.common.expression.SchemaPath;
@@ -18,6 +19,7 @@ import org.apache.drill.exec.expr.ValueVectorReadExpression;
 import org.apache.drill.exec.expr.ValueVectorWriteExpression;
 import org.apache.drill.exec.ops.FragmentContext;
 import org.apache.drill.exec.physical.config.Project;
+import org.apache.drill.exec.physical.impl.VectorHolder;
 import org.apache.drill.exec.proto.SchemaDefProtos.FieldDef;
 import org.apache.drill.exec.proto.SchemaDefProtos.NamePart;
 import org.apache.drill.exec.proto.SchemaDefProtos.NamePart.Type;
@@ -32,6 +34,7 @@ import org.apache.drill.exec.record.vector.SelectionVector4;
 import org.apache.drill.exec.record.vector.TypeHelper;
 import org.apache.drill.exec.record.vector.ValueVector;
 
+
 import com.google.common.base.Preconditions;
 import com.google.common.collect.Lists;
 
@@ -45,6 +48,7 @@ public class ProjectRecordBatch implements RecordBatch{
   private Projector projector;
   private List<ValueVector<?>> allocationVectors;
   private List<ValueVector<?>> outputVectors;
+  private VectorHolder vh;
   
   
   public ProjectRecordBatch(Project pop, RecordBatch incoming, FragmentContext context){
@@ -86,19 +90,20 @@ public class ProjectRecordBatch implements RecordBatch{
   }
 
   @Override
-  public TypedFieldId getValueVector(SchemaPath path) {
-    return null;
+  public TypedFieldId getValueVectorId(SchemaPath path) {
+    return vh.getValueVector(path);
   }
 
   @Override
-  public <T extends ValueVector<T>> T getValueVectorById(int fieldId, Class<?> vvClass) {
-    return null;
+  public <T extends ValueVector<T>> T getValueVectorById(int fieldId, Class<?> clazz) {
+    return vh.getValueVector(fieldId, clazz);
   }
 
   @Override
   public IterOutcome next() {
     
     IterOutcome upstream = incoming.next();
+    logger.debug("Upstream... {}", upstream);
     switch(upstream){
     case NONE:
     case NOT_YET:
@@ -109,6 +114,7 @@ public class ProjectRecordBatch implements RecordBatch{
         projector = createNewProjector();
       }catch(SchemaChangeException ex){
         incoming.kill();
+        logger.error("Failure during query", ex);
         context.fail(ex);
         return IterOutcome.STOP;
       }
@@ -119,6 +125,9 @@ public class ProjectRecordBatch implements RecordBatch{
         v.allocateNew(recordCount);
       }
       projector.projectRecords(recordCount, 0);
+      for(ValueVector<?> v : this.outputVectors){
+        v.setRecordCount(recordCount);
+      }
       return upstream; // change if upstream changed, otherwise normal.
     default:
       throw new UnsupportedOperationException();
@@ -134,24 +143,21 @@ public class ProjectRecordBatch implements RecordBatch{
       }
     }
     this.outputVectors = Lists.newArrayList();
-    
+    this.vh = new VectorHolder(outputVectors);
     final List<NamedExpression> exprs = pop.getExprs();
     final ErrorCollector collector = new ErrorCollectorImpl();
     final List<TransferPairing<?>> transfers = Lists.newArrayList();
     
-    final CodeGenerator cg = new CodeGenerator("setupEvaluators", "doPerRecordWork", context.getFunctionRegistry());
+    final CodeGenerator<Projector> cg = new CodeGenerator<Projector>(Projector.TEMPLATE_DEFINITION, context.getFunctionRegistry());
     
     for(int i =0; i < exprs.size(); i++){
       final NamedExpression namedExpression = exprs.get(i);
-      final MaterializedField outputField = getMaterializedField(namedExpression);
       final LogicalExpression expr = ExpressionTreeMaterializer.materialize(namedExpression.getExpr(), incoming, collector);
+      final MaterializedField outputField = getMaterializedField(namedExpression.getRef(), expr);
       if(collector.hasErrors()){
         throw new SchemaChangeException(String.format("Failure while trying to materialize incoming schema.  Errors:\n %s.", collector.toErrorString()));
       }
       
-      
-      
-      
       // add value vector to transfer if direct reference and this is allowed, otherwise, add to evaluation stack.
       if(expr instanceof ValueVectorReadExpression && incoming.getSchema().getSelectionVector() == SelectionVectorMode.NONE){
         ValueVectorReadExpression vectorRead = (ValueVectorReadExpression) expr;
@@ -179,21 +185,21 @@ public class ProjectRecordBatch implements RecordBatch{
     this.outSchema = bldr.build();
     
     try {
-      return context.getImplementationClass(Projector.TEMPLATE_DEFINITION, cg);
+      Projector projector = context.getImplementationClass(Projector.TEMPLATE_DEFINITION, cg);
+      projector.setup(context, incoming, this, transfers);
+      return projector;
     } catch (ClassTransformationException | IOException e) {
       throw new SchemaChangeException("Failure while attempting to load generated class", e);
     }
   }
   
-  
   @Override
   public WritableBatch getWritableBatch() {
-    return null;
+    return WritableBatch.get(incoming.getRecordCount(), outputVectors);
   }
   
-  
-  private MaterializedField getMaterializedField(NamedExpression ex){
-    return new MaterializedField(getFieldDef(ex.getRef(), ex.getExpr().getMajorType()));
+  private MaterializedField getMaterializedField(FieldReference reference, LogicalExpression expr){
+    return new MaterializedField(getFieldDef(reference, expr.getMajorType()));
   }
 
   private FieldDef getFieldDef(SchemaPath path, MajorType type){
@@ -212,7 +218,9 @@ public class ProjectRecordBatch implements RecordBatch{
       }else{
         parts.add(NamePart.newBuilder().setType(Type.NAME).setName(seg.getNameSegment().getPath().toString()).build());
       }
+      seg = seg.getChild();
     }
     return parts;
   }
+  
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/80b1d24f/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 31c418c..8f22b40 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
@@ -9,12 +9,12 @@ import org.apache.drill.exec.record.RecordBatch;
 
 public interface Projector {
 
-  public abstract void setup(FragmentContext context, RecordBatch incoming, List<TransferPairing<?>> transfers)  throws SchemaChangeException;
+  public abstract void setup(FragmentContext context, RecordBatch incoming,  RecordBatch outgoing, List<TransferPairing<?>> transfers)  throws SchemaChangeException;
 
   
-  public abstract void projectRecords(int recordCount, int firstOutputIndex);
+  public abstract int projectRecords(int recordCount, int firstOutputIndex);
 
-  public static TemplateClassDefinition<Projector, Void> TEMPLATE_DEFINITION = new TemplateClassDefinition<Projector, Void>( //
-      Projector.class, "org.apache.drill.exec.physical.impl.project.ProjectTemplate", ProjectEvaluator.class, Void.class);
+  public static TemplateClassDefinition<Projector> TEMPLATE_DEFINITION = new TemplateClassDefinition<Projector>( //
+      Projector.class, "org.apache.drill.exec.physical.impl.project.ProjectorTemplate", ProjectEvaluator.class, "setupEvaluators", "doPerRecordWork");
 
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/80b1d24f/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/ProjectorTemplate.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/ProjectorTemplate.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/ProjectorTemplate.java
index 60af7d2..1163fef 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/ProjectorTemplate.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/ProjectorTemplate.java
@@ -2,50 +2,28 @@ package org.apache.drill.exec.physical.impl.project;
 
 import java.util.List;
 
-import org.apache.drill.common.expression.ErrorCollector;
-import org.apache.drill.common.expression.ErrorCollectorImpl;
-import org.apache.drill.common.expression.LogicalExpression;
-import org.apache.drill.common.expression.PathSegment;
-import org.apache.drill.common.expression.SchemaPath;
-import org.apache.drill.common.logical.data.NamedExpression;
-import org.apache.drill.common.types.TypeProtos.MajorType;
 import org.apache.drill.exec.exception.SchemaChangeException;
-import org.apache.drill.exec.expr.CodeGenerator;
-import org.apache.drill.exec.expr.ExpressionTreeMaterializer;
-import org.apache.drill.exec.expr.ValueVectorReadExpression;
-import org.apache.drill.exec.expr.ValueVectorWriteExpression;
-import org.apache.drill.exec.expr.fn.FunctionImplementationRegistry;
 import org.apache.drill.exec.ops.FragmentContext;
-import org.apache.drill.exec.physical.config.Project;
-import org.apache.drill.exec.proto.SchemaDefProtos.FieldDef;
-import org.apache.drill.exec.proto.SchemaDefProtos.NamePart;
-import org.apache.drill.exec.proto.SchemaDefProtos.NamePart.Type;
 import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
-import org.apache.drill.exec.record.MaterializedField;
 import org.apache.drill.exec.record.RecordBatch;
 import org.apache.drill.exec.record.vector.SelectionVector2;
 import org.apache.drill.exec.record.vector.SelectionVector4;
-import org.apache.drill.exec.record.vector.TypeHelper;
-import org.apache.drill.exec.record.vector.ValueVector;
 
-import com.google.common.base.Preconditions;
 import com.google.common.collect.ImmutableList;
-import com.google.common.collect.Lists;
 
 public abstract class ProjectorTemplate implements Projector {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ProjectorTemplate.class);
-
+  
   private ImmutableList<TransferPairing<?>> transfers;
   private SelectionVector2 vector2;
   private SelectionVector4 vector4;
   private SelectionVectorMode svMode;
   
-  public ProjectorTemplate(final FragmentContext context, final RecordBatch incomingBatch, final Project pop, FunctionImplementationRegistry funcRegistry) throws SchemaChangeException{
-    super();
+  public ProjectorTemplate() throws SchemaChangeException{
   }
 
   @Override
-  public final void projectRecords(final int recordCount, int firstOutputIndex) {
+  public final int projectRecords(final int recordCount, int firstOutputIndex) {
     switch(svMode){
     case FOUR_BYTE:
       throw new UnsupportedOperationException();
@@ -56,7 +34,7 @@ public abstract class ProjectorTemplate implements Projector {
       for(int i = 0; i < count; i+=2, firstOutputIndex++){
         doPerRecordWork(vector2.getIndex(i), firstOutputIndex);
       }
-      return;
+      return recordCount;
       
       
     case NONE:
@@ -68,7 +46,7 @@ public abstract class ProjectorTemplate implements Projector {
       for (int i = 0; i < countN; i++, firstOutputIndex++) {
         doPerRecordWork(i, firstOutputIndex);
       }
-      return;
+      return recordCount;
       
       
     default:
@@ -77,7 +55,7 @@ public abstract class ProjectorTemplate implements Projector {
   }
 
   @Override
-  public final void setup(FragmentContext context, RecordBatch incoming, List<TransferPairing<?>> transfers)  throws SchemaChangeException{
+  public final void setup(FragmentContext context, RecordBatch incoming, RecordBatch outgoing, List<TransferPairing<?>> transfers)  throws SchemaChangeException{
 
     this.svMode = incoming.getSchema().getSelectionVector(); 
     switch(svMode){
@@ -89,10 +67,10 @@ public abstract class ProjectorTemplate implements Projector {
       break;
     }
     this.transfers = ImmutableList.copyOf(transfers);
-    setupEvaluators(context, incoming);
+    setupEvaluators(context, incoming, outgoing);
   }
 
-  protected abstract void setupEvaluators(FragmentContext context, RecordBatch incoming) throws SchemaChangeException;
+  protected abstract void setupEvaluators(FragmentContext context, RecordBatch incoming, RecordBatch outgoing) throws SchemaChangeException;
   protected abstract void doPerRecordWork(int inIndex, int outIndex);
 
   

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/80b1d24f/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatch.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatch.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatch.java
index 40447ec..0edb93e 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatch.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatch.java
@@ -85,10 +85,10 @@ public interface RecordBatch {
    * @param path The path where the vector should be located.
    * @return The local field id associated with this vector.
    */
-  public abstract TypedFieldId getValueVector(SchemaPath path);
+  public abstract TypedFieldId getValueVectorId(SchemaPath path);
   
   
-  public abstract <T extends ValueVector<T>> T getValueVectorById(int fieldId, Class<?> vvClass);
+  public abstract <T extends ValueVector<T>> T getValueVectorById(int fieldId, Class<?> clazz);
 
   /**
    * Update the data in each Field reading interface for the next range of records. Once a RecordBatch returns an

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/80b1d24f/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/SchemaBuilder.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/SchemaBuilder.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/SchemaBuilder.java
index 20cc82c..0989c1d 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/SchemaBuilder.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/SchemaBuilder.java
@@ -21,6 +21,7 @@ import java.util.List;
 import java.util.Set;
 
 import org.apache.drill.exec.exception.SchemaChangeException;
+import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
 
 import com.beust.jcommander.internal.Sets;
 import com.google.common.collect.Lists;
@@ -32,7 +33,7 @@ import com.google.common.collect.Lists;
 public class SchemaBuilder {
   private Set<MaterializedField> fields = Sets.newHashSet();
 
-  private BatchSchema.SelectionVectorMode selectionVectorMode;
+  private BatchSchema.SelectionVectorMode selectionVectorMode = SelectionVectorMode.NONE;
 
   SchemaBuilder() {
   }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/80b1d24f/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/NullableFixed8.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/NullableFixed8.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/NullableFixed8.java
index 3fc39eb..48aa0c2 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/NullableFixed8.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/NullableFixed8.java
@@ -32,5 +32,12 @@ public final class NullableFixed8 extends NullableValueVector<NullableFixed8, Fi
     return new Fixed8(this.field, allocator);
   }
 
+  public long get(int index){
+    return 1l;
+  }
+  
+  public void set(int index, long value){
+    
+  }
 
 }


[22/53] [abbrv] git commit: Clean up threading of client/server. Utilize command pattern for BitCom stuff to abstract away connection failures. Works on one bit single exchange remote query now. Next up, two bit single exchange query.

Posted by ja...@apache.org.
Clean up threading of client/server.  Utilize command pattern for BitCom stuff to abstract away connection failures.  Works on one bit single exchange remote query now.  Next up, two bit single exchange query.


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

Branch: refs/heads/master
Commit: b8db98ad7c159db3cf41a3866ff53013f87964b4
Parents: e57a8d6
Author: Jacques Nadeau <ja...@apache.org>
Authored: Tue May 21 18:38:56 2013 -0700
Committer: Jacques Nadeau <ja...@apache.org>
Committed: Tue May 21 18:38:56 2013 -0700

----------------------------------------------------------------------
 .../common/graph/AdjacencyListBuilder.java      |   2 +-
 .../org/apache/drill/exec/cache/LocalCache.java |   2 +
 .../apache/drill/exec/client/DrillClient.java   |  77 +++++---
 .../exec/coord/LocalClusterCoordinator.java     |   7 +-
 .../apache/drill/exec/ops/FragmentContext.java  |  19 +-
 .../org/apache/drill/exec/ops/QueryContext.java |   4 +
 .../exec/physical/config/MockRecordReader.java  |   1 -
 .../exec/physical/config/RandomReceiver.java    |   5 -
 .../drill/exec/physical/config/Screen.java      |   2 +-
 .../drill/exec/physical/impl/ScreenCreator.java |  79 ++++++--
 .../exec/physical/impl/SingleSenderCreator.java |  41 +++-
 .../exec/physical/impl/WireRecordBatch.java     |   8 +-
 .../impl/materialize/QueryWritableBatch.java    |   8 +
 .../materialize/VectorRecordMaterializer.java   |  11 +-
 .../exec/planner/fragment/Materializer.java     |   8 +-
 .../planner/fragment/SimpleParallelizer.java    |   4 +-
 .../exec/planner/fragment/StatsCollector.java   |   2 +-
 .../drill/exec/record/RawFragmentBatch.java     |   5 +
 .../exec/rpc/AbstractHandshakeHandler.java      |   5 +-
 .../drill/exec/rpc/BaseRpcOutcomeListener.java  |  32 ++++
 .../org/apache/drill/exec/rpc/BasicClient.java  | 176 +++++++++--------
 .../exec/rpc/BasicClientWithConnection.java     |   9 +-
 .../org/apache/drill/exec/rpc/BasicServer.java  |   7 +-
 .../rpc/ChannelListenerWithCoordinationId.java  |  25 +++
 .../drill/exec/rpc/CoordinationQueue.java       |  96 ++++++++--
 .../apache/drill/exec/rpc/DrillRpcFuture.java   |   2 -
 .../drill/exec/rpc/DrillRpcFutureImpl.java      |  70 ++-----
 .../java/org/apache/drill/exec/rpc/RpcBus.java  |  88 +++++----
 .../apache/drill/exec/rpc/RpcCheckedFuture.java |  33 ++++
 .../drill/exec/rpc/RpcConnectionHandler.java    |  28 +++
 .../org/apache/drill/exec/rpc/RpcException.java |  13 ++
 .../org/apache/drill/exec/rpc/RpcOutcome.java   |  26 +++
 .../drill/exec/rpc/RpcOutcomeListener.java      |   7 +-
 .../exec/rpc/ZeroCopyProtobufLengthDecoder.java |   2 +-
 .../apache/drill/exec/rpc/bit/BitClient.java    |  52 +++---
 .../org/apache/drill/exec/rpc/bit/BitCom.java   |   8 +-
 .../apache/drill/exec/rpc/bit/BitComImpl.java   | 129 ++-----------
 .../apache/drill/exec/rpc/bit/BitCommand.java   |  28 +++
 .../drill/exec/rpc/bit/BitConnection.java       |  79 ++------
 .../exec/rpc/bit/BitConnectionManager.java      | 175 +++++++++++++----
 .../apache/drill/exec/rpc/bit/BitServer.java    |  60 ++++--
 .../apache/drill/exec/rpc/bit/BitTunnel.java    | 187 +++++--------------
 .../exec/rpc/bit/ConnectionManagerRegistry.java |  73 ++++++++
 .../drill/exec/rpc/bit/FutureBitCommand.java    |  78 ++++++++
 .../apache/drill/exec/rpc/bit/ListenerPool.java |  15 +-
 .../drill/exec/rpc/bit/ListeningBitCommand.java |  73 ++++++++
 .../drill/exec/rpc/user/QueryResultBatch.java   |   7 +
 .../drill/exec/rpc/user/QueryResultHandler.java | 153 +++++++++++++++
 .../apache/drill/exec/rpc/user/UserClient.java  | 153 ++-------------
 .../exec/rpc/user/UserResultsListener.java      |  11 +-
 .../apache/drill/exec/rpc/user/UserServer.java  |  10 +-
 .../drill/exec/server/BootStrapContext.java     |   2 +-
 .../drill/exec/server/RemoteServiceSet.java     |   2 +
 .../drill/exec/service/ServiceEngine.java       |   6 +-
 .../drill/exec/work/EndpointListener.java       |   5 +-
 .../apache/drill/exec/work/FragmentRunner.java  |  10 +-
 .../org/apache/drill/exec/work/WorkManager.java |   7 +-
 .../work/batch/AbstractFragmentCollector.java   |   7 +-
 .../drill/exec/work/batch/BatchCollector.java   |   3 +-
 .../exec/work/batch/BitComHandlerImpl.java      |   9 +-
 .../drill/exec/work/batch/IncomingBuffers.java  |  21 ++-
 .../drill/exec/work/batch/MergingCollector.java |   5 +-
 .../exec/work/batch/PartitionedCollector.java   |   1 +
 .../exec/work/batch/UnlmitedRawBatchBuffer.java |   2 +-
 .../apache/drill/exec/work/foreman/Foreman.java |  22 +--
 .../work/foreman/RunningFragmentManager.java    |   8 +-
 .../work/fragment/IncomingFragmentHandler.java  |   2 +-
 .../work/fragment/LocalFragmentHandler.java     |   4 +-
 .../work/fragment/RemoteFragmentHandler.java    |   4 +-
 .../physical/impl/DistributedFragmentRun.java   |  17 +-
 .../apache/drill/exec/pop/CheckFragmenter.java  |  21 +--
 .../apache/drill/exec/pop/FragmentChecker.java  |  41 ++--
 .../apache/drill/exec/server/TestBitRpc.java    |  26 +--
 .../resources/physical_single_exchange.json     |   1 -
 74 files changed, 1498 insertions(+), 923 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/b8db98ad/sandbox/prototype/common/src/main/java/org/apache/drill/common/graph/AdjacencyListBuilder.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/graph/AdjacencyListBuilder.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/graph/AdjacencyListBuilder.java
index 1668477..4a385ce 100644
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/graph/AdjacencyListBuilder.java
+++ b/sandbox/prototype/common/src/main/java/org/apache/drill/common/graph/AdjacencyListBuilder.java
@@ -57,7 +57,7 @@ import java.util.Map;
   }
 
   public AdjacencyList<V> getAdjacencyList() {
-    logger.debug("Values; {}", ops.values().toArray());
+//    logger.debug("Values; {}", ops.values().toArray());
     AdjacencyList<V> a = new AdjacencyList<V>();
 
     for (AdjacencyList<V>.Node from : ops.values()) {

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/b8db98ad/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/cache/LocalCache.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/cache/LocalCache.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/cache/LocalCache.java
index ddb2a02..b656f2d 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/cache/LocalCache.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/cache/LocalCache.java
@@ -43,11 +43,13 @@ public class LocalCache implements DistributedCache {
 
   @Override
   public PlanFragment getFragment(FragmentHandle handle) {
+    logger.debug("looking for fragment with handle: {}", handle);
     return handles.get(handle);
   }
 
   @Override
   public void storeFragment(PlanFragment fragment) {
+    logger.debug("Storing fragment: {}", fragment);
     handles.put(fragment.getHandle(), fragment);
   }
   

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/b8db98ad/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/client/DrillClient.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/client/DrillClient.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/client/DrillClient.java
index bb7f77e..c35e834 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/client/DrillClient.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/client/DrillClient.java
@@ -30,22 +30,23 @@ import java.io.IOException;
 import java.util.Collection;
 import java.util.List;
 import java.util.Vector;
-import java.util.concurrent.ExecutionException;
-import java.util.concurrent.Future;
 
 import org.apache.drill.common.config.DrillConfig;
 import org.apache.drill.exec.coord.ClusterCoordinator;
 import org.apache.drill.exec.coord.ZKClusterCoordinator;
 import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
 import org.apache.drill.exec.proto.UserProtos.QueryType;
-import org.apache.drill.exec.proto.UserProtos.RpcType;
-import org.apache.drill.exec.proto.UserProtos.UserToBitHandshake;
+import org.apache.drill.exec.rpc.BasicClientWithConnection.ServerConnection;
+import org.apache.drill.exec.rpc.DrillRpcFuture;
 import org.apache.drill.exec.rpc.NamedThreadFactory;
+import org.apache.drill.exec.rpc.RpcConnectionHandler;
 import org.apache.drill.exec.rpc.RpcException;
 import org.apache.drill.exec.rpc.user.QueryResultBatch;
 import org.apache.drill.exec.rpc.user.UserClient;
 import org.apache.drill.exec.rpc.user.UserResultsListener;
-import org.apache.drill.exec.rpc.user.UserRpcConfig;
+
+import com.google.common.util.concurrent.AbstractCheckedFuture;
+import com.google.common.util.concurrent.SettableFuture;
 
 /**
  * Thin wrapper around a UserClient that handles connect/close and transforms String into ByteBuf
@@ -75,9 +76,6 @@ public class DrillClient implements Closeable{
   }
   
   
-  
-
-
   /**
    * Connects the client to a Drillbit server
    *
@@ -97,7 +95,9 @@ public class DrillClient implements Closeable{
     this.client = new UserClient(bb, new NioEventLoopGroup(1, new NamedThreadFactory("Client-")));
     try {
       logger.debug("Connecting to server {}:{}", endpoint.getAddress(), endpoint.getUserPort());
-      this.client.connect(endpoint);
+      FutureHandler f = new FutureHandler();
+      this.client.connect(f, endpoint);
+      f.checkedGet();
     } catch (InterruptedException e) {
       throw new IOException(e);
     }
@@ -120,34 +120,63 @@ public class DrillClient implements Closeable{
    * @throws RpcException
    */
   public List<QueryResultBatch> runQuery(QueryType type, String plan) throws RpcException {
-    try {
-      ListHoldingResultsListener listener = new ListHoldingResultsListener();
-      Future<Void> f = client.submitQuery(newBuilder().setResultsMode(STREAM_FULL).setType(type).setPlan(plan).build(), listener);
-      f.get();
-      if(listener.ex != null){
-        throw listener.ex;
-      }else{
-        return listener.results;
-      }
-    } catch (InterruptedException | ExecutionException e) {
-      throw new RpcException(e);
-    }
+    ListHoldingResultsListener listener = new ListHoldingResultsListener();
+    client.submitQuery(listener, newBuilder().setResultsMode(STREAM_FULL).setType(type).setPlan(plan).build());
+    return listener.getResults();
+
   }
   
-  private class ListHoldingResultsListener extends UserResultsListener{
-    private RpcException ex;
+  private class ListHoldingResultsListener implements UserResultsListener {
     private Vector<QueryResultBatch> results = new Vector<QueryResultBatch>();
+    private SettableFuture<List<QueryResultBatch>> future = SettableFuture.create();
     
     @Override
     public void submissionFailed(RpcException ex) {
       logger.debug("Submission failed.", ex);
-      this.ex = ex;
+      future.setException(ex);
     }
 
     @Override
     public void resultArrived(QueryResultBatch result) {
       logger.debug("Result arrived.  Is Last Chunk: {}.  Full Result: {}", result.getHeader().getIsLastChunk(), result);
       results.add(result);
+      if(result.getHeader().getIsLastChunk()){
+        future.set(results);
+      }
+    }
+  
+    public List<QueryResultBatch> getResults() throws RpcException{
+      try{
+        return future.get();
+      }catch(Throwable t){
+        throw RpcException.mapException(t);
+      }
+    }
+  }
+  
+  private class FutureHandler extends AbstractCheckedFuture<Void, RpcException> implements RpcConnectionHandler<ServerConnection>, DrillRpcFuture<Void>{
+
+    protected FutureHandler() {
+      super( SettableFuture.<Void>create());
+    }
+
+    @Override
+    public void connectionSucceeded(ServerConnection connection) {
+      getInner().set(null);
+    }
+
+    @Override
+    public void connectionFailed(FailureType type, Throwable t) {
+      getInner().setException(new RpcException(String.format("Failure connecting to server. Failure of type %s.", type.name()), t));
+    }
+
+    private SettableFuture<Void> getInner(){
+      return (SettableFuture<Void>) delegate();
+    }
+    
+    @Override
+    protected RpcException mapException(Exception e) {
+      return RpcException.mapException(e);
     }
     
   }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/b8db98ad/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/coord/LocalClusterCoordinator.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/coord/LocalClusterCoordinator.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/coord/LocalClusterCoordinator.java
index 43a5430..f7b3549 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/coord/LocalClusterCoordinator.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/coord/LocalClusterCoordinator.java
@@ -29,17 +29,16 @@ import com.google.common.collect.Maps;
 public class LocalClusterCoordinator extends ClusterCoordinator{
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(LocalClusterCoordinator.class);
 
-  private volatile Map<RegistrationHandle, DrillbitEndpoint> endpoints;
+  private volatile Map<RegistrationHandle, DrillbitEndpoint> endpoints = Maps.newConcurrentMap();
   
   @Override
   public void close() throws IOException {
-    endpoints = null;
+    endpoints.clear();
   }
 
   @Override
   public void start(long millis) throws Exception {
     logger.debug("Local Cluster Coordinator started.");
-    endpoints = Maps.newConcurrentMap();
   }
 
   @Override
@@ -52,6 +51,8 @@ public class LocalClusterCoordinator extends ClusterCoordinator{
 
   @Override
   public void unregister(RegistrationHandle handle) {
+    if(handle == null) return;
+    
     endpoints.remove(handle);
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/b8db98ad/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 e64453c..33707a0 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
@@ -52,7 +52,9 @@ public class FragmentContext {
   private final FragmentHandle handle;
   private final UserClientConnection connection;
   private final IncomingBuffers buffers;
-
+  private volatile Throwable failureCause;
+  private volatile boolean failed = false;
+  
   public FragmentContext(DrillbitContext dbContext, FragmentHandle handle, UserClientConnection connection, IncomingBuffers buffers) {
     this.fragmentTime = dbContext.getMetrics().timer(METRIC_TIMER_FRAGMENT_TIME);
     this.batchesCompleted = new SingleThreadNestedCounter(dbContext, METRIC_BATCHES_COMPLETED);
@@ -65,9 +67,10 @@ public class FragmentContext {
   }
 
   public void fail(Throwable cause) {
-
+    logger.debug("Fragment Context received failure. {}", cause);
+    failed = true;
+    failureCause = cause;
   }
-
   
   public DrillbitContext getDrillbitContext(){
     return context;
@@ -107,4 +110,14 @@ public class FragmentContext {
   public IncomingBuffers getBuffers(){
     return buffers;
   }
+
+  public Throwable getFailureCause() {
+    return failureCause;
+  }
+  
+  public boolean isFailed(){
+    return failed;
+  }
+  
+  
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/b8db98ad/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/QueryContext.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/QueryContext.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/QueryContext.java
index fd24deb..1c251b8 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/QueryContext.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/QueryContext.java
@@ -23,6 +23,7 @@ import org.apache.drill.exec.cache.DistributedCache;
 import org.apache.drill.exec.planner.PhysicalPlanReader;
 import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
 import org.apache.drill.exec.proto.UserBitShared.QueryId;
+import org.apache.drill.exec.rpc.bit.BitCom;
 import org.apache.drill.exec.server.DrillbitContext;
 
 public class QueryContext {
@@ -57,4 +58,7 @@ public class QueryContext {
     return drillbitContext.getPlanReader();
   }
   
+  public BitCom getBitCom(){
+    return drillbitContext.getBitCom();
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/b8db98ad/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockRecordReader.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockRecordReader.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockRecordReader.java
index eaaeaa3..6a1eba4 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockRecordReader.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockRecordReader.java
@@ -76,7 +76,6 @@ public class MockRecordReader implements RecordReader {
       int batchRecordCount = 250000 / estimateRowSize;
 
       for (int i = 0; i < config.getTypes().length; i++) {
-        logger.debug("Adding field {} of type {}", i, config.getTypes()[i]);
         valueVectors[i] = getVector(i, config.getTypes()[i].getName(), config.getTypes()[i].getMajorType(), batchRecordCount);
         output.addField(i, valueVectors[i]);
       }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/b8db98ad/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/RandomReceiver.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/RandomReceiver.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/RandomReceiver.java
index ed41586..6772fb0 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/RandomReceiver.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/RandomReceiver.java
@@ -72,11 +72,6 @@ public class RandomReceiver extends AbstractReceiver{
     return new Size(1,1);
   }
 
-  @Override
-  public int getOppositeMajorFragmentId() {
-    return 0;
-  }
-
   
 
   

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/b8db98ad/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/Screen.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/Screen.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/Screen.java
index 86a201d..688c6b5 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/Screen.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/Screen.java
@@ -64,7 +64,7 @@ public class Screen extends AbstractStore implements Root{
     // didn't get screwed up.
     if (endpoints.size() != 1) throw new PhysicalOperatorSetupException("A Screen operator can only be assigned to a single node.");
     DrillbitEndpoint endpoint = endpoints.iterator().next();
-    logger.debug("Endpoint this: {}, assignment: {}", this.endpoint, endpoint);
+//    logger.debug("Endpoint this: {}, assignment: {}", this.endpoint, endpoint);
     if (!endpoint.equals(this.endpoint)) {
       throw new PhysicalOperatorSetupException(String.format(
           "A Screen operator can only be assigned to its home node.  Expected endpoint %s, Actual endpoint: %s",

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/b8db98ad/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 c0711db..c20538d 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
@@ -17,21 +17,32 @@
  ******************************************************************************/
 package org.apache.drill.exec.physical.impl;
 
+import io.netty.buffer.ByteBuf;
+
 import java.util.List;
 
 import org.apache.drill.exec.ops.FragmentContext;
 import org.apache.drill.exec.physical.config.Screen;
+import org.apache.drill.exec.physical.impl.materialize.QueryWritableBatch;
 import org.apache.drill.exec.physical.impl.materialize.RecordMaterializer;
 import org.apache.drill.exec.physical.impl.materialize.VectorRecordMaterializer;
+import org.apache.drill.exec.proto.GeneralRPCProtos.Ack;
+import org.apache.drill.exec.proto.UserBitShared.DrillPBError;
+import org.apache.drill.exec.proto.UserBitShared.RecordBatchDef;
+import org.apache.drill.exec.proto.UserProtos.QueryResult;
 import org.apache.drill.exec.record.RecordBatch;
 import org.apache.drill.exec.record.RecordBatch.IterOutcome;
+import org.apache.drill.exec.rpc.BaseRpcOutcomeListener;
+import org.apache.drill.exec.rpc.RpcException;
 import org.apache.drill.exec.rpc.user.UserServer.UserClientConnection;
+import org.apache.drill.exec.work.foreman.ErrorHelper;
 
 import com.google.common.base.Preconditions;
 
 public class ScreenCreator implements RootCreator<Screen>{
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ScreenCreator.class);
-
+  
+  
   @Override
   public RootExec getRoot(FragmentContext context, Screen config, List<RecordBatch> children) {
     Preconditions.checkArgument(children.size() == 1);
@@ -40,7 +51,9 @@ public class ScreenCreator implements RootCreator<Screen>{
   
   
   private static class ScreenRoot implements RootExec{
-
+    static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ScreenRoot.class);
+    volatile boolean ok = true;
+    
     final RecordBatch incoming;
     final FragmentContext context;
     final UserClientConnection connection;
@@ -56,25 +69,53 @@ public class ScreenCreator implements RootCreator<Screen>{
     
     @Override
     public boolean next() {
+      if(!ok){
+        stop();
+        return false;
+      }
+      
       IterOutcome outcome = incoming.next();
-      boolean isLast = false;
+      logger.debug("Screen Outcome {}", outcome);
       switch(outcome){
-      case NONE:
-      case STOP:
-        connection.sendResult(materializer.convertNext(true));
-        context.batchesCompleted.inc(1);
-        context.recordsCompleted.inc(incoming.getRecordCount());
+      case STOP: {
+          QueryResult header1 = QueryResult.newBuilder() //
+              .setQueryId(context.getHandle().getQueryId()) //
+              .setRowCount(0) //
+              .addError(ErrorHelper.logAndConvertError(context.getIdentity(), "Screen received stop request sent.", context.getFailureCause(), logger))
+              .setDef(RecordBatchDef.getDefaultInstance()) //
+              .setIsLastChunk(true) //
+              .build();
+          QueryWritableBatch batch1 = new QueryWritableBatch(header1);
+
+          connection.sendResult(listener, batch1);
+          return false;
+      }
+      case NONE: {
+        if(materializer == null){
+          // receive no results.
+          context.batchesCompleted.inc(1);
+          context.recordsCompleted.inc(incoming.getRecordCount());
+          QueryResult header2 = QueryResult.newBuilder() //
+              .setQueryId(context.getHandle().getQueryId()) //
+              .setRowCount(0) //
+              .setDef(RecordBatchDef.getDefaultInstance()) //
+              .setIsLastChunk(true) //
+              .build();
+          QueryWritableBatch batch2 = new QueryWritableBatch(header2);
+          connection.sendResult(listener, batch2);
+        }else{
+          connection.sendResult(listener, materializer.convertNext(true));
+        }
         return false;
-        
+      }
       case OK_NEW_SCHEMA:
         materializer = new VectorRecordMaterializer(context, incoming);
         // fall through.
-        // fall through
       case OK:
-        connection.sendResult(materializer.convertNext(false));
         context.batchesCompleted.inc(1);
         context.recordsCompleted.inc(incoming.getRecordCount());
-        return !isLast;
+        connection.sendResult(listener, materializer.convertNext(false));
+        return true;
       default:
         throw new UnsupportedOperationException();
       }
@@ -85,6 +126,20 @@ public class ScreenCreator implements RootCreator<Screen>{
       incoming.kill();
     }
 
+    private SendListener listener = new SendListener();
+    
+    private class SendListener extends BaseRpcOutcomeListener<Ack>{
+
+      @Override
+      public void failed(RpcException ex) {
+        logger.error("Failure while sending data to user.", ex);
+        ErrorHelper.logAndConvertError(context.getIdentity(), "Failure while sending fragment to client.", ex, logger);
+        ok = false;
+      }
+      
+    }
     
   }
+  
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/b8db98ad/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/SingleSenderCreator.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/SingleSenderCreator.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/SingleSenderCreator.java
index 60c2d78..b7d4c7e 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/SingleSenderCreator.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/SingleSenderCreator.java
@@ -23,9 +23,12 @@ import org.apache.drill.common.exceptions.ExecutionSetupException;
 import org.apache.drill.exec.ops.FragmentContext;
 import org.apache.drill.exec.physical.config.SingleSender;
 import org.apache.drill.exec.proto.ExecProtos.FragmentHandle;
+import org.apache.drill.exec.proto.GeneralRPCProtos.Ack;
 import org.apache.drill.exec.record.FragmentWritableBatch;
 import org.apache.drill.exec.record.RecordBatch;
 import org.apache.drill.exec.record.RecordBatch.IterOutcome;
+import org.apache.drill.exec.rpc.BaseRpcOutcomeListener;
+import org.apache.drill.exec.rpc.RpcException;
 import org.apache.drill.exec.rpc.bit.BitTunnel;
 
 public class SingleSenderCreator implements RootCreator<SingleSender>{
@@ -45,9 +48,9 @@ public class SingleSenderCreator implements RootCreator<SingleSender>{
     private FragmentHandle handle;
     private int recMajor;
     private FragmentContext context;
+    private volatile boolean ok = true;
     
     public SingleSenderRootExec(FragmentContext context, RecordBatch batch, SingleSender config){
-      logger.debug("Creating single sender root exec base on config: {}", config);
       this.incoming = batch;
       this.handle = context.getHandle();
       this.recMajor = config.getOppositeMajorFragmentId();
@@ -57,20 +60,24 @@ public class SingleSenderCreator implements RootCreator<SingleSender>{
     
     @Override
     public boolean next() {
+      if(!ok){
+        incoming.kill();
+        
+        return false;
+      }
       IterOutcome out = incoming.next();
       logger.debug("Outcome of sender next {}", out);
       switch(out){
       case STOP:
       case NONE:
-        FragmentWritableBatch b2 = new FragmentWritableBatch(false, handle.getQueryId(), handle.getMajorFragmentId(), handle.getMinorFragmentId(), recMajor, 0, incoming.getWritableBatch());
-        tunnel.sendRecordBatch(context, b2);
+        FragmentWritableBatch b2 = new FragmentWritableBatch(true, handle.getQueryId(), handle.getMajorFragmentId(), handle.getMinorFragmentId(), recMajor, 0, incoming.getWritableBatch());
+        tunnel.sendRecordBatch(new RecordSendFailure(), context, b2);
         return false;
-        
 
       case OK:
       case OK_NEW_SCHEMA:
-        FragmentWritableBatch batch = new FragmentWritableBatch(true, handle.getQueryId(), handle.getMajorFragmentId(), handle.getMinorFragmentId(), recMajor, 0, incoming.getWritableBatch());
-        tunnel.sendRecordBatch(context, batch);
+        FragmentWritableBatch batch = new FragmentWritableBatch(false, handle.getQueryId(), handle.getMajorFragmentId(), handle.getMinorFragmentId(), recMajor, 0, incoming.getWritableBatch());
+        tunnel.sendRecordBatch(new RecordSendFailure(), context, batch);
         return true;
 
       case NOT_YET:
@@ -81,9 +88,31 @@ public class SingleSenderCreator implements RootCreator<SingleSender>{
 
     @Override
     public void stop() {
+      ok = false;
     }
     
     
+    private class RecordSendFailure extends BaseRpcOutcomeListener<Ack>{
+
+      @Override
+      public void failed(RpcException ex) {
+        context.fail(ex);
+        stop();
+      }
+
+      @Override
+      public void success(Ack value) {
+        if(value.getOk()) return;
+        
+        logger.error("Downstream fragment was not accepted.  Stopping future sends.");
+        // if we didn't get ack ok, we'll need to kill the query.
+        context.fail(new RpcException("A downstream fragment batch wasn't accepted.  This fragment thus fails."));
+        stop();
+      }
+      
+    }
     
   }
+  
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/b8db98ad/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/WireRecordBatch.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/WireRecordBatch.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/WireRecordBatch.java
index fc7f833..b41b0cd 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/WireRecordBatch.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/WireRecordBatch.java
@@ -38,6 +38,7 @@ public class WireRecordBatch implements RecordBatch{
   private RecordBatchLoader batchLoader;
   private RawFragmentBatchProvider fragProvider;
   private FragmentContext context;
+  private BatchSchema schema;
 
   
   public WireRecordBatch(FragmentContext context, RawFragmentBatchProvider fragProvider) {
@@ -53,7 +54,7 @@ public class WireRecordBatch implements RecordBatch{
 
   @Override
   public BatchSchema getSchema() {
-    return null;
+    return schema;
   }
 
   @Override
@@ -73,13 +74,16 @@ public class WireRecordBatch implements RecordBatch{
 
   @Override
   public IterOutcome next() {
-    RawFragmentBatch batch = this.fragProvider.getNext();
+    RawFragmentBatch batch = fragProvider.getNext();
     try{
       if(batch == null) return IterOutcome.NONE;
 
+      logger.debug("Next received batch {}", batch);
+
       RecordBatchDef rbd = batch.getHeader().getDef();
       boolean schemaChanged = batchLoader.load(rbd, batch.getBody());
       if(schemaChanged){
+        this.schema = batchLoader.getSchema();
         return IterOutcome.OK_NEW_SCHEMA;
       }else{
         return IterOutcome.OK;

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/b8db98ad/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/materialize/QueryWritableBatch.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/materialize/QueryWritableBatch.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/materialize/QueryWritableBatch.java
index 187e6e9..e8ed48a 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/materialize/QueryWritableBatch.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/materialize/QueryWritableBatch.java
@@ -17,6 +17,8 @@
  ******************************************************************************/
 package org.apache.drill.exec.physical.impl.materialize;
 
+import java.util.Arrays;
+
 import io.netty.buffer.ByteBuf;
 
 import org.apache.drill.exec.proto.UserProtos.QueryResult;
@@ -42,5 +44,11 @@ public class QueryWritableBatch {
   public QueryResult getHeader() {
     return header;
   }
+
+  @Override
+  public String toString() {
+    return "QueryWritableBatch [header=" + header + ", buffers=" + Arrays.toString(buffers) + "]";
+  }
+  
   
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/b8db98ad/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/materialize/VectorRecordMaterializer.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/materialize/VectorRecordMaterializer.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/materialize/VectorRecordMaterializer.java
index e2d2eb9..7929296 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/materialize/VectorRecordMaterializer.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/materialize/VectorRecordMaterializer.java
@@ -20,6 +20,7 @@ package org.apache.drill.exec.physical.impl.materialize;
 import org.apache.drill.exec.ops.FragmentContext;
 import org.apache.drill.exec.proto.UserBitShared.QueryId;
 import org.apache.drill.exec.proto.UserProtos.QueryResult;
+import org.apache.drill.exec.record.BatchSchema;
 import org.apache.drill.exec.record.MaterializedField;
 import org.apache.drill.exec.record.RecordBatch;
 import org.apache.drill.exec.record.WritableBatch;
@@ -33,10 +34,12 @@ public class VectorRecordMaterializer implements RecordMaterializer{
   public VectorRecordMaterializer(FragmentContext context, RecordBatch batch) {
     this.queryId = context.getHandle().getQueryId();
     this.batch = batch;
-
-    for (MaterializedField f : batch.getSchema()) {
-      logger.debug("New Field: {}", f);
-    }
+    BatchSchema schema = batch.getSchema();
+    assert schema != null : "Schema must be defined.";
+    
+//    for (MaterializedField f : batch.getSchema()) {
+//      logger.debug("New Field: {}", f);
+//    }
   }
 
   public QueryWritableBatch convertNext(boolean isLast) {

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/b8db98ad/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/Materializer.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/Materializer.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/Materializer.java
index 9fee586..da71271 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/Materializer.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/Materializer.java
@@ -41,13 +41,13 @@ public class Materializer extends AbstractPhysicalVisitor<PhysicalOperator, Mate
       // this is a sending exchange.
       PhysicalOperator child = exchange.getChild().accept(this, iNode);
       PhysicalOperator materializedSender = exchange.getSender(iNode.getMinorFragmentId(), child);
-      logger.debug("Visit sending exchange, materialized {} with child {}.", materializedSender, child);
+//      logger.debug("Visit sending exchange, materialized {} with child {}.", materializedSender, child);
       return materializedSender;
       
     }else{
       // receiving exchange.
       PhysicalOperator materializedReceiver = exchange.getReceiver(iNode.getMinorFragmentId());
-      logger.debug("Visit receiving exchange, materialized receiver: {}.", materializedReceiver);
+//      logger.debug("Visit receiving exchange, materialized receiver: {}.", materializedReceiver);
       return materializedReceiver;
     }
   }
@@ -63,7 +63,7 @@ public class Materializer extends AbstractPhysicalVisitor<PhysicalOperator, Mate
     
     try {
       PhysicalOperator o = store.getSpecificStore(child, iNode.getMinorFragmentId());
-      logger.debug("New materialized store node {} with child {}", o, child);
+//      logger.debug("New materialized store node {} with child {}", o, child);
       return o;
     } catch (PhysicalOperatorSetupException e) {
       throw new FragmentSetupException("Failure while generating a specific Store materialization.");
@@ -72,7 +72,7 @@ public class Materializer extends AbstractPhysicalVisitor<PhysicalOperator, Mate
 
   @Override
   public PhysicalOperator visitOp(PhysicalOperator op, IndexedFragmentNode iNode) throws ExecutionSetupException {
-    logger.debug("Visiting catch all: {}", op);
+//    logger.debug("Visiting catch all: {}", op);
     List<PhysicalOperator> children = Lists.newArrayList();
     for(PhysicalOperator child : op){
       children.add(child.accept(this, iNode));

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/b8db98ad/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/SimpleParallelizer.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/SimpleParallelizer.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/SimpleParallelizer.java
index fc03a23..8adb447 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/SimpleParallelizer.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/SimpleParallelizer.java
@@ -145,7 +145,7 @@ public class SimpleParallelizer {
       // figure out width.
       int width = Math.min(stats.getMaxWidth(), globalMaxWidth);
       float diskCost = stats.getDiskCost();
-      logger.debug("Frag max width: {} and diskCost: {}", stats.getMaxWidth(), diskCost);
+//      logger.debug("Frag max width: {} and diskCost: {}", stats.getMaxWidth(), diskCost);
 
       // TODO: right now we'll just assume that each task is cost 1 so we'll set the breadth at the lesser of the number
       // of tasks or the maximum width of the fragment.
@@ -154,7 +154,7 @@ public class SimpleParallelizer {
       }
 
       if (width < 1) width = 1;
-      logger.debug("Setting width {} on fragment {}", width, wrapper);
+//      logger.debug("Setting width {} on fragment {}", width, wrapper);
       wrapper.setWidth(width);
       // figure out endpoint assignments. also informs the exchanges about their respective endpoints.
       wrapper.assignEndpoints(allNodes);

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/b8db98ad/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/StatsCollector.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/StatsCollector.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/StatsCollector.java
index d53a78c..af8ec04 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/StatsCollector.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/StatsCollector.java
@@ -41,7 +41,7 @@ public class StatsCollector {
 
     Wrapper wrapper = planningSet.get(n);
     n.getRoot().accept(opStatCollector, wrapper);
-    logger.debug("Set stats to {}", wrapper.getStats());
+//    logger.debug("Set stats to {}", wrapper.getStats());
     // receivers...
     for (ExchangeFragmentPair child : n) {
       // get the fragment node that feeds this node.

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/b8db98ad/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/RawFragmentBatch.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/RawFragmentBatch.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/RawFragmentBatch.java
index c244cea..4f87224 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/RawFragmentBatch.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/RawFragmentBatch.java
@@ -41,4 +41,9 @@ public class RawFragmentBatch {
     return body;
   }
 
+  @Override
+  public String toString() {
+    return "RawFragmentBatch [header=" + header + ", body=" + body + "]";
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/b8db98ad/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/AbstractHandshakeHandler.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/AbstractHandshakeHandler.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/AbstractHandshakeHandler.java
index 859d385..ea591da 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/AbstractHandshakeHandler.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/AbstractHandshakeHandler.java
@@ -25,8 +25,7 @@ import com.google.protobuf.Internal.EnumLite;
 import com.google.protobuf.MessageLite;
 import com.google.protobuf.Parser;
 
-public abstract class AbstractHandshakeHandler<T extends MessageLite> extends
-    ChannelInboundMessageHandlerAdapter<InboundRpcMessage> {
+public abstract class AbstractHandshakeHandler<T extends MessageLite> extends ChannelInboundMessageHandlerAdapter<InboundRpcMessage> {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(AbstractHandshakeHandler.class);
 
   protected final EnumLite handshakeType;
@@ -41,7 +40,7 @@ public abstract class AbstractHandshakeHandler<T extends MessageLite> extends
 
   @Override
   public final void messageReceived(ChannelHandlerContext ctx, InboundRpcMessage inbound) throws Exception {
-    coordinationId = inbound.coordinationId;
+    this.coordinationId = inbound.coordinationId;
     ctx.channel().pipeline().remove(this);
     if (inbound.rpcType != handshakeType.getNumber())
       throw new RpcException(String.format("Handshake failure.  Expected %s[%d] but received number [%d]",

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/b8db98ad/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/BaseRpcOutcomeListener.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/BaseRpcOutcomeListener.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/BaseRpcOutcomeListener.java
new file mode 100644
index 0000000..1dab1c7
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/BaseRpcOutcomeListener.java
@@ -0,0 +1,32 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.rpc;
+
+public class BaseRpcOutcomeListener<T> implements RpcOutcomeListener<T> {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(BaseRpcOutcomeListener.class);
+
+  @Override
+  public void failed(RpcException ex) {
+  }
+
+  @Override
+  public void success(T value) {
+  }
+  
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/b8db98ad/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/BasicClient.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/BasicClient.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/BasicClient.java
index 0ff2b9d..0afc5d0 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/BasicClient.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/BasicClient.java
@@ -29,22 +29,30 @@ import io.netty.channel.socket.SocketChannel;
 import io.netty.channel.socket.nio.NioSocketChannel;
 import io.netty.util.concurrent.GenericFutureListener;
 
+import org.apache.drill.exec.rpc.RpcConnectionHandler.FailureType;
+
 import com.google.common.util.concurrent.SettableFuture;
 import com.google.protobuf.Internal.EnumLite;
 import com.google.protobuf.MessageLite;
 import com.google.protobuf.Parser;
 
-public abstract class BasicClient<T extends EnumLite, R extends RemoteConnection> extends RpcBus<T, R> {
+public abstract class BasicClient<T extends EnumLite, R extends RemoteConnection, HANDSHAKE_SEND extends MessageLite, HANDSHAKE_RESPONSE extends MessageLite>
+    extends RpcBus<T, R> {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(BasicClient.class);
 
-  private Bootstrap b;
+  private final Bootstrap b;
   private volatile boolean connect = false;
   protected R connection;
-  private EventLoopGroup eventLoop;
+  private final T handshakeType;
+  private final Class<HANDSHAKE_RESPONSE> responseClass;
+  private final Parser<HANDSHAKE_RESPONSE> handshakeParser;
 
-  public BasicClient(RpcConfig rpcMapping, ByteBufAllocator alloc, EventLoopGroup eventLoopGroup) {
+  public BasicClient(RpcConfig rpcMapping, ByteBufAllocator alloc, EventLoopGroup eventLoopGroup, T handshakeType,
+      Class<HANDSHAKE_RESPONSE> responseClass, Parser<HANDSHAKE_RESPONSE> handshakeParser) {
     super(rpcMapping);
-    this.eventLoop = eventLoopGroup;
+    this.responseClass = responseClass;
+    this.handshakeType = handshakeType;
+    this.handshakeParser = handshakeParser;
     
     b = new Bootstrap() //
         .group(eventLoopGroup) //
@@ -59,12 +67,12 @@ public abstract class BasicClient<T extends EnumLite, R extends RemoteConnection
             logger.debug("initializing client connection.");
             connection = initRemoteConnection(ch);
             ch.closeFuture().addListener(getCloseHandler(connection));
-
+            
             ch.pipeline().addLast( //
                 new ZeroCopyProtobufLengthDecoder(), //
                 new RpcDecoder(rpcConfig.getName()), //
                 new RpcEncoder(rpcConfig.getName()), //
-                getHandshakeHandler(), //
+                new ClientHandshakeHandler(), //
                 new InboundHandler(connection), //
                 new RpcExceptionHandler() //
                 );
@@ -75,26 +83,9 @@ public abstract class BasicClient<T extends EnumLite, R extends RemoteConnection
     ;
   }
 
-  protected abstract ClientHandshakeHandler<?> getHandshakeHandler();
-
-  protected abstract class ClientHandshakeHandler<T extends MessageLite> extends AbstractHandshakeHandler<T> {
-    private Class<T> responseType;
-
-    public ClientHandshakeHandler(EnumLite handshakeType, Class<T> responseType, Parser<T> parser) {
-      super(handshakeType, parser);
-      this.responseType = responseType;
-    }
-
-    @Override
-    protected final void consumeHandshake(Channel c, T msg) throws Exception {
-      validateHandshake(msg);
-      queue.getFuture(handshakeType.getNumber(), coordinationId, responseType).setValue(msg);
-    }
-
-    protected abstract void validateHandshake(T msg) throws Exception;
-
-  }
-
+  protected abstract void validateHandshake(HANDSHAKE_RESPONSE validateHandshake) throws RpcException;
+  protected abstract void finalizeConnection(HANDSHAKE_RESPONSE handshake, R connection);
+  
   protected GenericFutureListener<ChannelFuture> getCloseHandler(Channel channel) {
     return new ChannelClosedHandler();
   }
@@ -105,6 +96,11 @@ public abstract class BasicClient<T extends EnumLite, R extends RemoteConnection
         "This shouldn't be used in client mode as a client only has a single connection.");
   }
 
+  protected <SEND extends MessageLite, RECEIVE extends MessageLite> void send(RpcOutcomeListener<RECEIVE> listener,
+      T rpcType, SEND protobufBody, Class<RECEIVE> clazz, ByteBuf... dataBodies) throws RpcException {
+    super.send(listener, connection, rpcType, protobufBody, clazz, dataBodies);
+  }
+
   protected <SEND extends MessageLite, RECEIVE extends MessageLite> DrillRpcFuture<RECEIVE> send(T rpcType,
       SEND protobufBody, Class<RECEIVE> clazz, ByteBuf... dataBodies) throws RpcException {
     return super.send(connection, rpcType, protobufBody, clazz, dataBodies);
@@ -115,65 +111,91 @@ public abstract class BasicClient<T extends EnumLite, R extends RemoteConnection
     return true;
   }
 
-  /**
-   * TODO: This is a horrible hack to manage deadlock caused by creation of BitClient within BitCom.  Should be cleaned up.
-   */
-  private class HandshakeThread<SEND extends MessageLite, RECEIVE extends MessageLite> extends Thread {
-    final SettableFuture<RECEIVE> future;
-    T handshakeType;
-    SEND handshakeValue;
-    String host;
-    int port;
-    Class<RECEIVE> responseClass;
-
-    public HandshakeThread(T handshakeType, SEND handshakeValue, String host, int port, Class<RECEIVE> responseClass) {
-      super();
-      assert host != null && !host.isEmpty();
-      assert port > 0;
-      logger.debug("Creating new handshake thread to connec to {}:{}", host, port);
-      this.setName(String.format("handshake thread for %s", handshakeType.getClass().getCanonicalName()));
-      future = SettableFuture.create();
-      this.handshakeType = handshakeType;
+  protected void connectAsClient(RpcConnectionHandler<R> connectionListener, HANDSHAKE_SEND handshakeValue, String host, int port){
+    ConnectionMultiListener cml = new ConnectionMultiListener(connectionListener, handshakeValue);
+    b.connect(host, port).addListener(cml.connectionHandler);
+  }
+
+  private class ConnectionMultiListener {
+    private final RpcConnectionHandler<R> l;
+    private final HANDSHAKE_SEND handshakeValue;
+
+    public ConnectionMultiListener(RpcConnectionHandler<R> l, HANDSHAKE_SEND handshakeValue) {
+      assert l != null;
+      assert handshakeValue != null;
+          
+      this.l = l;
       this.handshakeValue = handshakeValue;
-      this.host = host;
-      this.port = port;
-      this.responseClass = responseClass;
     }
 
-    @Override
-    public void run() {
-      try {
-        logger.debug("Starting to get client connection on host {}, port {}.", host, port);
-        
-        ChannelFuture f = b.connect(host, port);
-        f.sync();
-        if (connection == null) throw new RpcException("Failure while attempting to connect to server.");
-        connect = !connect;
-        logger.debug("Client connected, sending handshake.");
-        DrillRpcFuture<RECEIVE> fut = send(handshakeType, handshakeValue, responseClass);
-        future.set(fut.checkedGet());
-        logger.debug("Got bit client connection.");
-      } catch (Exception e) {
-        logger.debug("Failed to get client connection.", e);
-        future.setException(e);
+    public final ConnectionHandler connectionHandler = new ConnectionHandler();
+    public final HandshakeSendHandler handshakeSendHandler = new HandshakeSendHandler();
+
+    /**
+     * Manages connection establishment outcomes.
+     */
+    private class ConnectionHandler implements GenericFutureListener<ChannelFuture> {
+
+      @Override
+      public void operationComplete(ChannelFuture future) throws Exception {
+//        logger.debug("Connection operation finished.  Success: {}", future.isSuccess());
+        try {
+          future.get();
+          if (future.isSuccess()) {
+            send(handshakeSendHandler, handshakeType, handshakeValue, responseClass);
+          } else {
+            l.connectionFailed(FailureType.CONNECTION, new RpcException("General connection failure."));
+          }
+//          logger.debug("Handshake queued for send.");
+        } catch (Exception ex) {
+          l.connectionFailed(FailureType.CONNECTION, ex);
+        }
       }
     }
 
+    /**
+     * manages handshake outcomes.
+     */
+    private class HandshakeSendHandler implements RpcOutcomeListener<HANDSHAKE_RESPONSE> {
+
+      @Override
+      public void failed(RpcException ex) {
+        logger.debug("Failure while initiating handshake", ex);
+        l.connectionFailed(FailureType.HANDSHAKE_COMMUNICATION, ex);
+      }
+
+      @Override
+      public void success(HANDSHAKE_RESPONSE value) {
+//        logger.debug("Handshake received. {}", value);
+        try {
+          BasicClient.this.validateHandshake(value);
+          BasicClient.this.finalizeConnection(value, connection);
+          BasicClient.this.connect = true;
+          l.connectionSucceeded(connection);
+//          logger.debug("Handshake completed succesfully.");
+        } catch (RpcException ex) {
+          l.connectionFailed(FailureType.HANDSHAKE_VALIDATION, ex);
+        }
+      }
+
+    }
+
   }
 
-  protected <SEND extends MessageLite, RECEIVE extends MessageLite> RECEIVE connectAsClient(T handshakeType,
-      SEND handshakeValue, String host, int port, Class<RECEIVE> responseClass) throws InterruptedException,
-      RpcException {
-    
-    
-    HandshakeThread<SEND, RECEIVE> ht = new HandshakeThread<SEND, RECEIVE>(handshakeType, handshakeValue, host, port, responseClass);
-    ht.start();
-    try{
-      return ht.future.get();  
-    }catch(Exception e){
-      throw new RpcException(e);
+  private class ClientHandshakeHandler extends AbstractHandshakeHandler<HANDSHAKE_RESPONSE> {
+
+    public ClientHandshakeHandler() {
+      super(BasicClient.this.handshakeType, BasicClient.this.handshakeParser);
     }
-    
+
+    @Override
+    protected final void consumeHandshake(Channel c, HANDSHAKE_RESPONSE msg) throws Exception {
+      // remove the handshake information from the queue so it doesn't sit there forever.
+      RpcOutcome<HANDSHAKE_RESPONSE> response = queue.getFuture(handshakeType.getNumber(), coordinationId,
+          responseClass);
+      response.set(msg);
+    }
+
   }
 
   public void close() {

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/b8db98ad/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/BasicClientWithConnection.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/BasicClientWithConnection.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/BasicClientWithConnection.java
index 0e62f14..2028db6 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/BasicClientWithConnection.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/BasicClientWithConnection.java
@@ -26,13 +26,16 @@ import io.netty.util.concurrent.GenericFutureListener;
 
 import org.apache.drill.exec.rpc.BasicClientWithConnection.ServerConnection;
 
+import com.google.protobuf.MessageLite;
+import com.google.protobuf.Parser;
 import com.google.protobuf.Internal.EnumLite;
 
-public abstract class BasicClientWithConnection<T extends EnumLite> extends BasicClient<T, ServerConnection>{
+public abstract class BasicClientWithConnection<T extends EnumLite, HANDSHAKE_SEND extends MessageLite, HANDSHAKE_RESPONSE extends MessageLite> extends BasicClient<T, ServerConnection, HANDSHAKE_SEND, HANDSHAKE_RESPONSE>{
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(BasicClientWithConnection.class);
 
-  public BasicClientWithConnection(RpcConfig rpcMapping, ByteBufAllocator alloc, EventLoopGroup eventLoopGroup) {
-    super(rpcMapping, alloc, eventLoopGroup);
+  public BasicClientWithConnection(RpcConfig rpcMapping, ByteBufAllocator alloc, EventLoopGroup eventLoopGroup, T handshakeType,
+      Class<HANDSHAKE_RESPONSE> responseClass, Parser<HANDSHAKE_RESPONSE> handshakeParser) {
+    super(rpcMapping, alloc, eventLoopGroup, handshakeType, responseClass, handshakeParser);
   }
   
   @Override

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/b8db98ad/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/BasicServer.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/BasicServer.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/BasicServer.java
index 52bb0a2..af5d9c9 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/BasicServer.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/BasicServer.java
@@ -72,7 +72,7 @@ public abstract class BasicServer<T extends EnumLite, C extends RemoteConnection
                 new ZeroCopyProtobufLengthDecoder(), //
                 new RpcDecoder(rpcConfig.getName()), //
                 new RpcEncoder(rpcConfig.getName()), //
-                getHandshakeHandler(),
+                getHandshakeHandler(connection),
                 new InboundHandler(connection), //
                 new RpcExceptionHandler() //
                 );            
@@ -88,7 +88,7 @@ public abstract class BasicServer<T extends EnumLite, C extends RemoteConnection
   }
 
   
-  protected abstract ServerHandshakeHandler<?> getHandshakeHandler();
+  protected abstract ServerHandshakeHandler<?> getHandshakeHandler(C connection);
 
   protected static abstract class ServerHandshakeHandler<T extends MessageLite> extends AbstractHandshakeHandler<T> {
 
@@ -104,9 +104,6 @@ public abstract class BasicServer<T extends EnumLite, C extends RemoteConnection
     
     public abstract MessageLite getHandshakeResponse(T inbound) throws Exception;
     
-
-      
-    
   }
   
   

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/b8db98ad/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/ChannelListenerWithCoordinationId.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/ChannelListenerWithCoordinationId.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/ChannelListenerWithCoordinationId.java
new file mode 100644
index 0000000..27e9dee
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/ChannelListenerWithCoordinationId.java
@@ -0,0 +1,25 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.rpc;
+
+import io.netty.channel.ChannelFuture;
+import io.netty.util.concurrent.GenericFutureListener;
+
+public interface ChannelListenerWithCoordinationId extends GenericFutureListener<ChannelFuture>{
+  public int getCoordinationId();
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/b8db98ad/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/CoordinationQueue.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/CoordinationQueue.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/CoordinationQueue.java
index 70142bb..9edbe11 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/CoordinationQueue.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/CoordinationQueue.java
@@ -17,8 +17,12 @@
  ******************************************************************************/
 package org.apache.drill.exec.rpc;
 
+import io.netty.channel.ChannelFuture;
+import io.netty.util.concurrent.GenericFutureListener;
+
 import java.util.Map;
 import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutionException;
 
 import org.apache.drill.exec.proto.GeneralRPCProtos.RpcFailure;
 
@@ -29,31 +33,93 @@ public class CoordinationQueue {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(CoordinationQueue.class);
 
   private final PositiveAtomicInteger circularInt = new PositiveAtomicInteger();
-  private final Map<Integer, DrillRpcFutureImpl<?>> map;
+  private final Map<Integer, RpcOutcome<?>> map;
 
   public CoordinationQueue(int segmentSize, int segmentCount) {
-    map = new ConcurrentHashMap<Integer, DrillRpcFutureImpl<?>>(segmentSize, 0.75f, segmentCount);
+    map = new ConcurrentHashMap<Integer, RpcOutcome<?>>(segmentSize, 0.75f, segmentCount);
   }
 
-  void channelClosed(Exception ex) {
-    for (DrillRpcFutureImpl<?> f : map.values()) {
-      f.setException(ex);
+  void channelClosed(Throwable ex) {
+    if(ex != null){
+      RpcException e;
+      if(ex instanceof RpcException){
+        e = (RpcException) ex;
+      }else{
+        e = new RpcException(ex);  
+      }
+      for (RpcOutcome<?> f : map.values()) {
+        f.setException(e);
+      }
     }
   }
 
-  public <V> DrillRpcFutureImpl<V> getNewFuture(Class<V> clazz) {
+  public <V> ChannelListenerWithCoordinationId get(RpcOutcomeListener<V> handler, Class<V> clazz){
     int i = circularInt.getNext();
-    DrillRpcFutureImpl<V> future = DrillRpcFutureImpl.getNewFuture(i, clazz);
-    // logger.debug("Writing to map coord {}, future {}", i, future);
+    RpcListener<V> future = new RpcListener<V>(handler, clazz, i);
     Object old = map.put(i, future);
     if (old != null)
       throw new IllegalStateException(
           "You attempted to reuse a coordination id when the previous coordination id has not been removed.  This is likely rpc future callback memory leak.");
     return future;
   }
+  
+  private class RpcListener<T> implements ChannelListenerWithCoordinationId, RpcOutcome<T>{
+    final RpcOutcomeListener<T> handler;
+    final Class<T> clazz;
+    final int coordinationId;
+    
+    public RpcListener(RpcOutcomeListener<T> handler, Class<T> clazz, int coordinationId) {
+      super();
+      this.handler = handler;
+      this.clazz = clazz;
+      this.coordinationId = coordinationId;
+    }
+
+    @Override
+    public void operationComplete(ChannelFuture future) throws Exception {
+      if(!future.isSuccess()){
+        removeFromMap(coordinationId);
+      }
+    }
+
+    @SuppressWarnings("unchecked")
+    @Override
+    public void set(Object value) {
+      assert clazz.isAssignableFrom(value.getClass());
+      handler.success( (T) value);
+    }
+
+    @Override
+    public void setException(Throwable t) {
+      handler.failed(RpcException.mapException(t));
+    }
+
+    @Override
+    public Class<T> getOutcomeType() {
+      return clazz;
+    }
+
+    @Override
+    public int getCoordinationId() {
+      return coordinationId;
+    }
+    
+    
+  }
+//  
+//  public <V> DrillRpcFutureImpl<V> getNewFuture(Class<V> clazz) {
+//    int i = circularInt.getNext();
+//    DrillRpcFutureImpl<V> future = DrillRpcFutureImpl.getNewFuture(i, clazz);
+//    // logger.debug("Writing to map coord {}, future {}", i, future);
+//    Object old = map.put(i, future);
+//    if (old != null)
+//      throw new IllegalStateException(
+//          "You attempted to reuse a coordination id when the previous coordination id has not been removed.  This is likely rpc future callback memory leak.");
+//    return future;
+//  }
 
-  private DrillRpcFutureImpl<?> removeFromMap(int coordinationId) {
-    DrillRpcFutureImpl<?> rpc = map.remove(coordinationId);
+  private RpcOutcome<?> removeFromMap(int coordinationId) {
+    RpcOutcome<?> rpc = map.remove(coordinationId);
     if (rpc == null) {
       logger.error("Rpc is null.");
       throw new IllegalStateException(
@@ -62,11 +128,11 @@ public class CoordinationQueue {
     return rpc;
   }
 
-  public <V> DrillRpcFutureImpl<V> getFuture(int rpcType, int coordinationId, Class<V> clazz) {
+  public <V> RpcOutcome<V> getFuture(int rpcType, int coordinationId, Class<V> clazz) {
     // logger.debug("Getting future for coordinationId {} and class {}", coordinationId, clazz);
-    DrillRpcFutureImpl<?> rpc = removeFromMap(coordinationId);
+    RpcOutcome<?> rpc = removeFromMap(coordinationId);
     // logger.debug("Got rpc from map {}", rpc);
-    Class<?> outcomeClass = rpc.getOutcomeClass();
+    Class<?> outcomeClass = rpc.getOutcomeType();
 
     if (outcomeClass != clazz) {
 
@@ -80,7 +146,7 @@ public class CoordinationQueue {
     }
 
     @SuppressWarnings("unchecked")
-    DrillRpcFutureImpl<V> crpc = (DrillRpcFutureImpl<V>) rpc;
+    RpcOutcome<V> crpc = (RpcOutcome<V>) rpc;
 
     // logger.debug("Returning casted future");
     return crpc;
@@ -88,7 +154,7 @@ public class CoordinationQueue {
 
   public void updateFailedFuture(int coordinationId, RpcFailure failure) {
     // logger.debug("Updating failed future.");
-    DrillRpcFutureImpl<?> rpc = removeFromMap(coordinationId);
+    RpcOutcome<?> rpc = removeFromMap(coordinationId);
     rpc.setException(new RemoteRpcException(failure));
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/b8db98ad/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/DrillRpcFuture.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/DrillRpcFuture.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/DrillRpcFuture.java
index bae947a..9033ea1 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/DrillRpcFuture.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/DrillRpcFuture.java
@@ -21,6 +21,4 @@ import com.google.common.util.concurrent.CheckedFuture;
 
 public interface DrillRpcFuture<T> extends CheckedFuture<T,RpcException> {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillRpcFuture.class);
-
-  public void addLightListener(RpcOutcomeListener<T> outcomeListener);
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/b8db98ad/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/DrillRpcFutureImpl.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/DrillRpcFutureImpl.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/DrillRpcFutureImpl.java
index ee14eeb..d5d3a9c 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/DrillRpcFutureImpl.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/DrillRpcFutureImpl.java
@@ -22,22 +22,12 @@ import java.util.concurrent.ExecutionException;
 import com.google.common.util.concurrent.AbstractCheckedFuture;
 import com.google.common.util.concurrent.AbstractFuture;
 import com.google.common.util.concurrent.ListenableFuture;
-import com.google.common.util.concurrent.MoreExecutors;
 
-class DrillRpcFutureImpl<V> extends AbstractCheckedFuture<V, RpcException> implements DrillRpcFuture<V>{
+class DrillRpcFutureImpl<V> extends AbstractCheckedFuture<V, RpcException> implements DrillRpcFuture<V>, RpcOutcomeListener<V>{
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillRpcFutureImpl.class);
 
-  final int coordinationId;
-  private final Class<V> clazz;
-
-  public DrillRpcFutureImpl(ListenableFuture<V> delegate, int coordinationId, Class<V> clazz) {
-    super(delegate);
-    this.coordinationId = coordinationId;
-    this.clazz = clazz;
-  }
-
-  public Class<V> getOutcomeClass(){
-    return clazz;
+  public DrillRpcFutureImpl() {
+    super(new InnerFuture<V>());
   }
   
   /**
@@ -53,24 +43,7 @@ class DrillRpcFutureImpl<V> extends AbstractCheckedFuture<V, RpcException> imple
 
   @Override
   protected RpcException mapException(Exception ex) {
-    Throwable e = ex;
-    while(e instanceof ExecutionException){
-      e = e.getCause();
-    }
-    if (e instanceof RpcException)  return (RpcException) e;
-
-    return new RpcException(ex);
-
-  }
-
-  @SuppressWarnings("unchecked")
-  void setValue(Object value) {
-    assert clazz.isAssignableFrom(value.getClass());
-    ((InnerFuture<V>) super.delegate()).setValue((V) value);
-  }
-
-  boolean setException(Throwable t) {
-    return ((InnerFuture<V>) super.delegate()).setException(t);
+    return RpcException.mapException(ex);
   }
 
   public static class InnerFuture<T> extends AbstractFuture<T> {
@@ -85,34 +58,17 @@ class DrillRpcFutureImpl<V> extends AbstractCheckedFuture<V, RpcException> imple
     }
   }
 
-  public class RpcOutcomeListenerWrapper implements Runnable{
-    final RpcOutcomeListener<V> inner;
-    
-    public RpcOutcomeListenerWrapper(RpcOutcomeListener<V> inner) {
-      super();
-      this.inner = inner;
-    }
-
-    @Override
-    public void run() {
-      try{
-        inner.success(DrillRpcFutureImpl.this.checkedGet());
-      }catch(RpcException e){
-        inner.failed(e);
-      }
-    }
-  }
-  
-  public void addLightListener(RpcOutcomeListener<V> outcomeListener){
-    this.addListener(new RpcOutcomeListenerWrapper(outcomeListener), MoreExecutors.sameThreadExecutor());
+  @Override
+  public void failed(RpcException ex) {
+    ( (InnerFuture<V>)delegate()).setException(ex);
   }
-  
-  
-  
-  public static <V> DrillRpcFutureImpl<V> getNewFuture(int coordinationId, Class<V> clazz) {
-    InnerFuture<V> f = new InnerFuture<V>();
-    return new DrillRpcFutureImpl<V>(f, coordinationId, clazz);
+
+  @Override
+  public void success(V value) {
+    ( (InnerFuture<V>)delegate()).setValue(value);
   }
 
 
+  
+  
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/b8db98ad/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/RpcBus.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/RpcBus.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/RpcBus.java
index 11764db..a680a97 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/RpcBus.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/RpcBus.java
@@ -64,6 +64,16 @@ public abstract class RpcBus<T extends EnumLite, C extends RemoteConnection> imp
 
   public <SEND extends MessageLite, RECEIVE extends MessageLite> DrillRpcFuture<RECEIVE> send(C connection, T rpcType,
       SEND protobufBody, Class<RECEIVE> clazz, ByteBuf... dataBodies) {
+    DrillRpcFutureImpl<RECEIVE> rpcFuture = new DrillRpcFutureImpl<RECEIVE>();
+    this.send(rpcFuture, connection, rpcType, protobufBody, clazz, dataBodies);
+    return rpcFuture;
+  }  
+  
+  public <SEND extends MessageLite, RECEIVE extends MessageLite> void send(RpcOutcomeListener<RECEIVE> listener, C connection, T rpcType,
+      SEND protobufBody, Class<RECEIVE> clazz, ByteBuf... dataBodies) {
+  
+    
+
 
     assert !Arrays.asList(dataBodies).contains(null);
     assert rpcConfig.checkSend(rpcType, protobufBody.getClass(), clazz);
@@ -72,14 +82,12 @@ public abstract class RpcBus<T extends EnumLite, C extends RemoteConnection> imp
     boolean completed = false;
 
     try {
-      // logger.debug("Seding message");
       Preconditions.checkNotNull(protobufBody);
-      DrillRpcFutureImpl<RECEIVE> rpcFuture = queue.getNewFuture(clazz);
-      OutboundRpcMessage m = new OutboundRpcMessage(RpcMode.REQUEST, rpcType, rpcFuture.coordinationId, protobufBody, dataBodies);
+      ChannelListenerWithCoordinationId futureListener = queue.get(listener, clazz);
+      OutboundRpcMessage m = new OutboundRpcMessage(RpcMode.REQUEST, rpcType, futureListener.getCoordinationId(), protobufBody, dataBodies);
       ChannelFuture channelFuture = connection.getChannel().write(m);
-      channelFuture.addListener(new Listener(rpcFuture.coordinationId, clazz));
+      channelFuture.addListener(futureListener);
       completed = true;
-      return rpcFuture;
     } finally {
       if (!completed) {
         if (pBuffer != null) pBuffer.release();
@@ -140,10 +148,10 @@ public abstract class RpcBus<T extends EnumLite, C extends RemoteConnection> imp
       case RESPONSE:
         MessageLite m = getResponseDefaultInstance(msg.rpcType);
         assert rpcConfig.checkReceive(msg.rpcType, m.getClass());
-        DrillRpcFutureImpl<?> rpcFuture = queue.getFuture(msg.rpcType, msg.coordinationId, m.getClass());
+        RpcOutcome<?> rpcFuture = queue.getFuture(msg.rpcType, msg.coordinationId, m.getClass());
         Parser<?> parser = m.getParserForType();
         Object value = parser.parseFrom(new ByteBufInputStream(msg.pBody, msg.pBody.readableBytes()));
-        rpcFuture.setValue(value);
+        rpcFuture.set(value);
         if (RpcConstants.EXTRA_DEBUGGING) logger.debug("Updated rpc future {} with value {}", rpcFuture, value);
 
         break;
@@ -162,39 +170,39 @@ public abstract class RpcBus<T extends EnumLite, C extends RemoteConnection> imp
 
   }
 
-  private class Listener implements GenericFutureListener<ChannelFuture> {
-
-    private int coordinationId;
-    private Class<?> clazz;
-
-    public Listener(int coordinationId, Class<?> clazz) {
-      this.coordinationId = coordinationId;
-      this.clazz = clazz;
-    }
-
-    @Override
-    public void operationComplete(ChannelFuture channelFuture) throws Exception {
-      // logger.debug("Completed channel write.");
-
-      if (channelFuture.isCancelled()) {
-        DrillRpcFutureImpl<?> rpcFuture = queue.getFuture(-1, coordinationId, clazz);
-        rpcFuture.setException(new CancellationException("Socket operation was canceled."));
-      } else if (!channelFuture.isSuccess()) {
-        try {
-          channelFuture.get();
-          throw new IllegalStateException("Future was described as completed and not succesful but did not throw an exception.");
-        } catch (Exception e) {
-          logger.error("Error occurred during Rpc", e);
-          DrillRpcFutureImpl<?> rpcFuture = queue.getFuture(-1, coordinationId, clazz);
-          rpcFuture.setException(e);
-        }
-      } else {
-        // send was successful. No need to modify DrillRpcFuture.
-        return;
-      }
-    }
-
-  }
+//  private class Listener implements GenericFutureListener<ChannelFuture> {
+//
+//    private int coordinationId;
+//    private Class<?> clazz;
+//
+//    public Listener(int coordinationId, Class<?> clazz) {
+//      this.coordinationId = coordinationId;
+//      this.clazz = clazz;
+//    }
+//
+//    @Override
+//    public void operationComplete(ChannelFuture channelFuture) throws Exception {
+//      // logger.debug("Completed channel write.");
+//
+//      if (channelFuture.isCancelled()) {
+//        RpcOutcome<?> rpcFuture = queue.getFuture(-1, coordinationId, clazz);
+//        rpcFuture.setException(new CancellationException("Socket operation was canceled."));
+//      } else if (!channelFuture.isSuccess()) {
+//        try {
+//          channelFuture.get();
+//          throw new IllegalStateException("Future was described as completed and not succesful but did not throw an exception.");
+//        } catch (Exception e) {
+//          logger.error("Error occurred during Rpc", e);
+//          RpcOutcome<?> rpcFuture = queue.getFuture(-1, coordinationId, clazz);
+//          rpcFuture.setException(e);
+//        }
+//      } else {
+//        // send was successful. No need to modify DrillRpcFuture.
+//        return;
+//      }
+//    }
+//
+//  }
 
   public static <T> T get(ByteBuf pBody, Parser<T> parser) throws RpcException{
     try {

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/b8db98ad/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/RpcCheckedFuture.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/RpcCheckedFuture.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/RpcCheckedFuture.java
new file mode 100644
index 0000000..7c300d3
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/RpcCheckedFuture.java
@@ -0,0 +1,33 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.rpc;
+
+import com.google.common.util.concurrent.AbstractCheckedFuture;
+import com.google.common.util.concurrent.ListenableFuture;
+
+public class RpcCheckedFuture<T> extends AbstractCheckedFuture<T, RpcException> implements DrillRpcFuture<T>{
+  public RpcCheckedFuture(ListenableFuture<T> delegate) {
+    super(delegate);
+  }
+
+  @Override
+  protected RpcException mapException(Exception e) {
+    return RpcException.mapException(e);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/b8db98ad/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/RpcConnectionHandler.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/RpcConnectionHandler.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/RpcConnectionHandler.java
new file mode 100644
index 0000000..0f55488
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/RpcConnectionHandler.java
@@ -0,0 +1,28 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.rpc;
+
+public interface RpcConnectionHandler<T extends RemoteConnection> {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(RpcConnectionHandler.class);
+  
+  public static enum FailureType{CONNECTION, HANDSHAKE_COMMUNICATION, HANDSHAKE_VALIDATION}
+  
+  public void connectionSucceeded(T connection);
+  public void connectionFailed(FailureType type, Throwable t);
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/b8db98ad/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/RpcException.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/RpcException.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/RpcException.java
index ca66481..500f959 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/RpcException.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/RpcException.java
@@ -17,6 +17,8 @@
  ******************************************************************************/
 package org.apache.drill.exec.rpc;
 
+import java.util.concurrent.ExecutionException;
+
 import org.apache.drill.common.exceptions.DrillIOException;
 
 /**
@@ -41,5 +43,16 @@ public class RpcException extends DrillIOException{
     super(cause);
   }
   
+  public static RpcException mapException(Throwable t){
+    while(t instanceof ExecutionException) t = ((ExecutionException)t).getCause();
+    if(t instanceof RpcException) return ((RpcException) t);
+    return new RpcException(t);
+  }
+  
+  public static RpcException mapException(String message, Throwable t){
+    while(t instanceof ExecutionException) t = ((ExecutionException)t).getCause();
+    return new RpcException(message, t);
+  }
+  
   
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/b8db98ad/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/RpcOutcome.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/RpcOutcome.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/RpcOutcome.java
new file mode 100644
index 0000000..a25e5e7
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/RpcOutcome.java
@@ -0,0 +1,26 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.rpc;
+
+public interface RpcOutcome<T> {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(RpcOutcome.class);
+  
+  public void set(Object value);
+  public void setException(Throwable t);
+  public Class<T> getOutcomeType();
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/b8db98ad/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/RpcOutcomeListener.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/RpcOutcomeListener.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/RpcOutcomeListener.java
index fac908c..771edcf 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/RpcOutcomeListener.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/RpcOutcomeListener.java
@@ -17,11 +17,10 @@
  ******************************************************************************/
 package org.apache.drill.exec.rpc;
 
-public abstract class RpcOutcomeListener<V> {
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(RpcOutcomeListener.class);
+public interface RpcOutcomeListener<V> {
   
-  public void failed(RpcException ex){};
-  public void success(V value){};
+  public void failed(RpcException ex);
+  public void success(V value);
   
   
 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/b8db98ad/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/ZeroCopyProtobufLengthDecoder.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/ZeroCopyProtobufLengthDecoder.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/ZeroCopyProtobufLengthDecoder.java
index 20a7d7d..318abb1 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/ZeroCopyProtobufLengthDecoder.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/ZeroCopyProtobufLengthDecoder.java
@@ -36,7 +36,7 @@ public class ZeroCopyProtobufLengthDecoder extends ByteToMessageDecoder {
   protected void decode(ChannelHandlerContext ctx, ByteBuf in, MessageBuf<Object> out) throws Exception {
 
     if(!ctx.channel().isOpen()){
-      logger.info("Channel is closed, discarding remaining {} byte(s) in buffer.", in.readableBytes());
+      if(in.readableBytes() > 0) logger.info("Channel is closed, discarding remaining {} byte(s) in buffer.", in.readableBytes());
       in.skipBytes(in.readableBytes());
       return;
     }


[30/53] [abbrv] git commit: Merge branch 'master' into execwork. Fix conflicts.

Posted by ja...@apache.org.
Merge branch 'master' into execwork.  Fix conflicts.

Conflicts:
	sandbox/prototype/common/src/test/resources/logback.xml
	sandbox/prototype/pom.xml


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

Branch: refs/heads/master
Commit: 5a5d07f69c17ed4c0f63d17f27cd652830a5d03e
Parents: 1be6bd4 97eb07a
Author: Jacques Nadeau <ja...@apache.org>
Authored: Wed Jun 19 16:30:02 2013 -0700
Committer: Jacques Nadeau <ja...@apache.org>
Committed: Wed Jun 19 16:30:02 2013 -0700

----------------------------------------------------------------------
 .../org/apache/drill/common/JSONOptions.java    |   4 +
 .../drill/common/logical/data/Constant.java     |  52 +++++++++
 .../apache/drill/common/logical/data/Scan.java  |  14 +--
 .../common/logical/data/SourceOperator.java     |   9 ++
 .../data/visitors/AbstractLogicalVisitor.java   |  32 ++++--
 .../logical/data/visitors/LogicalVisitor.java   |  18 +++
 .../apache/drill/exec/ref/rops/ConstantROP.java | 112 +++++++++++++++++++
 .../drill/exec/ref/rse/JSONRecordReader.java    |   2 +-
 .../drill/exec/ref/rops/ConstantROPTest.java    |  95 ++++++++++++++++
 .../exec/ref/src/test/resources/constant.json   |   4 +
 .../exec/ref/src/test/resources/constant2.json  |  42 +++++++
 sandbox/prototype/pom.xml                       |  46 ++++----
 sandbox/prototype/sqlline                       |  16 +++
 13 files changed, 404 insertions(+), 42 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/5a5d07f6/sandbox/prototype/common/src/main/java/org/apache/drill/common/JSONOptions.java
----------------------------------------------------------------------
diff --cc sandbox/prototype/common/src/main/java/org/apache/drill/common/JSONOptions.java
index e309c60,0000000..8a185a4
mode 100644,000000..100644
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/JSONOptions.java
+++ b/sandbox/prototype/common/src/main/java/org/apache/drill/common/JSONOptions.java
@@@ -1,111 -1,0 +1,115 @@@
 +/*******************************************************************************
 + * Licensed to the Apache Software Foundation (ASF) under one
 + * or more contributor license agreements.  See the NOTICE file
 + * distributed with this work for additional information
 + * regarding copyright ownership.  The ASF licenses this file
 + * to you under the Apache License, Version 2.0 (the
 + * "License"); you may not use this file except in compliance
 + * with the License.  You may obtain a copy of the License at
 + * 
 + * http://www.apache.org/licenses/LICENSE-2.0
 + * 
 + * Unless required by applicable law or agreed to in writing, software
 + * distributed under the License is distributed on an "AS IS" BASIS,
 + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 + * See the License for the specific language governing permissions and
 + * limitations under the License.
 + ******************************************************************************/
 +package org.apache.drill.common;
 +
 +import java.io.IOException;
 +import java.util.ArrayList;
 +import java.util.List;
 +
 +import com.fasterxml.jackson.core.*;
 +import com.fasterxml.jackson.core.type.TypeReference;
 +import com.fasterxml.jackson.databind.ObjectMapper;
 +import org.apache.drill.common.JSONOptions.De;
 +import org.apache.drill.common.JSONOptions.Se;
 +import org.apache.drill.common.config.DrillConfig;
 +import org.apache.drill.common.exceptions.LogicalPlanParsingException;
 +import org.slf4j.Logger;
 +import org.slf4j.LoggerFactory;
 +
 +import com.fasterxml.jackson.databind.DeserializationContext;
 +import com.fasterxml.jackson.databind.JsonNode;
 +import com.fasterxml.jackson.databind.SerializerProvider;
 +import com.fasterxml.jackson.databind.annotation.JsonDeserialize;
 +import com.fasterxml.jackson.databind.annotation.JsonSerialize;
 +import com.fasterxml.jackson.databind.deser.std.StdDeserializer;
 +import com.fasterxml.jackson.databind.ser.std.StdSerializer;
 +
 +@JsonSerialize(using = Se.class)
 +@JsonDeserialize(using = De.class)
 +public class JSONOptions {
 +  
 +  final static Logger logger = LoggerFactory.getLogger(JSONOptions.class);
 +  
 +  private JsonNode root;
 +  private JsonLocation location;
 +  
 +  private JSONOptions(JsonNode n, JsonLocation location){
 +    this.root = n;
 +    this.location = location;
 +  }
 +  
 +  public <T> T getWith(DrillConfig config, Class<T> c){
 +    try {
 +      //logger.debug("Read tree {}", root);
 +      return config.getMapper().treeToValue(root, c);
 +    } catch (JsonProcessingException e) {
 +      throw new LogicalPlanParsingException(String.format("Failure while trying to convert late bound json options to type of %s. Reference was originally located at line %d, column %d.", c.getCanonicalName(), location.getLineNr(), location.getColumnNr()), e);
 +    }
 +  }
 +
 +  public <T> T getListWith(DrillConfig config, TypeReference<T> t) throws IOException {
 +      ObjectMapper mapper = config.getMapper();
 +      return mapper.treeAsTokens(root).readValueAs(t);
 +     // return mapper.treeToValue(root,  mapper.getTypeFactory().constructCollectionType(List.class, c));
 +  }
 +  
 +  public JsonNode path(String name){
 +    return root.path(name);
 +  }
++
++  public JsonNode getRoot(){
++      return root;
++  }
 +  
 +  public static class De extends StdDeserializer<JSONOptions> {
 +    
 +    public De() {
 +      super(JSONOptions.class);
 +      logger.debug("Creating Deserializer.");
 +    }
 +
 +    @Override
 +    public JSONOptions deserialize(JsonParser jp, DeserializationContext ctxt) throws IOException,
 +        JsonProcessingException {
 +      JsonLocation l = jp.getTokenLocation();
 +//      logger.debug("Reading tree.");
 +      TreeNode n = jp.readValueAsTree();
 +//      logger.debug("Tree {}", n);
 +      if(n instanceof JsonNode){
 +        return new JSONOptions( (JsonNode) n, l); 
 +      }else{
 +        throw new IllegalArgumentException(String.format("Received something other than a JsonNode %s", n));
 +      }
 +    }
 +
 +  }
 +
 +  public static class Se extends StdSerializer<JSONOptions> {
 +
 +    public Se() {
 +      super(JSONOptions.class);
 +    }
 +
 +    @Override
 +    public void serialize(JSONOptions value, JsonGenerator jgen, SerializerProvider provider) throws IOException,
 +        JsonGenerationException {
 +      jgen.writeTree(value.root);
 +    }
 +
 +  }
 +}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/5a5d07f6/sandbox/prototype/common/src/main/java/org/apache/drill/common/logical/data/Constant.java
----------------------------------------------------------------------
diff --cc sandbox/prototype/common/src/main/java/org/apache/drill/common/logical/data/Constant.java
index 0000000,84d5939..8a0024c
mode 000000,100644..100644
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/logical/data/Constant.java
+++ b/sandbox/prototype/common/src/main/java/org/apache/drill/common/logical/data/Constant.java
@@@ -1,0 -1,44 +1,52 @@@
+ /*******************************************************************************
+  * Licensed to the Apache Software Foundation (ASF) under one
+  * or more contributor license agreements.  See the NOTICE file
+  * distributed with this work for additional information
+  * regarding copyright ownership.  The ASF licenses this file
+  * to you under the Apache License, Version 2.0 (the
+  * "License"); you may not use this file except in compliance
+  * with the License.  You may obtain a copy of the License at
+  *
+  * http://www.apache.org/licenses/LICENSE-2.0
+  *
+  * Unless required by applicable law or agreed to in writing, software
+  * distributed under the License is distributed on an "AS IS" BASIS,
+  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  * See the License for the specific language governing permissions and
+  * limitations under the License.
+  ******************************************************************************/
+ package org.apache.drill.common.logical.data;
+ 
 -import com.google.common.base.Preconditions;
 -import org.apache.drill.common.expression.FieldReference;
 -import org.apache.drill.common.logical.JSONOptions;
++import java.util.Iterator;
++
++import org.apache.drill.common.JSONOptions;
++import org.apache.drill.common.logical.data.visitors.LogicalVisitor;
+ 
+ import com.fasterxml.jackson.annotation.JsonCreator;
+ import com.fasterxml.jackson.annotation.JsonProperty;
+ import com.fasterxml.jackson.annotation.JsonTypeName;
++import com.google.common.base.Preconditions;
+ 
+ @JsonTypeName("constant")
+ public class Constant extends SourceOperator{
+ 
+     private final JSONOptions content;
+ 
+     @JsonCreator
+     public Constant(@JsonProperty("content") JSONOptions content){
+         super();
+         this.content = content;
+         Preconditions.checkNotNull(content, "content attribute is required for source operator 'constant'.");
+     }
+ 
+     public JSONOptions getContent() {
+         return content;
+     }
+ 
++    @Override
++    public <T, X, E extends Throwable> T accept(LogicalVisitor<T, X, E> logicalVisitor, X value) throws E {
++      return logicalVisitor.visitConstant(this, value);
++    }
++
++    
+ }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/5a5d07f6/sandbox/prototype/common/src/main/java/org/apache/drill/common/logical/data/Scan.java
----------------------------------------------------------------------
diff --cc sandbox/prototype/common/src/main/java/org/apache/drill/common/logical/data/Scan.java
index 14f44b4,f22a5bc..d56d3f3
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/logical/data/Scan.java
+++ b/sandbox/prototype/common/src/main/java/org/apache/drill/common/logical/data/Scan.java
@@@ -17,16 -17,12 +17,14 @@@
   ******************************************************************************/
  package org.apache.drill.common.logical.data;
  
- import com.google.common.collect.Iterators;
 +import org.apache.drill.common.JSONOptions;
  import org.apache.drill.common.expression.FieldReference;
 -import org.apache.drill.common.logical.JSONOptions;
  
  import com.fasterxml.jackson.annotation.JsonCreator;
  import com.fasterxml.jackson.annotation.JsonProperty;
  import com.fasterxml.jackson.annotation.JsonTypeName;
 +import org.apache.drill.common.logical.data.visitors.LogicalVisitor;
 +
- import java.util.Iterator;
  
  @JsonTypeName("scan")
  public class Scan extends SourceOperator{
@@@ -55,18 -51,7 +53,14 @@@
    public FieldReference getOutputReference() {
      return outputReference;
    }
 -	
 -  
 +
-     @Override
-     public <T, X, E extends Throwable> T accept(LogicalVisitor<T, X, E> logicalVisitor, X value) throws E {
-         return logicalVisitor.visitScan(this, value);
-     }
++  @Override
++  public <T, X, E extends Throwable> T accept(LogicalVisitor<T, X, E> logicalVisitor, X value) throws E {
++      return logicalVisitor.visitScan(this, value);
++  }
 +
-     @Override
-     public Iterator<LogicalOperator> iterator() {
-         return Iterators.emptyIterator();
-     }
 +
 +
 +
  
  }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/5a5d07f6/sandbox/prototype/common/src/main/java/org/apache/drill/common/logical/data/SourceOperator.java
----------------------------------------------------------------------
diff --cc sandbox/prototype/common/src/main/java/org/apache/drill/common/logical/data/SourceOperator.java
index 73373e9,73373e9..2ad94a3
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/logical/data/SourceOperator.java
+++ b/sandbox/prototype/common/src/main/java/org/apache/drill/common/logical/data/SourceOperator.java
@@@ -17,10 -17,10 +17,19 @@@
   ******************************************************************************/
  package org.apache.drill.common.logical.data;
  
++import java.util.Iterator;
++
++import com.google.common.collect.Iterators;
++
  
  /**
   * An operator that produces data without any parents.  (zero input operator)
   */
  public abstract class SourceOperator extends LogicalOperatorBase{
  
++  @Override
++  public Iterator<LogicalOperator> iterator() {
++      return Iterators.emptyIterator();
++  }
++  
  }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/5a5d07f6/sandbox/prototype/common/src/main/java/org/apache/drill/common/logical/data/visitors/AbstractLogicalVisitor.java
----------------------------------------------------------------------
diff --cc sandbox/prototype/common/src/main/java/org/apache/drill/common/logical/data/visitors/AbstractLogicalVisitor.java
index a293d0b,0000000..d8a1378
mode 100644,000000..100644
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/logical/data/visitors/AbstractLogicalVisitor.java
+++ b/sandbox/prototype/common/src/main/java/org/apache/drill/common/logical/data/visitors/AbstractLogicalVisitor.java
@@@ -1,94 -1,0 +1,112 @@@
++/*******************************************************************************
++ * Licensed to the Apache Software Foundation (ASF) under one
++ * or more contributor license agreements.  See the NOTICE file
++ * distributed with this work for additional information
++ * regarding copyright ownership.  The ASF licenses this file
++ * to you under the Apache License, Version 2.0 (the
++ * "License"); you may not use this file except in compliance
++ * with the License.  You may obtain a copy of the License at
++ * 
++ * http://www.apache.org/licenses/LICENSE-2.0
++ * 
++ * Unless required by applicable law or agreed to in writing, software
++ * distributed under the License is distributed on an "AS IS" BASIS,
++ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
++ * See the License for the specific language governing permissions and
++ * limitations under the License.
++ ******************************************************************************/
 +package org.apache.drill.common.logical.data.visitors;
 +
 +import org.apache.drill.common.logical.data.*;
 +
- /**
-  * Created with IntelliJ IDEA.
-  * User: jaltekruse
-  * Date: 6/10/13
-  * Time: 1:55 PM
-  * To change this template use File | Settings | File Templates.
-  */
++
 +public abstract class AbstractLogicalVisitor<T, X, E extends Throwable> implements LogicalVisitor<T, X, E> {
 +
 +    public T visitOp(LogicalOperator op, X value) throws E{
 +        throw new UnsupportedOperationException(String.format(
 +                "The LogicalVisitor of type %s does not currently support visiting the PhysicalOperator type %s.", this
 +                .getClass().getCanonicalName(), op.getClass().getCanonicalName()));
 +    }
 +
 +    @Override
 +    public T visitScan(Scan scan, X value) throws E {
 +        return visitOp(scan, value);
 +    }
 +
 +    @Override
 +    public T visitStore(Store store, X value) throws E {
 +        return visitOp(store, value);
 +    }
 +
 +    @Override
 +    public T visitFilter(Filter filter, X value) throws E {
 +        return visitOp(filter, value);
 +    }
 +
 +    @Override
 +    public T visitFlatten(Flatten flatten, X value) throws E {
 +        return visitOp(flatten, value);
 +    }
 +
 +    @Override
 +    public T visitProject(Project project, X value) throws E {
 +        return visitOp(project, value);
 +    }
 +
 +    @Override
 +    public T visitOrder(Order order, X value) throws E {
 +        return visitOp(order, value);
 +    }
 +
 +    @Override
 +    public T visitJoin(Join join, X value) throws E {
 +        return visitOp(join, value);
 +    }
 +
 +    @Override
 +    public T visitLimit(Limit limit, X value) throws E {
 +        return visitOp(limit, value);
 +    }
 +
 +    @Override
 +    public T visitRunningAggregate(RunningAggregate runningAggregate, X value) throws E {
 +        return visitOp(runningAggregate, value);
 +    }
 +
 +    @Override
 +    public T visitSegment(Segment segment, X value) throws E {
 +        return visitOp(segment, value);
 +    }
 +
 +    @Override
 +    public T visitSequence(Sequence sequence, X value) throws E {
 +        return visitOp(sequence, value);
 +    }
 +
 +    @Override
 +    public T visitTransform(Transform transform, X value) throws E {
 +        return visitOp(transform, value);
 +    }
 +
 +    @Override
 +    public T visitUnion(Union union, X value) throws E {
 +        return visitOp(union, value);
 +    }
 +
 +    @Override
 +    public T visitCollapsingAggregate(CollapsingAggregate collapsingAggregate, X value) throws E {
 +        return visitOp(collapsingAggregate, value);
 +    }
 +
 +    @Override
 +    public T visitWindowFrame(WindowFrame windowFrame, X value) throws E {
 +        return visitOp(windowFrame, value);
 +    }
++
++    @Override
++    public T visitConstant(Constant constant, X value) throws E {
++       return visitOp(constant, value);
++    }
++    
++    
 +}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/5a5d07f6/sandbox/prototype/common/src/main/java/org/apache/drill/common/logical/data/visitors/LogicalVisitor.java
----------------------------------------------------------------------
diff --cc sandbox/prototype/common/src/main/java/org/apache/drill/common/logical/data/visitors/LogicalVisitor.java
index 1f08913,0000000..10e6ec2
mode 100644,000000..100644
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/logical/data/visitors/LogicalVisitor.java
+++ b/sandbox/prototype/common/src/main/java/org/apache/drill/common/logical/data/visitors/LogicalVisitor.java
@@@ -1,35 -1,0 +1,53 @@@
++/*******************************************************************************
++ * Licensed to the Apache Software Foundation (ASF) under one
++ * or more contributor license agreements.  See the NOTICE file
++ * distributed with this work for additional information
++ * regarding copyright ownership.  The ASF licenses this file
++ * to you under the Apache License, Version 2.0 (the
++ * "License"); you may not use this file except in compliance
++ * with the License.  You may obtain a copy of the License at
++ * 
++ * http://www.apache.org/licenses/LICENSE-2.0
++ * 
++ * Unless required by applicable law or agreed to in writing, software
++ * distributed under the License is distributed on an "AS IS" BASIS,
++ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
++ * See the License for the specific language governing permissions and
++ * limitations under the License.
++ ******************************************************************************/
 +package org.apache.drill.common.logical.data.visitors;
 +
 +
 +import org.apache.drill.common.graph.GraphVisitor;
 +import org.apache.drill.common.logical.data.*;
 +
 +/**
 + * Visitor class designed to traversal of a operator tree.  Basis for a number of operator manipulations including fragmentation and materialization.
 + * @param <RETURN> The class associated with the return of each visit method.
 + * @param <EXTRA> The class object associated with additional data required for a particular operator modification.
 + * @param <EXCEP> An optional exception class that can be thrown when a portion of a modification or traversal fails.  Must extend Throwable.  In the case where the visitor does not throw any caught exception, this can be set as RuntimeException.
 + */
 +public interface LogicalVisitor<RETURN, EXTRA, EXCEP extends Throwable> {
 +    static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(LogicalVisitor.class);
 +
 +
 +    //public RETURN visitExchange(Exchange exchange, EXTRA value) throws EXCEP;
 +    public RETURN visitScan(Scan scan, EXTRA value) throws EXCEP;
 +    public RETURN visitStore(Store store, EXTRA value) throws EXCEP;
 +    public RETURN visitCollapsingAggregate(CollapsingAggregate collapsingAggregate, EXTRA value) throws EXCEP;
 +
 +    public RETURN visitFilter(Filter filter, EXTRA value) throws EXCEP;
 +    public RETURN visitFlatten(Flatten flatten, EXTRA value) throws EXCEP;
 +
 +    public RETURN visitProject(Project project, EXTRA value) throws EXCEP;
++    public RETURN visitConstant(Constant constant, EXTRA value) throws EXCEP;
 +    public RETURN visitOrder(Order order, EXTRA value) throws EXCEP;
 +    public RETURN visitJoin(Join join, EXTRA value) throws EXCEP;
 +    public RETURN visitLimit(Limit limit, EXTRA value) throws EXCEP;
 +    public RETURN visitRunningAggregate(RunningAggregate runningAggregate, EXTRA value) throws EXCEP;
 +    public RETURN visitSegment(Segment segment, EXTRA value) throws EXCEP;
 +    public RETURN visitSequence(Sequence sequence, EXTRA value) throws EXCEP;
 +    public RETURN visitTransform(Transform transform, EXTRA value) throws EXCEP;
 +    public RETURN visitUnion(Union union, EXTRA value) throws EXCEP;
 +    public RETURN visitWindowFrame(WindowFrame windowFrame, EXTRA value) throws EXCEP;
 +}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/5a5d07f6/sandbox/prototype/exec/ref/src/test/java/org/apache/drill/exec/ref/rops/ConstantROPTest.java
----------------------------------------------------------------------
diff --cc sandbox/prototype/exec/ref/src/test/java/org/apache/drill/exec/ref/rops/ConstantROPTest.java
index 0000000,c7d4719..499b335
mode 000000,100644..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,0 -1,97 +1,95 @@@
+ 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 org.apache.drill.common.config.DrillConfig;
+ 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.eval.BasicEvaluatorFactory;
+ import org.apache.drill.exec.ref.rse.RSERegistry;
+ import org.apache.drill.exec.ref.values.ScalarValues;
+ import org.junit.Test;
+ 
+ 
+ import java.io.File;
+ import java.util.Collection;
+ import java.util.Iterator;
+ 
+ /**
+  * Created with IntelliJ IDEA.
+  * User: jaltekruse
+  * Date: 6/4/13
+  * Time: 4:15 PM
+  * To change this template use File | Settings | File Templates.
+  */
+ public class ConstantROPTest {
+ 
+     @Test
+     public void testConstant(){
+ 
+         ObjectMapper map = DrillConfig.create().getMapper();
+         Constant con;
+         try{
+             con = map.readValue( FileUtils.getResourceAsString("/constant.json"), Constant.class);
+             if (con == null){
+                 System.out.println("constant is null");
+             }
+             System.out.println(con);
+ 
+             ConstantROP rop = new ConstantROP(con);
+ 
+             rop.setupIterators(new IteratorRegistry());
+             RecordIterator iter = rop.getIteratorInternal();
+             RecordPointer ptr = iter.getRecordPointer();
+ 
+             int i = 1;
+             while (iter.next() != RecordIterator.NextOutcome.NONE_LEFT){
+                 System.out.println(ptr);
+                 org.junit.Assert.assertEquals("Integer value in record " + i + " is incorrect.",
+                         ptr.getField(new SchemaPath("c1")), new ScalarValues.IntegerScalar(i));
+                 org.junit.Assert.assertEquals("String value in record " + i + " is incorrect.",
+                         ptr.getField(new SchemaPath("c2")), new ScalarValues.StringScalar("string " + i));
+                 i++;
+             }
+             org.junit.Assert.assertEquals("Incorrect number of records returned by 'constant' record iterator.", 3, i - 1);
+         } catch (Exception ex){ ex.printStackTrace(); }
+         System.out.println("end test");
+     }
+ 
+     // 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{
+             DrillConfig config = DrillConfig.create();
+             final String jsonFile = "/constant2.json";
+             LogicalPlan plan = LogicalPlan.parse(config, FileUtils.getResourceAsString(jsonFile));
+             org.junit.Assert.assertEquals("Constant operator not read in properly or not recognized as a source operator.",
 -                    plan.getGraph().getSources().toString(), "[Constant [memo=null]]");
++                    plan.getGraph().getLeaves().toString(), "[Constant [memo=null]]");
+ 
+             org.junit.Assert.assertEquals("Edge between constant operator and sink not recognized.",
 -                    plan.getGraph().getSinks().toString(), "[Store [memo=output sink]]");
 -
 -            org.junit.Assert.assertEquals("Constant operator not read in properly or not recognized as a sink operator.",
 -                    plan.getGraph().getAdjList().getAllEdges().toString(), "[Edge [from=Node [val=Constant [memo=null]], to=Node [val=Store [memo=output sink]]]]");
++                    plan.getGraph().getRoots().toString(), "[Store [memo=output sink]]");
+ 
++            
+             IteratorRegistry ir = new IteratorRegistry();
+             ReferenceInterpreter i = new ReferenceInterpreter(plan, ir, new BasicEvaluatorFactory(ir), new RSERegistry(config));
+             i.setup();
+             Collection<RunOutcome> outcomes = i.run();
+ 
+             for(RunOutcome outcome : outcomes){
+                 System.out.println("============");
+                 System.out.println(outcome);
+                 if(outcome.outcome == RunOutcome.OutcomeType.FAILED && outcome.exception != null){
+                     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/5a5d07f6/sandbox/prototype/pom.xml
----------------------------------------------------------------------


[07/53] [abbrv] Update typing system. Update RPC system. Add Fragmenting Implementation. Working single node. Distributed failing due to threading issues.

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/pom.xml
----------------------------------------------------------------------
diff --git a/sandbox/prototype/pom.xml b/sandbox/prototype/pom.xml
index fa89417..25f156d 100644
--- a/sandbox/prototype/pom.xml
+++ b/sandbox/prototype/pom.xml
@@ -137,7 +137,7 @@
 		<dependency>
 			<groupId>de.huxhorn.lilith</groupId>
 			<artifactId>de.huxhorn.lilith.logback.appender.multiplex-classic</artifactId>
-			<version>0.9.42.1</version>
+			<version>0.9.43</version>
 			<scope>test</scope>
 		</dependency>
 


[31/53] [abbrv] Create new generated value vectors utilizing fmpp. Includes: - First pass; integrate build system and some cursory implementations - starting to split common logic into base class - implement most of varlen value vector functionality, mi

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/7075cca1/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/Fixed4.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/Fixed4.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/Fixed4.java
deleted file mode 100644
index ff54784..0000000
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/Fixed4.java
+++ /dev/null
@@ -1,59 +0,0 @@
-/*******************************************************************************
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- ******************************************************************************/
-package org.apache.drill.exec.record.vector;
-
-import org.apache.drill.exec.memory.BufferAllocator;
-import org.apache.drill.exec.proto.SchemaDefProtos;
-import org.apache.drill.exec.record.MaterializedField;
-
-public class Fixed4 extends AbstractFixedValueVector<Fixed4>{
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(Fixed4.class);
-
-  public Fixed4(MaterializedField field, BufferAllocator allocator) {
-    super(field, allocator, 4*8);
-  }
-
-  public final void setInt(int index, int value){
-    index*=4;
-    data.setInt(index, value);
-  }
-  
-  public final int getInt(int index){
-    index*=4;
-    return data.getInt(index);
-  }
-  
-  public final void setFloat4(int index, float value){
-    index*=4;
-    data.setFloat(index, value);
-  }
-  
-  public final float getFloat4(int index){
-    index*=4;
-    return data.getFloat(index);
-  }
-  
-  @Override
-  public Object getObject(int index) {
-    if (field != null && field.getType().getMinorType() == SchemaDefProtos.MinorType.FLOAT4) {
-      return getFloat4(index);
-    } else {
-      return getInt(index);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/7075cca1/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/Fixed8.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/Fixed8.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/Fixed8.java
deleted file mode 100644
index 3629f5c..0000000
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/Fixed8.java
+++ /dev/null
@@ -1,58 +0,0 @@
-/*******************************************************************************
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- ******************************************************************************/
-package org.apache.drill.exec.record.vector;
-
-import org.apache.drill.exec.memory.BufferAllocator;
-import org.apache.drill.exec.record.MaterializedField;
-
-public class Fixed8 extends AbstractFixedValueVector<Fixed8>{
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(Fixed8.class);
-
-  public Fixed8(MaterializedField field, BufferAllocator allocator) {
-    super(field, allocator, 8*8);
-  }
-
-  public final void setBigInt(int index, long value){
-    index*=8;
-    data.setLong(index, value);
-  }
-  
-  public final long getBigInt(int index){
-    index*=8;
-    return data.getLong(index);
-  }
-  
-  public final void setFloat8(int index, double value){
-    index*=8;
-    data.setDouble(index, value);
-  }
-  
-  public final double getFloat8(int index){
-    index*=8;
-    return data.getDouble(index);
-  }
-
-  @Override
-  public Object getObject(int index) {
-    return getBigInt(index);
-  }
-  
-  
-  
-  
-}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/7075cca1/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/FixedLen.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/FixedLen.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/FixedLen.java
deleted file mode 100644
index 594af23..0000000
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/FixedLen.java
+++ /dev/null
@@ -1,45 +0,0 @@
-/*******************************************************************************
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- ******************************************************************************/
-package org.apache.drill.exec.record.vector;
-
-import io.netty.buffer.ByteBuf;
-
-import org.apache.drill.exec.memory.BufferAllocator;
-import org.apache.drill.exec.record.MaterializedField;
-
-public class FixedLen extends AbstractFixedValueVector<FixedLen>{
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(FixedLen.class);
-
-  
-  public FixedLen(MaterializedField field, BufferAllocator allocator) {
-    super(field, allocator, field.getWidth());
-  }
-
-  public void set(ByteBuf b){
-     
-  }
-  
-  public void get(ByteBuf b){
-    
-  }
-
-  @Override
-  public Object getObject(int index) {
-    return null;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/7075cca1/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/NullValueException.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/NullValueException.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/NullValueException.java
new file mode 100644
index 0000000..027b698
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/NullValueException.java
@@ -0,0 +1,9 @@
+package org.apache.drill.exec.record.vector;
+
+import org.apache.drill.common.exceptions.DrillRuntimeException;
+
+public class NullValueException extends DrillRuntimeException {
+  public NullValueException(int index) {
+    super("Element at index position: " + index + " is null");
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/7075cca1/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/NullableBit.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/NullableBit.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/NullableBit.java
deleted file mode 100644
index ac8c6b6..0000000
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/NullableBit.java
+++ /dev/null
@@ -1,20 +0,0 @@
-package org.apache.drill.exec.record.vector;
-
-import org.apache.drill.exec.memory.BufferAllocator;
-import org.apache.drill.exec.record.MaterializedField;
-
-public class NullableBit extends NullableValueVector<NullableBit, Bit>{
-    public NullableBit(MaterializedField field, BufferAllocator allocator) {
-        super(field, allocator);
-    }
-
-    @Override
-    protected Bit getNewValueVector(BufferAllocator allocator) {
-        return new Bit(this.field, allocator);
-    }
-
-    public void set(int index) {
-        setNotNull(index);
-        value.set(index);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/7075cca1/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/NullableFixed4.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/NullableFixed4.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/NullableFixed4.java
deleted file mode 100644
index d09b04a..0000000
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/NullableFixed4.java
+++ /dev/null
@@ -1,44 +0,0 @@
-/*******************************************************************************
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- ******************************************************************************/
-package org.apache.drill.exec.record.vector;
-
-import org.apache.drill.exec.memory.BufferAllocator;
-import org.apache.drill.exec.record.MaterializedField;
-
-public final class NullableFixed4 extends NullableValueVector<NullableFixed4, Fixed4>{
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(NullableFixed4.class);
-
-  public NullableFixed4(MaterializedField field, BufferAllocator allocator) {
-    super(field, allocator);
-  }
-
-  @Override
-  protected Fixed4 getNewValueVector(BufferAllocator allocator) {
-    return new Fixed4(this.field, allocator);
-  }
-
-  public void setInt(int index, int newVal) {
-      setNotNull(index);
-      value.setInt(index, newVal);
-  }
-
-  public void setFloat4(int index, float val) {
-      setNotNull(index);
-      value.setFloat4(index, val);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/7075cca1/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/NullableValueVector.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/NullableValueVector.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/NullableValueVector.java
deleted file mode 100644
index 8c3d7f6..0000000
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/NullableValueVector.java
+++ /dev/null
@@ -1,108 +0,0 @@
-/*******************************************************************************
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- ******************************************************************************/
-package org.apache.drill.exec.record.vector;
-
-import io.netty.buffer.ByteBuf;
-
-import org.apache.drill.exec.memory.BufferAllocator;
-import org.apache.drill.exec.proto.UserBitShared.FieldMetadata;
-import org.apache.drill.exec.record.MaterializedField;
-
-/**
- * Abstract class supports null versions.
- */
-abstract class NullableValueVector<T extends NullableValueVector<T, E>, E extends BaseValueVector<E>> extends BaseValueVector<T> {
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(NullableValueVector.class);
-
-  protected Bit bits;
-  protected E value;
-
-  public NullableValueVector(MaterializedField field, BufferAllocator allocator) {
-    super(field, allocator);
-    bits = new Bit(null, allocator);
-    value = getNewValueVector(allocator);
-  }
-  
-  protected abstract E getNewValueVector(BufferAllocator allocator);
-
-  public void setNull(int index) {
-      bits.set(index);
-  }
-
-  public void setNotNull(int index) {
-      bits.clear(index);
-  }
-
-  public int isNull(int index){
-    return bits.getBit(index);
-  }
-  
-  @Override
-  protected int getAllocationSize(int valueCount) {
-    return bits.getAllocationSize(valueCount) + value.getAllocationSize(valueCount);
-  }
-  
-  @Override
-  public MaterializedField getField() {
-    return field;
-  }
-
-  @Override
-  protected void childResetAllocation(int valueCount, ByteBuf buf) {
-    int firstSize = bits.getAllocationSize(valueCount);
-    value.resetAllocation(valueCount, buf.slice(firstSize, value.getAllocationSize(valueCount)));
-    bits.resetAllocation(valueCount, buf.slice(0, firstSize));
-    bits.setAllFalse();
-  }
-
-  @Override
-  protected void childCloneMetadata(T other) {
-    bits.cloneMetadata(other.bits);
-    value.cloneInto(value);
-  }
-
-  @Override
-  protected void childClear() {
-    bits.clear();
-    value.clear();
-  }
-
-  
-  @Override
-  public ByteBuf[] getBuffers() {
-    return new ByteBuf[]{bits.data, value.data};
-  }
-
-  @Override
-  public void setRecordCount(int recordCount) {
-    super.setRecordCount(recordCount);
-    bits.setRecordCount(recordCount);
-    value.setRecordCount(recordCount);
-  }
-
-  @Override
-  public Object getObject(int index) {
-    if(isNull(index) == 1){
-      return null;
-    }else{
-      return value.getObject(index);
-    }
-  }
-
-}
-

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/7075cca1/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/NullableVarLen4.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/NullableVarLen4.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/NullableVarLen4.java
deleted file mode 100644
index f5a2770..0000000
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/NullableVarLen4.java
+++ /dev/null
@@ -1,21 +0,0 @@
-package org.apache.drill.exec.record.vector;
-
-import org.apache.drill.exec.memory.BufferAllocator;
-import org.apache.drill.exec.record.MaterializedField;
-
-public class NullableVarLen4 extends NullableValueVector<NullableVarLen4, VarLen4> {
-
-    public NullableVarLen4(MaterializedField field, BufferAllocator allocator) {
-        super(field, allocator);
-    }
-
-    @Override
-    protected VarLen4 getNewValueVector(BufferAllocator allocator) {
-        return new VarLen4(field, allocator);
-    }
-
-    public void setBytes(int index, byte[] bytes) {
-        setNotNull(index);
-        value.setBytes(index, bytes);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/7075cca1/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/RepeatMap.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/RepeatMap.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/RepeatMap.java
deleted file mode 100644
index 2c08551..0000000
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/RepeatMap.java
+++ /dev/null
@@ -1,57 +0,0 @@
-/*******************************************************************************
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- ******************************************************************************/
-package org.apache.drill.exec.record.vector;
-
-import io.netty.buffer.ByteBuf;
-
-import org.apache.drill.exec.memory.BufferAllocator;
-import org.apache.drill.exec.record.MaterializedField;
-
-public class RepeatMap extends BaseValueVector<RepeatMap>{
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(RepeatMap.class);
-
-  
-  public RepeatMap(MaterializedField field, BufferAllocator allocator) {
-    super(field, allocator);
-  }
-
-  @Override
-  protected int getAllocationSize(int valueCount) {
-    return 4 * valueCount;
-  }
-
-  @Override
-  protected void childResetAllocation(int valueCount, ByteBuf buf) {
-  }
-
-  @Override
-  protected void childCloneMetadata(RepeatMap other) {
-  }
-
-  @Override
-  protected void childClear() {
-  }
-
-  @Override
-  public Object getObject(int index) {
-    return null;
-  }
-
-  
-  
-}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/7075cca1/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/SelectionVector.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/SelectionVector.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/SelectionVector.java
index 323b55f..e79a525 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/SelectionVector.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/SelectionVector.java
@@ -25,7 +25,7 @@ import org.apache.drill.exec.record.MaterializedField;
 /**
  * Convenience/Clarification Fixed2 wrapper.
  */
-public class SelectionVector extends Fixed2{
+public class SelectionVector extends ValueVector.UInt2 {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(SelectionVector.class);
 
   public SelectionVector(MaterializedField field, BufferAllocator allocator) {

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/7075cca1/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/TypeHelper.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/TypeHelper.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/TypeHelper.java
deleted file mode 100644
index 543a222..0000000
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/TypeHelper.java
+++ /dev/null
@@ -1,259 +0,0 @@
-/*******************************************************************************
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- ******************************************************************************/
-package org.apache.drill.exec.record.vector;
-
-import org.apache.drill.exec.memory.BufferAllocator;
-import org.apache.drill.exec.proto.SchemaDefProtos.DataMode;
-import org.apache.drill.exec.proto.SchemaDefProtos.MajorType;
-import org.apache.drill.exec.proto.SchemaDefProtos.MinorType;
-import org.apache.drill.exec.record.MaterializedField;
-
-public class TypeHelper {
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TypeHelper.class);
-
-  private static final int WIDTH_ESTIMATE_1 = 10;
-  private static final int WIDTH_ESTIMATE_2 = 50000;
-  private static final int WIDTH_ESTIMATE_4 = 1024*1024;
-
-  public static int getSize(MajorType major){
-    switch(major.getMinorType()){
-    case TINYINT: return 1;
-    case SMALLINT: return 2;
-    case INT: return 4;
-    case BIGINT: return 8;
-    case DECIMAL4: return 4;
-    case DECIMAL8: return 8;
-    case DECIMAL12: return 12;
-    case DECIMAL16: return 16;
-    case MONEY: return 8;
-    case DATE: return 4;
-    case TIME: return 8;
-    case TIMETZ: return 12;
-    case TIMESTAMP: return 8;
-    case DATETIME: return 8;
-    case INTERVAL: return 12;
-    case FLOAT4: return 4;
-    case FLOAT8: return 8;
-    case BOOLEAN: return 1/8;
-    case FIXEDCHAR: return major.getWidth();
-    case VARCHAR1: return 1 + WIDTH_ESTIMATE_1;
-    case VARCHAR2: return 2 + WIDTH_ESTIMATE_2;
-    case VARCHAR4: return 4 + WIDTH_ESTIMATE_4;
-    case FIXEDBINARY: return major.getWidth();
-    case VARBINARY1: return 1 + WIDTH_ESTIMATE_1;
-    case VARBINARY2: return 2 + WIDTH_ESTIMATE_2;
-    case VARBINARY4: return 4 + WIDTH_ESTIMATE_4;
-    case UINT1: return 1;
-    case UINT2: return 2;
-    case UINT4: return 4;
-    case UINT8: return 8;
-    case PROTO2: return 2 + WIDTH_ESTIMATE_2;
-    case PROTO4: return 4 + WIDTH_ESTIMATE_4;
-    case MSGPACK2: return 2 + WIDTH_ESTIMATE_2;
-    case MSGPACK4: return 4 + WIDTH_ESTIMATE_4;
-    }
-    return 4;
-  }
-
-  public static Class<?> getValueVectorClass(MinorType type, DataMode mode){
-    switch(mode){
-    case OPTIONAL:
-      switch(type){
-        case REPEATMAP: return RepeatMap.class;
-        case TINYINT: return Fixed1.class;
-        case SMALLINT: return Fixed2.class;
-        case INT: return Fixed4.class;
-        case BIGINT: return Fixed8.class;
-        case DECIMAL4: return Fixed4.class;
-        case DECIMAL8: return Fixed8.class;
-        case DECIMAL12: return Fixed12.class;
-        case DECIMAL16: return Fixed16.class;
-        case MONEY: return Fixed8.class;
-        case DATE: return Fixed4.class;
-        case TIME: return Fixed8.class;
-        case TIMETZ: return Fixed12.class;
-        case TIMESTAMP: return Fixed8.class;
-        case DATETIME: return Fixed8.class;
-        case INTERVAL: return Fixed12.class;
-        case FLOAT4: return Fixed4.class;
-        case FLOAT8: return Fixed8.class;
-        case BOOLEAN: return Bit.class;
-        case FIXEDCHAR: return FixedLen.class;
-        case VARCHAR1: return VarLen1.class;
-        case VARCHAR2: return VarLen2.class;
-        case VARCHAR4: return VarLen4.class;
-        case FIXEDBINARY: return FixedLen.class;
-        case VARBINARY1: return VarLen1.class;
-        case VARBINARY2: return VarLen2.class;
-        case VARBINARY4: return VarLen4.class;
-        case UINT1: return Fixed1.class;
-        case UINT2: return Fixed2.class;
-        case UINT4: return Fixed4.class;
-        case UINT8: return Fixed8.class;
-        case PROTO2: return VarLen2.class;
-        case PROTO4: return VarLen4.class;
-        case MSGPACK2: return VarLen2.class;
-        case MSGPACK4: return VarLen4.class;
-      }
-      break;
-    case REQUIRED:
-      switch(type){
-//        case TINYINT: return NullableFixed1.class;
-//        case SMALLINT: return NullableFixed2.class;
-//        case INT: return NullableFixed4.class;
-//        case BIGINT: return NullableFixed8.class;
-//        case DECIMAL4: return NullableFixed4.class;
-//        case DECIMAL8: return NullableFixed8.class;
-//        case DECIMAL12: return NullableFixed12.class;
-//        case DECIMAL16: return NullableFixed16.class;
-//        case MONEY: return NullableFixed8.class;
-//        case DATE: return NullableFixed4.class;
-//        case TIME: return NullableFixed8.class;
-//        case TIMETZ: return NullableFixed12.class;
-//        case TIMESTAMP: return NullableFixed8.class;
-//        case DATETIME: return NullableFixed8.class;
-//        case INTERVAL: return NullableFixed12.class;
-//        case FLOAT4: return NullableFixed4.class;
-//        case FLOAT8: return NullableFixed8.class;
-//        case BOOLEAN: return NullableBit.class;
-//        case FIXEDCHAR: return NullableFixedLen.class;
-//        case VARCHAR1: return NullableVarLen1.class;
-//        case VARCHAR2: return NullableVarLen2.class;
-//        case VARCHAR4: return NullableVarLen4.class;
-//        case FIXEDBINARY: return NullableFixedLen.class;
-//        case VARBINARY1: return NullableVarLen1.class;
-//        case VARBINARY2: return NullableVarLen2.class;
-//        case VARBINARY4: return NullableVarLen4.class;
-//        case UINT1: return NullableFixed1.class;
-//        case UINT2: return NullableFixed2.class;
-//        case UINT4: return NullableFixed4.class;
-//        case UINT8: return NullableFixed8.class;
-//        case PROTO2: return NullableVarLen2.class;
-//        case PROTO4: return NullableVarLen4.class;
-//        case MSGPACK2: return NullableVarLen2.class;
-//        case MSGPACK4: return NullableVarLen4.class;      
-      }
-      break;
-    case REPEATED:
-      switch(type){
-//        case TINYINT: return RepeatedFixed1.class;
-//        case SMALLINT: return RepeatedFixed2.class;
-//        case INT: return RepeatedFixed4.class;
-//        case BIGINT: return RepeatedFixed8.class;
-//        case DECIMAL4: return RepeatedFixed4.class;
-//        case DECIMAL8: return RepeatedFixed8.class;
-//        case DECIMAL12: return RepeatedFixed12.class;
-//        case DECIMAL16: return RepeatedFixed16.class;
-//        case MONEY: return RepeatedFixed8.class;
-//        case DATE: return RepeatedFixed4.class;
-//        case TIME: return RepeatedFixed8.class;
-//        case TIMETZ: return RepeatedFixed12.class;
-//        case TIMESTAMP: return RepeatedFixed8.class;
-//        case DATETIME: return RepeatedFixed8.class;
-//        case INTERVAL: return RepeatedFixed12.class;
-//        case FLOAT4: return RepeatedFixed4.class;
-//        case FLOAT8: return RepeatedFixed8.class;
-//        case BOOLEAN: return RepeatedBit.class;
-//        case FIXEDCHAR: return RepeatedFixedLen.class;
-//        case VARCHAR1: return RepeatedVarLen1.class;
-//        case VARCHAR2: return RepeatedVarLen2.class;
-//        case VARCHAR4: return RepeatedVarLen4.class;
-//        case FIXEDBINARY: return RepeatedFixedLen.class;
-//        case VARBINARY1: return RepeatedVarLen1.class;
-//        case VARBINARY2: return RepeatedVarLen2.class;
-//        case VARBINARY4: return RepeatedVarLen4.class;
-//        case UINT1: return RepeatedFixed1.class;
-//        case UINT2: return RepeatedFixed2.class;
-//        case UINT4: return RepeatedFixed4.class;
-//        case UINT8: return RepeatedFixed8.class;
-//        case PROTO2: return RepeatedVarLen2.class;
-//        case PROTO4: return RepeatedVarLen4.class;
-//        case MSGPACK2: return RepeatedVarLen2.class;
-//        case MSGPACK4: return RepeatedVarLen4.class;      
-      }
-      break;
-    default:
-      break;
-
-    }
-    throw new UnsupportedOperationException();
-  }
-
-
-  public static ValueVector<?> getNewVector(MaterializedField field, BufferAllocator allocator){
-    MajorType type = field.getType();
-    switch(type.getMode()){
-    case REQUIRED:
-      switch(type.getMinorType()){
-      case TINYINT: return new Fixed1(field, allocator);
-      case SMALLINT: return new Fixed2(field, allocator);
-      case INT: return new Fixed4(field, allocator);
-      case BIGINT: return new Fixed8(field, allocator);
-      case DECIMAL4: return new Fixed4(field, allocator);
-      case DECIMAL8: return new Fixed8(field, allocator);
-      case DECIMAL12: return new Fixed12(field, allocator);
-      case DECIMAL16: return new Fixed16(field, allocator);
-      case MONEY: return new Fixed8(field, allocator);
-      case DATE: return new Fixed4(field, allocator);
-      case TIME: return new Fixed8(field, allocator);
-      case TIMETZ: return new Fixed12(field, allocator);
-      case TIMESTAMP: return new Fixed8(field, allocator);
-      case DATETIME: return new Fixed8(field, allocator);
-      case INTERVAL: return new Fixed12(field, allocator);
-      case FLOAT4: return new Fixed4(field, allocator);
-      case FLOAT8: return new Fixed8(field, allocator);
-      case BOOLEAN: return new Bit(field, allocator);
-      case FIXEDCHAR: return new FixedLen(field, allocator);
-      case VARCHAR1: return new VarLen1(field, allocator);
-      case VARCHAR2: return new VarLen2(field, allocator);
-      case VARCHAR4: return new VarLen4(field, allocator);
-      case FIXEDBINARY: return new FixedLen(field, allocator);
-      case VARBINARY1: return new VarLen1(field, allocator);
-      case VARBINARY2: return new VarLen2(field, allocator);
-      case VARBINARY4: return new VarLen4(field, allocator);
-      case UINT1: return new Fixed1(field, allocator);
-      case UINT2: return new Fixed2(field, allocator);
-      case UINT4: return new Fixed4(field, allocator);
-      case UINT8: return new Fixed8(field, allocator);
-      case PROTO2: return new VarLen2(field, allocator);
-      case PROTO4: return new VarLen4(field, allocator);
-      case MSGPACK2: return new VarLen2(field, allocator);
-      case MSGPACK4: return new VarLen4(field, allocator);
-      }
-      break;
-    case REPEATED:
-        switch(type.getMinorType()) {
-            case MAP: return new RepeatMap(field, allocator);
-        }
-      break;
-    case OPTIONAL:
-        switch(type.getMinorType()) {
-            case BOOLEAN: return new NullableBit(field, allocator);
-            case INT: return new NullableFixed4(field, allocator);
-            case FLOAT4: return new NullableFixed4(field, allocator);
-            case VARCHAR4: return new NullableVarLen4(field, allocator);
-        }
-      break;
-    default:
-      break;
-
-    }
-    throw new UnsupportedOperationException(type.getMinorType() + " type is not supported. Mode: " + type.getMode());
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/7075cca1/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/ValueVector.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/ValueVector.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/ValueVector.java
deleted file mode 100644
index 8a5a822..0000000
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/ValueVector.java
+++ /dev/null
@@ -1,118 +0,0 @@
-/*******************************************************************************
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- ******************************************************************************/
-package org.apache.drill.exec.record.vector;
-
-import io.netty.buffer.ByteBuf;
-
-import java.io.Closeable;
-
-import org.apache.drill.exec.proto.UserBitShared.FieldMetadata;
-import org.apache.drill.exec.record.MaterializedField;
-
-/**
- * A vector of values.  Acts a containing instance that may rotate its internal buffer depending on what it needs to hold.  Should be language agnostic so that it can be passed between Java and JNI without modification.
- */
-public interface ValueVector<T extends ValueVector<T>> extends Closeable {
-
-  /**
-   * Copies the data from this vector into its pair.
-   * 
-   * @param vector
-   */
-  public abstract void cloneInto(T vector);
-
-  /**
-   * Allocate a new memory space for this vector.
-   * 
-   * @param valueCount
-   *          The number of possible values which should be contained in this vector.
-   */
-  public abstract void allocateNew(int valueCount);
-
-  /**
-   * Update the value vector to the provided record information.
-   * @param metadata
-   * @param data
-   */
-  public abstract void setTo(FieldMetadata metadata, ByteBuf data);
-  
-  /**
-   * Zero copy move of data from this vector to the target vector. Any future access to this vector without being
-   * populated by a new vector will cause problems.
-   * 
-   * @param vector
-   */
-  public abstract void transferTo(T vector);
-
-  /**
-   * Return the underlying buffers associated with this vector. Note that this doesn't impact the reference counts for this buffer so it only should be
-   * used for in context access. Also note that this buffer changes regularly thus external classes shouldn't hold a
-   * reference to it (unless they change it).
-   * 
-   * @return The underlying ByteBuf.
-   */
-  public abstract ByteBuf[] getBuffers();
-
-  /**
-   * Returns the maximum number of values contained within this vector.
-   * @return Vector size
-   */
-  public abstract int capacity();
-
-
-  /**
-   * Release supporting resources.
-   */
-  public abstract void close();
-
-  /**
-   * Get information about how this field is materialized.
-   * 
-   * @return
-   */
-  public abstract MaterializedField getField();
-
-  /**
-   * Define the number of records that are in this value vector.
-   * @param recordCount Number of records active in this vector.  Used for purposes such as getting a writable range of the data.
-   */
-  public abstract void setRecordCount(int recordCount);
-  public abstract int getRecordCount();
-  
-  
-  /**
-   * Get the metadata for this field.
-   * @return
-   */
-  public abstract FieldMetadata getMetadata();
-  
-  /**
-   * Debug interface to get values per record.
-   * @param index The record index.
-   * @return The value in the vector.
-   */
-  public Object getObject(int index);
-  
-  
-  /**
-   * Useful for generating random data.
-   */
-  public void randomizeData();
-    
-  
-}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/7075cca1/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/VarLen1.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/VarLen1.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/VarLen1.java
deleted file mode 100644
index 6c2b39e..0000000
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/VarLen1.java
+++ /dev/null
@@ -1,36 +0,0 @@
-/*******************************************************************************
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- ******************************************************************************/
-package org.apache.drill.exec.record.vector;
-
-import org.apache.drill.exec.memory.BufferAllocator;
-import org.apache.drill.exec.record.MaterializedField;
-
-public class VarLen1 extends VariableVector<VarLen1, Fixed1>{
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(VarLen1.class);
-
-  public VarLen1(MaterializedField field, BufferAllocator allocator) {
-    super(field, allocator, 1);
-  }
-
-  @Override
-  protected Fixed1 getNewLengthVector(BufferAllocator allocator) {
-    return new Fixed1(null, allocator);
-  }
-
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/7075cca1/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/VarLen2.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/VarLen2.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/VarLen2.java
deleted file mode 100644
index 3bcdd94..0000000
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/VarLen2.java
+++ /dev/null
@@ -1,36 +0,0 @@
-/*******************************************************************************
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- ******************************************************************************/
-package org.apache.drill.exec.record.vector;
-
-import org.apache.drill.exec.memory.BufferAllocator;
-import org.apache.drill.exec.record.MaterializedField;
-
-public class VarLen2 extends VariableVector<VarLen2, Fixed2>{
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(VarLen2.class);
-
-  public VarLen2(MaterializedField field, BufferAllocator allocator) {
-    super(field, allocator, 2);
-  }
-
-  @Override
-  protected Fixed2 getNewLengthVector(BufferAllocator allocator) {
-    return new Fixed2(null, allocator);
-  }
-
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/7075cca1/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/VarLen4.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/VarLen4.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/VarLen4.java
deleted file mode 100644
index 71ed7a1..0000000
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/VarLen4.java
+++ /dev/null
@@ -1,48 +0,0 @@
-/*******************************************************************************
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- ******************************************************************************/
-package org.apache.drill.exec.record.vector;
-
-import org.apache.drill.exec.memory.BufferAllocator;
-import org.apache.drill.exec.record.MaterializedField;
-
-import static com.google.common.base.Preconditions.checkArgument;
-
-public class VarLen4 extends VariableVector<VarLen4, Fixed4>{
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(VarLen4.class);
-
-  public VarLen4(MaterializedField field, BufferAllocator allocator) {
-    super(field, allocator, 4);
-  }
-
-  @Override
-  protected Fixed4 getNewLengthVector(BufferAllocator allocator) {
-    return new Fixed4(null, allocator);
-  }
-
-  public void setBytes(int index, byte[] bytes) {
-      checkArgument(index >= 0);
-      if(index == 0) {
-          lengthVector.setInt(0, bytes.length);
-          data.setBytes(0, bytes);
-      } else {
-          int previousOffset = lengthVector.getInt(index - 1);
-          lengthVector.setInt(index, previousOffset + bytes.length);
-          data.setBytes(previousOffset, bytes);
-      }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/7075cca1/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/VariableVector.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/VariableVector.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/VariableVector.java
deleted file mode 100644
index dc8fec4..0000000
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/VariableVector.java
+++ /dev/null
@@ -1,102 +0,0 @@
-/*******************************************************************************
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- ******************************************************************************/
-package org.apache.drill.exec.record.vector;
-
-import io.netty.buffer.ByteBuf;
-
-import org.apache.drill.exec.memory.BufferAllocator;
-import org.apache.drill.exec.record.DeadBuf;
-import org.apache.drill.exec.record.MaterializedField;
-
-import static com.google.common.base.Preconditions.checkArgument;
-import static com.google.common.base.Preconditions.checkState;
-
-/** 
- * A vector of variable length bytes.  Constructed as a vector of lengths or positions and a vector of values.  Random access is only possible if the variable vector stores positions as opposed to lengths.
- */
-public abstract class VariableVector<T extends VariableVector<T, E>, E extends BaseValueVector<E>> extends BaseValueVector<T>{
-
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(VariableVector.class);
-  
-  protected final E lengthVector;
-  protected int expectedValueLength;
-  
-  public VariableVector(MaterializedField field, BufferAllocator allocator, int expectedValueLength) {
-    super(field, allocator);
-    this.lengthVector = getNewLengthVector(allocator);
-    this.expectedValueLength = expectedValueLength;
-  }
-  
-  protected abstract E getNewLengthVector(BufferAllocator allocator);
-  
-  @Override
-  protected int getAllocationSize(int valueCount) {
-    return lengthVector.getAllocationSize(valueCount) + (expectedValueLength * valueCount);
-  }
-  
-  @Override
-  protected void childResetAllocation(int valueCount, ByteBuf buf) {
-    int firstSize = lengthVector.getAllocationSize(valueCount);
-    lengthVector.resetAllocation(valueCount, buf.slice(0, firstSize));
-    data = buf.slice(firstSize, expectedValueLength * valueCount);
-  }
-
-  @Override
-  protected void childCloneMetadata(T other) {
-    lengthVector.cloneMetadata(other.lengthVector);
-    other.expectedValueLength = expectedValueLength;
-  }
-
-  @Override
-  protected void childClear() {
-    lengthVector.clear();
-    if(data != DeadBuf.DEAD_BUFFER){
-      data.release();
-      data = DeadBuf.DEAD_BUFFER;
-    }
-  }
-
-  @Override
-  public ByteBuf[] getBuffers() {
-    return new ByteBuf[]{lengthVector.data, data};
-  }
-
-  @Override
-  public void setRecordCount(int recordCount) {
-    super.setRecordCount(recordCount);
-    lengthVector.setRecordCount(recordCount);
-  }  
-  
-  public void setTotalBytes(int totalBytes){
-    data.writerIndex(totalBytes);
-  }
-
-  @Override
-  public Object getObject(int index) {
-      checkArgument(index >= 0);
-      int startIdx = 0;
-      if(index > 0) {
-          startIdx = (int) lengthVector.getObject(index - 1);
-      }
-      int size = (int) lengthVector.getObject(index) - startIdx;
-      checkState(size >= 0);
-      byte[] dst = new byte[size];
-      data.getBytes(startIdx, dst, 0, size);
-      return dst;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/7075cca1/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/RpcBus.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/RpcBus.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/RpcBus.java
index a680a97..9ce3f3c 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/RpcBus.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/RpcBus.java
@@ -85,6 +85,7 @@ public abstract class RpcBus<T extends EnumLite, C extends RemoteConnection> imp
       Preconditions.checkNotNull(protobufBody);
       ChannelListenerWithCoordinationId futureListener = queue.get(listener, clazz);
       OutboundRpcMessage m = new OutboundRpcMessage(RpcMode.REQUEST, rpcType, futureListener.getCoordinationId(), protobufBody, dataBodies);
+      logger.debug("Outbound RPC message: {}.    DATA BODIES: {}", m, dataBodies);
       ChannelFuture channelFuture = connection.getChannel().write(m);
       channelFuture.addListener(futureListener);
       completed = true;
@@ -132,7 +133,6 @@ public abstract class RpcBus<T extends EnumLite, C extends RemoteConnection> imp
     @Override
     public void messageReceived(ChannelHandlerContext ctx, InboundRpcMessage msg) throws Exception {
       if (!ctx.channel().isOpen()) return;
-
       if (RpcConstants.EXTRA_DEBUGGING) logger.debug("Received message {}", msg);
       switch (msg.mode) {
       case REQUEST:

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/7075cca1/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/JSONRecordReader.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/JSONRecordReader.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/JSONRecordReader.java
index 42d1be5..f2f97b7 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/JSONRecordReader.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/JSONRecordReader.java
@@ -341,44 +341,42 @@ public class JSONRecordReader implements RecordReader {
             switch (minorType) {
                 case INT: {
                     holder.incAndCheckLength(32);
-                    NullableFixed4 fixed4 = (NullableFixed4) holder.getValueVector();
+                    ValueVector.NullableInt int4 = (ValueVector.NullableInt) holder.getValueVector();
                     if (val == null) {
-                        fixed4.setNull(index);
+                      int4.setNull(index);
                     } else {
-                        fixed4.setInt(index, (Integer) val);
+                      int4.set(index, (Integer) val);
                     }
                     return holder.hasEnoughSpace(32);
                 }
                 case FLOAT4: {
                     holder.incAndCheckLength(32);
-                    NullableFixed4 fixed4 = (NullableFixed4) holder.getValueVector();
+                    ValueVector.NullableFloat4 float4 = (ValueVector.NullableFloat4) holder.getValueVector();
                     if (val == null) {
-                        fixed4.setNull(index);
+                      float4.setNull(index);
                     } else {
-                        fixed4.setFloat4(index, (Float) val);
+                      float4.set(index, (Float) val);
                     }
                     return holder.hasEnoughSpace(32);
                 }
                 case VARCHAR4: {
                     if (val == null) {
-                        ((NullableVarLen4) holder.getValueVector()).setNull(index);
+                        ((ValueVector.NullableVarChar4) holder.getValueVector()).setNull(index);
                         return (index + 1) * 4 <= holder.getLength();
                     } else {
                         byte[] bytes = ((String) val).getBytes(UTF_8);
-                        int length = bytes.length * 8;
+                        int length = bytes.length;
                         holder.incAndCheckLength(length);
-                        NullableVarLen4 varLen4 = (NullableVarLen4) holder.getValueVector();
-                        varLen4.setBytes(index, bytes);
+                        ValueVector.NullableVarChar4 varLen4 = (ValueVector.NullableVarChar4) holder.getValueVector();
+                        varLen4.set(index, bytes);
                         return holder.hasEnoughSpace(length);
                     }
                 }
                 case BOOLEAN: {
                     holder.incAndCheckLength(1);
-                    NullableBit bit = (NullableBit) holder.getValueVector();
-                    if (val == null) {
-                        bit.setNull(index);
-                    } else if ((Boolean) val) {
-                        bit.set(index);
+                    ValueVector.NullableBit bit = (ValueVector.NullableBit) holder.getValueVector();
+                    if (val != null) {
+                        bit.set(index, (Boolean)val ? 1 : 0);
                     }
                     return holder.hasEnoughSpace(1);
                 }
@@ -411,7 +409,7 @@ public class JSONRecordReader implements RecordReader {
             SchemaDefProtos.MajorType type = field.getFieldType();
             int fieldId = field.getFieldId();
             MaterializedField f = MaterializedField.create(new SchemaPath(field.getFieldName()), fieldId, parentFieldId, type);
-            ValueVector<?> v = TypeHelper.getNewVector(f, allocator);
+            ValueVector.Base v = TypeHelper.getNewVector(f, allocator);
             v.allocateNew(batchSize);
             VectorHolder holder = new VectorHolder(batchSize, v);
             valueVectorMap.put(fieldId, holder);

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/7075cca1/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/VectorHolder.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/VectorHolder.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/VectorHolder.java
index 59590d2..4043913 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/VectorHolder.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/VectorHolder.java
@@ -25,15 +25,15 @@ import org.apache.drill.exec.record.vector.ValueVector;
 
 public class VectorHolder {
     private int length;
-    private ValueVector vector;
+    private ValueVector.Base vector;
     private int currentLength;
 
-    VectorHolder(int length, ValueVector<?> vector) {
+    VectorHolder(int length, ValueVector.Base vector) {
         this.length = length;
         this.vector = vector;
     }
 
-    public ValueVector getValueVector() {
+    public ValueVector.Base getValueVector() {
         return vector;
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/7075cca1/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/AbstractFragmentRunnerListener.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/AbstractFragmentRunnerListener.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/AbstractFragmentRunnerListener.java
index 0e8edd5..7668bdc 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/AbstractFragmentRunnerListener.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/AbstractFragmentRunnerListener.java
@@ -103,7 +103,8 @@ public class AbstractFragmentRunnerListener implements FragmentRunnerListener{
 
   protected void fail(FragmentHandle handle, FragmentStatus.Builder statusBuilder){
     statusChange(handle, statusBuilder.build());
+    // TODO: ensure the foreman handles the exception
   }
 
-  
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/7075cca1/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/FragmentRunner.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/FragmentRunner.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/FragmentRunner.java
index 554b398..ec9f6db 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/FragmentRunner.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/FragmentRunner.java
@@ -82,7 +82,7 @@ public class FragmentRunner implements Runnable, CancelableQuery, StatusProvider
           }else{
             updateState(FragmentState.RUNNING, FragmentState.FINISHED, false);
           }
-          
+
         }
       }
       
@@ -92,6 +92,7 @@ public class FragmentRunner implements Runnable, CancelableQuery, StatusProvider
       }
       
     }catch(Exception ex){
+      logger.debug("Caught exception while running fragment: {} ", ex);
       internalFail(ex);
     }finally{
       t.stop();

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/7075cca1/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 594a3a2..3edf283 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
@@ -42,61 +42,59 @@ public class TestSimpleFragmentRun extends PopUnitTestBase {
   @Test
   public void runNoExchangeFragment() throws Exception {
     try(RemoteServiceSet serviceSet = RemoteServiceSet.getLocalServiceSet(); 
-            Drillbit bit = new Drillbit(CONFIG, serviceSet);
-            DrillClient client = new DrillClient(CONFIG, serviceSet.getCoordinator());){
+        Drillbit bit = new Drillbit(CONFIG, serviceSet); 
+        DrillClient client = new DrillClient(CONFIG, serviceSet.getCoordinator());){
     
-        // run query.
-        bit.run();
-        client.connect();
-        List<QueryResultBatch> results = client.runQuery(QueryType.PHYSICAL, Files.toString(FileUtils.getResourceAsFile("/physical_test2.json"), Charsets.UTF_8));
+    // run query.
+    bit.run();
+    client.connect();
+    List<QueryResultBatch> results = client.runQuery(QueryType.PHYSICAL, Files.toString(FileUtils.getResourceAsFile("/physical_test2.json"), Charsets.UTF_8));
 
-        // look at records
-        RecordBatchLoader batchLoader = new RecordBatchLoader(bit.getContext().getAllocator());
-        int recordCount = 0;
-        for (QueryResultBatch batch : results) {
-          if(!batch.hasData()) continue;
-          boolean schemaChanged = batchLoader.load(batch.getHeader().getDef(), batch.getData());
-          boolean firstColumn = true;
+    // look at records
+    RecordBatchLoader batchLoader = new RecordBatchLoader(bit.getContext().getAllocator());
+    int recordCount = 0;
+    for (QueryResultBatch batch : results) {
+      if(!batch.hasData()) continue;
+      boolean schemaChanged = batchLoader.load(batch.getHeader().getDef(), batch.getData());
+      boolean firstColumn = true;
 
-          // print headers.
-          if (schemaChanged) {
-            System.out.println("\n\n========NEW SCHEMA=========\n\n");
-            for (IntObjectCursor<ValueVector<?>> v : batchLoader) {
+      // print headers.
+      if (schemaChanged) {
+        System.out.println("\n\n========NEW SCHEMA=========\n\n");
+        for (IntObjectCursor<ValueVector.Base> v : batchLoader) {
 
-              if (firstColumn) {
-                firstColumn = false;
-              } else {
-                System.out.print("\t");
-              }
-              System.out.print(v.value.getField().getName());
-              System.out.print("[");
-              System.out.print(v.value.getField().getType().getMinorType());
-              System.out.print("]");
-            }
-            System.out.println();
+          if (firstColumn) {
+            firstColumn = false;
+          } else {
+            System.out.print("\t");
           }
+          System.out.print(v.value.getField().getName());
+          System.out.print("[");
+          System.out.print(v.value.getField().getType().getMinorType());
+          System.out.print("]");
+        }
+        System.out.println();
+      }
 
 
-          for (int i = 0; i < batchLoader.getRecordCount(); i++) {
-            boolean first = true;
-            recordCount++;
-            for (IntObjectCursor<ValueVector<?>> v : batchLoader) {
-              if (first) {
-                first = false;
-              } else {
-                System.out.print("\t");
-              }
-              System.out.print(v.value.getObject(i));
-            }
-            if(!first) System.out.println();
+      for (int i = 0; i < batchLoader.getRecordCount(); i++) {
+        boolean first = true;
+        recordCount++;
+        for (IntObjectCursor<ValueVector.Base> v : batchLoader) {
+          if (first) {
+            first = false;
+          } else {
+            System.out.print("\t");
           }
-
-        }
-        logger.debug("Received results {}", results);
-        assertEquals(recordCount, 200);
+          System.out.print(v.value.getObject(i));
         }
-  }
-
+        if(!first) System.out.println();
+      }
 
+    }
+    logger.debug("Received results {}", results);
+    assertEquals(recordCount, 200);
+    }
+  }
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/7075cca1/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/record/vector/TestValueVector.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/record/vector/TestValueVector.java b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/record/vector/TestValueVector.java
new file mode 100644
index 0000000..5924f7d
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/record/vector/TestValueVector.java
@@ -0,0 +1,252 @@
+package org.apache.drill.exec.record.vector;
+
+import io.netty.buffer.ByteBuf;
+import org.apache.drill.exec.memory.DirectBufferAllocator;
+import org.apache.drill.exec.proto.SchemaDefProtos;
+import org.apache.drill.exec.record.MaterializedField;
+
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+
+import org.apache.hadoop.io.UTF8;
+import org.junit.Test;
+
+import java.nio.charset.Charset;
+
+public class TestValueVector {
+
+  DirectBufferAllocator allocator = new DirectBufferAllocator();
+
+  @Test
+  public void testFixedType() {
+    // Build a required uint field definition
+    SchemaDefProtos.MajorType.Builder typeBuilder = SchemaDefProtos.MajorType.newBuilder();
+    SchemaDefProtos.FieldDef.Builder defBuilder = SchemaDefProtos.FieldDef.newBuilder();
+    typeBuilder
+        .setMinorType(SchemaDefProtos.MinorType.UINT4)
+        .setMode(SchemaDefProtos.DataMode.REQUIRED)
+        .setWidth(4);
+    defBuilder
+        .setFieldId(1)
+        .setParentId(0)
+        .setMajorType(typeBuilder.build());
+        MaterializedField field = MaterializedField.create(defBuilder.build());
+
+    // Create a new value vector for 1024 integers
+    ValueVector.MutableUInt4 v = new ValueVector.MutableUInt4(field, allocator);
+    v.allocateNew(1024);
+
+    // Put and set a few values
+    v.set(0, 100);
+    v.set(1, 101);
+    v.set(100, 102);
+    v.set(1022, 103);
+    v.set(1023, 104);
+    assertEquals(100, v.get(0));
+    assertEquals(101, v.get(1));
+    assertEquals(102, v.get(100));
+    assertEquals(103, v.get(1022));
+    assertEquals(104, v.get(1023));
+
+    // Ensure unallocated space returns 0
+    assertEquals(0, v.get(3));
+  }
+
+  @Test
+  public void testNullableVarLen2() {
+    // Build an optional varchar field definition
+    SchemaDefProtos.MajorType.Builder typeBuilder = SchemaDefProtos.MajorType.newBuilder();
+    SchemaDefProtos.FieldDef.Builder defBuilder = SchemaDefProtos.FieldDef.newBuilder();
+    typeBuilder
+        .setMinorType(SchemaDefProtos.MinorType.VARCHAR2)
+        .setMode(SchemaDefProtos.DataMode.OPTIONAL)
+        .setWidth(2);
+    defBuilder
+        .setFieldId(1)
+        .setParentId(0)
+        .setMajorType(typeBuilder.build());
+    MaterializedField field = MaterializedField.create(defBuilder.build());
+
+    // Create a new value vector for 1024 integers
+    ValueVector.NullableVarChar2 v = new ValueVector.NullableVarChar2(field, allocator);
+    v.allocateNew(1024);
+
+    // Create and set 3 sample strings
+    String str1 = new String("AAAAA1");
+    String str2 = new String("BBBBBBBBB2");
+    String str3 = new String("CCCC3");
+    v.set(0, str1.getBytes(Charset.forName("UTF-8")));
+    v.set(1, str2.getBytes(Charset.forName("UTF-8")));
+    v.set(2, str3.getBytes(Charset.forName("UTF-8")));
+
+    // Check the sample strings
+    assertEquals(str1, new String(v.get(0), Charset.forName("UTF-8")));
+    assertEquals(str2, new String(v.get(1), Charset.forName("UTF-8")));
+    assertEquals(str3, new String(v.get(2), Charset.forName("UTF-8")));
+
+    // Ensure null value throws
+    try {
+      v.get(3);
+      assertFalse(false);
+    } catch(NullValueException e) { }
+
+  }
+
+
+  @Test
+  public void testNullableFixedType() {
+    // Build an optional uint field definition
+    SchemaDefProtos.MajorType.Builder typeBuilder = SchemaDefProtos.MajorType.newBuilder();
+    SchemaDefProtos.FieldDef.Builder defBuilder = SchemaDefProtos.FieldDef.newBuilder();
+    typeBuilder
+        .setMinorType(SchemaDefProtos.MinorType.UINT4)
+        .setMode(SchemaDefProtos.DataMode.OPTIONAL)
+        .setWidth(4);
+    defBuilder
+        .setFieldId(1)
+        .setParentId(0)
+        .setMajorType(typeBuilder.build());
+    MaterializedField field = MaterializedField.create(defBuilder.build());
+
+    // Create a new value vector for 1024 integers
+    ValueVector.NullableUInt4 v = new ValueVector.NullableUInt4(field, allocator);
+    v.allocateNew(1024);
+
+    // Put and set a few values
+    v.set(0, 100);
+    v.set(1, 101);
+    v.set(100, 102);
+    v.set(1022, 103);
+    v.set(1023, 104);
+    assertEquals(100, v.get(0));
+    assertEquals(101, v.get(1));
+    assertEquals(102, v.get(100));
+    assertEquals(103, v.get(1022));
+    assertEquals(104, v.get(1023));
+
+    // Ensure null values throw
+    try {
+      v.get(3);
+      assertFalse(false);
+    } catch(NullValueException e) { }
+
+    v.allocateNew(2048);
+    try {
+      v.get(0);
+      assertFalse(false);
+    } catch(NullValueException e) { }
+
+    v.set(0, 100);
+    v.set(1, 101);
+    v.set(100, 102);
+    v.set(1022, 103);
+    v.set(1023, 104);
+    assertEquals(100, v.get(0));
+    assertEquals(101, v.get(1));
+    assertEquals(102, v.get(100));
+    assertEquals(103, v.get(1022));
+    assertEquals(104, v.get(1023));
+
+    // Ensure null values throw
+    try {
+      v.get(3);
+      assertFalse(false);
+    } catch(NullValueException e) { }
+
+  }
+
+  @Test
+  public void testNullableFloat() {
+    // Build an optional float field definition
+    SchemaDefProtos.MajorType.Builder typeBuilder = SchemaDefProtos.MajorType.newBuilder();
+    SchemaDefProtos.FieldDef.Builder defBuilder = SchemaDefProtos.FieldDef.newBuilder();
+    typeBuilder
+        .setMinorType(SchemaDefProtos.MinorType.FLOAT4)
+        .setMode(SchemaDefProtos.DataMode.OPTIONAL)
+        .setWidth(4);
+    defBuilder
+        .setFieldId(1)
+        .setParentId(0)
+        .setMajorType(typeBuilder.build());
+    MaterializedField field = MaterializedField.create(defBuilder.build());
+
+    // Create a new value vector for 1024 integers
+    ValueVector.NullableFloat4 v = (ValueVector.NullableFloat4) TypeHelper.getNewVector(field, allocator);
+
+    v.allocateNew(1024);
+
+    // Put and set a few values
+    v.set(0, 100.1f);
+    v.set(1, 101.2f);
+    v.set(100, 102.3f);
+    v.set(1022, 103.4f);
+    v.set(1023, 104.5f);
+    assertEquals(100.1f, v.get(0), 0);
+    assertEquals(101.2f, v.get(1), 0);
+    assertEquals(102.3f, v.get(100), 0);
+    assertEquals(103.4f, v.get(1022), 0);
+    assertEquals(104.5f, v.get(1023), 0);
+
+    // Ensure null values throw
+    try {
+      v.get(3);
+      assertFalse(false);
+    } catch(NullValueException e) { }
+
+    v.allocateNew(2048);
+    try {
+      v.get(0);
+      assertFalse(false);
+    } catch(NullValueException e) { }
+
+  }
+
+  @Test
+  public void testBitVector() {
+    // Build a required boolean field definition
+    SchemaDefProtos.MajorType.Builder typeBuilder = SchemaDefProtos.MajorType.newBuilder();
+    SchemaDefProtos.FieldDef.Builder defBuilder = SchemaDefProtos.FieldDef.newBuilder();
+    typeBuilder
+        .setMinorType(SchemaDefProtos.MinorType.BOOLEAN)
+        .setMode(SchemaDefProtos.DataMode.REQUIRED)
+        .setWidth(4);
+    defBuilder
+        .setFieldId(1)
+        .setParentId(0)
+        .setMajorType(typeBuilder.build());
+    MaterializedField field = MaterializedField.create(defBuilder.build());
+
+    // Create a new value vector for 1024 integers
+    ValueVector.MutableBit v = new ValueVector.MutableBit(field, allocator);
+    v.allocateNew(1024);
+
+    // Put and set a few values
+    v.set(0, 1);
+    v.set(1, 0);
+    v.set(100, 0);
+    v.set(1022, 1);
+    assertEquals(1, v.get(0));
+    assertEquals(0, v.get(1));
+    assertEquals(0, v.get(100));
+    assertEquals(1, v.get(1022));
+
+    // test setting the same value twice
+    v.set(0, 1);
+    v.set(0, 1);
+    v.set(1, 0);
+    v.set(1, 0);
+    assertEquals(1, v.get(0));
+    assertEquals(0, v.get(1));
+
+    // test toggling the values
+    v.set(0, 0);
+    v.set(1, 1);
+    assertEquals(0, v.get(0));
+    assertEquals(1, v.get(1));
+
+    // Ensure unallocated space returns 0
+    assertEquals(0, v.get(3));
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/7075cca1/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/JSONRecordReaderTest.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/JSONRecordReaderTest.java b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/JSONRecordReaderTest.java
index a29e9f0..cef40ff 100644
--- a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/JSONRecordReaderTest.java
+++ b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/JSONRecordReaderTest.java
@@ -11,6 +11,7 @@ import org.apache.drill.exec.physical.impl.OutputMutator;
 import org.apache.drill.exec.proto.SchemaDefProtos;
 import org.apache.drill.exec.proto.UserBitShared;
 import org.apache.drill.exec.record.vector.ValueVector;
+import org.junit.Ignore;
 import org.junit.Test;
 
 import java.io.IOException;
@@ -30,7 +31,7 @@ public class JSONRecordReaderTest {
 
     class MockOutputMutator implements OutputMutator {
         List<Integer> removedFields = Lists.newArrayList();
-        List<ValueVector> addFields = Lists.newArrayList();
+        List<ValueVector.Base> addFields = Lists.newArrayList();
 
         @Override
         public void removeField(int fieldId) throws SchemaChangeException {
@@ -38,7 +39,7 @@ public class JSONRecordReaderTest {
         }
 
         @Override
-        public void addField(int fieldId, ValueVector<?> vector) throws SchemaChangeException {
+        public void addField(int fieldId, ValueVector.Base vector) throws SchemaChangeException {
             addFields.add(vector);
         }
 
@@ -50,16 +51,16 @@ public class JSONRecordReaderTest {
             return removedFields;
         }
 
-        List<ValueVector> getAddFields() {
+        List<ValueVector.Base> getAddFields() {
             return addFields;
         }
     }
 
-    private <T> void assertField(ValueVector valueVector, int index, SchemaDefProtos.MinorType expectedMinorType, T value, String name) {
+    private <T> void assertField(ValueVector.Base valueVector, int index, SchemaDefProtos.MinorType expectedMinorType, T value, String name) {
         assertField(valueVector, index, expectedMinorType, value, name, 0);
     }
 
-    private <T> void assertField(ValueVector valueVector, int index, SchemaDefProtos.MinorType expectedMinorType, T value, String name, int parentFieldId) {
+    private <T> void assertField(ValueVector.Base valueVector, int index, SchemaDefProtos.MinorType expectedMinorType, T value, String name, int parentFieldId) {
         UserBitShared.FieldMetadata metadata = valueVector.getMetadata();
         SchemaDefProtos.FieldDef def = metadata.getDef();
         assertEquals(expectedMinorType, def.getMajorType().getMinorType());
@@ -89,15 +90,15 @@ public class JSONRecordReaderTest {
         JSONRecordReader jr = new JSONRecordReader(context, getResource("scan_json_test_1.json"));
 
         MockOutputMutator mutator = new MockOutputMutator();
-        List<ValueVector> addFields = mutator.getAddFields();
+        List<ValueVector.Base> addFields = mutator.getAddFields();
         jr.setup(mutator);
         assertEquals(2, jr.next());
         assertEquals(3, addFields.size());
         assertField(addFields.get(0), 0, SchemaDefProtos.MinorType.INT, 123, "test");
-        assertField(addFields.get(1), 0, SchemaDefProtos.MinorType.BOOLEAN, 1, "b");
+        assertField(addFields.get(1), 0, SchemaDefProtos.MinorType.BOOLEAN, true, "b");
         assertField(addFields.get(2), 0, SchemaDefProtos.MinorType.VARCHAR4, "hi!".getBytes(UTF_8), "c");
         assertField(addFields.get(0), 1, SchemaDefProtos.MinorType.INT, 1234, "test");
-        assertField(addFields.get(1), 1, SchemaDefProtos.MinorType.BOOLEAN, 0, "b");
+        assertField(addFields.get(1), 1, SchemaDefProtos.MinorType.BOOLEAN, false, "b");
         assertField(addFields.get(2), 1, SchemaDefProtos.MinorType.VARCHAR4, "drill!".getBytes(UTF_8), "c");
 
         assertEquals(0, jr.next());
@@ -115,7 +116,7 @@ public class JSONRecordReaderTest {
 
         JSONRecordReader jr = new JSONRecordReader(context, getResource("scan_json_test_2.json"));
         MockOutputMutator mutator = new MockOutputMutator();
-        List<ValueVector> addFields = mutator.getAddFields();
+        List<ValueVector.Base> addFields = mutator.getAddFields();
 
         jr.setup(mutator);
         assertEquals(3, jr.next());
@@ -123,18 +124,18 @@ public class JSONRecordReaderTest {
         assertField(addFields.get(0), 0, SchemaDefProtos.MinorType.INT, 123, "test");
         assertField(addFields.get(1), 0, SchemaDefProtos.MinorType.INT, 1, "b");
         assertField(addFields.get(2), 0, SchemaDefProtos.MinorType.FLOAT4, (float) 2.15, "c");
-        assertField(addFields.get(3), 0, SchemaDefProtos.MinorType.BOOLEAN, 1, "bool");
+        assertField(addFields.get(3), 0, SchemaDefProtos.MinorType.BOOLEAN, true, "bool");
         assertField(addFields.get(4), 0, SchemaDefProtos.MinorType.VARCHAR4, "test1".getBytes(UTF_8), "str1");
 
         assertField(addFields.get(0), 1, SchemaDefProtos.MinorType.INT, 1234, "test");
         assertField(addFields.get(1), 1, SchemaDefProtos.MinorType.INT, 3, "b");
-        assertField(addFields.get(3), 1, SchemaDefProtos.MinorType.BOOLEAN, 0, "bool");
+        assertField(addFields.get(3), 1, SchemaDefProtos.MinorType.BOOLEAN, false, "bool");
         assertField(addFields.get(4), 1, SchemaDefProtos.MinorType.VARCHAR4, "test2".getBytes(UTF_8), "str1");
         assertField(addFields.get(5), 1, SchemaDefProtos.MinorType.INT, 4, "d");
 
         assertField(addFields.get(0), 2, SchemaDefProtos.MinorType.INT, 12345, "test");
         assertField(addFields.get(2), 2, SchemaDefProtos.MinorType.FLOAT4, (float) 5.16, "c");
-        assertField(addFields.get(3), 2, SchemaDefProtos.MinorType.BOOLEAN, 1, "bool");
+        assertField(addFields.get(3), 2, SchemaDefProtos.MinorType.BOOLEAN, true, "bool");
         assertField(addFields.get(5), 2, SchemaDefProtos.MinorType.INT, 6, "d");
         assertField(addFields.get(6), 2, SchemaDefProtos.MinorType.VARCHAR4, "test3".getBytes(UTF_8), "str2");
         assertTrue(mutator.getRemovedFields().isEmpty());
@@ -152,7 +153,7 @@ public class JSONRecordReaderTest {
 
         JSONRecordReader jr = new JSONRecordReader(context, getResource("scan_json_test_2.json"), 64); // batch only fits 1 int
         MockOutputMutator mutator = new MockOutputMutator();
-        List<ValueVector> addFields = mutator.getAddFields();
+        List<ValueVector.Base> addFields = mutator.getAddFields();
         List<Integer> removedFields = mutator.getRemovedFields();
 
         jr.setup(mutator);
@@ -161,14 +162,14 @@ public class JSONRecordReaderTest {
         assertField(addFields.get(0), 0, SchemaDefProtos.MinorType.INT, 123, "test");
         assertField(addFields.get(1), 0, SchemaDefProtos.MinorType.INT, 1, "b");
         assertField(addFields.get(2), 0, SchemaDefProtos.MinorType.FLOAT4, (float) 2.15, "c");
-        assertField(addFields.get(3), 0, SchemaDefProtos.MinorType.BOOLEAN, 1, "bool");
+        assertField(addFields.get(3), 0, SchemaDefProtos.MinorType.BOOLEAN, true, "bool");
         assertField(addFields.get(4), 0, SchemaDefProtos.MinorType.VARCHAR4, "test1".getBytes(UTF_8), "str1");
         assertTrue(removedFields.isEmpty());
         assertEquals(1, jr.next());
         assertEquals(6, addFields.size());
         assertField(addFields.get(0), 0, SchemaDefProtos.MinorType.INT, 1234, "test");
         assertField(addFields.get(1), 0, SchemaDefProtos.MinorType.INT, 3, "b");
-        assertField(addFields.get(3), 0, SchemaDefProtos.MinorType.BOOLEAN, 0, "bool");
+        assertField(addFields.get(3), 0, SchemaDefProtos.MinorType.BOOLEAN, false, "bool");
         assertField(addFields.get(4), 0, SchemaDefProtos.MinorType.VARCHAR4, "test2".getBytes(UTF_8), "str1");
         assertField(addFields.get(5), 0, SchemaDefProtos.MinorType.INT, 4, "d");
         assertEquals(1, removedFields.size());
@@ -177,7 +178,7 @@ public class JSONRecordReaderTest {
         assertEquals(1, jr.next());
         assertEquals(8, addFields.size()); // The reappearing of field 'c' is also included
         assertField(addFields.get(0), 0, SchemaDefProtos.MinorType.INT, 12345, "test");
-        assertField(addFields.get(3), 0, SchemaDefProtos.MinorType.BOOLEAN, 1, "bool");
+        assertField(addFields.get(3), 0, SchemaDefProtos.MinorType.BOOLEAN, true, "bool");
         assertField(addFields.get(5), 0, SchemaDefProtos.MinorType.INT, 6, "d");
         assertField(addFields.get(6), 0, SchemaDefProtos.MinorType.FLOAT4, (float) 5.16, "c");
         assertField(addFields.get(7), 0, SchemaDefProtos.MinorType.VARCHAR4, "test3".getBytes(UTF_8), "str2");
@@ -187,6 +188,7 @@ public class JSONRecordReaderTest {
         assertEquals(0, jr.next());
     }
 
+    @Ignore("Pending repeated map implementation")
     @Test
     public void testNestedFieldInSameBatch(@Injectable final FragmentContext context) throws ExecutionSetupException {
         new Expectations() {
@@ -199,7 +201,7 @@ public class JSONRecordReaderTest {
         JSONRecordReader jr = new JSONRecordReader(context, getResource("scan_json_test_3.json"));
 
         MockOutputMutator mutator = new MockOutputMutator();
-        List<ValueVector> addFields = mutator.getAddFields();
+        List<ValueVector.Base> addFields = mutator.getAddFields();
         jr.setup(mutator);
         assertEquals(2, jr.next());
         assertEquals(5, addFields.size());


Re: [51/53] [abbrv] git commit: Initial working filter operator

Posted by Ted Dunning <te...@gmail.com>.
On Fri, Jul 19, 2013 at 6:58 PM, <ja...@apache.org> wrote:

> Initial working filter operator
>
>
> Project: http://git-wip-us.apache.org/repos/asf/incubator-drill/repo
> Commit:
> http://git-wip-us.apache.org/repos/asf/incubator-drill/commit/65e2cfce
> Tree: http://git-wip-us.apache.org/repos/asf/incubator-drill/tree/65e2cfce
> Diff: http://git-wip-us.apache.org/repos/asf/incubator-drill/diff/65e2cfce
>
> Branch: refs/heads/master
> Commit: 65e2cfce5364ccfe30b83b9dddeb304b79efbc76
> Parents: 57de7ed
> Author: Jacques Nadeau <ja...@apache.org>
> Authored: Wed Jul 17 00:27:31 2013 -0700
> Committer: Jacques Nadeau <ja...@apache.org>
> Committed: Fri Jul 19 14:53:31 2013 -0700
>
> ----------------------------------------------------------------------
>  .../org/apache/drill/exec/cache/HazelCache.java |   1 -
>  .../drill/exec/compile/ClassTransformer.java    |  17 +-
>  .../drill/exec/compile/JDKClassCompiler.java    |   5 +-
>  .../exec/compile/TemplateClassDefinition.java   |  24 +--
>  .../apache/drill/exec/expr/CodeGenerator.java   |  20 +-
>  .../drill/exec/expr/EvaluationVisitor.java      |  22 +-
>  .../drill/exec/expr/fn/FunctionConverter.java   |   2 +-
>  .../exec/expr/fn/MethodGrabbingVisitor.java     |   3 +-
>  .../apache/drill/exec/ops/FragmentContext.java  |   9 +-
>  .../drill/exec/physical/impl/ImplCreator.java   |  10 +-
>  .../drill/exec/physical/impl/ScanBatch.java     |  11 +-
>  .../exec/physical/impl/WireRecordBatch.java     |   8 +-
>  .../physical/impl/filter/ExampleFilter.java     | 111 ----------
>  .../impl/filter/FilterBatchCreator.java         |  23 +++
>  .../physical/impl/filter/FilterEvaluator.java   |  10 +
>  .../physical/impl/filter/FilterRecordBatch.java | 200 +++++++++++++++++++
>  .../physical/impl/filter/FilterTemplate.java    |  63 ++++--
>  .../exec/physical/impl/filter/Filterer.java     |  19 ++
>  .../impl/filter/ReturnValueExpression.java      |  39 ++++
>  .../SelectionVectorPopulationExpression.java    |  39 ----
>  .../physical/impl/project/ProjectEvaluator.java |   4 +-
>  .../impl/project/ProjectRecordBatch.java        |  11 +-
>  .../exec/physical/impl/project/Projector.java   |   2 +-
>  .../impl/project/ProjectorTemplate.java         |  10 +-
>  .../apache/drill/exec/record/RecordBatch.java   |   3 +-
>  .../drill/exec/record/RecordBatchLoader.java    |   4 +-
>  .../apache/drill/exec/record/SchemaBuilder.java |   2 +-
>  .../exec/record/selection/SelectionVector2.java |  36 +++-
>  .../drill/exec/store/JSONRecordReader.java      |   3 +-
>  .../exec/compile/TestClassTransformation.java   |   2 +-
>  .../apache/drill/exec/expr/ExpressionTest.java  |   2 +-
>  .../exec/physical/impl/SimpleRootExec.java      |   4 +
>  .../physical/impl/filter/TestSimpleFilter.java  |  58 ++++++
>  .../record/ExpressionTreeMaterializerTest.java  |   2 +-
>  .../drill/exec/store/JSONRecordReaderTest.java  |   3 +-
>  .../src/test/resources/filter/test1.json        |  34 ++++
>  36 files changed, 566 insertions(+), 250 deletions(-)
> ----------------------------------------------------------------------
>
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/65e2cfce/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/cache/HazelCache.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/cache/HazelCache.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/cache/HazelCache.java
> index f4fdbfa..fe4a212 100644
> ---
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/cache/HazelCache.java
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/cache/HazelCache.java
> @@ -29,7 +29,6 @@ import
> org.apache.drill.exec.proto.ExecProtos.FragmentHandle;
>  import org.apache.drill.exec.proto.ExecProtos.PlanFragment;
>  import org.apache.drill.exec.proto.ExecProtos.WorkQueueStatus;
>
> -import com.beust.jcommander.internal.Lists;
>  import com.google.common.cache.Cache;
>  import com.google.common.cache.CacheBuilder;
>  import com.google.protobuf.InvalidProtocolBufferException;
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/65e2cfce/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/compile/ClassTransformer.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/compile/ClassTransformer.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/compile/ClassTransformer.java
> index 4bf6e7e..d7cde2a 100644
> ---
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/compile/ClassTransformer.java
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/compile/ClassTransformer.java
> @@ -44,11 +44,11 @@ import org.objectweb.asm.tree.ClassNode;
>  import org.objectweb.asm.tree.FieldNode;
>  import org.objectweb.asm.tree.MethodNode;
>
> -import com.beust.jcommander.internal.Sets;
>  import com.google.common.base.Preconditions;
>  import com.google.common.cache.CacheBuilder;
>  import com.google.common.cache.CacheLoader;
>  import com.google.common.cache.LoadingCache;
> +import com.google.common.collect.Sets;
>  import com.google.common.io.Files;
>  import com.google.common.io.Resources;
>
> @@ -112,15 +112,16 @@ public class ClassTransformer {
>        String materializedClassName) throws ClassTransformationException {
>
>      try {
> -
> +      long t0 = System.nanoTime();
>        final byte[] implementationClass =
> classLoader.getClassByteCode(materializedClassName, entireClass);
>
>        // Get Template Class
>        final String templateClassName =
> templateDefinition.getTemplateClassName().replaceAll("\\.", File.separator);
>        final String templateClassPath = File.separator + templateClassName
> + ".class";
> +      long t1 = System.nanoTime();
>        final byte[] templateClass =
> getClassByteCodeFromPath(templateClassPath);
> -      int fileNum = new Random().nextInt(100);
> -      Files.write(templateClass, new
> File(String.format("/tmp/%d-template.class", fileNum)));
> +//      int fileNum = new Random().nextInt(100);
> +      //Files.write(templateClass, new
> File(String.format("/tmp/%d-template.class", fileNum)));
>        // Generate Merge Class
>
>        // Setup adapters for merging, remapping class names and class
> writing. This is done in reverse order of how they
> @@ -130,7 +131,11 @@ public class ClassTransformer {
>        RemapClasses remapper = new RemapClasses(oldTemplateSlashName,
> materializedSlashName);
>
>        {
> +
>          ClassNode impl = getClassNodeFromByteCode(implementationClass);
> +        long t2 = System.nanoTime();
> +        logger.debug("Compile {}, decode template {}", (t1 -
> t0)/1000/1000, (t2- t1)/1000/1000);
> +
>          ClassWriter cw = new ClassWriter(ClassWriter.COMPUTE_FRAMES);
>
>          ClassVisitor remappingAdapter = new RemappingClassAdapter(cw,
> remapper);
> @@ -139,7 +144,7 @@ public class ClassTransformer {
>          ClassReader tReader = new ClassReader(templateClass);
>          tReader.accept(mergingAdapter, ClassReader.EXPAND_FRAMES);
>          byte[] outputClass = cw.toByteArray();
> -        Files.write(outputClass, new
> File(String.format("/tmp/%d-output.class", fileNum)));
> +//        Files.write(outputClass, new
> File(String.format("/tmp/%d-output.class", fileNum)));
>          outputClass = cw.toByteArray();
>
>          // Load the class
> @@ -160,7 +165,7 @@ public class ClassTransformer {
>          reader.accept(remap, ClassReader.EXPAND_FRAMES);
>          byte[] newByteCode = subcw.toByteArray();
>          classLoader.injectByteCode(s.replace(oldTemplateSlashName,
> materializedSlashName).replace('/', '.'), newByteCode);
> -        Files.write(subcw.toByteArray(), new
> File(String.format("/tmp/%d-sub-%d.class", fileNum, i)));
> +//        Files.write(subcw.toByteArray(), new
> File(String.format("/tmp/%d-sub-%d.class", fileNum, i)));
>          i++;
>        }
>
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/65e2cfce/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/compile/JDKClassCompiler.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/compile/JDKClassCompiler.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/compile/JDKClassCompiler.java
> index 15e87fe..8f6e572 100644
> ---
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/compile/JDKClassCompiler.java
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/compile/JDKClassCompiler.java
> @@ -30,9 +30,9 @@ import java.util.List;
>  import javax.tools.Diagnostic;
>  import javax.tools.DiagnosticListener;
>  import javax.tools.JavaCompiler;
> +import javax.tools.JavaCompiler.CompilationTask;
>  import javax.tools.JavaFileManager;
>  import javax.tools.JavaFileObject;
> -import javax.tools.JavaCompiler.CompilationTask;
>  import javax.tools.JavaFileObject.Kind;
>  import javax.tools.SimpleJavaFileObject;
>  import javax.tools.StandardLocation;
> @@ -43,7 +43,8 @@ import org.codehaus.commons.compiler.Location;
>  import org.codehaus.commons.compiler.jdk.ByteArrayJavaFileManager;
>  import
> org.codehaus.commons.compiler.jdk.ByteArrayJavaFileManager.ByteArrayJavaFileObject;
>
> -import com.beust.jcommander.internal.Lists;
> +import com.google.common.collect.Lists;
> +
>
>  class JDKClassCompiler implements ClassCompiler {
>    static final org.slf4j.Logger logger =
> org.slf4j.LoggerFactory.getLogger(JDKClassCompiler.class);
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/65e2cfce/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 5a01dce..20ef361 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,7 +17,6 @@
>
> ******************************************************************************/
>  package org.apache.drill.exec.compile;
>
> -import java.lang.reflect.Method;
>
>
>  public class TemplateClassDefinition<T>{
> @@ -25,24 +24,19 @@ public class TemplateClassDefinition<T>{
>    private final Class<T> externalInterface;
>    private final String templateClassName;
>    private final Class<?> internalInterface;
> -  private final String setupName;
> -  private final String evalName;
> -
> +  private final Class<?> evalReturnType;
>
> -  public TemplateClassDefinition(Class<T> externalInterface, String
> templateClassName, Class<?> internalInterface, String setupName, String
> evalName) {
> +  public TemplateClassDefinition(Class<T> externalInterface, String
> templateClassName, Class<?> internalInterface, Class<?> evalReturnType) {
>      super();
>      this.externalInterface = externalInterface;
>      this.templateClassName = templateClassName;
>      this.internalInterface = internalInterface;
> -    this.setupName = setupName;
> -    this.evalName = evalName;
> +    this.evalReturnType = evalReturnType;
>    }
>
> -
>    public Class<T> getExternalInterface() {
>      return externalInterface;
>    }
> -
>
>    public Class<?> getInternalInterface() {
>      return internalInterface;
> @@ -52,16 +46,8 @@ public class TemplateClassDefinition<T>{
>      return templateClassName;
>    }
>
> -  public String getSetupName() {
> -    return setupName;
> +  public Class<?> getEvalReturnType() {
> +    return evalReturnType;
>    }
> -
> -
> -  public String getEvalName() {
> -    return evalName;
> -  }
> -
> -
> -
>
>  }
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/65e2cfce/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 ed6bd9b..241c1cc 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
> @@ -1,10 +1,13 @@
>  package org.apache.drill.exec.expr;
>
>  import java.io.IOException;
> +import java.lang.reflect.Method;
>
> +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.exec.compile.TemplateClassDefinition;
> +import org.apache.drill.exec.exception.SchemaChangeException;
>  import org.apache.drill.exec.expr.fn.FunctionImplementationRegistry;
>  import org.apache.drill.exec.expr.holders.BooleanHolder;
>  import org.apache.drill.exec.expr.holders.IntHolder;
> @@ -36,8 +39,6 @@ public class CodeGenerator<T> {
>    private JBlock currentEvalBlock;
>    private JBlock currentSetupBlock;
>    private final EvaluationVisitor evaluationVisitor;
> -  private final String setupName;
> -  private final String perRecordName;
>    private final TemplateClassDefinition<T> definition;
>    private JCodeModel model;
>    private int index = 0;
> @@ -46,8 +47,6 @@ public class CodeGenerator<T> {
>      super();
>      try{
>        this.definition = definition;
> -      this.setupName = definition.getSetupName();
> -      this.perRecordName = definition.getEvalName();
>        this.model = new JCodeModel();
>        this.clazz =
> model._package("org.apache.drill.exec.test.generated")._class("Test1");
>        clazz._implements(definition.getInternalInterface());
> @@ -59,7 +58,7 @@ public class CodeGenerator<T> {
>      }
>    }
>
> -  public void addNextWrite(ValueVectorWriteExpression ex){
> +  public void addExpr(LogicalExpression ex){
>      logger.debug("Adding next write {}", ex);
>      currentEvalBlock = new JBlock();
>      parentEvalBlock.add(currentEvalBlock);
> @@ -80,20 +79,27 @@ public class CodeGenerator<T> {
>      return currentSetupBlock;
>    }
>
> +
> +  public TemplateClassDefinition<T> getDefinition() {
> +    return definition;
> +  }
> +
>    public String generate() throws IOException{
>
>      {
>        //setup method
> -      JMethod m = clazz.method(JMod.PUBLIC, model.VOID, this.setupName);
> +      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.
> -      JMethod m = clazz.method(JMod.PUBLIC, model.VOID,
> this.perRecordName);
> +      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);
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/65e2cfce/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 6b0e499..c219d9c 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
> @@ -14,7 +14,7 @@ import org.apache.drill.common.types.Types;
>  import org.apache.drill.exec.expr.CodeGenerator.HoldingContainer;
>  import org.apache.drill.exec.expr.fn.FunctionHolder;
>  import org.apache.drill.exec.expr.fn.FunctionImplementationRegistry;
> -import
> org.apache.drill.exec.physical.impl.filter.SelectionVectorPopulationExpression;
> +import org.apache.drill.exec.physical.impl.filter.ReturnValueExpression;
>  import org.apache.drill.exec.record.selection.SelectionVector2;
>  import org.apache.drill.exec.vector.TypeHelper;
>
> @@ -117,7 +117,7 @@ public class EvaluationVisitor extends
> AbstractExprVisitor<HoldingContainer, Cod
>    @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()));
> +    generator.getBlock().assign(out.getValue(), JExpr.lit(e.getBoolean()
> ? 1 : 0));
>      return out;
>    }
>
> @@ -127,8 +127,8 @@ public class EvaluationVisitor extends
> AbstractExprVisitor<HoldingContainer, Cod
>        return visitValueVectorExpression((ValueVectorReadExpression) e,
> generator);
>      }else if(e instanceof ValueVectorWriteExpression){
>        return visitValueVectorWriteExpression((ValueVectorWriteExpression)
> e, generator);
> -    }else if(e instanceof SelectionVectorPopulationExpression){
> -      return
> visitSelectionVectorExpression((SelectionVectorPopulationExpression) e,
> generator);
> +    }else if(e instanceof ReturnValueExpression){
> +      return visitReturnValueExpression((ReturnValueExpression) e,
> generator);
>      }else{
>        return super.visitUnknown(e, generator);
>      }
> @@ -196,21 +196,11 @@ public class EvaluationVisitor extends
> AbstractExprVisitor<HoldingContainer, Cod
>    }
>
>
> -  private HoldingContainer
> visitSelectionVectorExpression(SelectionVectorPopulationExpression e,
> CodeGenerator<?> generator){
> -    JType svClass = generator.getModel()._ref(SelectionVector2.class);
> -    JVar sv = generator.declareClassField("sv", svClass);
> -    JVar index = generator.declareClassField("svIndex",
> generator.getModel().CHAR);
> +  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());
> -
> -//    JBlock blk = generator.getSetupBlock();
> -//    blk.assign(sv,
> JExpr.direct("outgoing").invoke("getSelectionVector2"));
> -//    JConditional jc = blk._if(hc.getValue());
> -//    JBlock body = jc._then();
> -//    body.add(sv.invoke("set").arg(index).arg(JExpr.direct("inIndex")));
> -//    body.assign(index, index.plus(JExpr.lit(1)));
> +    generator.getBlock()._return(hc.getValue().eq(JExpr.lit(1)));
>      return null;
>    }
>
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/65e2cfce/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 84f04f0..8e0f1be 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
> @@ -25,7 +25,7 @@ import org.codehaus.janino.Parser;
>  import org.codehaus.janino.Scanner;
>  import org.mortbay.util.IO;
>
> -import com.beust.jcommander.internal.Lists;
> +import com.google.common.collect.Lists;
>  import com.google.common.io.InputSupplier;
>  import com.google.common.io.Resources;
>
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/65e2cfce/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/MethodGrabbingVisitor.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/MethodGrabbingVisitor.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/MethodGrabbingVisitor.java
> index 22b9046..57268ee 100644
> ---
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/MethodGrabbingVisitor.java
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/MethodGrabbingVisitor.java
> @@ -8,7 +8,8 @@ import org.codehaus.janino.Java.ClassDeclaration;
>  import org.codehaus.janino.Java.MethodDeclarator;
>  import org.codehaus.janino.util.Traverser;
>
> -import com.beust.jcommander.internal.Maps;
> +import com.google.common.collect.Maps;
> +
>
>  public class MethodGrabbingVisitor{
>    static final org.slf4j.Logger logger =
> org.slf4j.LoggerFactory.getLogger(MethodGrabbingVisitor.class);
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/65e2cfce/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 876b873..2ae4afa 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
> @@ -103,11 +103,14 @@ public class FragmentContext {
>      return context.getAllocator();
>    }
>
> -  public <T> T getImplementationClass(TemplateClassDefinition<T>
> templateDefinition, CodeGenerator<T> cg) throws
> ClassTransformationException, IOException{
> -    return transformer.getImplementationClass(this.loader,
> templateDefinition, cg.generate(), cg.getMaterializedClassName());
> +  public <T> T getImplementationClass(CodeGenerator<T> cg) throws
> ClassTransformationException, IOException{
> +    long t1 = System.nanoTime();
> +    T t= transformer.getImplementationClass(this.loader,
> cg.getDefinition(), cg.generate(), cg.getMaterializedClassName());
> +    System.out.println( (System.nanoTime() - t1)/1000/1000 );
> +    return t;
> +
>    }
>
> -
>    public void addMetricsToStatus(FragmentStatus.Builder stats){
>      stats.setBatchesCompleted(batchesCompleted.get());
>      stats.setDataProcessed(dataProcessed.get());
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/65e2cfce/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 739c0d4..f96d6f3 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
> @@ -26,12 +26,14 @@ import
> org.apache.drill.exec.physical.base.AbstractPhysicalVisitor;
>  import org.apache.drill.exec.physical.base.FragmentRoot;
>  import org.apache.drill.exec.physical.base.PhysicalOperator;
>  import org.apache.drill.exec.physical.base.Scan;
> +import org.apache.drill.exec.physical.config.Filter;
>  import org.apache.drill.exec.physical.config.MockScanBatchCreator;
>  import org.apache.drill.exec.physical.config.MockScanPOP;
>  import org.apache.drill.exec.physical.config.Project;
>  import org.apache.drill.exec.physical.config.RandomReceiver;
>  import org.apache.drill.exec.physical.config.Screen;
>  import org.apache.drill.exec.physical.config.SingleSender;
> +import org.apache.drill.exec.physical.impl.filter.FilterBatchCreator;
>  import org.apache.drill.exec.physical.impl.project.ProjectBatchCreator;
>  import org.apache.drill.exec.record.RecordBatch;
>
> @@ -46,6 +48,7 @@ public class ImplCreator extends
> AbstractPhysicalVisitor<RecordBatch, FragmentCo
>    private RandomReceiverCreator rrc = new RandomReceiverCreator();
>    private SingleSenderCreator ssc = new SingleSenderCreator();
>    private ProjectBatchCreator pbc = new ProjectBatchCreator();
> +  private FilterBatchCreator fbc = new FilterBatchCreator();
>    private RootExec root = null;
>
>    private ImplCreator(){}
> @@ -78,10 +81,13 @@ public class ImplCreator extends
> AbstractPhysicalVisitor<RecordBatch, FragmentCo
>      root = sc.getRoot(context, op, getChildren(op, context));
>      return null;
>    }
> -
> -
>
>    @Override
> +  public RecordBatch visitFilter(Filter filter, FragmentContext context)
> throws ExecutionSetupException {
> +    return fbc.getBatch(context, filter, getChildren(filter, 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/65e2cfce/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScanBatch.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScanBatch.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScanBatch.java
> index 5688bb1..084db54 100644
> ---
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScanBatch.java
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScanBatch.java
> @@ -26,7 +26,6 @@ import org.apache.drill.common.expression.SchemaPath;
>  import org.apache.drill.exec.exception.SchemaChangeException;
>  import org.apache.drill.exec.ops.FragmentContext;
>  import org.apache.drill.exec.record.BatchSchema;
> -import org.apache.drill.exec.record.InvalidValueAccessor;
>  import org.apache.drill.exec.record.MaterializedField;
>  import org.apache.drill.exec.record.RecordBatch;
>  import org.apache.drill.exec.record.SchemaBuilder;
> @@ -36,9 +35,8 @@ import
> org.apache.drill.exec.record.selection.SelectionVector4;
>  import org.apache.drill.exec.store.RecordReader;
>  import org.apache.drill.exec.vector.ValueVector;
>
> -import com.beust.jcommander.internal.Lists;
> -import com.beust.jcommander.internal.Maps;
> -import com.carrotsearch.hppc.procedures.IntObjectProcedure;
> +import com.google.common.collect.Lists;
> +import com.google.common.collect.Maps;
>
>  /**
>   * Record batch used for a particular scan. Operators against one or more
> @@ -171,6 +169,11 @@ public class ScanBatch implements RecordBatch {
>    }
>
>    @Override
> +  public Iterator<ValueVector> iterator() {
> +    return vectors.iterator();
> +  }
> +
> +  @Override
>    public WritableBatch getWritableBatch() {
>      return WritableBatch.get(this.getRecordCount(), vectors);
>    }
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/65e2cfce/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/WireRecordBatch.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/WireRecordBatch.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/WireRecordBatch.java
> index d2b8bfd..a575f69 100644
> ---
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/WireRecordBatch.java
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/WireRecordBatch.java
> @@ -17,6 +17,8 @@
>
> ******************************************************************************/
>  package org.apache.drill.exec.physical.impl;
>
> +import java.util.Iterator;
> +
>  import org.apache.drill.common.expression.SchemaPath;
>  import org.apache.drill.exec.exception.SchemaChangeException;
>  import org.apache.drill.exec.ops.FragmentContext;
> @@ -66,9 +68,13 @@ public class WireRecordBatch implements RecordBatch{
>    public void kill() {
>      fragProvider.kill(context);
>    }
> -
>
>    @Override
> +  public Iterator<ValueVector> iterator() {
> +    return batchLoader.iterator();
> +  }
> +
> +  @Override
>    public SelectionVector2 getSelectionVector2() {
>      throw new UnsupportedOperationException();
>    }
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/65e2cfce/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/filter/ExampleFilter.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/filter/ExampleFilter.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/filter/ExampleFilter.java
> deleted file mode 100644
> index 85f598f..0000000
> ---
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/filter/ExampleFilter.java
> +++ /dev/null
> @@ -1,111 +0,0 @@
>
> -/*******************************************************************************
> - * Licensed to the Apache Software Foundation (ASF) under one
> - * or more contributor license agreements.  See the NOTICE file
> - * distributed with this work for additional information
> - * regarding copyright ownership.  The ASF licenses this file
> - * to you under the Apache License, Version 2.0 (the
> - * "License"); you may not use this file except in compliance
> - * with the License.  You may obtain a copy of the License at
> - *
> - * http://www.apache.org/licenses/LICENSE-2.0
> - *
> - * Unless required by applicable law or agreed to in writing, software
> - * distributed under the License is distributed on an "AS IS" BASIS,
> - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
> implied.
> - * See the License for the specific language governing permissions and
> - * limitations under the License.
> -
> ******************************************************************************/
> -
> -package org.apache.drill.exec.physical.impl.filter;
> -
> -import org.apache.drill.common.expression.SchemaPath;
> -import org.apache.drill.exec.exception.SchemaChangeException;
> -import org.apache.drill.exec.ops.FragmentContext;
> -import org.apache.drill.exec.record.BatchSchema;
> -import org.apache.drill.exec.record.RecordBatch;
> -import org.apache.drill.exec.record.WritableBatch;
> -import org.apache.drill.exec.record.selection.SelectionVector2;
> -import org.apache.drill.exec.record.selection.SelectionVector4;
> -import org.apache.drill.exec.vector.ValueVector;
> -
> -public class ExampleFilter implements RecordBatch {
> -  static final org.slf4j.Logger logger =
> org.slf4j.LoggerFactory.getLogger(ExampleFilter.class);
> -
> -  //private EvalutationPredicates []
> -  private RecordBatch incoming;
> -  private BatchSchema outboundSchema;
> -  private int recordCount;
> -
> -  private void reconfigureSchema() throws SchemaChangeException {
> -    BatchSchema in = incoming.getSchema();
> -    outboundSchema =
> BatchSchema.newBuilder().addFields(in).setSelectionVectorMode(BatchSchema.SelectionVectorMode.TWO_BYTE).build();
> -  }
> -
> -  private int generateSelectionVector(){
> -                    return -1;
> -  }
> -
> -  @Override
> -  public FragmentContext getContext() {
> -    return incoming.getContext();
> -  }
> -
> -  @Override
> -  public BatchSchema getSchema() {
> -    return outboundSchema;
> -  }
> -
> -  @Override
> -  public int getRecordCount() {
> -    return recordCount;  //To change body of implemented methods use File
> | Settings | File Templates.
> -  }
> -
> -  @Override
> -  public void kill() {
> -    //To change body of implemented methods use File | Settings | File
> Templates.
> -  }
> -
> -  @Override
> -  public SelectionVector2 getSelectionVector2() {
> -    return null;
> -  }
> -
> -  @Override
> -  public SelectionVector4 getSelectionVector4() {
> -    return null;
> -  }
> -
> -  @Override
> -  public TypedFieldId getValueVectorId(SchemaPath path) {
> -    return null;
> -  }
> -
> -  @Override
> -  public <T extends ValueVector> T getValueVectorById(int fieldId,
> Class<?> vvClass) {
> -    return null;
> -  }
> -
> -  @Override
> -  public IterOutcome next() {
> -    IterOutcome out = incoming.next();
> -    switch (incoming.next()) {
> -
> -      case NONE:
> -        return IterOutcome.NONE;
> -      case OK_NEW_SCHEMA:
> -        //reconfigureSchema();
> -      case OK:
> -        this.recordCount = generateSelectionVector();
> -        return out;
> -      case STOP:
> -        return IterOutcome.STOP;
> -      default:
> -        throw new UnsupportedOperationException();
> -    }
> -  }
> -
> -  @Override
> -  public WritableBatch getWritableBatch() {
> -    return null;  //To change body of implemented methods use File |
> Settings | File Templates.
> -  }
> -}
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/65e2cfce/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/filter/FilterBatchCreator.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/filter/FilterBatchCreator.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/filter/FilterBatchCreator.java
> new file mode 100644
> index 0000000..df2518b
> --- /dev/null
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/filter/FilterBatchCreator.java
> @@ -0,0 +1,23 @@
> +package org.apache.drill.exec.physical.impl.filter;
> +
> +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.Filter;
> +import org.apache.drill.exec.physical.impl.BatchCreator;
> +import org.apache.drill.exec.record.RecordBatch;
> +
> +import com.google.common.base.Preconditions;
> +
> +public class FilterBatchCreator implements BatchCreator<Filter>{
> +  static final org.slf4j.Logger logger =
> org.slf4j.LoggerFactory.getLogger(FilterBatchCreator.class);
> +
> +  @Override
> +  public RecordBatch getBatch(FragmentContext context, Filter config,
> List<RecordBatch> children) throws ExecutionSetupException {
> +    Preconditions.checkArgument(children.size() == 1);
> +    return new FilterRecordBatch(config, children.iterator().next(),
> context);
> +  }
> +
> +
> +}
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/65e2cfce/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/filter/FilterEvaluator.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/filter/FilterEvaluator.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/filter/FilterEvaluator.java
> new file mode 100644
> index 0000000..0fad224
> --- /dev/null
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/filter/FilterEvaluator.java
> @@ -0,0 +1,10 @@
> +package org.apache.drill.exec.physical.impl.filter;
> +
> +import org.apache.drill.exec.exception.SchemaChangeException;
> +import org.apache.drill.exec.ops.FragmentContext;
> +import org.apache.drill.exec.record.RecordBatch;
> +
> +public interface FilterEvaluator {
> +  public void doSetup(FragmentContext context, RecordBatch incoming,
> RecordBatch outgoing) throws SchemaChangeException;
> +  public boolean doEval(int inIndex, int outIndex);
> +}
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/65e2cfce/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/filter/FilterRecordBatch.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/filter/FilterRecordBatch.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/filter/FilterRecordBatch.java
> new file mode 100644
> index 0000000..fc9dbc6
> --- /dev/null
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/filter/FilterRecordBatch.java
> @@ -0,0 +1,200 @@
> +package org.apache.drill.exec.physical.impl.filter;
> +
> +import java.io.IOException;
> +import java.util.Iterator;
> +import java.util.List;
> +
> +import org.apache.drill.common.expression.ErrorCollector;
> +import org.apache.drill.common.expression.ErrorCollectorImpl;
> +import org.apache.drill.common.expression.FieldReference;
> +import org.apache.drill.common.expression.LogicalExpression;
> +import org.apache.drill.common.expression.PathSegment;
> +import org.apache.drill.common.expression.SchemaPath;
> +import org.apache.drill.common.logical.data.NamedExpression;
> +import org.apache.drill.common.types.TypeProtos.MajorType;
> +import org.apache.drill.exec.exception.ClassTransformationException;
> +import org.apache.drill.exec.exception.SchemaChangeException;
> +import org.apache.drill.exec.expr.CodeGenerator;
> +import org.apache.drill.exec.expr.ExpressionTreeMaterializer;
> +import org.apache.drill.exec.expr.ValueVectorReadExpression;
> +import org.apache.drill.exec.expr.ValueVectorWriteExpression;
> +import org.apache.drill.exec.ops.FragmentContext;
> +import org.apache.drill.exec.physical.config.Filter;
> +import org.apache.drill.exec.physical.config.Project;
> +import org.apache.drill.exec.physical.impl.VectorHolder;
> +import org.apache.drill.exec.physical.impl.project.Projector;
> +import org.apache.drill.exec.proto.SchemaDefProtos.FieldDef;
> +import org.apache.drill.exec.proto.SchemaDefProtos.NamePart;
> +import org.apache.drill.exec.proto.SchemaDefProtos.NamePart.Type;
> +import org.apache.drill.exec.record.BatchSchema;
> +import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
> +import org.apache.drill.exec.record.MaterializedField;
> +import org.apache.drill.exec.record.RecordBatch;
> +import org.apache.drill.exec.record.SchemaBuilder;
> +import org.apache.drill.exec.record.TransferPair;
> +import org.apache.drill.exec.record.WritableBatch;
> +import org.apache.drill.exec.record.selection.SelectionVector2;
> +import org.apache.drill.exec.record.selection.SelectionVector4;
> +import org.apache.drill.exec.vector.AllocationHelper;
> +import org.apache.drill.exec.vector.NonRepeatedMutator;
> +import org.apache.drill.exec.vector.TypeHelper;
> +import org.apache.drill.exec.vector.ValueVector;
> +
> +import com.google.common.base.Preconditions;
> +import com.google.common.collect.Lists;
> +
> +public class FilterRecordBatch implements RecordBatch{
> +  static final org.slf4j.Logger logger =
> org.slf4j.LoggerFactory.getLogger(FilterRecordBatch.class);
> +
> +  private final Filter filterConfig;
> +  private final RecordBatch incoming;
> +  private final FragmentContext context;
> +  private final SelectionVector2 sv;
> +  private BatchSchema outSchema;
> +  private Filterer filter;
> +  private List<ValueVector> outputVectors;
> +  private VectorHolder vh;
> +
> +  public FilterRecordBatch(Filter pop, RecordBatch incoming,
> FragmentContext context){
> +    this.filterConfig = pop;
> +    this.incoming = incoming;
> +    this.context = context;
> +    sv = new SelectionVector2(context.getAllocator());
> +  }
> +
> +
> +  @Override
> +  public FragmentContext getContext() {
> +    return context;
> +  }
> +
> +  @Override
> +  public BatchSchema getSchema() {
> +    Preconditions.checkNotNull(outSchema);
> +    return outSchema;
> +  }
> +
> +  @Override
> +  public int getRecordCount() {
> +    return sv.getCount();
> +  }
> +
> +  @Override
> +  public void kill() {
> +    incoming.kill();
> +  }
> +
> +
> +  @Override
> +  public Iterator<ValueVector> iterator() {
> +    return outputVectors.iterator();
> +  }
> +
> +  @Override
> +  public SelectionVector2 getSelectionVector2() {
> +    return sv;
> +  }
> +
> +  @Override
> +  public SelectionVector4 getSelectionVector4() {
> +    throw new UnsupportedOperationException();
> +  }
> +
> +  @Override
> +  public TypedFieldId getValueVectorId(SchemaPath path) {
> +    return vh.getValueVector(path);
> +  }
> +
> +  @Override
> +  public <T extends ValueVector> T getValueVectorById(int fieldId,
> Class<?> clazz) {
> +    return vh.getValueVector(fieldId, clazz);
> +  }
> +
> +  @Override
> +  public IterOutcome next() {
> +
> +    IterOutcome upstream = incoming.next();
> +    logger.debug("Upstream... {}", upstream);
> +    switch(upstream){
> +    case NONE:
> +    case NOT_YET:
> +    case STOP:
> +      return upstream;
> +    case OK_NEW_SCHEMA:
> +      try{
> +        filter = createNewFilterer();
> +      }catch(SchemaChangeException ex){
> +        incoming.kill();
> +        logger.error("Failure during query", ex);
> +        context.fail(ex);
> +        return IterOutcome.STOP;
> +      }
> +      // fall through.
> +    case OK:
> +      int recordCount = incoming.getRecordCount();
> +      sv.allocateNew(recordCount);
> +      filter.filterBatch(recordCount);
> +      for(ValueVector v : this.outputVectors){
> +        ValueVector.Mutator m = v.getMutator();
> +        if(m instanceof NonRepeatedMutator){
> +          ((NonRepeatedMutator) m).setValueCount(recordCount);
> +        }else{
> +          throw new UnsupportedOperationException();
> +        }
> +      }
> +      return upstream; // change if upstream changed, otherwise normal.
> +    default:
> +      throw new UnsupportedOperationException();
> +    }
> +  }
> +
> +
> +  private Filterer createNewFilterer() throws SchemaChangeException{
> +    if(outputVectors != null){
> +      for(ValueVector v : outputVectors){
> +        v.close();
> +      }
> +    }
> +    this.outputVectors = Lists.newArrayList();
> +    this.vh = new VectorHolder(outputVectors);
> +    LogicalExpression filterExpression = filterConfig.getExpr();
> +    final ErrorCollector collector = new ErrorCollectorImpl();
> +    final List<TransferPair> transfers = Lists.newArrayList();
> +    final CodeGenerator<Filterer> cg = new
> CodeGenerator<Filterer>(Filterer.TEMPLATE_DEFINITION,
> context.getFunctionRegistry());
> +
> +    final LogicalExpression expr =
> ExpressionTreeMaterializer.materialize(filterExpression, incoming,
> collector);
> +    if(collector.hasErrors()){
> +      throw new SchemaChangeException(String.format("Failure while trying
> to materialize incoming schema.  Errors:\n %s.",
> collector.toErrorString()));
> +    }
> +
> +    cg.addExpr(new ReturnValueExpression(expr));
> +
> +    for(ValueVector v : incoming){
> +      TransferPair pair = v.getTransferPair();
> +      outputVectors.add(pair.getTo());
> +      transfers.add(pair);
> +    }
> +
> +    SchemaBuilder bldr =
> BatchSchema.newBuilder().setSelectionVectorMode(SelectionVectorMode.TWO_BYTE);
> +    for(ValueVector v : outputVectors){
> +      bldr.addField(v.getField());
> +    }
> +    this.outSchema = bldr.build();
> +
> +    try {
> +      TransferPair[] tx = transfers.toArray(new
> TransferPair[transfers.size()]);
> +      Filterer filterer = context.getImplementationClass(cg);
> +      filterer.setup(context, incoming, this, tx);
> +      return filterer;
> +    } catch (ClassTransformationException | IOException e) {
> +      throw new SchemaChangeException("Failure while attempting to load
> generated class", e);
> +    }
> +  }
> +
> +  @Override
> +  public WritableBatch getWritableBatch() {
> +    return WritableBatch.get(sv.getCount(), outputVectors);
> +  }
> +
> +
> +}
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/65e2cfce/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/filter/FilterTemplate.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/filter/FilterTemplate.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/filter/FilterTemplate.java
> index 216bfec..4092911 100644
> ---
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/filter/FilterTemplate.java
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/filter/FilterTemplate.java
> @@ -1,18 +1,27 @@
>  package org.apache.drill.exec.physical.impl.filter;
>
> +import org.apache.drill.exec.exception.SchemaChangeException;
> +import org.apache.drill.exec.ops.FragmentContext;
> +import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
>  import org.apache.drill.exec.record.RecordBatch;
> +import org.apache.drill.exec.record.TransferPair;
>  import org.apache.drill.exec.record.selection.SelectionVector2;
>
> -public abstract class FilterTemplate {
> +public abstract class FilterTemplate implements Filterer{
>    static final org.slf4j.Logger logger =
> org.slf4j.LoggerFactory.getLogger(FilterTemplate.class);
>
> -  SelectionVector2 outgoingSelectionVector;
> -  SelectionVector2 incomingSelectionVector;
> +  private SelectionVector2 outgoingSelectionVector;
> +  private SelectionVector2 incomingSelectionVector;
> +  private SelectionVectorMode svMode;
> +  private TransferPair[] transfers;
>
> -  public void setup(RecordBatch incoming, RecordBatch outgoing){
> -    outgoingSelectionVector = outgoing.getSelectionVector2();
> -
> -    switch(incoming.getSchema().getSelectionVector()){
> +  @Override
> +  public void setup(FragmentContext context, RecordBatch incoming,
> RecordBatch outgoing, TransferPair[] transfers) throws
> SchemaChangeException{
> +    this.transfers = transfers;
> +    this.outgoingSelectionVector = outgoing.getSelectionVector2();
> +    this.svMode = incoming.getSchema().getSelectionVector();
> +
> +    switch(svMode){
>      case NONE:
>        break;
>      case TWO_BYTE:
> @@ -21,28 +30,54 @@ public abstract class FilterTemplate {
>      default:
>        throw new UnsupportedOperationException();
>      }
> +    doSetup(context, incoming, outgoing);
> +  }
> +
> +  private void doTransfers(){
> +    for(TransferPair t : transfers){
> +      t.transfer();
> +    }
>    }
>
> -  public void filterBatchSV2(int recordCount){
> +  public void filterBatch(int recordCount){
> +    doTransfers();
> +    switch(svMode){
> +    case NONE:
> +      filterBatchNoSV(recordCount);
> +      break;
> +    case TWO_BYTE:
> +      filterBatchSV2(recordCount);
> +      break;
> +    default:
> +      throw new UnsupportedOperationException();
> +    }
> +  }
> +
> +  private void filterBatchSV2(int recordCount){
>      int svIndex = 0;
> -    for(char i =0; i < recordCount; i++){
> -      if(include(i)){
> -        outgoingSelectionVector.setIndex(svIndex, i);
> +    final int count = recordCount*2;
> +    for(int i = 0; i < count; i+=2){
> +      char index = incomingSelectionVector.getIndex(i);
> +      if(doEval(i, 0)){
> +        outgoingSelectionVector.setIndex(svIndex, index);
>          svIndex+=2;
>        }
>      }
> +    outgoingSelectionVector.setRecordCount(svIndex/2);
>    }
>
> -  public void filterBatchNoSV(int recordCount){
> +  private void filterBatchNoSV(int recordCount){
>      int svIndex = 0;
>      for(char i =0; i < recordCount; i++){
>
> -      if(include(i)){
> +      if(doEval(i, 0)){
>          outgoingSelectionVector.setIndex(svIndex, i);
>          svIndex+=2;
>        }
>      }
> +    outgoingSelectionVector.setRecordCount(svIndex/2);
>    }
>
> -  protected abstract boolean include(int index);
> +  protected abstract void doSetup(FragmentContext context, RecordBatch
> incoming, RecordBatch outgoing) throws SchemaChangeException;
> +  protected abstract boolean doEval(int inIndex, int outIndex);
>  }
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/65e2cfce/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
> new file mode 100644
> index 0000000..b270869
> --- /dev/null
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/filter/Filterer.java
> @@ -0,0 +1,19 @@
> +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;
> +
> +public interface Filterer {
> +  static final org.slf4j.Logger logger =
> org.slf4j.LoggerFactory.getLogger(Filterer.class);
> +
> +  public void setup(FragmentContext context, RecordBatch incoming,
> RecordBatch outgoing, TransferPair[] transfers) throws
> SchemaChangeException;
> +  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);
> +
> +}
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/65e2cfce/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
> new file mode 100644
> index 0000000..a794d63
> --- /dev/null
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/filter/ReturnValueExpression.java
> @@ -0,0 +1,39 @@
> +package org.apache.drill.exec.physical.impl.filter;
> +
> +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;
> +
> +public class ReturnValueExpression implements LogicalExpression{
> +  static final org.slf4j.Logger logger =
> org.slf4j.LoggerFactory.getLogger(ReturnValueExpression.class);
> +
> +  private LogicalExpression child;
> +
> +  public ReturnValueExpression(LogicalExpression child) {
> +    this.child = child;
> +  }
> +
> +  public LogicalExpression getChild() {
> +    return child;
> +  }
> +
> +  @Override
> +  public MajorType getMajorType() {
> +    return Types.NULL;
> +  }
> +
> +  @Override
> +  public <T, V, E extends Exception> T accept(ExprVisitor<T, V, E>
> visitor, V value) throws E {
> +    return visitor.visitUnknown(this, value);
> +  }
> +
> +  @Override
> +  public ExpressionPosition getPosition() {
> +    return ExpressionPosition.UNKNOWN;
> +  }
> +
> +
> +
> +}
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/65e2cfce/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/filter/SelectionVectorPopulationExpression.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/filter/SelectionVectorPopulationExpression.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/filter/SelectionVectorPopulationExpression.java
> deleted file mode 100644
> index f253695..0000000
> ---
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/filter/SelectionVectorPopulationExpression.java
> +++ /dev/null
> @@ -1,39 +0,0 @@
> -package org.apache.drill.exec.physical.impl.filter;
> -
> -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;
> -
> -public class SelectionVectorPopulationExpression implements
> LogicalExpression{
> -  static final org.slf4j.Logger logger =
> org.slf4j.LoggerFactory.getLogger(SelectionVectorPopulationExpression.class);
> -
> -  private LogicalExpression child;
> -
> -  public SelectionVectorPopulationExpression(LogicalExpression child) {
> -    this.child = child;
> -  }
> -
> -  public LogicalExpression getChild() {
> -    return child;
> -  }
> -
> -  @Override
> -  public MajorType getMajorType() {
> -    return Types.NULL;
> -  }
> -
> -  @Override
> -  public <T, V, E extends Exception> T accept(ExprVisitor<T, V, E>
> visitor, V value) throws E {
> -    return visitor.visitUnknown(this, value);
> -  }
> -
> -  @Override
> -  public ExpressionPosition getPosition() {
> -    return ExpressionPosition.UNKNOWN;
> -  }
> -
> -
> -
> -}
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/65e2cfce/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 86caf28..5fd1fb4 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
> @@ -7,6 +7,6 @@ import org.apache.drill.exec.record.RecordBatch;
>  public interface ProjectEvaluator {
>    static final org.slf4j.Logger logger =
> org.slf4j.LoggerFactory.getLogger(ProjectEvaluator.class);
>
> -  public abstract void setupEvaluators(FragmentContext context,
> RecordBatch incoming, RecordBatch outgoing) throws SchemaChangeException;
> -  public abstract void doPerRecordWork(int inIndex, int outIndex);
> +  public abstract void doSetup(FragmentContext context, RecordBatch
> incoming, RecordBatch outgoing) throws SchemaChangeException;
> +  public abstract void doEval(int inIndex, int outIndex);
>  }
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/65e2cfce/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/ProjectRecordBatch.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/ProjectRecordBatch.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/ProjectRecordBatch.java
> index 3d1e3f7..060cd92 100644
> ---
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/ProjectRecordBatch.java
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/ProjectRecordBatch.java
> @@ -1,6 +1,7 @@
>  package org.apache.drill.exec.physical.impl.project;
>
>  import java.io.IOException;
> +import java.util.Iterator;
>  import java.util.List;
>
>  import org.apache.drill.common.expression.ErrorCollector;
> @@ -59,7 +60,11 @@ public class ProjectRecordBatch implements RecordBatch{
>      this.context = context;
>    }
>
> -
> +  @Override
> +  public Iterator<ValueVector> iterator() {
> +    return outputVectors.iterator();
> +  }
> +
>    @Override
>    public FragmentContext getContext() {
>      return context;
> @@ -180,7 +185,7 @@ public class ProjectRecordBatch implements RecordBatch{
>          allocationVectors.add(vector);
>          outputVectors.add(vector);
>          ValueVectorWriteExpression write = new
> ValueVectorWriteExpression(outputVectors.size() - 1, expr);
> -        cg.addNextWrite(write);
> +        cg.addExpr(write);
>        }
>
>      }
> @@ -192,7 +197,7 @@ public class ProjectRecordBatch implements RecordBatch{
>      this.outSchema = bldr.build();
>
>      try {
> -      Projector projector =
> context.getImplementationClass(Projector.TEMPLATE_DEFINITION, cg);
> +      Projector projector = context.getImplementationClass(cg);
>        projector.setup(context, incoming, this, transfers);
>        return projector;
>      } catch (ClassTransformationException | IOException e) {
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/65e2cfce/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 2787f0c..0d1e201 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
> @@ -16,6 +16,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, "setupEvaluators", "doPerRecordWork");
> +      Projector.class,
> "org.apache.drill.exec.physical.impl.project.ProjectorTemplate",
> ProjectEvaluator.class, null);
>
>  }
> \ No newline at end of file
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/65e2cfce/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/ProjectorTemplate.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/ProjectorTemplate.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/ProjectorTemplate.java
> index 486c7b0..735d355 100644
> ---
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/ProjectorTemplate.java
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/ProjectorTemplate.java
> @@ -33,7 +33,7 @@ public abstract class ProjectorTemplate implements
> Projector {
>      case TWO_BYTE:
>        final int count = recordCount*2;
>        for(int i = 0; i < count; i+=2, firstOutputIndex++){
> -        doPerRecordWork(vector2.getIndex(i), firstOutputIndex);
> +        doEval(vector2.getIndex(i), firstOutputIndex);
>        }
>        return recordCount;
>
> @@ -45,7 +45,7 @@ public abstract class ProjectorTemplate implements
> Projector {
>        }
>        final int countN = recordCount;
>        for (int i = 0; i < countN; i++, firstOutputIndex++) {
> -        doPerRecordWork(i, firstOutputIndex);
> +        doEval(i, firstOutputIndex);
>        }
>        return recordCount;
>
> @@ -68,11 +68,11 @@ public abstract class ProjectorTemplate implements
> Projector {
>        break;
>      }
>      this.transfers = ImmutableList.copyOf(transfers);
> -    setupEvaluators(context, incoming, outgoing);
> +    setupEval(context, incoming, outgoing);
>    }
>
> -  protected abstract void setupEvaluators(FragmentContext context,
> RecordBatch incoming, RecordBatch outgoing) throws SchemaChangeException;
> -  protected abstract void doPerRecordWork(int inIndex, int outIndex);
> +  protected abstract void setupEval(FragmentContext context, RecordBatch
> incoming, RecordBatch outgoing) throws SchemaChangeException;
> +  protected abstract void doEval(int inIndex, int outIndex);
>
>
>
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/65e2cfce/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatch.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatch.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatch.java
> index 650a148..ff856d4 100644
> ---
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatch.java
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatch.java
> @@ -29,7 +29,7 @@ import org.apache.drill.exec.vector.ValueVector;
>   * composed of ValueVectors, ideally a batch fits within L2 cache (~256k
> per core). The set of value vectors do not
>   * change unless the next() IterOutcome is a *_NEW_SCHEMA type.
>   */
> -public interface RecordBatch {
> +public interface RecordBatch extends Iterable<ValueVector>{
>
>    /**
>     * Describes the outcome of a RecordBatch being incremented forward.
> @@ -88,6 +88,7 @@ public interface RecordBatch {
>    public abstract TypedFieldId getValueVectorId(SchemaPath path);
>
>
> +
>    public abstract <T extends ValueVector> T getValueVectorById(int
> fieldId, Class<?> clazz);
>
>    /**
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/65e2cfce/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatchLoader.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatchLoader.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatchLoader.java
> index e2a1648..5f7648b 100644
> ---
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatchLoader.java
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatchLoader.java
> @@ -33,9 +33,9 @@ import
> org.apache.drill.exec.record.RecordBatch.TypedFieldId;
>  import org.apache.drill.exec.vector.TypeHelper;
>  import org.apache.drill.exec.vector.ValueVector;
>
> -import com.beust.jcommander.internal.Lists;
> -import com.beust.jcommander.internal.Maps;
>  import com.google.common.collect.ImmutableList;
> +import com.google.common.collect.Lists;
> +import com.google.common.collect.Maps;
>
>  public class RecordBatchLoader implements Iterable<ValueVector>{
>    static final org.slf4j.Logger logger =
> org.slf4j.LoggerFactory.getLogger(RecordBatchLoader.class);
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/65e2cfce/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/SchemaBuilder.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/SchemaBuilder.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/SchemaBuilder.java
> index 0989c1d..34e4043 100644
> ---
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/SchemaBuilder.java
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/SchemaBuilder.java
> @@ -23,8 +23,8 @@ import java.util.Set;
>  import org.apache.drill.exec.exception.SchemaChangeException;
>  import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
>
> -import com.beust.jcommander.internal.Sets;
>  import com.google.common.collect.Lists;
> +import com.google.common.collect.Sets;
>
>  /**
>   * A reusable builder that supports the creation of BatchSchemas. Can
> have a supporting expected object. If the expected Schema object is
> defined, the
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/65e2cfce/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/selection/SelectionVector2.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/selection/SelectionVector2.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/selection/SelectionVector2.java
> index cdc136e..88f0c79 100644
> ---
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/selection/SelectionVector2.java
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/selection/SelectionVector2.java
> @@ -19,16 +19,20 @@ package org.apache.drill.exec.record.selection;
>
>  import io.netty.buffer.ByteBuf;
>
> +import java.io.Closeable;
> +import java.io.IOException;
> +
>  import org.apache.drill.exec.memory.BufferAllocator;
>  import org.apache.drill.exec.record.DeadBuf;
>
>  /**
>   * A selection vector that fronts, at most, a
>   */
> -public class SelectionVector2{
> +public class SelectionVector2 implements Closeable{
>    static final org.slf4j.Logger logger =
> org.slf4j.LoggerFactory.getLogger(SelectionVector2.class);
>
>    private final BufferAllocator allocator;
> +  private int recordCount;
>    private ByteBuf buffer = DeadBuf.DEAD_BUFFER;
>
>    public SelectionVector2(BufferAllocator allocator) {
> @@ -36,14 +40,40 @@ public class SelectionVector2{
>    }
>
>    public int getCount(){
> -    return -1;
> +    return recordCount;
>    }
>
> -  public int getIndex(int directIndex){
> +  public char getIndex(int directIndex){
>      return buffer.getChar(directIndex);
>    }
>
>    public void setIndex(int directIndex, char value){
>      buffer.setChar(directIndex, value);
>    }
> +
> +  public void allocateNew(int size){
> +    clear();
> +    buffer = allocator.buffer(size * 2);
> +  }
> +
> +
> +  public void clear() {
> +    if (buffer != DeadBuf.DEAD_BUFFER) {
> +      buffer.release();
> +      buffer = DeadBuf.DEAD_BUFFER;
> +      recordCount = 0;
> +    }
> +  }
> +
> +  public void setRecordCount(int recordCount){
> +    logger.debug("Seting record count to {}", recordCount);
> +    this.recordCount = recordCount;
> +  }
> +
> +  @Override
> +  public void close() throws IOException {
> +    clear();
> +  }
> +
> +
>  }
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/65e2cfce/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/JSONRecordReader.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/JSONRecordReader.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/JSONRecordReader.java
> index 8513dfe..3a57410 100644
> ---
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/JSONRecordReader.java
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/JSONRecordReader.java
> @@ -39,13 +39,14 @@ import
> org.apache.drill.exec.vector.NullableVarChar4Vector;
>  import org.apache.drill.exec.vector.TypeHelper;
>  import org.apache.drill.exec.vector.ValueVector;
>
> -import com.beust.jcommander.internal.Maps;
> +
>  import com.fasterxml.jackson.core.JsonFactory;
>  import com.fasterxml.jackson.core.JsonParser;
>  import com.fasterxml.jackson.core.JsonToken;
>  import com.google.common.base.Charsets;
>  import com.google.common.collect.Iterables;
>  import com.google.common.collect.Lists;
> +import com.google.common.collect.Maps;
>  import com.google.common.io.Files;
>  import com.google.common.io.InputSupplier;
>  import com.google.common.io.Resources;
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/65e2cfce/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 cb1e1d6..d2889ed 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, "a", "b");
> +        ExampleInternalInterface.class, null);
>
>
>      ClassTransformer ct = new ClassTransformer();
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/65e2cfce/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/expr/ExpressionTest.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/expr/ExpressionTest.java
> b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/expr/ExpressionTest.java
> index 623af0e..c610374 100644
> ---
> a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/expr/ExpressionTest.java
> +++
> b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/expr/ExpressionTest.java
> @@ -92,7 +92,7 @@ public class ExpressionTest {
>      }
>
>      CodeGenerator<Projector> cg = new
> CodeGenerator<Projector>(Projector.TEMPLATE_DEFINITION, new
> FunctionImplementationRegistry(DrillConfig.create()));
> -    cg.addNextWrite(new ValueVectorWriteExpression(-1, materializedExpr));
> +    cg.addExpr(new ValueVectorWriteExpression(-1, materializedExpr));
>      return cg.generate();
>    }
>
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/65e2cfce/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/SimpleRootExec.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/SimpleRootExec.java
> b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/SimpleRootExec.java
> index d125ec0..c6434f7 100644
> ---
> a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/SimpleRootExec.java
> +++
> b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/SimpleRootExec.java
> @@ -5,6 +5,7 @@ import
> org.apache.drill.exec.physical.impl.ScreenCreator.ScreenRoot;
>  import org.apache.drill.exec.record.RecordBatch;
>  import org.apache.drill.exec.record.RecordBatch.IterOutcome;
>  import org.apache.drill.exec.record.RecordBatch.TypedFieldId;
> +import org.apache.drill.exec.record.selection.SelectionVector2;
>  import org.apache.drill.exec.vector.ValueVector;
>
>  public class SimpleRootExec implements RootExec{
> @@ -21,6 +22,9 @@ public class SimpleRootExec implements RootExec{
>
>    }
>
> +  public SelectionVector2 getSelectionVector2(){
> +    return incoming.getSelectionVector2();
> +  }
>
>    public <T extends ValueVector> T getValueVectorById(SchemaPath path,
> Class<?> vvClass){
>      TypedFieldId tfid = incoming.getValueVectorId(path);
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/65e2cfce/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
> new file mode 100644
> index 0000000..df11aa7
> --- /dev/null
> +++
> b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/filter/TestSimpleFilter.java
> @@ -0,0 +1,58 @@
> +package org.apache.drill.exec.physical.impl.filter;
> +
> +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.FragmentHandle;
> +import org.apache.drill.exec.rpc.user.UserServer.UserClientConnection;
> +import org.apache.drill.exec.server.DrillbitContext;
> +import org.junit.After;
> +import org.junit.Test;
> +
> +import com.google.common.base.Charsets;
> +import com.google.common.io.Files;
> +import com.yammer.metrics.MetricRegistry;
> +
> +public class TestSimpleFilter {
> +  static final org.slf4j.Logger logger =
> org.slf4j.LoggerFactory.getLogger(TestSimpleFilter.class);
> +  DrillConfig c = DrillConfig.create();
> +
> +
> +  @Test
> +  public void project(@Injectable final DrillbitContext bitContext,
> @Injectable UserClientConnection connection) throws Exception{
> +    System.out.println(System.getProperty("java.class.path"));
> +
> +
> +    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("/filter/test1.json"),
> Charsets.UTF_8));
> +    FunctionImplementationRegistry registry = new
> FunctionImplementationRegistry(c);
> +    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()){
> +      System.out.println(exec.getSelectionVector2().getCount());
> +    }
> +  }
> +
> +  @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/65e2cfce/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 925faf7..68b8881 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
> @@ -31,7 +31,7 @@ import
> org.apache.drill.exec.proto.SchemaDefProtos.NamePart;
>  import org.apache.drill.exec.record.RecordBatch.TypedFieldId;
>  import org.junit.Test;
>
> -import com.beust.jcommander.internal.Lists;
> +import com.google.common.collect.Lists;
>  import com.google.common.collect.Range;
>
>  public class ExpressionTreeMaterializerTest {
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/65e2cfce/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/JSONRecordReaderTest.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/JSONRecordReaderTest.java
> b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/JSONRecordReaderTest.java
> index 7c9e8f4..4a0358e 100644
> ---
> a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/JSONRecordReaderTest.java
> +++
> b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/JSONRecordReaderTest.java
> @@ -24,7 +24,8 @@ import org.apache.drill.exec.vector.ValueVector;
>  import org.junit.Ignore;
>  import org.junit.Test;
>
> -import com.beust.jcommander.internal.Lists;
> +import com.google.common.collect.Lists;
> +
>
>  public class JSONRecordReaderTest {
>    private static final Charset UTF_8 = Charset.forName("UTF-8");
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/65e2cfce/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
> new file mode 100644
> index 0000000..a892c70
> --- /dev/null
> +++ b/sandbox/prototype/exec/java-exec/src/test/resources/filter/test1.json
> @@ -0,0 +1,34 @@
> +{
> +    head:{
> +        type:"APACHE_DRILL_PHYSICAL",
> +        version:"1",
> +        generator:{
> +            type:"manual"
> +        }
> +    },
> +       graph:[
> +        {
> +            @id:1,
> +            pop:"mock-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,
> +            child: 1,
> +            pop:"filter",
> +            expr: "true"
> +        },
> +        {
> +            @id: 3,
> +            child: 2,
> +            pop: "screen"
> +        }
> +    ]
> +}
> \ No newline at end of file
>
>

[51/53] [abbrv] git commit: Initial working filter operator

Posted by ja...@apache.org.
Initial working filter operator


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

Branch: refs/heads/master
Commit: 65e2cfce5364ccfe30b83b9dddeb304b79efbc76
Parents: 57de7ed
Author: Jacques Nadeau <ja...@apache.org>
Authored: Wed Jul 17 00:27:31 2013 -0700
Committer: Jacques Nadeau <ja...@apache.org>
Committed: Fri Jul 19 14:53:31 2013 -0700

----------------------------------------------------------------------
 .../org/apache/drill/exec/cache/HazelCache.java |   1 -
 .../drill/exec/compile/ClassTransformer.java    |  17 +-
 .../drill/exec/compile/JDKClassCompiler.java    |   5 +-
 .../exec/compile/TemplateClassDefinition.java   |  24 +--
 .../apache/drill/exec/expr/CodeGenerator.java   |  20 +-
 .../drill/exec/expr/EvaluationVisitor.java      |  22 +-
 .../drill/exec/expr/fn/FunctionConverter.java   |   2 +-
 .../exec/expr/fn/MethodGrabbingVisitor.java     |   3 +-
 .../apache/drill/exec/ops/FragmentContext.java  |   9 +-
 .../drill/exec/physical/impl/ImplCreator.java   |  10 +-
 .../drill/exec/physical/impl/ScanBatch.java     |  11 +-
 .../exec/physical/impl/WireRecordBatch.java     |   8 +-
 .../physical/impl/filter/ExampleFilter.java     | 111 ----------
 .../impl/filter/FilterBatchCreator.java         |  23 +++
 .../physical/impl/filter/FilterEvaluator.java   |  10 +
 .../physical/impl/filter/FilterRecordBatch.java | 200 +++++++++++++++++++
 .../physical/impl/filter/FilterTemplate.java    |  63 ++++--
 .../exec/physical/impl/filter/Filterer.java     |  19 ++
 .../impl/filter/ReturnValueExpression.java      |  39 ++++
 .../SelectionVectorPopulationExpression.java    |  39 ----
 .../physical/impl/project/ProjectEvaluator.java |   4 +-
 .../impl/project/ProjectRecordBatch.java        |  11 +-
 .../exec/physical/impl/project/Projector.java   |   2 +-
 .../impl/project/ProjectorTemplate.java         |  10 +-
 .../apache/drill/exec/record/RecordBatch.java   |   3 +-
 .../drill/exec/record/RecordBatchLoader.java    |   4 +-
 .../apache/drill/exec/record/SchemaBuilder.java |   2 +-
 .../exec/record/selection/SelectionVector2.java |  36 +++-
 .../drill/exec/store/JSONRecordReader.java      |   3 +-
 .../exec/compile/TestClassTransformation.java   |   2 +-
 .../apache/drill/exec/expr/ExpressionTest.java  |   2 +-
 .../exec/physical/impl/SimpleRootExec.java      |   4 +
 .../physical/impl/filter/TestSimpleFilter.java  |  58 ++++++
 .../record/ExpressionTreeMaterializerTest.java  |   2 +-
 .../drill/exec/store/JSONRecordReaderTest.java  |   3 +-
 .../src/test/resources/filter/test1.json        |  34 ++++
 36 files changed, 566 insertions(+), 250 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/65e2cfce/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/cache/HazelCache.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/cache/HazelCache.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/cache/HazelCache.java
index f4fdbfa..fe4a212 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/cache/HazelCache.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/cache/HazelCache.java
@@ -29,7 +29,6 @@ import org.apache.drill.exec.proto.ExecProtos.FragmentHandle;
 import org.apache.drill.exec.proto.ExecProtos.PlanFragment;
 import org.apache.drill.exec.proto.ExecProtos.WorkQueueStatus;
 
-import com.beust.jcommander.internal.Lists;
 import com.google.common.cache.Cache;
 import com.google.common.cache.CacheBuilder;
 import com.google.protobuf.InvalidProtocolBufferException;

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/65e2cfce/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/compile/ClassTransformer.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/compile/ClassTransformer.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/compile/ClassTransformer.java
index 4bf6e7e..d7cde2a 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/compile/ClassTransformer.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/compile/ClassTransformer.java
@@ -44,11 +44,11 @@ import org.objectweb.asm.tree.ClassNode;
 import org.objectweb.asm.tree.FieldNode;
 import org.objectweb.asm.tree.MethodNode;
 
-import com.beust.jcommander.internal.Sets;
 import com.google.common.base.Preconditions;
 import com.google.common.cache.CacheBuilder;
 import com.google.common.cache.CacheLoader;
 import com.google.common.cache.LoadingCache;
+import com.google.common.collect.Sets;
 import com.google.common.io.Files;
 import com.google.common.io.Resources;
 
@@ -112,15 +112,16 @@ public class ClassTransformer {
       String materializedClassName) throws ClassTransformationException {
 
     try {
-
+      long t0 = System.nanoTime();
       final byte[] implementationClass = classLoader.getClassByteCode(materializedClassName, entireClass);
 
       // Get Template Class
       final String templateClassName = templateDefinition.getTemplateClassName().replaceAll("\\.", File.separator);
       final String templateClassPath = File.separator + templateClassName + ".class";
+      long t1 = System.nanoTime();
       final byte[] templateClass = getClassByteCodeFromPath(templateClassPath);
-      int fileNum = new Random().nextInt(100);
-      Files.write(templateClass, new File(String.format("/tmp/%d-template.class", fileNum)));
+//      int fileNum = new Random().nextInt(100);
+      //Files.write(templateClass, new File(String.format("/tmp/%d-template.class", fileNum)));
       // Generate Merge Class
 
       // Setup adapters for merging, remapping class names and class writing. This is done in reverse order of how they
@@ -130,7 +131,11 @@ public class ClassTransformer {
       RemapClasses remapper = new RemapClasses(oldTemplateSlashName, materializedSlashName);
       
       {
+        
         ClassNode impl = getClassNodeFromByteCode(implementationClass);
+        long t2 = System.nanoTime();
+        logger.debug("Compile {}, decode template {}", (t1 - t0)/1000/1000, (t2- t1)/1000/1000);
+        
         ClassWriter cw = new ClassWriter(ClassWriter.COMPUTE_FRAMES);
 
         ClassVisitor remappingAdapter = new RemappingClassAdapter(cw, remapper);
@@ -139,7 +144,7 @@ public class ClassTransformer {
         ClassReader tReader = new ClassReader(templateClass);
         tReader.accept(mergingAdapter, ClassReader.EXPAND_FRAMES);
         byte[] outputClass = cw.toByteArray();
-        Files.write(outputClass, new File(String.format("/tmp/%d-output.class", fileNum)));
+//        Files.write(outputClass, new File(String.format("/tmp/%d-output.class", fileNum)));
         outputClass = cw.toByteArray();
 
         // Load the class
@@ -160,7 +165,7 @@ public class ClassTransformer {
         reader.accept(remap, ClassReader.EXPAND_FRAMES);
         byte[] newByteCode = subcw.toByteArray();
         classLoader.injectByteCode(s.replace(oldTemplateSlashName, materializedSlashName).replace('/', '.'), newByteCode);
-        Files.write(subcw.toByteArray(), new File(String.format("/tmp/%d-sub-%d.class", fileNum, i)));
+//        Files.write(subcw.toByteArray(), new File(String.format("/tmp/%d-sub-%d.class", fileNum, i)));
         i++;
       }
 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/65e2cfce/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/compile/JDKClassCompiler.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/compile/JDKClassCompiler.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/compile/JDKClassCompiler.java
index 15e87fe..8f6e572 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/compile/JDKClassCompiler.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/compile/JDKClassCompiler.java
@@ -30,9 +30,9 @@ import java.util.List;
 import javax.tools.Diagnostic;
 import javax.tools.DiagnosticListener;
 import javax.tools.JavaCompiler;
+import javax.tools.JavaCompiler.CompilationTask;
 import javax.tools.JavaFileManager;
 import javax.tools.JavaFileObject;
-import javax.tools.JavaCompiler.CompilationTask;
 import javax.tools.JavaFileObject.Kind;
 import javax.tools.SimpleJavaFileObject;
 import javax.tools.StandardLocation;
@@ -43,7 +43,8 @@ import org.codehaus.commons.compiler.Location;
 import org.codehaus.commons.compiler.jdk.ByteArrayJavaFileManager;
 import org.codehaus.commons.compiler.jdk.ByteArrayJavaFileManager.ByteArrayJavaFileObject;
 
-import com.beust.jcommander.internal.Lists;
+import com.google.common.collect.Lists;
+
 
 class JDKClassCompiler implements ClassCompiler {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(JDKClassCompiler.class);

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/65e2cfce/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 5a01dce..20ef361 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,7 +17,6 @@
  ******************************************************************************/
 package org.apache.drill.exec.compile;
 
-import java.lang.reflect.Method;
 
 
 public class TemplateClassDefinition<T>{
@@ -25,24 +24,19 @@ public class TemplateClassDefinition<T>{
   private final Class<T> externalInterface;
   private final String templateClassName;
   private final Class<?> internalInterface;
-  private final String setupName;
-  private final String evalName;
-  
+  private final Class<?> evalReturnType;
 
-  public TemplateClassDefinition(Class<T> externalInterface, String templateClassName, Class<?> internalInterface, String setupName, String evalName) {
+  public TemplateClassDefinition(Class<T> externalInterface, String templateClassName, Class<?> internalInterface, Class<?> evalReturnType) {
     super();
     this.externalInterface = externalInterface;
     this.templateClassName = templateClassName;
     this.internalInterface = internalInterface;
-    this.setupName = setupName;
-    this.evalName = evalName;
+    this.evalReturnType = evalReturnType;
   }
 
-
   public Class<T> getExternalInterface() {
     return externalInterface;
   }
-
   
   public Class<?> getInternalInterface() {
     return internalInterface;
@@ -52,16 +46,8 @@ public class TemplateClassDefinition<T>{
     return templateClassName;
   }
 
-  public String getSetupName() {
-    return setupName;
+  public Class<?> getEvalReturnType() {
+    return evalReturnType;
   }
-
-
-  public String getEvalName() {
-    return evalName;
-  }
-  
-  
-  
   
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/65e2cfce/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 ed6bd9b..241c1cc 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
@@ -1,10 +1,13 @@
 package org.apache.drill.exec.expr;
 
 import java.io.IOException;
+import java.lang.reflect.Method;
 
+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.exec.compile.TemplateClassDefinition;
+import org.apache.drill.exec.exception.SchemaChangeException;
 import org.apache.drill.exec.expr.fn.FunctionImplementationRegistry;
 import org.apache.drill.exec.expr.holders.BooleanHolder;
 import org.apache.drill.exec.expr.holders.IntHolder;
@@ -36,8 +39,6 @@ public class CodeGenerator<T> {
   private JBlock currentEvalBlock;
   private JBlock currentSetupBlock;
   private final EvaluationVisitor evaluationVisitor;
-  private final String setupName;
-  private final String perRecordName;
   private final TemplateClassDefinition<T> definition;
   private JCodeModel model;
   private int index = 0;
@@ -46,8 +47,6 @@ public class CodeGenerator<T> {
     super();
     try{
       this.definition = definition;
-      this.setupName = definition.getSetupName();
-      this.perRecordName = definition.getEvalName();
       this.model = new JCodeModel();
       this.clazz = model._package("org.apache.drill.exec.test.generated")._class("Test1");
       clazz._implements(definition.getInternalInterface());
@@ -59,7 +58,7 @@ public class CodeGenerator<T> {
     }
   }
 
-  public void addNextWrite(ValueVectorWriteExpression ex){
+  public void addExpr(LogicalExpression ex){
     logger.debug("Adding next write {}", ex);
     currentEvalBlock = new JBlock();
     parentEvalBlock.add(currentEvalBlock);
@@ -80,20 +79,27 @@ public class CodeGenerator<T> {
     return currentSetupBlock;
   }
   
+  
+  public TemplateClassDefinition<T> getDefinition() {
+    return definition;
+  }
+
   public String generate() throws IOException{
 
     {
       //setup method
-      JMethod m = clazz.method(JMod.PUBLIC, model.VOID, this.setupName);
+      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.
-      JMethod m = clazz.method(JMod.PUBLIC, model.VOID, this.perRecordName);
+      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);

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/65e2cfce/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 6b0e499..c219d9c 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
@@ -14,7 +14,7 @@ import org.apache.drill.common.types.Types;
 import org.apache.drill.exec.expr.CodeGenerator.HoldingContainer;
 import org.apache.drill.exec.expr.fn.FunctionHolder;
 import org.apache.drill.exec.expr.fn.FunctionImplementationRegistry;
-import org.apache.drill.exec.physical.impl.filter.SelectionVectorPopulationExpression;
+import org.apache.drill.exec.physical.impl.filter.ReturnValueExpression;
 import org.apache.drill.exec.record.selection.SelectionVector2;
 import org.apache.drill.exec.vector.TypeHelper;
 
@@ -117,7 +117,7 @@ public class EvaluationVisitor extends AbstractExprVisitor<HoldingContainer, Cod
   @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()));
+    generator.getBlock().assign(out.getValue(), JExpr.lit(e.getBoolean() ? 1 : 0));
     return out;
   }
   
@@ -127,8 +127,8 @@ public class EvaluationVisitor extends AbstractExprVisitor<HoldingContainer, Cod
       return visitValueVectorExpression((ValueVectorReadExpression) e, generator);
     }else if(e instanceof ValueVectorWriteExpression){
       return visitValueVectorWriteExpression((ValueVectorWriteExpression) e, generator);
-    }else if(e instanceof SelectionVectorPopulationExpression){
-      return visitSelectionVectorExpression((SelectionVectorPopulationExpression) e, generator);
+    }else if(e instanceof ReturnValueExpression){
+      return visitReturnValueExpression((ReturnValueExpression) e, generator);
     }else{
       return super.visitUnknown(e, generator);  
     }
@@ -196,21 +196,11 @@ public class EvaluationVisitor extends AbstractExprVisitor<HoldingContainer, Cod
   }
   
   
-  private HoldingContainer visitSelectionVectorExpression(SelectionVectorPopulationExpression e, CodeGenerator<?> generator){
-    JType svClass = generator.getModel()._ref(SelectionVector2.class);
-    JVar sv = generator.declareClassField("sv", svClass);
-    JVar index = generator.declareClassField("svIndex", generator.getModel().CHAR);
+  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());
-    
-//    JBlock blk = generator.getSetupBlock();
-//    blk.assign(sv, JExpr.direct("outgoing").invoke("getSelectionVector2"));
-//    JConditional jc = blk._if(hc.getValue());
-//    JBlock body = jc._then();
-//    body.add(sv.invoke("set").arg(index).arg(JExpr.direct("inIndex")));
-//    body.assign(index, index.plus(JExpr.lit(1)));
+    generator.getBlock()._return(hc.getValue().eq(JExpr.lit(1)));
     return null;
   }
   

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/65e2cfce/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 84f04f0..8e0f1be 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
@@ -25,7 +25,7 @@ import org.codehaus.janino.Parser;
 import org.codehaus.janino.Scanner;
 import org.mortbay.util.IO;
 
-import com.beust.jcommander.internal.Lists;
+import com.google.common.collect.Lists;
 import com.google.common.io.InputSupplier;
 import com.google.common.io.Resources;
 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/65e2cfce/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/MethodGrabbingVisitor.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/MethodGrabbingVisitor.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/MethodGrabbingVisitor.java
index 22b9046..57268ee 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/MethodGrabbingVisitor.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/MethodGrabbingVisitor.java
@@ -8,7 +8,8 @@ import org.codehaus.janino.Java.ClassDeclaration;
 import org.codehaus.janino.Java.MethodDeclarator;
 import org.codehaus.janino.util.Traverser;
 
-import com.beust.jcommander.internal.Maps;
+import com.google.common.collect.Maps;
+
 
 public class MethodGrabbingVisitor{
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(MethodGrabbingVisitor.class);

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/65e2cfce/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 876b873..2ae4afa 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
@@ -103,11 +103,14 @@ public class FragmentContext {
     return context.getAllocator();
   }
 
-  public <T> T getImplementationClass(TemplateClassDefinition<T> templateDefinition, CodeGenerator<T> cg) throws ClassTransformationException, IOException{
-    return transformer.getImplementationClass(this.loader, templateDefinition, cg.generate(), cg.getMaterializedClassName());
+  public <T> T getImplementationClass(CodeGenerator<T> cg) throws ClassTransformationException, IOException{
+    long t1 = System.nanoTime();
+    T t= transformer.getImplementationClass(this.loader, cg.getDefinition(), cg.generate(), cg.getMaterializedClassName());
+    System.out.println( (System.nanoTime() - t1)/1000/1000 );
+    return t;
+    
   }
   
-
   public void addMetricsToStatus(FragmentStatus.Builder stats){
     stats.setBatchesCompleted(batchesCompleted.get());
     stats.setDataProcessed(dataProcessed.get());

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/65e2cfce/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 739c0d4..f96d6f3 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
@@ -26,12 +26,14 @@ import org.apache.drill.exec.physical.base.AbstractPhysicalVisitor;
 import org.apache.drill.exec.physical.base.FragmentRoot;
 import org.apache.drill.exec.physical.base.PhysicalOperator;
 import org.apache.drill.exec.physical.base.Scan;
+import org.apache.drill.exec.physical.config.Filter;
 import org.apache.drill.exec.physical.config.MockScanBatchCreator;
 import org.apache.drill.exec.physical.config.MockScanPOP;
 import org.apache.drill.exec.physical.config.Project;
 import org.apache.drill.exec.physical.config.RandomReceiver;
 import org.apache.drill.exec.physical.config.Screen;
 import org.apache.drill.exec.physical.config.SingleSender;
+import org.apache.drill.exec.physical.impl.filter.FilterBatchCreator;
 import org.apache.drill.exec.physical.impl.project.ProjectBatchCreator;
 import org.apache.drill.exec.record.RecordBatch;
 
@@ -46,6 +48,7 @@ public class ImplCreator extends AbstractPhysicalVisitor<RecordBatch, FragmentCo
   private RandomReceiverCreator rrc = new RandomReceiverCreator();
   private SingleSenderCreator ssc = new SingleSenderCreator();
   private ProjectBatchCreator pbc = new ProjectBatchCreator();
+  private FilterBatchCreator fbc = new FilterBatchCreator();
   private RootExec root = null;
   
   private ImplCreator(){}
@@ -78,10 +81,13 @@ public class ImplCreator extends AbstractPhysicalVisitor<RecordBatch, FragmentCo
     root = sc.getRoot(context, op, getChildren(op, context));
     return null;
   }
-
-  
   
   @Override
+  public RecordBatch visitFilter(Filter filter, FragmentContext context) throws ExecutionSetupException {
+    return fbc.getBatch(context, filter, getChildren(filter, 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/65e2cfce/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScanBatch.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScanBatch.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScanBatch.java
index 5688bb1..084db54 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScanBatch.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScanBatch.java
@@ -26,7 +26,6 @@ import org.apache.drill.common.expression.SchemaPath;
 import org.apache.drill.exec.exception.SchemaChangeException;
 import org.apache.drill.exec.ops.FragmentContext;
 import org.apache.drill.exec.record.BatchSchema;
-import org.apache.drill.exec.record.InvalidValueAccessor;
 import org.apache.drill.exec.record.MaterializedField;
 import org.apache.drill.exec.record.RecordBatch;
 import org.apache.drill.exec.record.SchemaBuilder;
@@ -36,9 +35,8 @@ import org.apache.drill.exec.record.selection.SelectionVector4;
 import org.apache.drill.exec.store.RecordReader;
 import org.apache.drill.exec.vector.ValueVector;
 
-import com.beust.jcommander.internal.Lists;
-import com.beust.jcommander.internal.Maps;
-import com.carrotsearch.hppc.procedures.IntObjectProcedure;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
 
 /**
  * Record batch used for a particular scan. Operators against one or more
@@ -171,6 +169,11 @@ public class ScanBatch implements RecordBatch {
   }
 
   @Override
+  public Iterator<ValueVector> iterator() {
+    return vectors.iterator();
+  }
+
+  @Override
   public WritableBatch getWritableBatch() {
     return WritableBatch.get(this.getRecordCount(), vectors);
   }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/65e2cfce/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/WireRecordBatch.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/WireRecordBatch.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/WireRecordBatch.java
index d2b8bfd..a575f69 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/WireRecordBatch.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/WireRecordBatch.java
@@ -17,6 +17,8 @@
  ******************************************************************************/
 package org.apache.drill.exec.physical.impl;
 
+import java.util.Iterator;
+
 import org.apache.drill.common.expression.SchemaPath;
 import org.apache.drill.exec.exception.SchemaChangeException;
 import org.apache.drill.exec.ops.FragmentContext;
@@ -66,9 +68,13 @@ public class WireRecordBatch implements RecordBatch{
   public void kill() {
     fragProvider.kill(context);
   }
-
   
   @Override
+  public Iterator<ValueVector> iterator() {
+    return batchLoader.iterator();
+  }
+
+  @Override
   public SelectionVector2 getSelectionVector2() {
     throw new UnsupportedOperationException();
   }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/65e2cfce/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/filter/ExampleFilter.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/filter/ExampleFilter.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/filter/ExampleFilter.java
deleted file mode 100644
index 85f598f..0000000
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/filter/ExampleFilter.java
+++ /dev/null
@@ -1,111 +0,0 @@
-/*******************************************************************************
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- ******************************************************************************/
-
-package org.apache.drill.exec.physical.impl.filter;
-
-import org.apache.drill.common.expression.SchemaPath;
-import org.apache.drill.exec.exception.SchemaChangeException;
-import org.apache.drill.exec.ops.FragmentContext;
-import org.apache.drill.exec.record.BatchSchema;
-import org.apache.drill.exec.record.RecordBatch;
-import org.apache.drill.exec.record.WritableBatch;
-import org.apache.drill.exec.record.selection.SelectionVector2;
-import org.apache.drill.exec.record.selection.SelectionVector4;
-import org.apache.drill.exec.vector.ValueVector;
-
-public class ExampleFilter implements RecordBatch {
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ExampleFilter.class);
-
-  //private EvalutationPredicates []
-  private RecordBatch incoming;
-  private BatchSchema outboundSchema;
-  private int recordCount;
-
-  private void reconfigureSchema() throws SchemaChangeException {
-    BatchSchema in = incoming.getSchema();
-    outboundSchema = BatchSchema.newBuilder().addFields(in).setSelectionVectorMode(BatchSchema.SelectionVectorMode.TWO_BYTE).build();
-  }
-
-  private int generateSelectionVector(){
-                    return -1;
-  }
-
-  @Override
-  public FragmentContext getContext() {
-    return incoming.getContext();
-  }
-
-  @Override
-  public BatchSchema getSchema() {
-    return outboundSchema;
-  }
-
-  @Override
-  public int getRecordCount() {
-    return recordCount;  //To change body of implemented methods use File | Settings | File Templates.
-  }
-
-  @Override
-  public void kill() {
-    //To change body of implemented methods use File | Settings | File Templates.
-  }
-
-  @Override
-  public SelectionVector2 getSelectionVector2() {
-    return null;
-  }
-
-  @Override
-  public SelectionVector4 getSelectionVector4() {
-    return null;
-  }
-
-  @Override
-  public TypedFieldId getValueVectorId(SchemaPath path) {
-    return null;
-  }
-
-  @Override
-  public <T extends ValueVector> T getValueVectorById(int fieldId, Class<?> vvClass) {
-    return null;
-  }
-
-  @Override
-  public IterOutcome next() {
-    IterOutcome out = incoming.next();
-    switch (incoming.next()) {
-
-      case NONE:
-        return IterOutcome.NONE;
-      case OK_NEW_SCHEMA:
-        //reconfigureSchema();
-      case OK:
-        this.recordCount = generateSelectionVector();
-        return out;
-      case STOP:
-        return IterOutcome.STOP;
-      default:
-        throw new UnsupportedOperationException();
-    }
-  }
-
-  @Override
-  public WritableBatch getWritableBatch() {
-    return null;  //To change body of implemented methods use File | Settings | File Templates.
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/65e2cfce/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/filter/FilterBatchCreator.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/filter/FilterBatchCreator.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/filter/FilterBatchCreator.java
new file mode 100644
index 0000000..df2518b
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/filter/FilterBatchCreator.java
@@ -0,0 +1,23 @@
+package org.apache.drill.exec.physical.impl.filter;
+
+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.Filter;
+import org.apache.drill.exec.physical.impl.BatchCreator;
+import org.apache.drill.exec.record.RecordBatch;
+
+import com.google.common.base.Preconditions;
+
+public class FilterBatchCreator implements BatchCreator<Filter>{
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(FilterBatchCreator.class);
+
+  @Override
+  public RecordBatch getBatch(FragmentContext context, Filter config, List<RecordBatch> children) throws ExecutionSetupException {
+    Preconditions.checkArgument(children.size() == 1);
+    return new FilterRecordBatch(config, children.iterator().next(), context);
+  }
+  
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/65e2cfce/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/filter/FilterEvaluator.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/filter/FilterEvaluator.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/filter/FilterEvaluator.java
new file mode 100644
index 0000000..0fad224
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/filter/FilterEvaluator.java
@@ -0,0 +1,10 @@
+package org.apache.drill.exec.physical.impl.filter;
+
+import org.apache.drill.exec.exception.SchemaChangeException;
+import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.record.RecordBatch;
+
+public interface FilterEvaluator {
+  public void doSetup(FragmentContext context, RecordBatch incoming, RecordBatch outgoing) throws SchemaChangeException;
+  public boolean doEval(int inIndex, int outIndex);
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/65e2cfce/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/filter/FilterRecordBatch.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/filter/FilterRecordBatch.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/filter/FilterRecordBatch.java
new file mode 100644
index 0000000..fc9dbc6
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/filter/FilterRecordBatch.java
@@ -0,0 +1,200 @@
+package org.apache.drill.exec.physical.impl.filter;
+
+import java.io.IOException;
+import java.util.Iterator;
+import java.util.List;
+
+import org.apache.drill.common.expression.ErrorCollector;
+import org.apache.drill.common.expression.ErrorCollectorImpl;
+import org.apache.drill.common.expression.FieldReference;
+import org.apache.drill.common.expression.LogicalExpression;
+import org.apache.drill.common.expression.PathSegment;
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.common.logical.data.NamedExpression;
+import org.apache.drill.common.types.TypeProtos.MajorType;
+import org.apache.drill.exec.exception.ClassTransformationException;
+import org.apache.drill.exec.exception.SchemaChangeException;
+import org.apache.drill.exec.expr.CodeGenerator;
+import org.apache.drill.exec.expr.ExpressionTreeMaterializer;
+import org.apache.drill.exec.expr.ValueVectorReadExpression;
+import org.apache.drill.exec.expr.ValueVectorWriteExpression;
+import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.physical.config.Filter;
+import org.apache.drill.exec.physical.config.Project;
+import org.apache.drill.exec.physical.impl.VectorHolder;
+import org.apache.drill.exec.physical.impl.project.Projector;
+import org.apache.drill.exec.proto.SchemaDefProtos.FieldDef;
+import org.apache.drill.exec.proto.SchemaDefProtos.NamePart;
+import org.apache.drill.exec.proto.SchemaDefProtos.NamePart.Type;
+import org.apache.drill.exec.record.BatchSchema;
+import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
+import org.apache.drill.exec.record.MaterializedField;
+import org.apache.drill.exec.record.RecordBatch;
+import org.apache.drill.exec.record.SchemaBuilder;
+import org.apache.drill.exec.record.TransferPair;
+import org.apache.drill.exec.record.WritableBatch;
+import org.apache.drill.exec.record.selection.SelectionVector2;
+import org.apache.drill.exec.record.selection.SelectionVector4;
+import org.apache.drill.exec.vector.AllocationHelper;
+import org.apache.drill.exec.vector.NonRepeatedMutator;
+import org.apache.drill.exec.vector.TypeHelper;
+import org.apache.drill.exec.vector.ValueVector;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+
+public class FilterRecordBatch implements RecordBatch{
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(FilterRecordBatch.class);
+
+  private final Filter filterConfig;
+  private final RecordBatch incoming;
+  private final FragmentContext context;
+  private final SelectionVector2 sv;
+  private BatchSchema outSchema;
+  private Filterer filter;
+  private List<ValueVector> outputVectors;
+  private VectorHolder vh;
+  
+  public FilterRecordBatch(Filter pop, RecordBatch incoming, FragmentContext context){
+    this.filterConfig = pop;
+    this.incoming = incoming;
+    this.context = context;
+    sv = new SelectionVector2(context.getAllocator());
+  }
+  
+  
+  @Override
+  public FragmentContext getContext() {
+    return context;
+  }
+
+  @Override
+  public BatchSchema getSchema() {
+    Preconditions.checkNotNull(outSchema);
+    return outSchema;
+  }
+
+  @Override
+  public int getRecordCount() {
+    return sv.getCount();
+  }
+
+  @Override
+  public void kill() {
+    incoming.kill();
+  }
+
+  
+  @Override
+  public Iterator<ValueVector> iterator() {
+    return outputVectors.iterator();
+  }
+
+  @Override
+  public SelectionVector2 getSelectionVector2() {
+    return sv;
+  }
+
+  @Override
+  public SelectionVector4 getSelectionVector4() {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public TypedFieldId getValueVectorId(SchemaPath path) {
+    return vh.getValueVector(path);
+  }
+
+  @Override
+  public <T extends ValueVector> T getValueVectorById(int fieldId, Class<?> clazz) {
+    return vh.getValueVector(fieldId, clazz);
+  }
+
+  @Override
+  public IterOutcome next() {
+    
+    IterOutcome upstream = incoming.next();
+    logger.debug("Upstream... {}", upstream);
+    switch(upstream){
+    case NONE:
+    case NOT_YET:
+    case STOP:
+      return upstream;
+    case OK_NEW_SCHEMA:
+      try{
+        filter = createNewFilterer();
+      }catch(SchemaChangeException ex){
+        incoming.kill();
+        logger.error("Failure during query", ex);
+        context.fail(ex);
+        return IterOutcome.STOP;
+      }
+      // fall through.
+    case OK:
+      int recordCount = incoming.getRecordCount();
+      sv.allocateNew(recordCount);
+      filter.filterBatch(recordCount);
+      for(ValueVector v : this.outputVectors){
+        ValueVector.Mutator m = v.getMutator();
+        if(m instanceof NonRepeatedMutator){
+          ((NonRepeatedMutator) m).setValueCount(recordCount);
+        }else{
+          throw new UnsupportedOperationException();
+        }
+      }
+      return upstream; // change if upstream changed, otherwise normal.
+    default:
+      throw new UnsupportedOperationException();
+    }
+  }
+  
+
+  private Filterer createNewFilterer() throws SchemaChangeException{
+    if(outputVectors != null){
+      for(ValueVector v : outputVectors){
+        v.close();
+      }
+    }
+    this.outputVectors = Lists.newArrayList();
+    this.vh = new VectorHolder(outputVectors);
+    LogicalExpression filterExpression = filterConfig.getExpr();
+    final ErrorCollector collector = new ErrorCollectorImpl();
+    final List<TransferPair> transfers = Lists.newArrayList();
+    final CodeGenerator<Filterer> cg = new CodeGenerator<Filterer>(Filterer.TEMPLATE_DEFINITION, context.getFunctionRegistry());
+    
+    final LogicalExpression expr = ExpressionTreeMaterializer.materialize(filterExpression, incoming, collector);
+    if(collector.hasErrors()){
+      throw new SchemaChangeException(String.format("Failure while trying to materialize incoming schema.  Errors:\n %s.", collector.toErrorString()));
+    }
+    
+    cg.addExpr(new ReturnValueExpression(expr));
+    
+    for(ValueVector v : incoming){
+      TransferPair pair = v.getTransferPair();
+      outputVectors.add(pair.getTo());
+      transfers.add(pair);
+    }
+    
+    SchemaBuilder bldr = BatchSchema.newBuilder().setSelectionVectorMode(SelectionVectorMode.TWO_BYTE);
+    for(ValueVector v : outputVectors){
+      bldr.addField(v.getField());
+    }
+    this.outSchema = bldr.build();
+    
+    try {
+      TransferPair[] tx = transfers.toArray(new TransferPair[transfers.size()]);
+      Filterer filterer = context.getImplementationClass(cg);
+      filterer.setup(context, incoming, this, tx);
+      return filterer;
+    } catch (ClassTransformationException | IOException e) {
+      throw new SchemaChangeException("Failure while attempting to load generated class", e);
+    }
+  }
+  
+  @Override
+  public WritableBatch getWritableBatch() {
+    return WritableBatch.get(sv.getCount(), outputVectors);
+  }
+  
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/65e2cfce/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/filter/FilterTemplate.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/filter/FilterTemplate.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/filter/FilterTemplate.java
index 216bfec..4092911 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/filter/FilterTemplate.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/filter/FilterTemplate.java
@@ -1,18 +1,27 @@
 package org.apache.drill.exec.physical.impl.filter;
 
+import org.apache.drill.exec.exception.SchemaChangeException;
+import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
 import org.apache.drill.exec.record.RecordBatch;
+import org.apache.drill.exec.record.TransferPair;
 import org.apache.drill.exec.record.selection.SelectionVector2;
 
-public abstract class FilterTemplate {
+public abstract class FilterTemplate implements Filterer{
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(FilterTemplate.class);
   
-  SelectionVector2 outgoingSelectionVector;
-  SelectionVector2 incomingSelectionVector;
+  private SelectionVector2 outgoingSelectionVector;
+  private SelectionVector2 incomingSelectionVector;
+  private SelectionVectorMode svMode;
+  private TransferPair[] transfers;
   
-  public void setup(RecordBatch incoming, RecordBatch outgoing){
-    outgoingSelectionVector = outgoing.getSelectionVector2();
-
-    switch(incoming.getSchema().getSelectionVector()){
+  @Override
+  public void setup(FragmentContext context, RecordBatch incoming, RecordBatch outgoing, TransferPair[] transfers) throws SchemaChangeException{
+    this.transfers = transfers;
+    this.outgoingSelectionVector = outgoing.getSelectionVector2();
+    this.svMode = incoming.getSchema().getSelectionVector();
+    
+    switch(svMode){
     case NONE:
       break;
     case TWO_BYTE:
@@ -21,28 +30,54 @@ public abstract class FilterTemplate {
     default:
       throw new UnsupportedOperationException();
     }
+    doSetup(context, incoming, outgoing);
+  }
+
+  private void doTransfers(){
+    for(TransferPair t : transfers){
+      t.transfer();
+    }
   }
   
-  public void filterBatchSV2(int recordCount){
+  public void filterBatch(int recordCount){
+    doTransfers();
+    switch(svMode){
+    case NONE:
+      filterBatchNoSV(recordCount);
+      break;
+    case TWO_BYTE:
+      filterBatchSV2(recordCount);
+      break;
+    default:
+      throw new UnsupportedOperationException();
+    }
+  }
+  
+  private void filterBatchSV2(int recordCount){
     int svIndex = 0;
-    for(char i =0; i < recordCount; i++){
-      if(include(i)){
-        outgoingSelectionVector.setIndex(svIndex, i);
+    final int count = recordCount*2;
+    for(int i = 0; i < count; i+=2){
+      char index = incomingSelectionVector.getIndex(i);
+      if(doEval(i, 0)){
+        outgoingSelectionVector.setIndex(svIndex, index);
         svIndex+=2;
       }
     }
+    outgoingSelectionVector.setRecordCount(svIndex/2);
   }
   
-  public void filterBatchNoSV(int recordCount){
+  private void filterBatchNoSV(int recordCount){
     int svIndex = 0;
     for(char i =0; i < recordCount; i++){
       
-      if(include(i)){
+      if(doEval(i, 0)){
         outgoingSelectionVector.setIndex(svIndex, i);
         svIndex+=2;
       }
     }
+    outgoingSelectionVector.setRecordCount(svIndex/2);
   }
   
-  protected abstract boolean include(int index);
+  protected abstract void doSetup(FragmentContext context, RecordBatch incoming, RecordBatch outgoing) throws SchemaChangeException;
+  protected abstract boolean doEval(int inIndex, int outIndex);
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/65e2cfce/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
new file mode 100644
index 0000000..b270869
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/filter/Filterer.java
@@ -0,0 +1,19 @@
+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;
+
+public interface Filterer {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(Filterer.class);
+  
+  public void setup(FragmentContext context, RecordBatch incoming, RecordBatch outgoing, TransferPair[] transfers) throws SchemaChangeException;
+  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);
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/65e2cfce/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
new file mode 100644
index 0000000..a794d63
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/filter/ReturnValueExpression.java
@@ -0,0 +1,39 @@
+package org.apache.drill.exec.physical.impl.filter;
+
+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;
+
+public class ReturnValueExpression implements LogicalExpression{
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ReturnValueExpression.class);
+
+  private LogicalExpression child;
+  
+  public ReturnValueExpression(LogicalExpression child) {
+    this.child = child;
+  }
+
+  public LogicalExpression getChild() {
+    return child;
+  }
+
+  @Override
+  public MajorType getMajorType() {
+    return Types.NULL;
+  }
+
+  @Override
+  public <T, V, E extends Exception> T accept(ExprVisitor<T, V, E> visitor, V value) throws E {
+    return visitor.visitUnknown(this, value);
+  }
+
+  @Override
+  public ExpressionPosition getPosition() {
+    return ExpressionPosition.UNKNOWN;
+  }
+  
+  
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/65e2cfce/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/filter/SelectionVectorPopulationExpression.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/filter/SelectionVectorPopulationExpression.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/filter/SelectionVectorPopulationExpression.java
deleted file mode 100644
index f253695..0000000
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/filter/SelectionVectorPopulationExpression.java
+++ /dev/null
@@ -1,39 +0,0 @@
-package org.apache.drill.exec.physical.impl.filter;
-
-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;
-
-public class SelectionVectorPopulationExpression implements LogicalExpression{
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(SelectionVectorPopulationExpression.class);
-
-  private LogicalExpression child;
-  
-  public SelectionVectorPopulationExpression(LogicalExpression child) {
-    this.child = child;
-  }
-
-  public LogicalExpression getChild() {
-    return child;
-  }
-
-  @Override
-  public MajorType getMajorType() {
-    return Types.NULL;
-  }
-
-  @Override
-  public <T, V, E extends Exception> T accept(ExprVisitor<T, V, E> visitor, V value) throws E {
-    return visitor.visitUnknown(this, value);
-  }
-
-  @Override
-  public ExpressionPosition getPosition() {
-    return ExpressionPosition.UNKNOWN;
-  }
-  
-  
-  
-}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/65e2cfce/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 86caf28..5fd1fb4 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
@@ -7,6 +7,6 @@ import org.apache.drill.exec.record.RecordBatch;
 public interface ProjectEvaluator {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ProjectEvaluator.class);
   
-  public abstract void setupEvaluators(FragmentContext context, RecordBatch incoming, RecordBatch outgoing) throws SchemaChangeException;
-  public abstract void doPerRecordWork(int inIndex, int outIndex);
+  public abstract void doSetup(FragmentContext context, RecordBatch incoming, RecordBatch outgoing) throws SchemaChangeException;
+  public abstract void doEval(int inIndex, int outIndex);
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/65e2cfce/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/ProjectRecordBatch.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/ProjectRecordBatch.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/ProjectRecordBatch.java
index 3d1e3f7..060cd92 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/ProjectRecordBatch.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/ProjectRecordBatch.java
@@ -1,6 +1,7 @@
 package org.apache.drill.exec.physical.impl.project;
 
 import java.io.IOException;
+import java.util.Iterator;
 import java.util.List;
 
 import org.apache.drill.common.expression.ErrorCollector;
@@ -59,7 +60,11 @@ public class ProjectRecordBatch implements RecordBatch{
     this.context = context;
   }
   
-  
+  @Override
+  public Iterator<ValueVector> iterator() {
+    return outputVectors.iterator();
+  }
+
   @Override
   public FragmentContext getContext() {
     return context;
@@ -180,7 +185,7 @@ public class ProjectRecordBatch implements RecordBatch{
         allocationVectors.add(vector);
         outputVectors.add(vector);
         ValueVectorWriteExpression write = new ValueVectorWriteExpression(outputVectors.size() - 1, expr);
-        cg.addNextWrite(write);
+        cg.addExpr(write);
       }
       
     }
@@ -192,7 +197,7 @@ public class ProjectRecordBatch implements RecordBatch{
     this.outSchema = bldr.build();
     
     try {
-      Projector projector = context.getImplementationClass(Projector.TEMPLATE_DEFINITION, cg);
+      Projector projector = context.getImplementationClass(cg);
       projector.setup(context, incoming, this, transfers);
       return projector;
     } catch (ClassTransformationException | IOException e) {

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/65e2cfce/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 2787f0c..0d1e201 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
@@ -16,6 +16,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, "setupEvaluators", "doPerRecordWork");
+      Projector.class, "org.apache.drill.exec.physical.impl.project.ProjectorTemplate", ProjectEvaluator.class, null);
 
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/65e2cfce/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/ProjectorTemplate.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/ProjectorTemplate.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/ProjectorTemplate.java
index 486c7b0..735d355 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/ProjectorTemplate.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/ProjectorTemplate.java
@@ -33,7 +33,7 @@ public abstract class ProjectorTemplate implements Projector {
     case TWO_BYTE:
       final int count = recordCount*2;
       for(int i = 0; i < count; i+=2, firstOutputIndex++){
-        doPerRecordWork(vector2.getIndex(i), firstOutputIndex);
+        doEval(vector2.getIndex(i), firstOutputIndex);
       }
       return recordCount;
       
@@ -45,7 +45,7 @@ public abstract class ProjectorTemplate implements Projector {
       }
       final int countN = recordCount;
       for (int i = 0; i < countN; i++, firstOutputIndex++) {
-        doPerRecordWork(i, firstOutputIndex);
+        doEval(i, firstOutputIndex);
       }
       return recordCount;
       
@@ -68,11 +68,11 @@ public abstract class ProjectorTemplate implements Projector {
       break;
     }
     this.transfers = ImmutableList.copyOf(transfers);
-    setupEvaluators(context, incoming, outgoing);
+    setupEval(context, incoming, outgoing);
   }
 
-  protected abstract void setupEvaluators(FragmentContext context, RecordBatch incoming, RecordBatch outgoing) throws SchemaChangeException;
-  protected abstract void doPerRecordWork(int inIndex, int outIndex);
+  protected abstract void setupEval(FragmentContext context, RecordBatch incoming, RecordBatch outgoing) throws SchemaChangeException;
+  protected abstract void doEval(int inIndex, int outIndex);
 
   
 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/65e2cfce/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatch.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatch.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatch.java
index 650a148..ff856d4 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatch.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatch.java
@@ -29,7 +29,7 @@ import org.apache.drill.exec.vector.ValueVector;
  * composed of ValueVectors, ideally a batch fits within L2 cache (~256k per core). The set of value vectors do not
  * change unless the next() IterOutcome is a *_NEW_SCHEMA type.
  */
-public interface RecordBatch {
+public interface RecordBatch extends Iterable<ValueVector>{
 
   /**
    * Describes the outcome of a RecordBatch being incremented forward.
@@ -88,6 +88,7 @@ public interface RecordBatch {
   public abstract TypedFieldId getValueVectorId(SchemaPath path);
   
   
+  
   public abstract <T extends ValueVector> T getValueVectorById(int fieldId, Class<?> clazz);
 
   /**

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/65e2cfce/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatchLoader.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatchLoader.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatchLoader.java
index e2a1648..5f7648b 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatchLoader.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatchLoader.java
@@ -33,9 +33,9 @@ import org.apache.drill.exec.record.RecordBatch.TypedFieldId;
 import org.apache.drill.exec.vector.TypeHelper;
 import org.apache.drill.exec.vector.ValueVector;
 
-import com.beust.jcommander.internal.Lists;
-import com.beust.jcommander.internal.Maps;
 import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
 
 public class RecordBatchLoader implements Iterable<ValueVector>{
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(RecordBatchLoader.class);

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/65e2cfce/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/SchemaBuilder.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/SchemaBuilder.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/SchemaBuilder.java
index 0989c1d..34e4043 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/SchemaBuilder.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/SchemaBuilder.java
@@ -23,8 +23,8 @@ import java.util.Set;
 import org.apache.drill.exec.exception.SchemaChangeException;
 import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
 
-import com.beust.jcommander.internal.Sets;
 import com.google.common.collect.Lists;
+import com.google.common.collect.Sets;
 
 /**
  * A reusable builder that supports the creation of BatchSchemas. Can have a supporting expected object. If the expected Schema object is defined, the

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/65e2cfce/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/selection/SelectionVector2.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/selection/SelectionVector2.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/selection/SelectionVector2.java
index cdc136e..88f0c79 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/selection/SelectionVector2.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/selection/SelectionVector2.java
@@ -19,16 +19,20 @@ package org.apache.drill.exec.record.selection;
 
 import io.netty.buffer.ByteBuf;
 
+import java.io.Closeable;
+import java.io.IOException;
+
 import org.apache.drill.exec.memory.BufferAllocator;
 import org.apache.drill.exec.record.DeadBuf;
 
 /**
  * A selection vector that fronts, at most, a
  */
-public class SelectionVector2{
+public class SelectionVector2 implements Closeable{
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(SelectionVector2.class);
 
   private final BufferAllocator allocator;
+  private int recordCount;
   private ByteBuf buffer = DeadBuf.DEAD_BUFFER;
 
   public SelectionVector2(BufferAllocator allocator) {
@@ -36,14 +40,40 @@ public class SelectionVector2{
   }
 
   public int getCount(){
-    return -1;
+    return recordCount;
   }
 
-  public int getIndex(int directIndex){
+  public char getIndex(int directIndex){
     return buffer.getChar(directIndex);
   }
 
   public void setIndex(int directIndex, char value){
     buffer.setChar(directIndex, value);
   }
+  
+  public void allocateNew(int size){
+    clear();
+    buffer = allocator.buffer(size * 2);
+  }
+  
+  
+  public void clear() {
+    if (buffer != DeadBuf.DEAD_BUFFER) {
+      buffer.release();
+      buffer = DeadBuf.DEAD_BUFFER;
+      recordCount = 0;
+    }
+  }
+  
+  public void setRecordCount(int recordCount){
+    logger.debug("Seting record count to {}", recordCount);
+    this.recordCount = recordCount;
+  }
+
+  @Override
+  public void close() throws IOException {
+    clear();
+  }
+  
+  
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/65e2cfce/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/JSONRecordReader.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/JSONRecordReader.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/JSONRecordReader.java
index 8513dfe..3a57410 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/JSONRecordReader.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/JSONRecordReader.java
@@ -39,13 +39,14 @@ import org.apache.drill.exec.vector.NullableVarChar4Vector;
 import org.apache.drill.exec.vector.TypeHelper;
 import org.apache.drill.exec.vector.ValueVector;
 
-import com.beust.jcommander.internal.Maps;
+
 import com.fasterxml.jackson.core.JsonFactory;
 import com.fasterxml.jackson.core.JsonParser;
 import com.fasterxml.jackson.core.JsonToken;
 import com.google.common.base.Charsets;
 import com.google.common.collect.Iterables;
 import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
 import com.google.common.io.Files;
 import com.google.common.io.InputSupplier;
 import com.google.common.io.Resources;

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/65e2cfce/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 cb1e1d6..d2889ed 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, "a", "b");
+        ExampleInternalInterface.class, null);
     
     
     ClassTransformer ct = new ClassTransformer();

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/65e2cfce/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/expr/ExpressionTest.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/expr/ExpressionTest.java b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/expr/ExpressionTest.java
index 623af0e..c610374 100644
--- a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/expr/ExpressionTest.java
+++ b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/expr/ExpressionTest.java
@@ -92,7 +92,7 @@ public class ExpressionTest {
     }
 
     CodeGenerator<Projector> cg = new CodeGenerator<Projector>(Projector.TEMPLATE_DEFINITION, new FunctionImplementationRegistry(DrillConfig.create()));
-    cg.addNextWrite(new ValueVectorWriteExpression(-1, materializedExpr));
+    cg.addExpr(new ValueVectorWriteExpression(-1, materializedExpr));
     return cg.generate();
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/65e2cfce/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/SimpleRootExec.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/SimpleRootExec.java b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/SimpleRootExec.java
index d125ec0..c6434f7 100644
--- a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/SimpleRootExec.java
+++ b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/SimpleRootExec.java
@@ -5,6 +5,7 @@ import org.apache.drill.exec.physical.impl.ScreenCreator.ScreenRoot;
 import org.apache.drill.exec.record.RecordBatch;
 import org.apache.drill.exec.record.RecordBatch.IterOutcome;
 import org.apache.drill.exec.record.RecordBatch.TypedFieldId;
+import org.apache.drill.exec.record.selection.SelectionVector2;
 import org.apache.drill.exec.vector.ValueVector;
 
 public class SimpleRootExec implements RootExec{
@@ -21,6 +22,9 @@ public class SimpleRootExec implements RootExec{
     
   }
 
+  public SelectionVector2 getSelectionVector2(){
+    return incoming.getSelectionVector2();
+  }
 
   public <T extends ValueVector> T getValueVectorById(SchemaPath path, Class<?> vvClass){
     TypedFieldId tfid = incoming.getValueVectorId(path);

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/65e2cfce/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
new file mode 100644
index 0000000..df11aa7
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/filter/TestSimpleFilter.java
@@ -0,0 +1,58 @@
+package org.apache.drill.exec.physical.impl.filter;
+
+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.FragmentHandle;
+import org.apache.drill.exec.rpc.user.UserServer.UserClientConnection;
+import org.apache.drill.exec.server.DrillbitContext;
+import org.junit.After;
+import org.junit.Test;
+
+import com.google.common.base.Charsets;
+import com.google.common.io.Files;
+import com.yammer.metrics.MetricRegistry;
+
+public class TestSimpleFilter {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestSimpleFilter.class);
+  DrillConfig c = DrillConfig.create();
+  
+  
+  @Test
+  public void project(@Injectable final DrillbitContext bitContext, @Injectable UserClientConnection connection) throws Exception{
+    System.out.println(System.getProperty("java.class.path"));
+
+    
+    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("/filter/test1.json"), Charsets.UTF_8));
+    FunctionImplementationRegistry registry = new FunctionImplementationRegistry(c);
+    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()){
+      System.out.println(exec.getSelectionVector2().getCount());
+    }
+  }
+  
+  @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/65e2cfce/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 925faf7..68b8881 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
@@ -31,7 +31,7 @@ import org.apache.drill.exec.proto.SchemaDefProtos.NamePart;
 import org.apache.drill.exec.record.RecordBatch.TypedFieldId;
 import org.junit.Test;
 
-import com.beust.jcommander.internal.Lists;
+import com.google.common.collect.Lists;
 import com.google.common.collect.Range;
 
 public class ExpressionTreeMaterializerTest {

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/65e2cfce/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/JSONRecordReaderTest.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/JSONRecordReaderTest.java b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/JSONRecordReaderTest.java
index 7c9e8f4..4a0358e 100644
--- a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/JSONRecordReaderTest.java
+++ b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/JSONRecordReaderTest.java
@@ -24,7 +24,8 @@ import org.apache.drill.exec.vector.ValueVector;
 import org.junit.Ignore;
 import org.junit.Test;
 
-import com.beust.jcommander.internal.Lists;
+import com.google.common.collect.Lists;
+
 
 public class JSONRecordReaderTest {
   private static final Charset UTF_8 = Charset.forName("UTF-8");

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/65e2cfce/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
new file mode 100644
index 0000000..a892c70
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/test/resources/filter/test1.json
@@ -0,0 +1,34 @@
+{
+    head:{
+        type:"APACHE_DRILL_PHYSICAL",
+        version:"1",
+        generator:{
+            type:"manual"
+        }
+    },
+	graph:[
+        {
+            @id:1,
+            pop:"mock-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,
+            child: 1,
+            pop:"filter",
+            expr: "true"
+        },
+        {
+            @id: 3,
+            child: 2,
+            pop: "screen"
+        }
+    ]
+}
\ No newline at end of file


[08/53] [abbrv] Update typing system. Update RPC system. Add Fragmenting Implementation. Working single node. Distributed failing due to threading issues.

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/LocalFragmentHandler.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/LocalFragmentHandler.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/LocalFragmentHandler.java
new file mode 100644
index 0000000..3f710ed
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/LocalFragmentHandler.java
@@ -0,0 +1,69 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.work.fragment;
+
+import org.apache.drill.exec.exception.FragmentSetupException;
+import org.apache.drill.exec.proto.ExecProtos.FragmentHandle;
+import org.apache.drill.exec.record.RawFragmentBatch;
+import org.apache.drill.exec.rpc.RemoteConnection.ConnectionThrottle;
+import org.apache.drill.exec.work.FragmentRunner;
+import org.apache.drill.exec.work.batch.IncomingBuffers;
+
+public class LocalFragmentHandler implements IncomingFragmentHandler{
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(LocalFragmentHandler.class);
+
+  private final IncomingBuffers buffers;
+  private final FragmentRunner runner;
+  private final FragmentHandle handle;
+  private volatile boolean cancel = false;
+  
+  public LocalFragmentHandler(FragmentHandle handle, IncomingBuffers buffers, FragmentRunner runner) {
+    super();
+    this.handle = handle;
+    this.buffers = buffers;
+    this.runner = runner;
+  }
+
+  @Override
+  public boolean handle(ConnectionThrottle throttle, RawFragmentBatch batch) throws FragmentSetupException {
+    return buffers.batchArrived(throttle, batch);
+  }
+
+  @Override
+  public FragmentRunner getRunnable() {
+    return runner;
+  }
+
+  
+  public FragmentHandle getHandle() {
+    return handle;
+  }
+
+  @Override
+  public void cancel() {
+    cancel = true;
+  }
+
+  @Override
+  public boolean isDone() {
+    return cancel || isDone();
+  }
+  
+  
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/RemoteFragmentHandler.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/RemoteFragmentHandler.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/RemoteFragmentHandler.java
new file mode 100644
index 0000000..70d7e93
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/RemoteFragmentHandler.java
@@ -0,0 +1,123 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.work.fragment;
+
+import java.io.IOException;
+
+import org.apache.drill.common.exceptions.ExecutionSetupException;
+import org.apache.drill.exec.exception.FragmentSetupException;
+import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.physical.base.FragmentLeaf;
+import org.apache.drill.exec.physical.base.FragmentRoot;
+import org.apache.drill.exec.physical.impl.ImplCreator;
+import org.apache.drill.exec.physical.impl.RootExec;
+import org.apache.drill.exec.planner.PhysicalPlanReader;
+import org.apache.drill.exec.proto.ExecProtos.FragmentHandle;
+import org.apache.drill.exec.proto.ExecProtos.PlanFragment;
+import org.apache.drill.exec.record.RawFragmentBatch;
+import org.apache.drill.exec.rpc.RemoteConnection.ConnectionThrottle;
+import org.apache.drill.exec.rpc.bit.BitTunnel;
+import org.apache.drill.exec.server.DrillbitContext;
+import org.apache.drill.exec.work.FragmentRunner;
+import org.apache.drill.exec.work.FragmentRunnerListener;
+import org.apache.drill.exec.work.RemotingFragmentRunnerListener;
+import org.apache.drill.exec.work.batch.IncomingBuffers;
+
+/**
+ * This handler receives all incoming traffic for a particular FragmentHandle.  It will monitor the state of incoming batches
+ */
+public class RemoteFragmentHandler implements IncomingFragmentHandler {
+  private final PlanFragment fragment;
+  private FragmentLeaf root;
+  private final IncomingBuffers buffers;
+  private final FragmentRunnerListener runnerListener;
+  private volatile FragmentRunner runner;
+  private volatile boolean cancel = false;
+  private final FragmentContext context;
+  private final PhysicalPlanReader reader;
+  
+  public RemoteFragmentHandler(PlanFragment fragment, DrillbitContext context, BitTunnel foremanTunnel) throws FragmentSetupException{
+    try{
+      this.fragment = fragment;
+      this.root = context.getPlanReader().readFragmentOperator(fragment.getFragmentJson());
+      this.buffers = new IncomingBuffers(root);
+      this.context = new FragmentContext(context, fragment.getHandle(), null, buffers);
+      this.runnerListener = new RemotingFragmentRunnerListener(this.context, foremanTunnel);
+      this.reader = context.getPlanReader();
+      
+    }catch(IOException e){
+      throw new FragmentSetupException("Failure while decoding fragment.", e);
+    }
+  }
+
+  /* (non-Javadoc)
+   * @see org.apache.drill.exec.work.fragment.FragmentHandler#handle(org.apache.drill.exec.rpc.RemoteConnection.ConnectionThrottle, org.apache.drill.exec.record.RawFragmentBatch)
+   */
+  @Override
+  public boolean handle(ConnectionThrottle throttle, RawFragmentBatch batch) throws FragmentSetupException {
+    return buffers.batchArrived(throttle, batch);
+  }
+
+  /* (non-Javadoc)
+   * @see org.apache.drill.exec.work.fragment.FragmentHandler#getRunnable()
+   */
+  @Override
+  public FragmentRunner getRunnable(){
+    synchronized(this){
+      if(runner != null) throw new IllegalStateException("Get Runnable can only be run once.");
+      if(cancel) return null;
+      try {
+        FragmentRoot fragRoot = reader.readFragmentOperator(fragment.getFragmentJson());
+        RootExec exec = ImplCreator.getExec(context, fragRoot);
+        runner = new FragmentRunner(context, exec, runnerListener);
+        return this.runner;
+      } catch (IOException | ExecutionSetupException e) {
+        runnerListener.fail(fragment.getHandle(), "Failure while setting up remote fragment.", e);
+        return null;
+      }
+    }
+    
+  }
+
+  /* (non-Javadoc)
+   * @see org.apache.drill.exec.work.fragment.FragmentHandler#cancel()
+   */
+  @Override
+  public void cancel(){
+    synchronized(this){
+      cancel = true;
+      if(runner != null){
+        runner.cancel();
+      }
+    }
+  }
+
+  @Override
+  public FragmentHandle getHandle() {
+    return fragment.getHandle();
+  }
+
+  @Override
+  public boolean isDone() {
+    return cancel || buffers.isDone();
+  }
+  
+  
+
+  
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/user/UserWorker.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/user/UserWorker.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/user/UserWorker.java
new file mode 100644
index 0000000..621c7cb
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/user/UserWorker.java
@@ -0,0 +1,72 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.work.user;
+
+import java.util.UUID;
+
+import org.apache.drill.exec.proto.ExecProtos.FragmentHandle;
+import org.apache.drill.exec.proto.GeneralRPCProtos.Ack;
+import org.apache.drill.exec.proto.UserBitShared.QueryId;
+import org.apache.drill.exec.proto.UserProtos.QueryResult;
+import org.apache.drill.exec.proto.UserProtos.QueryResult.QueryState;
+import org.apache.drill.exec.proto.UserProtos.RequestResults;
+import org.apache.drill.exec.proto.UserProtos.RunQuery;
+import org.apache.drill.exec.rpc.Acks;
+import org.apache.drill.exec.rpc.user.UserServer.UserClientConnection;
+import org.apache.drill.exec.work.FragmentRunner;
+import org.apache.drill.exec.work.WorkManager.WorkerBee;
+import org.apache.drill.exec.work.foreman.Foreman;
+
+public class UserWorker{
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(UserWorker.class);
+  
+  private final WorkerBee bee;
+    
+  public UserWorker(WorkerBee bee) {
+    super();
+    this.bee = bee;
+  }
+
+  public QueryId submitWork(UserClientConnection connection, RunQuery query){
+    UUID uuid = UUID.randomUUID();
+    QueryId id = QueryId.newBuilder().setPart1(uuid.getMostSignificantBits()).setPart2(uuid.getLeastSignificantBits()).build();
+    Foreman foreman = new Foreman(bee, bee.getContext(), connection, id, query);
+    bee.addNewForeman(foreman);
+    return id;
+  }
+  
+  public QueryResult getResult(UserClientConnection connection, RequestResults req){
+    Foreman foreman = bee.getForemanForQueryId(req.getQueryId());
+    if(foreman == null) return QueryResult.newBuilder().setQueryState(QueryState.UNKNOWN_QUERY).build();
+    return foreman.getResult(connection, req);
+  }
+
+  public Ack cancelQuery(QueryId query){
+    Foreman foreman = bee.getForemanForQueryId(query);
+    if(foreman != null){
+      foreman.cancel();
+    }
+    return Acks.OK;
+  }
+  
+  public Ack cancelFragment(FragmentHandle handle){
+    FragmentRunner runner = bee.getFragmentRunner(handle);
+    if(runner != null) runner.cancel();
+    return Acks.OK;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/protobuf/Coordination.proto
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/protobuf/Coordination.proto b/sandbox/prototype/exec/java-exec/src/main/protobuf/Coordination.proto
new file mode 100644
index 0000000..5cc5cab
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/protobuf/Coordination.proto
@@ -0,0 +1,26 @@
+package exec;
+
+option java_package = "org.apache.drill.exec.proto";
+option java_outer_classname = "CoordinationProtos";
+option optimize_for = SPEED;
+
+message DrillbitEndpoint{
+  optional string address = 1;
+  optional int32 user_port = 2;
+  optional int32 bit_port = 3;
+  optional Roles roles = 4;
+}
+
+message DrillServiceInstance{
+  optional string id = 1;
+  optional int64 registrationTimeUTC = 2;
+  optional DrillbitEndpoint endpoint = 3;
+}
+
+message Roles{
+	optional bool sql_query = 1 [default = true];
+	optional bool logical_plan = 2 [default = true];
+	optional bool physical_plan = 3 [default = true];
+	optional bool java_executor = 4 [default = true];
+	optional bool distributed_cache = 5 [default = true];
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/protobuf/ExecutionProtos.proto
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/protobuf/ExecutionProtos.proto b/sandbox/prototype/exec/java-exec/src/main/protobuf/ExecutionProtos.proto
index 77a7ee1..7501d7c 100644
--- a/sandbox/prototype/exec/java-exec/src/main/protobuf/ExecutionProtos.proto
+++ b/sandbox/prototype/exec/java-exec/src/main/protobuf/ExecutionProtos.proto
@@ -2,9 +2,11 @@ package exec.bit;
 
 option java_package = "org.apache.drill.exec.proto";
 option java_outer_classname = "ExecProtos";
-option optimize_for = LITE_RUNTIME;
-import "SchemaDef.proto";
+option optimize_for = SPEED;
+
 import "Coordination.proto";
+import "UserBitShared.proto";
+
 
 
 ////// UserToBit RPC ///////
@@ -16,7 +18,7 @@ enum RpcType {
     // bit requests
     REQ_INIATILIZE_FRAGMENT = 3; // Returns Handle
     REQ_RECORD_BATCH = 4; // send record batch overview, returns Ack
-    REQ_BATCH_CHUNK = 5; // send additional batch chunk, returns Ack.
+    
     REQ_CANCEL_FRAGMENT = 6; // send a cancellation message for a fragment, returns Ack
 	REQ_FRAGMENT_STATUS = 7; // get a fragment status, returns FragmentStatus
 	REQ_BIT_STATUS = 8; // get bit status.
@@ -25,34 +27,29 @@ enum RpcType {
     RESP_FRAGMENT_HANDLE = 9;
     RESP_FRAGMENT_STATUS = 10;
 	RESP_BIT_STATUS = 11;
-	RESP_BATCH_CHUNK = 12;
 }
 
 
 message BitHandshake{
-	optional DrillbitEndpoint endpoint = 1;
+	optional int32 rpc_version = 1;
+	optional DrillbitEndpoint endpoint = 2;
 }
 
 message BitBatchChunk {}
 
 message BitStatus {
-	repeated ActiveFragment fragment = 1;
-}
-
-message ActiveFragment {
-	optional FragmentStatus status = 1;
-	optional int64 fragment_id = 2;
-	optional int64 query_id = 3; 
+	repeated FragmentStatus fragment_status = 1;
 }
 
 message FragmentStatus {
 	
 	enum FragmentState {
-	  AWAITING_ALLOCATION = 0;
-	  RUNNING = 1;
-	  FINISHED = 2;
-	  CANCELLED = 3;
-	  FAILED = 4;
+	  SENDING = 0;
+	  AWAITING_ALLOCATION = 1;
+	  RUNNING = 2;
+	  FINISHED = 3;
+	  CANCELLED = 4;
+	  FAILED = 5;
 	}
 	
 	optional int64 memory_use = 1;
@@ -61,27 +58,37 @@ message FragmentStatus {
 	optional int32 estimated_completion_percentage = 4;
 	optional FragmentState state = 5;
 	optional int64 data_processed = 6;
+	
+	optional FragmentHandle handle = 7;
+	optional exec.shared.DrillPBError error = 8;
+	optional int64 running_time = 9;
 }
 
-message RecordBatchHeader {
+message FragmentRecordBatch{
+	optional FragmentHandle handle = 1;
+	optional int32 sending_major_fragment_id = 2;
+	optional int32 sending_minor_fragment_id = 3;
+	optional exec.shared.RecordBatchDef def = 4;
+	optional bool isLastBatch = 5;
 }
 
 message PlanFragment {
-	optional int64 query_id = 1;
-	optional int32 major_fragment_id = 2;
-	optional int32 minor_fragment_id = 3;
+	optional FragmentHandle handle = 1;
 	optional float network_cost = 4;
 	optional float cpu_cost = 5;
 	optional float disk_cost = 6;
 	optional float memory_cost = 7;
 	optional string fragment_json = 8;
-	optional bool self_driven = 9;
 	optional DrillbitEndpoint assignment = 10;
+	optional bool leaf_fragment = 9;
+	optional DrillbitEndpoint foreman = 11;
+
 }
 
 message FragmentHandle {
-	optional int32 major_fragment_id = 1;
-	optional int32 minor_fragment_id = 1;
+	optional exec.shared.QueryId query_id = 1;
+	optional int32 major_fragment_id = 2;
+	optional int32 minor_fragment_id = 3;
 }
 
 message WorkQueueStatus{

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/protobuf/GeneralRPC.proto
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/protobuf/GeneralRPC.proto b/sandbox/prototype/exec/java-exec/src/main/protobuf/GeneralRPC.proto
index ebc7dca..48011bf 100644
--- a/sandbox/prototype/exec/java-exec/src/main/protobuf/GeneralRPC.proto
+++ b/sandbox/prototype/exec/java-exec/src/main/protobuf/GeneralRPC.proto
@@ -2,7 +2,9 @@ package exec.rpc;
 
 option java_package = "org.apache.drill.exec.proto";
 option java_outer_classname = "GeneralRPCProtos";
-option optimize_for = LITE_RUNTIME;
+option optimize_for = SPEED;
+
+import "Coordination.proto";
 
 message Ack{
 	optional bool ok = 1;
@@ -33,3 +35,5 @@ message RpcFailure {
   optional string short_error = 3;
   optional string long_error = 4;
 }
+
+

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/protobuf/SchemaDef.proto
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/protobuf/SchemaDef.proto b/sandbox/prototype/exec/java-exec/src/main/protobuf/SchemaDef.proto
index 6e983d4..de0009a 100644
--- a/sandbox/prototype/exec/java-exec/src/main/protobuf/SchemaDef.proto
+++ b/sandbox/prototype/exec/java-exec/src/main/protobuf/SchemaDef.proto
@@ -2,29 +2,62 @@ package exec;
 
 option java_package = "org.apache.drill.exec.proto";
 option java_outer_classname = "SchemaDefProtos";
-option optimize_for = LITE_RUNTIME;
+option optimize_for = SPEED;
 
 
 // Schema Definitions //
-enum DataType {
-  LATE = 0;
-  INT32 = 1;
-  INT64 = 2;
-  FLOAT32 = 3;
-  FLOAT64 = 4;
-  UTF8 = 5;
-  BYTES = 6;
+enum MinorType {
+    LATE = 0;   //  late binding type
+    MAP = 1;   //  an empty map column.  Useful for conceptual setup.  Children listed within here
+    REPEATMAP = 2;   //  a repeated map column (means that multiple children sit below this)
+    TINYINT = 3;   //  single byte signed integer
+    SMALLINT = 4;   //  two byte signed integer
+    INT = 5;   //  four byte signed integer
+    BIGINT = 6;   //  eight byte signed integer
+    DECIMAL4 = 7;   //  a decimal supporting precision between 1 and 8 (4 bits for decimal location, 1 sign)
+    DECIMAL8 = 8;   //  a decimal supporting precision between 9 and 18 (5 bits for decimal location, 1 sign)
+    DECIMAL12 = 9;   //  a decimal supporting precision between19 and 28 (5 bits for decimal location, 1 sign)
+    DECIMAL16 = 10;   //  a decimal supporting precision between 29 and 37 (6 bits for decimal location, 1 sign)
+    MONEY = 11;   //  signed decimal with two digit precision
+    DATE = 12;   //  days since 4713bc 
+    TIME = 13;   //  time in micros before or after 2000/1/1
+    TIMETZ = 14;   //  time in micros before or after 2000/1/1 with timezone
+    TIMESTAMP = 15;   //  unix epoch time in millis
+    DATETIME = 16;   //  TBD
+    INTERVAL = 17;   //  TBD
+    FLOAT4 = 18;   //  4 byte ieee 754 
+    FLOAT8 = 19;   //  8 byte ieee 754
+    BOOLEAN = 20;   //  single bit value
+    FIXEDCHAR = 21;   //  utf8 fixed length string, padded with spaces
+    VARCHAR1 = 22;   //  utf8 variable length string (up to 2^8 in length)
+    VARCHAR2 = 23;   //  utf8 variable length string (up to 2^16 in length)
+    VARCHAR4 = 24;   //  utf8 variable length string (up to 2^32 in length)
+    FIXEDBINARY = 25;   //  fixed length binary, padded with 0 bytes
+    VARBINARY1 = 26;   //  variable length binary (up to 2^8 in length)
+    VARBINARY2 = 27;   //  variable length binary (up to 2^16 in length)
+    VARBINARY4 = 28;   //  variable length binary (up to 2^32 in length)
+    UINT1 = 29;   //  unsigned 1 byte integer
+    UINT2 = 30;   //  unsigned 2 byte integer
+    UINT4 = 31;   //  unsigned 4 byte integer
+    UINT8 = 32;   //  unsigned 8 byte integer
+    PROTO2 = 33;   //  protobuf encoded complex type. (up to 2^16 in length)
+    PROTO4 = 34;   //  protobuf encoded complex type. (up to 2^32 in length)
+    MSGPACK2 = 35;   //  msgpack encoded complex type. (up to 2^16 in length)
+    MSGPACK4 = 36;   //  msgpack encoded complex type. (up to 2^32 in length)
 }
 
-enum DataMode {
-  REQUIRED = 0;
-  OPTIONAL = 1;
-  REPEATED = 2;
-  MAP = 3; 
+message MajorType {
+  optional MinorType minor_type = 1;
+  optional DataMode mode = 2;
+  optional int32 width = 3; // optional width for fixed size values.
+  optional int32 precision = 4; // used for decimal types
+  optional int32 scale = 5; // used for decimal types 
 }
 
-message SchemaDef {
-  repeated FieldDef field = 1;
+enum DataMode {
+  OPTIONAL = 0; // nullable
+  REQUIRED = 1; // non-nullable
+  REPEATED = 2; // single, repeated-field
 }
 
 enum ValueMode {
@@ -33,12 +66,21 @@ enum ValueMode {
 	DICT = 2;
 }
 
+message NamePart {
+  enum Type{
+    NAME = 0;
+    ARRAY = 1;
+  }
+  
+  optional Type type = 1;
+  optional string name = 2; // only required if this is a named type.
+}
+
 message FieldDef {
-  optional string name = 1;
-  optional DataMode data_mode = 2;
-  optional ValueMode value_mode = 3;
+  optional int32 field_id = 1;
+  optional int32 parent_id = 2; // the field_id of the parent of this field.  populated when this is a repeated field.  a field_id of 0 means that the record is the parent of this repeated field.
+  repeated NamePart name = 3; // multipart description of entire field name
+  optional MajorType major_type = 4; // the type associated with this field.
+  repeated FieldDef field = 5; // only in the cases of type == MAP or REPEATMAP
   
-  // If DataMode == 0-2, type should be populated and fields should be empty.  Otherwise, type should empty and fields should be defined. 
-  optional DataType type = 4;
-  repeated FieldDef fields = 5;
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/protobuf/User.proto
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/protobuf/User.proto b/sandbox/prototype/exec/java-exec/src/main/protobuf/User.proto
index 225d1a0..cbf5b4c 100644
--- a/sandbox/prototype/exec/java-exec/src/main/protobuf/User.proto
+++ b/sandbox/prototype/exec/java-exec/src/main/protobuf/User.proto
@@ -2,8 +2,12 @@ package exec.user;
 
 option java_package = "org.apache.drill.exec.proto";
 option java_outer_classname = "UserProtos";
-option optimize_for = LITE_RUNTIME;
+option optimize_for = SPEED;
+
 import "SchemaDef.proto";
+import "UserBitShared.proto";
+
+
 
 ////// UserToBit RPC ///////
 enum RpcType {
@@ -13,7 +17,8 @@ enum RpcType {
     
     // user to bit
     RUN_QUERY = 3;
-    REQUEST_RESULTS = 4;
+    CANCEL_QUERY = 4;
+    REQUEST_RESULTS = 5;
     
     // bit to user
 	QUERY_RESULT = 6;
@@ -21,33 +26,36 @@ enum RpcType {
 }
 
 message UserToBitHandshake {
-    optional bool support_listening = 1;
-    optional int32 rpc_version = 2;
+    optional bool support_listening = 2;
+    optional int32 rpc_version = 3;
 }
 
 message RequestResults {
-  optional int64 query_id = 1;
+  optional exec.shared.QueryId query_id = 1;
   optional int32 maximum_responses = 2;
 }
 
 message RunQuery {
-  optional QueryResultsMode mode = 1;
-  optional string plan = 2;
+  optional QueryResultsMode results_mode = 1;
+  optional QueryType type = 2;
+  optional string plan = 3;
+}
+
+enum QueryType {
+  SQL = 1;
+  LOGICAL = 2;
+  PHYSICAL = 3;
 }
 
 enum QueryResultsMode {
 	STREAM_FULL = 1; // Server will inform the client regularly on the status of the query. Once the query is completed, service will inform the client as each query chunk is made available.
-	STREAM_FIRST = 2; // Server will inform the client regularly on the status of the query.  Once the query is completed, server will inform the client of the first query chunk.
-	QUERY_FOR_STATUS = 3; // Client will need to query for status of query.
+	// STREAM_FIRST = 2; // Server will inform the client regularly on the status of the query.  Once the query is completed, server will inform the client of the first query chunk.
+	// QUERY_FOR_STATUS = 3; // Client will need to query for status of query.
 }
 
 
 message BitToUserHandshake {
-	optional int32 rpc_version = 1;
-}
-
-message QueryHandle {
-  	optional int64 query_id = 1;
+	optional int32 rpc_version = 2;
 }
 
 message NodeStatus {
@@ -56,37 +64,26 @@ message NodeStatus {
 }
 
 message QueryResult {
-	enum Outcome {
-	  RUNNING = 0;
-	  FAILED = 1;
+	enum QueryState {
+	  PENDING = 0;
+	  RUNNING = 1;
 	  COMPLETED = 2;
-	  WAITING = 3;
+	  CANCELED = 3;
+	  FAILED = 4;
+	  UNKNOWN_QUERY = 5;
 	}
 	
-	optional Outcome outcome = 1;
-	optional SchemaDef schema = 2;
+	optional QueryState query_state = 1;
+	optional exec.shared.QueryId query_id = 2;
 	optional bool is_last_chunk = 3;
 	optional int32 row_count = 4;
 	optional int64 records_scan = 5;
 	optional int64 records_error = 6;
 	optional int64 submission_time = 7;
 	repeated NodeStatus node_status = 8;	
-	repeated Error error = 9;
-}
-
-message TextErrorLocation{
-    optional int32 start_column = 2;
-    optional int32 start_row = 3;
-    optional int32 end_column = 4;
-    optional int32 end_row = 5;
-}
-
-message Error{
-    optional int64 error_id = 1; // for debug tracing purposes
-    optional string host = 2;
-    optional int32 error_type = 3; 
-    optional string message = 4;
-    optional TextErrorLocation error = 5; //optional, used when providing location of error within a piece of text.
+	repeated exec.shared.DrillPBError error = 9;
+	optional exec.shared.RecordBatchDef def = 10;
+	optional bool schema_changed = 11;
 }
 
 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/protobuf/UserBitShared.proto
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/protobuf/UserBitShared.proto b/sandbox/prototype/exec/java-exec/src/main/protobuf/UserBitShared.proto
new file mode 100644
index 0000000..5643c0f
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/protobuf/UserBitShared.proto
@@ -0,0 +1,46 @@
+package exec.shared;
+
+option java_package = "org.apache.drill.exec.proto";
+option java_outer_classname = "UserBitShared";
+option optimize_for = SPEED;
+
+import "Coordination.proto";
+import "SchemaDef.proto";
+
+message QueryId {
+  	optional sfixed64 part1 = 1;
+  	optional sfixed64 part2 = 2;
+}
+
+message DrillPBError{
+    optional string error_id = 1; // for debug tracing purposes
+    optional DrillbitEndpoint endpoint = 2;
+    optional int32 error_type = 3; 
+    optional string message = 4;
+    repeated ParsingError parsing_error = 5; //optional, used when providing location of error within a piece of text.
+}
+
+message ParsingError{
+    optional int32 start_column = 2;
+    optional int32 start_row = 3;
+    optional int32 end_column = 4;
+    optional int32 end_row = 5;
+}
+
+message RecordBatch{
+    
+}
+
+message RecordBatchDef {
+	repeated FieldMetadata field = 1;
+	optional int32 record_count = 2;
+	
+}
+
+message FieldMetadata {
+  optional FieldDef def = 1;
+  optional int32 value_count = 2;
+  optional int32 var_byte_length = 3;
+  optional int32 group_count = 4; // number of groups.  (number of repeated records)
+  optional int32 buffer_length = 5;
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/DrillSystemTestBase.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/DrillSystemTestBase.java b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/DrillSystemTestBase.java
index 37ba12b..d113ca3 100644
--- a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/DrillSystemTestBase.java
+++ b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/DrillSystemTestBase.java
@@ -103,4 +103,11 @@ public class DrillSystemTestBase {
     }
   }
 
+  public Drillbit getABit(){
+    return this.servers.iterator().next();
+  }
+  
+  public static DrillConfig getConfig(){
+    return config;
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/client/DrillClientSystemTest.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/client/DrillClientSystemTest.java b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/client/DrillClientSystemTest.java
index 09a06d7..dc463e3 100644
--- a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/client/DrillClientSystemTest.java
+++ b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/client/DrillClientSystemTest.java
@@ -1,17 +1,20 @@
 package org.apache.drill.exec.client;
 
-import com.google.common.base.Charsets;
-import com.google.common.io.Resources;
+import java.util.List;
+
 import org.apache.drill.exec.DrillSystemTestBase;
-import org.apache.drill.exec.proto.UserProtos;
-import org.apache.drill.exec.rpc.DrillRpcFuture;
+import org.apache.drill.exec.proto.UserProtos.QueryType;
+import org.apache.drill.exec.rpc.user.QueryResultBatch;
 import org.junit.After;
 import org.junit.BeforeClass;
+import org.junit.Ignore;
 import org.junit.Test;
 
-/**
- * @author David Alves
- */
+import com.google.common.base.Charsets;
+import com.google.common.io.Resources;
+
+
+@Ignore
 public class DrillClientSystemTest extends DrillSystemTestBase {
 
   private static String plan;
@@ -34,8 +37,8 @@ public class DrillClientSystemTest extends DrillSystemTestBase {
     startCluster(1);
     DrillClient client = new DrillClient();
     client.connect();
-    DrillRpcFuture<UserProtos.QueryHandle> result = client.submitPlan(plan);
-    System.out.println(result.get());
+    List<QueryResultBatch> result = client.runQuery(QueryType.LOGICAL, plan);
+    System.out.println(result);
     client.close();
   }
 
@@ -45,8 +48,8 @@ public class DrillClientSystemTest extends DrillSystemTestBase {
     startCluster(2);
     DrillClient client = new DrillClient();
     client.connect();
-    DrillRpcFuture<UserProtos.QueryHandle> result = client.submitPlan(plan);
-    System.out.println(result.get());
+    List<QueryResultBatch> result = client.runQuery(QueryType.LOGICAL, plan);
+    System.out.println(result);
     client.close();
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/compile/TestClassCompilationTypes.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/compile/TestClassCompilationTypes.java b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/compile/TestClassCompilationTypes.java
index 3d5d84e..2f8aa18 100644
--- a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/compile/TestClassCompilationTypes.java
+++ b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/compile/TestClassCompilationTypes.java
@@ -18,14 +18,15 @@
 package org.apache.drill.exec.compile;
 
 import org.codehaus.commons.compiler.jdk.ExpressionEvaluator;
+import org.junit.Ignore;
 import org.junit.Test;
 
 public class TestClassCompilationTypes {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestClassCompilationTypes.class);
 
-  @Test
+  @Ignore @Test
   public void comparePerfs() throws Exception {
-    for(int i =0; i < 50000; i++){
+    for(int i =0; i < 500; i++){
       int r = 0;
       long n0 = System.nanoTime();
       r += janino();

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/config/ParsePhysicalPlan.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/config/ParsePhysicalPlan.java b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/config/ParsePhysicalPlan.java
new file mode 100644
index 0000000..3b6bf6a
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/config/ParsePhysicalPlan.java
@@ -0,0 +1,42 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.physical.config;
+
+import org.apache.drill.common.config.DrillConfig;
+import org.apache.drill.common.util.FileUtils;
+import org.apache.drill.exec.physical.PhysicalPlan;
+import org.junit.Test;
+
+import com.fasterxml.jackson.databind.ObjectReader;
+import com.fasterxml.jackson.databind.ObjectWriter;
+import com.google.common.base.Charsets;
+import com.google.common.io.Files;
+
+public class ParsePhysicalPlan {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ParsePhysicalPlan.class);
+  
+  
+  @Test 
+  public void parseSimplePlan() throws Exception{
+    DrillConfig c = DrillConfig.create();
+    ObjectReader r = c.getMapper().reader(PhysicalPlan.class);
+    ObjectWriter writer = c.getMapper().writer();
+    PhysicalPlan plan = PhysicalPlan.parse(r, Files.toString(FileUtils.getResourceAsFile("/physical_test1.json"), Charsets.UTF_8));
+    System.out.println(plan.unparse(writer));
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/DistributedFragmentRun.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/DistributedFragmentRun.java b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/DistributedFragmentRun.java
new file mode 100644
index 0000000..7c6bfe5
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/DistributedFragmentRun.java
@@ -0,0 +1,53 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.physical.impl;
+
+import java.util.List;
+
+import org.apache.drill.common.util.FileUtils;
+import org.apache.drill.exec.client.DrillClient;
+import org.apache.drill.exec.pop.PopUnitTestBase;
+import org.apache.drill.exec.proto.UserProtos.QueryType;
+import org.apache.drill.exec.rpc.user.QueryResultBatch;
+import org.apache.drill.exec.server.Drillbit;
+import org.apache.drill.exec.server.RemoteServiceSet;
+import org.junit.Ignore;
+import org.junit.Test;
+
+import com.google.common.base.Charsets;
+import com.google.common.io.Files;
+
+@Ignore
+public class DistributedFragmentRun extends PopUnitTestBase{
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DistributedFragmentRun.class);
+  
+  
+  @Test 
+  public void simpleDistributedQuery() throws Exception{
+    RemoteServiceSet serviceSet = RemoteServiceSet.getLocalServiceSet();
+    try(Drillbit bit1 = new Drillbit(CONFIG, serviceSet); Drillbit bit2 = new Drillbit(CONFIG, serviceSet); DrillClient client = new DrillClient(CONFIG, serviceSet.getCoordinator());){
+      bit1.run();
+      bit2.run();
+      client.connect();
+      List<QueryResultBatch> results = client.runQuery(QueryType.PHYSICAL, Files.toString(FileUtils.getResourceAsFile("/physical_single_exchange.json"), Charsets.UTF_8));
+      System.out.println(results);
+    }
+    
+
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/SimpleFragmentRun.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/SimpleFragmentRun.java b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/SimpleFragmentRun.java
new file mode 100644
index 0000000..6755bb6
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/SimpleFragmentRun.java
@@ -0,0 +1,100 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.physical.impl;
+
+import 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;
+import org.apache.drill.exec.proto.UserProtos.QueryType;
+import org.apache.drill.exec.record.RecordBatchLoader;
+import org.apache.drill.exec.record.vector.ValueVector;
+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.Test;
+
+import com.carrotsearch.hppc.cursors.IntObjectCursor;
+import com.google.common.base.Charsets;
+import com.google.common.io.Files;
+
+public class SimpleFragmentRun extends PopUnitTestBase {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(SimpleFragmentRun.class);
+
+  @Test
+  public void runNoExchangeFragment() throws Exception {
+    try(RemoteServiceSet serviceSet = RemoteServiceSet.getLocalServiceSet(); 
+        Drillbit bit = new Drillbit(CONFIG, serviceSet); 
+        DrillClient client = new DrillClient(CONFIG, serviceSet.getCoordinator());){
+    
+    // run query.
+    bit.run();
+    client.connect();
+    List<QueryResultBatch> results = client.runQuery(QueryType.PHYSICAL, Files.toString(FileUtils.getResourceAsFile("/physical_test2.json"), Charsets.UTF_8));
+    
+    // look at records
+    RecordBatchLoader batchLoader = new RecordBatchLoader(bit.getContext().getAllocator());
+    int recordCount = 0;
+    for (QueryResultBatch batch : results) {
+      if(!batch.hasData()) continue;
+      boolean schemaChanged = batchLoader.load(batch.getHeader().getDef(), batch.getData());
+      boolean firstColumn = true;
+
+      // print headers.
+      if (schemaChanged) {
+        System.out.println("\n\n========NEW SCHEMA=========\n\n");
+        for (IntObjectCursor<ValueVector<?>> v : batchLoader) {
+
+          if (firstColumn) {
+            firstColumn = false;
+          } else {
+            System.out.print("\t");
+          }
+          System.out.print(v.value.getField().getName());
+          System.out.print("[");
+          System.out.print(v.value.getField().getType().getMinorType());
+          System.out.print("]");
+        }
+        System.out.println();
+      }
+
+
+      for (int i = 0; i < batchLoader.getRecordCount(); i++) {
+        boolean first = true;
+        recordCount++;
+        for (IntObjectCursor<ValueVector<?>> v : batchLoader) {
+          if (first) {
+            first = false;
+          } else {
+            System.out.print("\t");
+          }
+          System.out.print(v.value.getObject(i));
+        }
+        if(!first) System.out.println();
+      }
+
+    }
+    logger.debug("Received results {}", results);
+    assertEquals(recordCount, 200);
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/pop/CheckFragmenter.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/pop/CheckFragmenter.java b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/pop/CheckFragmenter.java
index 98bb874..7b7ab8e 100644
--- a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/pop/CheckFragmenter.java
+++ b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/pop/CheckFragmenter.java
@@ -24,63 +24,63 @@ import static org.junit.Assert.assertNull;
 import java.io.IOException;
 
 import org.apache.drill.common.config.DrillConfig;
-import org.apache.drill.common.physical.PhysicalPlan;
-import org.apache.drill.common.physical.pop.base.PhysicalOperator;
 import org.apache.drill.common.util.FileUtils;
 import org.apache.drill.exec.exception.FragmentSetupException;
-import org.apache.drill.exec.planner.FragmentNode;
-import org.apache.drill.exec.planner.FragmentingPhysicalVisitor;
-import org.apache.drill.exec.planner.FragmentNode.ExchangeFragmentPair;
+import org.apache.drill.exec.ops.QueryContext;
+import org.apache.drill.exec.physical.PhysicalPlan;
+import org.apache.drill.exec.physical.base.PhysicalOperator;
+import org.apache.drill.exec.planner.fragment.Fragment;
+import org.apache.drill.exec.planner.fragment.PlanningSet;
+import org.apache.drill.exec.planner.fragment.StatsCollector;
+import org.apache.drill.exec.planner.fragment.MakeFragmentsVisitor;
+import org.apache.drill.exec.planner.fragment.Fragment.ExchangeFragmentPair;
+import org.apache.drill.exec.planner.PhysicalPlanReader;
+import org.apache.drill.exec.planner.SimpleExecPlanner;
+import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
+import org.apache.drill.exec.server.DrillbitContext;
+import org.apache.drill.exec.work.QueryWorkUnit;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
 import com.google.common.base.Charsets;
 import com.google.common.io.Files;
 
-public class CheckFragmenter {
+public class CheckFragmenter extends PopUnitTestBase {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(CheckFragmenter.class);
-  
-  static DrillConfig config;
-  
-  @BeforeClass
-  public static void setup(){
-    config = DrillConfig.create();
-  }
-  
+
+
   @Test
-  public void ensureOneFragment() throws FragmentSetupException, IOException{
-    FragmentNode b = getRootFragment("/physical_test1.json");
+  public void ensureOneFragment() throws FragmentSetupException, IOException {
+    PhysicalPlanReader ppr = new PhysicalPlanReader(CONFIG, CONFIG.getMapper(), DrillbitEndpoint.getDefaultInstance());
+    Fragment b = getRootFragment(ppr, "/physical_test1.json");
     assertEquals(1, getFragmentCount(b));
     assertEquals(0, b.getReceivingExchangePairs().size());
     assertNull(b.getSendingExchange());
   }
-  
+
   @Test
-  public void ensureTwoFragments() throws FragmentSetupException, IOException{
-    FragmentNode b = getRootFragment("/physical_simpleexchange.json");
-    assertEquals(2, getFragmentCount(b));
+  public void ensureThreeFragments() throws FragmentSetupException, IOException {
+    PhysicalPlanReader ppr = new PhysicalPlanReader(CONFIG, CONFIG.getMapper(), DrillbitEndpoint.getDefaultInstance());
+    Fragment b = getRootFragment(ppr, "/physical_simpleexchange.json");
+    logger.debug("Fragment Node {}", b);
+    assertEquals(3, getFragmentCount(b));
     assertEquals(1, b.getReceivingExchangePairs().size());
     assertNull(b.getSendingExchange());
-    
+
     // get first child.
     b = b.iterator().next().getNode();
+    assertEquals(1, b.getReceivingExchangePairs().size());
+    assertNotNull(b.getSendingExchange());
+
+    b = b.iterator().next().getNode();
     assertEquals(0, b.getReceivingExchangePairs().size());
     assertNotNull(b.getSendingExchange());
   }
+
   
-  private int getFragmentCount(FragmentNode b){
-    int i =1;
-    for(ExchangeFragmentPair p : b){
-      i += getFragmentCount(p.getNode());
-    }
-    return i;
-  }
+
+
   
-  private FragmentNode getRootFragment(String file) throws FragmentSetupException, IOException{
-    FragmentingPhysicalVisitor f = new FragmentingPhysicalVisitor();
-    
-    PhysicalPlan plan = PhysicalPlan.parse(config.getMapper().reader(PhysicalPlan.class), Files.toString(FileUtils.getResourceAsFile(file), Charsets.UTF_8));
-    PhysicalOperator o = plan.getSortedOperators(false).iterator().next();
-    return o.accept(f, null);
-  }
+  
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/pop/CheckInjectionValue.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/pop/CheckInjectionValue.java b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/pop/CheckInjectionValue.java
index b8fd278..1d0fb91 100644
--- a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/pop/CheckInjectionValue.java
+++ b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/pop/CheckInjectionValue.java
@@ -22,12 +22,12 @@ import static org.junit.Assert.*;
 import java.util.List;
 
 import org.apache.drill.common.config.DrillConfig;
-import org.apache.drill.common.physical.PhysicalPlan;
-import org.apache.drill.common.physical.pop.Screen;
-import org.apache.drill.common.physical.pop.base.PhysicalOperator;
-import org.apache.drill.common.proto.CoordinationProtos.DrillbitEndpoint;
 import org.apache.drill.common.util.FileUtils;
+import org.apache.drill.exec.physical.PhysicalPlan;
+import org.apache.drill.exec.physical.base.PhysicalOperator;
+import org.apache.drill.exec.physical.config.Screen;
 import org.apache.drill.exec.planner.PhysicalPlanReader;
+import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
 import org.apache.drill.exec.server.DrillbitContext;
 import org.apache.drill.exec.store.StorageEngineRegistry;
 import org.junit.BeforeClass;
@@ -48,8 +48,8 @@ public class CheckInjectionValue {
   
   @Test
   public void testInjected() throws Exception{
-    PhysicalPlanReader r = new PhysicalPlanReader(config.getMapper(), DrillbitEndpoint.getDefaultInstance());
-    PhysicalPlan p = r.read(Files.toString(FileUtils.getResourceAsFile("/physical_screen.json"), Charsets.UTF_8));
+    PhysicalPlanReader r = new PhysicalPlanReader(config, config.getMapper(), DrillbitEndpoint.getDefaultInstance());
+    PhysicalPlan p = r.readPhysicalPlan(Files.toString(FileUtils.getResourceAsFile("/physical_screen.json"), Charsets.UTF_8));
     
     List<PhysicalOperator> o = p.getSortedOperators(false);
     

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/pop/FragmentChecker.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/pop/FragmentChecker.java b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/pop/FragmentChecker.java
new file mode 100644
index 0000000..6f229a3
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/pop/FragmentChecker.java
@@ -0,0 +1,66 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.pop;
+
+import static org.junit.Assert.*;
+
+import java.util.Collection;
+import java.util.Collections;
+
+import org.apache.drill.exec.planner.PhysicalPlanReader;
+import org.apache.drill.exec.planner.fragment.Fragment;
+import org.apache.drill.exec.planner.fragment.PlanningSet;
+import org.apache.drill.exec.planner.fragment.StatsCollector;
+import org.apache.drill.exec.planner.fragment.SimpleParallelizer;
+import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
+import org.apache.drill.exec.proto.ExecProtos.PlanFragment;
+import org.apache.drill.exec.proto.UserBitShared.QueryId;
+import org.apache.drill.exec.work.QueryWorkUnit;
+import org.junit.Test;
+
+import com.google.common.collect.Lists;
+
+public class FragmentChecker extends PopUnitTestBase{
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(FragmentChecker.class);
+  
+  
+  @Test
+  public void checkSimpleExchangePlan() throws Exception{
+    
+    PhysicalPlanReader ppr = new PhysicalPlanReader(CONFIG, CONFIG.getMapper(), DrillbitEndpoint.getDefaultInstance());
+    Fragment fragmentRoot = getRootFragment(ppr, "/physical_simpleexchange.json");
+    PlanningSet planningSet = StatsCollector.collectStats(fragmentRoot);
+    SimpleParallelizer par = new SimpleParallelizer();
+    
+    DrillbitEndpoint b1 = DrillbitEndpoint.newBuilder().setAddress("localhost").setBitPort(1234).build();
+    DrillbitEndpoint b2 = DrillbitEndpoint.newBuilder().setAddress("localhost").setBitPort(2345).build();
+    
+    QueryWorkUnit qwu = par.getFragments(b1, QueryId.getDefaultInstance(), Lists.newArrayList(b1, b2), ppr, fragmentRoot, planningSet, 10);
+    assertEquals(qwu.getFragments().size(), 3);
+    System.out.println("=========ROOT FRAGMENT=========");
+    System.out.print(qwu.getRootFragment().getFragmentJson());
+    
+    
+    for(PlanFragment f : qwu.getFragments()){
+      System.out.println("=========");
+      System.out.print(f.getFragmentJson());
+    }
+    logger.debug("Planning Set {}", planningSet);
+
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/pop/PopUnitTestBase.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/pop/PopUnitTestBase.java b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/pop/PopUnitTestBase.java
new file mode 100644
index 0000000..e5e109e
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/pop/PopUnitTestBase.java
@@ -0,0 +1,71 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.pop;
+
+import java.io.IOException;
+
+import org.apache.drill.common.config.DrillConfig;
+import org.apache.drill.common.util.FileUtils;
+import org.apache.drill.exec.exception.FragmentSetupException;
+import org.apache.drill.exec.physical.PhysicalPlan;
+import org.apache.drill.exec.physical.base.PhysicalOperator;
+import org.apache.drill.exec.planner.PhysicalPlanReader;
+import org.apache.drill.exec.planner.fragment.Fragment;
+import org.apache.drill.exec.planner.fragment.Fragment.ExchangeFragmentPair;
+import org.apache.drill.exec.planner.fragment.MakeFragmentsVisitor;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.rules.TestName;
+import org.junit.rules.TestRule;
+import org.junit.rules.Timeout;
+
+import com.google.common.base.Charsets;
+import com.google.common.io.Files;
+
+public abstract class PopUnitTestBase {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(PopUnitTestBase.class);
+  
+  // Determine if we are in Eclipse Debug mode.
+  static final boolean IS_DEBUG = java.lang.management.ManagementFactory.getRuntimeMXBean().getInputArguments().toString().indexOf("-agentlib:jdwp") > 0;
+  protected static DrillConfig CONFIG;
+
+  // Set a timeout unless we're debugging.
+  @Rule public TestRule globalTimeout = IS_DEBUG ? new TestName() : new Timeout(10000);
+  
+  @BeforeClass
+  public static void setup() {
+    CONFIG = DrillConfig.create();
+  }
+
+  
+  public static int getFragmentCount(Fragment b) {
+    int i = 1;
+    for (ExchangeFragmentPair p : b) {
+      i += getFragmentCount(p.getNode());
+    }
+    return i;
+  }
+
+  public Fragment getRootFragment(PhysicalPlanReader reader, String file) throws FragmentSetupException, IOException {
+    MakeFragmentsVisitor f = new MakeFragmentsVisitor();
+
+    PhysicalPlan plan = reader.readPhysicalPlan(Files.toString(FileUtils.getResourceAsFile(file), Charsets.UTF_8));
+    PhysicalOperator o = plan.getSortedOperators(false).iterator().next();
+    return o.accept(f, null);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/rpc/user/RunRemoteQuery.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/rpc/user/RunRemoteQuery.java b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/rpc/user/RunRemoteQuery.java
deleted file mode 100644
index d003373..0000000
--- a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/rpc/user/RunRemoteQuery.java
+++ /dev/null
@@ -1,41 +0,0 @@
-/*******************************************************************************
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- ******************************************************************************/
-package org.apache.drill.exec.rpc.user;
-
-import io.netty.buffer.UnpooledByteBufAllocator;
-import io.netty.channel.nio.NioEventLoopGroup;
-
-import org.apache.drill.exec.proto.UserProtos.QueryHandle;
-import org.apache.drill.exec.proto.UserProtos.RunQuery;
-import org.apache.drill.exec.rpc.DrillRpcFuture;
-import org.junit.Test;
-
-public class RunRemoteQuery {
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(RunRemoteQuery.class);
-  
-  @Test 
-  public void runRemoteQuery() throws Exception{
-    UserClient c = new UserClient(UnpooledByteBufAllocator.DEFAULT, new NioEventLoopGroup(1));
-    c.connectAsClient("localhost", 31010);
-    DrillRpcFuture<QueryHandle> futureHandle = c.submitQuery(RunQuery.getDefaultInstance());
-    QueryHandle h = futureHandle.checkedGet();
-    System.out.println(h);
-  }
-  
-  
-}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/rpc/user/UserRpcTest.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/rpc/user/UserRpcTest.java b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/rpc/user/UserRpcTest.java
deleted file mode 100644
index c8ce877..0000000
--- a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/rpc/user/UserRpcTest.java
+++ /dev/null
@@ -1,107 +0,0 @@
-/*******************************************************************************
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- ******************************************************************************/
-package org.apache.drill.exec.rpc.user;
-
-import io.netty.buffer.ByteBuf;
-import io.netty.buffer.ByteBufAllocator;
-import io.netty.buffer.PooledByteBufAllocator;
-import io.netty.channel.nio.NioEventLoopGroup;
-
-import java.util.concurrent.TimeUnit;
-
-import org.apache.drill.exec.proto.UserProtos.QueryHandle;
-import org.apache.drill.exec.proto.UserProtos.QueryResultsMode;
-import org.apache.drill.exec.proto.UserProtos.RunQuery;
-import org.apache.drill.exec.rpc.DrillRpcFuture;
-import org.apache.drill.exec.rpc.NamedThreadFactory;
-import org.junit.Test;
-
-public class UserRpcTest {
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(UserRpcTest.class);
-  
-  
-  
-  
-  @Test
-  public void doBasicRpcTest() throws Exception {
-    final int bufferSize = 25000;
-    final int batchSize = 1000;
-    final int batchCount = 100;
-
-    
-    int sends = 0;
-    int receives = 0;
-    long nanoSend = 0;
-    long nanoReceive = 0;
-
-    
-    try {
-      ByteBufAllocator bb = new PooledByteBufAllocator(true);
-//      ByteBufAllocator bb = UnpooledByteBufAllocator.DEFAULT;
-      UserServer s = new UserServer(bb, new NioEventLoopGroup(1, new NamedThreadFactory("Server-")), null);
-      s.bind(31515);
-
-      logger.debug("Starting user client.");
-      UserClient c = new UserClient(bb, new NioEventLoopGroup(1, new NamedThreadFactory("Client-")));
-
-      logger.debug("Connecting as client to server.");
-      c.connectAsClient("localhost", 31515);
-
-      
-      @SuppressWarnings("unchecked")
-      DrillRpcFuture<QueryHandle>[] handles = new DrillRpcFuture[batchSize];
-
-      for (int x = 0; x < batchCount; x++) {
-        long s1 = System.nanoTime();
-        for (int i = 0; i < batchSize; i++) {
-          sends++;
-          ByteBuf rawBody = bb.buffer(bufferSize);
-          rawBody.writerIndex(bufferSize);
-          if(rawBody.readableBytes() != bufferSize) throw new RuntimeException();
-          handles[i] = c.submitQuery(RunQuery.newBuilder().setMode(QueryResultsMode.QUERY_FOR_STATUS).build(), rawBody);
-        }
-        
-        long s2 = System.nanoTime();
-
-        for (int i = 0; i < batchSize; i++) {
-          handles[i].checkedGet(2, TimeUnit.SECONDS).getQueryId();
-          receives++;
-        }
-
-        long s3 = System.nanoTime();
-        nanoSend += (s2-s1);
-        nanoReceive += (s3-s2);
-        logger.debug("Submission time {}ms, return time {}ms", (s2 - s1) / 1000 / 1000, (s3 - s2) / 1000 / 1000);
-      }
-      // logger.debug("Submitting query.");
-      // DrillRpcFuture<QueryHandle> handleFuture =
-      // c.submitQuery(RunQuery.newBuilder().setMode(QueryResultsMode.QUERY_FOR_STATUS).build());
-      //
-      // logger.debug("Got query id handle of {}", handleFuture.get(2, TimeUnit.SECONDS).getQueryId());
-    } catch (Exception e) {
-      logger.error("Exception of type {} occurred while doing test.", e.getClass().getCanonicalName());
-      throw e;
-    } finally{
-      long mbsTransferred = (1l * bufferSize * batchSize * batchCount)/1024/1024;
-      double sSend = nanoSend*1.0d/1000/1000/1000;
-      double sReceive = nanoReceive*1.0d/1000/1000/1000;
-      logger.info(String.format("Completed %d sends and %d receives.  Total data transferred was %d.  Send bw: %f, Receive bw: %f.", sends, receives, mbsTransferred, mbsTransferred*1.0/sSend, mbsTransferred*1.0/sReceive));
-      logger.info("Completed {} sends and {} receives.", sends, receives);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/server/TestBitRpc.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/server/TestBitRpc.java b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/server/TestBitRpc.java
new file mode 100644
index 0000000..9684e9f
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/server/TestBitRpc.java
@@ -0,0 +1,84 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.server;
+
+import io.netty.buffer.ByteBuf;
+
+import java.util.concurrent.ConcurrentMap;
+
+import org.apache.drill.common.config.DrillConfig;
+import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
+import org.apache.drill.exec.proto.ExecProtos.FragmentHandle;
+import org.apache.drill.exec.proto.ExecProtos.PlanFragment;
+import org.apache.drill.exec.proto.GeneralRPCProtos.Ack;
+import org.apache.drill.exec.rpc.Response;
+import org.apache.drill.exec.rpc.RpcException;
+import org.apache.drill.exec.rpc.bit.BitClient;
+import org.apache.drill.exec.rpc.bit.BitComImpl;
+import org.apache.drill.exec.rpc.bit.BitConnection;
+import org.apache.drill.exec.rpc.bit.BitRpcConfig;
+import org.apache.drill.exec.rpc.bit.BitServer;
+import org.apache.drill.exec.rpc.bit.ListenerPool;
+import org.apache.drill.exec.work.batch.BitComHandler;
+import org.apache.drill.exec.work.fragment.IncomingFragmentHandler;
+import org.junit.Test;
+
+import com.google.common.collect.Maps;
+
+public class TestBitRpc {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestBitRpc.class);
+  
+  @Test
+  public void testBasicConnectionAndHandshake() throws Exception{
+    int port = 1234;
+    BootStrapContext c = new BootStrapContext(DrillConfig.create());
+    ConcurrentMap<DrillbitEndpoint, BitConnection> registry = Maps.newConcurrentMap();
+    BitServer server = new BitServer(new BitComTestHandler(), c, registry, new ListenerPool(2));
+    port = server.bind(port);
+    for(int i =0; i < 10; i++){
+      BitClient client = new BitClient(DrillbitEndpoint.newBuilder().setAddress("localhost").setBitPort(port).build(), null, new BitComTestHandler(), c, registry, new ListenerPool(2));
+      client.connect();
+      
+    }
+    System.out.println("connected");
+  }
+  
+  
+  
+  private class BitComTestHandler implements BitComHandler{
+
+    @Override
+    public Response handle(BitConnection connection, int rpcType, ByteBuf pBody, ByteBuf dBody) throws RpcException {
+      return BitRpcConfig.OK;
+    }
+
+    @Override
+    public void startNewRemoteFragment(PlanFragment fragment) {
+    }
+
+    @Override
+    public Ack cancelFragment(FragmentHandle handle) {
+      return null;
+    }
+
+    @Override
+    public void registerIncomingFragmentHandler(IncomingFragmentHandler handler) {
+    }
+    
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/MockRecordConfig.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/MockRecordConfig.java b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/MockRecordConfig.java
deleted file mode 100644
index 18c6955..0000000
--- a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/MockRecordConfig.java
+++ /dev/null
@@ -1,46 +0,0 @@
-/*******************************************************************************
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- ******************************************************************************/
-package org.apache.drill.exec.store;
-
-import org.apache.drill.common.expression.types.DataType;
-import org.apache.drill.common.logical.StorageEngineConfigBase;
-
-import com.fasterxml.jackson.annotation.JsonTypeName;
-
-@JsonTypeName("mock-config")
-public class MockRecordConfig extends StorageEngineConfigBase{
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(MockRecordConfig.class);
-  
-  private int recordCount;
-  private DataType[] types;
-  
-  public int getRecordCount() {
-    return recordCount;
-  }
-  public void setRecordCount(int recordCount) {
-    this.recordCount = recordCount;
-  }
-  public DataType[] getTypes() {
-    return types;
-  }
-  public void setTypes(DataType[] types) {
-    this.types = types;
-  }
-  
-  
-}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/MockRecordReader.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/MockRecordReader.java b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/MockRecordReader.java
deleted file mode 100644
index e1f56bd..0000000
--- a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/MockRecordReader.java
+++ /dev/null
@@ -1,108 +0,0 @@
-/*******************************************************************************
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- ******************************************************************************/
-package org.apache.drill.exec.store;
-
-import org.apache.drill.common.expression.types.DataType;
-import org.apache.drill.exec.exception.ExecutionSetupException;
-import org.apache.drill.exec.exception.SchemaChangeException;
-import org.apache.drill.exec.ops.FragmentContext;
-import org.apache.drill.exec.ops.OutputMutator;
-import org.apache.drill.exec.record.BatchSchema;
-import org.apache.drill.exec.record.vector.Int16Vector;
-import org.apache.drill.exec.record.vector.Int32Vector;
-import org.apache.drill.exec.record.vector.ValueVector;
-
-public class MockRecordReader implements RecordReader {
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(MockRecordReader.class);
-
-  private BatchSchema expectedSchema;
-  private OutputMutator output;
-  private MockRecordConfig config;
-  private FragmentContext context;
-  private ValueVector<?>[] valueVectors;
-  private int recordsRead;
-
-  public MockRecordReader(FragmentContext context, MockRecordConfig config) {
-    this.config = config;
-  }
-
-  private int getEstimatedRecordSize(DataType[] types) {
-    int x = 0;
-    for (int i = 0; i < types.length; i++) {
-      x += getEstimatedColumnSize(i);
-    }
-    return x;
-  }
-
-  private int getEstimatedColumnSize(int fieldId) {
-    return 4;
-  }
-
-  private ValueVector<?> getVector(int fieldId, DataType dt, int length) {
-    ValueVector<?> v;
-    if (dt == DataType.INT16) {
-      v = new Int16Vector(fieldId, context.getAllocator());
-    } else if (dt == DataType.INT32) {
-      v = new Int32Vector(fieldId, context.getAllocator());
-    } else {
-      throw new UnsupportedOperationException();
-    }
-    v.allocateNew(length);
-    return v;
-
-  }
-
-  @Override
-  public void setup(BatchSchema expectedSchema, OutputMutator output) throws ExecutionSetupException {
-    try {
-      this.expectedSchema = expectedSchema;
-      this.output = output;
-      int estimateRowSize = getEstimatedRecordSize(config.getTypes());
-      valueVectors = new ValueVector<?>[config.getTypes().length];
-      int batchRecordCount = 250000 / estimateRowSize;
-
-      for (int i = 0; i < config.getTypes().length; i++) {
-        valueVectors[i] = getVector(i, config.getTypes()[i], batchRecordCount);
-        output.addField(i, valueVectors[i]);
-      }
-    } catch (SchemaChangeException e) {
-      throw new ExecutionSetupException("Failure while setting up fields", e);
-    }
-
-  }
-
-  @Override
-  public int next() {
-    int recordSetSize = Math.min(valueVectors[0].size(), this.config.getRecordCount()- recordsRead);
-    recordsRead += recordSetSize;
-    return recordSetSize;
-  }
-
-  @Override
-  public void cleanup() {
-    for (int i = 0; i < valueVectors.length; i++) {
-      try {
-        output.removeField(valueVectors[i].getField().getFieldId());
-      } catch (SchemaChangeException e) {
-        logger.warn("Failure while trying tremove field.", e);
-      }
-      valueVectors[i].close();
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/MockStorageEngine.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/MockStorageEngine.java b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/MockStorageEngine.java
deleted file mode 100644
index cc82540..0000000
--- a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/MockStorageEngine.java
+++ /dev/null
@@ -1,54 +0,0 @@
-/*******************************************************************************
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- ******************************************************************************/
-package org.apache.drill.exec.store;
-
-import java.io.IOException;
-import java.util.Collection;
-
-import org.apache.drill.common.logical.data.Scan;
-import org.apache.drill.common.proto.CoordinationProtos.DrillbitEndpoint;
-import org.apache.drill.exec.ops.FragmentContext;
-
-import com.google.common.collect.ListMultimap;
-
-public class MockStorageEngine extends AbstractStorageEngine{
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(MockStorageEngine.class);
-
-  @Override
-  public boolean supportsRead() {
-    return true;
-  }
-
-  @Override
-  public Collection<ReadEntry> getReadEntries(Scan scan) throws IOException {
-    return null;
-  }
-
-  @Override
-  public ListMultimap<ReadEntry, DrillbitEndpoint> getReadLocations(Collection<ReadEntry> entries) {
-    return null;
-  }
-
-  @Override
-  public RecordReader getReader(FragmentContext context, ReadEntry readEntry) throws IOException {
-    return null;
-  }
-
-  
-  
-}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/test/resources/drill-module.conf
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/resources/drill-module.conf b/sandbox/prototype/exec/java-exec/src/test/resources/drill-module.conf
index 8785736..771a2fd 100644
--- a/sandbox/prototype/exec/java-exec/src/test/resources/drill-module.conf
+++ b/sandbox/prototype/exec/java-exec/src/test/resources/drill-module.conf
@@ -7,6 +7,9 @@ drill.exec: {
   	user.port : 31010,
   	bit.port : 32010
   },
+  operator: {
+    packages += "org.apache.drill.exec.physical.config"
+  },
   optimizer: {
     implementation: "org.apache.drill.exec.opt.IdentityOptimizer"
   },
@@ -30,4 +33,4 @@ drill.exec: {
   network: {
     start: 35000
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/test/resources/logback.xml
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/resources/logback.xml b/sandbox/prototype/exec/java-exec/src/test/resources/logback.xml
index b79b811..3af6e10 100644
--- a/sandbox/prototype/exec/java-exec/src/test/resources/logback.xml
+++ b/sandbox/prototype/exec/java-exec/src/test/resources/logback.xml
@@ -1,16 +1,14 @@
 <?xml version="1.0" encoding="UTF-8" ?>
 <configuration>
-
+<!-- 
   <appender name="SOCKET" class="de.huxhorn.lilith.logback.appender.ClassicMultiplexSocketAppender">
     <Compressing>true</Compressing> 
     <ReconnectionDelay>10000</ReconnectionDelay>
     <IncludeCallerData>true</IncludeCallerData>
     <RemoteHosts>localhost</RemoteHosts>
   </appender>
-
+-->
  <appender name="STDOUT" class="ch.qos.logback.core.ConsoleAppender">
-    <!-- encoders are assigned the type
-         ch.qos.logback.classic.encoder.PatternLayoutEncoder by default -->
     <encoder>
       <pattern>%d{HH:mm:ss.SSS} [%thread] %-5level %logger{36} - %msg%n</pattern>
     </encoder>
@@ -30,14 +28,14 @@
   --> 
   <logger name="org.apache.drill" additivity="false">
     <level value="debug" />
-    <appender-ref ref="SOCKET" />
+<!--     <appender-ref ref="SOCKET" /> -->
     <appender-ref ref="STDOUT" />
 <!--     <appender-ref ref="FILE" /> -->
   </logger>
 
   <root>
     <level value="error" />
-    <appender-ref ref="SOCKET" />
+<!--     <appender-ref ref="SOCKET" /> -->
     <appender-ref ref="STDOUT" />
 <!--     <appender-ref ref="FILE" /> -->
   </root>

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/test/resources/physical_screen.json
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/resources/physical_screen.json b/sandbox/prototype/exec/java-exec/src/test/resources/physical_screen.json
index c63aef1..8bb97db 100644
--- a/sandbox/prototype/exec/java-exec/src/test/resources/physical_screen.json
+++ b/sandbox/prototype/exec/java-exec/src/test/resources/physical_screen.json
@@ -12,9 +12,8 @@
             pop:"mock-scan",
             url: "http://apache.org",
             entries:[
-            	{id:1}
-            ],
-            cost: { disk: 1.0, memory: 1.0, cpu: 1.0, network: 1.0}
+            	{id:1, records: 100, size: 100}
+            ]
         },
         {
             @id:2,

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/test/resources/physical_simpleexchange.json
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/resources/physical_simpleexchange.json b/sandbox/prototype/exec/java-exec/src/test/resources/physical_simpleexchange.json
index e332785..85823cf 100644
--- a/sandbox/prototype/exec/java-exec/src/test/resources/physical_simpleexchange.json
+++ b/sandbox/prototype/exec/java-exec/src/test/resources/physical_simpleexchange.json
@@ -12,30 +12,44 @@
             pop:"mock-scan",
             url: "http://apache.org",
             entries:[
-            	{id:1}
-            ],
-            cost: { disk: 1.0, memory: 1.0, cpu: 1.0, network: 1.0}
+            	{records: 100, types: [
+            	  {name: "blue", type: "INT", mode: "REQUIRED"},
+            	  {name: "red", type: "BIGINT", mode: "REQUIRED"},
+            	  {name: "green", type: "INT", mode: "REQUIRED"}
+            	]},
+            	{records: 100, types: [
+            	  {name: "blue", type: "INT", mode: "REQUIRED"},
+            	  {name: "red", type: "BIGINT", mode: "REQUIRED"},
+            	  {name: "green", type: "INT", mode: "REQUIRED"}
+            	]}
+            ]
         },
         {
             @id:2,
-            pop: "partition-to-random-exchange",
-            child: 1,
-            partition: {
-              mode: "DUPLICATE"
-            }
+            pop: "hash-to-random-exchange",
+            child: 1
         },
         {
             @id:3,
             child: 2,
             pop:"filter",
             expr: "b > 5",
-            cost: { disk: 1.0, memory: 1.0, cpu: 1.0, network: 1.0}
+            selectivity: 0.8
         },
         {
             @id: 4,
             child: 3,
-            pop: "mock-store",
-            cost: { disk: 1.0, memory: 1.0, cpu: 1.0, network: 1.0}
+            pop: "mock-store"
+        },
+        {
+            @id:5,
+            child: 4,
+            pop: "union-exchange"
+        },
+        {
+            @id: 6,
+            child: 5,
+            pop: "screen"
         }
     ]
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/test/resources/physical_single_exchange.json
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/resources/physical_single_exchange.json b/sandbox/prototype/exec/java-exec/src/test/resources/physical_single_exchange.json
new file mode 100644
index 0000000..675ecfb
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/test/resources/physical_single_exchange.json
@@ -0,0 +1,34 @@
+{
+    head:{
+        type:"APACHE_DRILL_PHYSICAL",
+        version:"1",
+        generator:{
+            type:"manual"
+        }
+    },
+    graph:[
+        {
+            @id:1,
+            pop:"mock-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,
+            child: 1,
+            pop: "union-exchange"
+        },
+        {
+            @id: 3,
+            child: 2,
+            pop: "screen"
+        }
+    ]
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/test/resources/physical_test1.json
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/resources/physical_test1.json b/sandbox/prototype/exec/java-exec/src/test/resources/physical_test1.json
new file mode 100644
index 0000000..0ddd48f
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/test/resources/physical_test1.json
@@ -0,0 +1,40 @@
+{
+    head:{
+        type:"APACHE_DRILL_PHYSICAL",
+        version:"1",
+        generator:{
+            type:"manual"
+        }
+    },
+	graph:[
+        {
+            @id:1,
+            pop:"mock-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,
+            child: 1,
+            pop:"filter",
+            expr: "b > 5",
+            selectivity: 0.80
+        },
+        {
+            @id: 3,
+            child: 2,
+            pop: "mock-store"
+        },
+        {
+            @id: 4,
+            child: 3,
+            pop: "screen"
+        }
+    ]
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/test/resources/physical_test2.json
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/resources/physical_test2.json b/sandbox/prototype/exec/java-exec/src/test/resources/physical_test2.json
new file mode 100644
index 0000000..b001921
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/test/resources/physical_test2.json
@@ -0,0 +1,34 @@
+{
+    head:{
+        type:"APACHE_DRILL_PHYSICAL",
+        version:"1",
+        generator:{
+            type:"manual"
+        }
+    },
+    graph:[
+        {
+            @id:1,
+            pop:"mock-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"}
+            	]},
+            	{records: 100, types: [
+            	  {name: "blue", type: "INT", mode: "REQUIRED"},
+            	  {name: "red", type: "BIGINT", mode: "REQUIRED"},
+            	  {name: "green", type: "INT", mode: "REQUIRED"}
+            	]}
+            ]
+        },
+        {
+            @id: 2,
+            child: 1,
+            pop: "screen"
+        }
+    ]    
+    
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/ref/src/test/resources/donuts.json
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/ref/src/test/resources/donuts.json b/sandbox/prototype/exec/ref/src/test/resources/donuts.json
index 2d98b59..9fd6e3e 100644
--- a/sandbox/prototype/exec/ref/src/test/resources/donuts.json
+++ b/sandbox/prototype/exec/ref/src/test/resources/donuts.json
@@ -20,7 +20,7 @@
 			[
 				{ "id": "5001", "type": "None" },
 				{ "id": "5002", "type": "Glazed" },
-				{ "id": "5005", "type": "Sugar" },
+				{ "id": "5005", "type": "Sugar", color: "White"},
 				{ "id": "5007", "type": "Powdered Sugar" },
 				{ "id": "5006", "type": "Chocolate with Sprinkles" },
 				{ "id": "5003", "type": "Chocolate" },


[18/53] [abbrv] Update typing system. Update RPC system. Add Fragmenting Implementation. Working single node. Distributed failing due to threading issues.

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/AbstractStore.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/AbstractStore.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/AbstractStore.java
deleted file mode 100644
index 58edf03..0000000
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/AbstractStore.java
+++ /dev/null
@@ -1,42 +0,0 @@
-/*******************************************************************************
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- ******************************************************************************/
-package org.apache.drill.common.physical.pop.base;
-
-import java.util.List;
-
-import org.apache.drill.common.physical.EndpointAffinity;
-import org.apache.drill.common.proto.CoordinationProtos.DrillbitEndpoint;
-
-
-
-public abstract class AbstractStore extends AbstractSingle implements Store{
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(AbstractStore.class);
-
-  public AbstractStore(PhysicalOperator child) {
-    super(child);
-  }
-
-  @Override
-  public <T, X, E extends Throwable> T accept(PhysicalVisitor<T, X, E> physicalVisitor, X value) throws E{
-    return physicalVisitor.visitStore(this, value);
-  }
-
-
-  
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/Exchange.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/Exchange.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/Exchange.java
deleted file mode 100644
index d779eb8..0000000
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/Exchange.java
+++ /dev/null
@@ -1,69 +0,0 @@
-/*******************************************************************************
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- ******************************************************************************/
-package org.apache.drill.common.physical.pop.base;
-
-import java.util.List;
-
-import org.apache.drill.common.physical.OperatorCost;
-import org.apache.drill.common.proto.CoordinationProtos.DrillbitEndpoint;
-
-import com.fasterxml.jackson.annotation.JsonIgnore;
-import com.fasterxml.jackson.annotation.JsonProperty;
-
-public interface Exchange extends PhysicalOperator{
-
-  @JsonIgnore
-  public abstract OperatorCost getAggregateSendCost();
-
-  @JsonIgnore
-  public abstract OperatorCost getAggregateReceiveCost();
-
-  @JsonProperty("cost")
-  public abstract ExchangeCost getExchangeCost();
-
-  /**
-   * Inform this Exchange node about its sender locations.
-   * @param senderLocations
-   */
-  public abstract void setupSenders(List<DrillbitEndpoint> senderLocations);
-
-  /**
-   * Inform this Exchange node about its receiver locations.
-   * @param receiverLocations
-   */
-  public abstract void setupReceivers(List<DrillbitEndpoint> receiverLocations);
-
-  /**
-   * Get the Sender associated with the given minorFragmentId.  
-   * @param minorFragmentId The minor fragment id, must be in the range [0, fragment.width).
-   * @param child The feeding node for the requested sender.
-   * @return The materialized sender for the given arguments.
-   */
-  public abstract Sender getSender(int minorFragmentId, PhysicalOperator child);
-
-  /**
-   * Get the Receiver associated with the given minorFragmentId.
-   * @param minorFragmentId The minor fragment id, must be in the range [0, fragment.width).
-   * @return The materialized recevier for the given arguments.
-   */
-  public abstract Receiver getReceiver(int minorFragmentId);
-
-  public abstract int getMaxSendWidth();
-
-  public PhysicalOperator getChild();
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/ExchangeCost.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/ExchangeCost.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/ExchangeCost.java
deleted file mode 100644
index f17203e..0000000
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/ExchangeCost.java
+++ /dev/null
@@ -1,55 +0,0 @@
-/*******************************************************************************
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- ******************************************************************************/
-package org.apache.drill.common.physical.pop.base;
-
-import org.apache.drill.common.physical.OperatorCost;
-
-import com.fasterxml.jackson.annotation.JsonCreator;
-import com.fasterxml.jackson.annotation.JsonIgnore;
-import com.fasterxml.jackson.annotation.JsonProperty;
-
-public class ExchangeCost {
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ExchangeCost.class);
-  
-  private final OperatorCost send;
-  private final OperatorCost receive;
-  private final OperatorCost combined;
-  
-  @JsonCreator
-  public ExchangeCost(@JsonProperty("send") OperatorCost send, @JsonProperty("receive") OperatorCost receive) {
-    this.send = send;
-    this.receive = receive;
-    this.combined =  OperatorCost.combine(send,  receive);
-  }
-
-  @JsonIgnore
-  public OperatorCost getCombinedCost(){
-    return combined;
-  }
-
-  public OperatorCost getSend() {
-    return send;
-  }
-
-  public OperatorCost getReceive() {
-    return receive;
-  }
-  
-  
-  
-}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/FragmentLeaf.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/FragmentLeaf.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/FragmentLeaf.java
deleted file mode 100644
index 4557df4..0000000
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/FragmentLeaf.java
+++ /dev/null
@@ -1,25 +0,0 @@
-/*******************************************************************************
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- ******************************************************************************/
-package org.apache.drill.common.physical.pop.base;
-
-
-/**
- * A POP which relies on no other nodes within the current fragment.
- */
-public interface FragmentLeaf extends PhysicalOperator{
-}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/FragmentRoot.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/FragmentRoot.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/FragmentRoot.java
deleted file mode 100644
index 8d87d56..0000000
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/FragmentRoot.java
+++ /dev/null
@@ -1,25 +0,0 @@
-/*******************************************************************************
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- ******************************************************************************/
-package org.apache.drill.common.physical.pop.base;
-
-
-/**
- * Describes the root operation within a particular Fragment. This includes things like Sinks, and Sender nodes. 
- */
-public interface FragmentRoot extends PhysicalOperator{
-}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/HasAffinity.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/HasAffinity.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/HasAffinity.java
deleted file mode 100644
index feb32ec..0000000
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/HasAffinity.java
+++ /dev/null
@@ -1,26 +0,0 @@
-/*******************************************************************************
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- ******************************************************************************/
-package org.apache.drill.common.physical.pop.base;
-
-import java.util.List;
-
-import org.apache.drill.common.physical.EndpointAffinity;
-
-public interface HasAffinity extends PhysicalOperator{
-  public List<EndpointAffinity> getOperatorAffinity();
-}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/Leaf.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/Leaf.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/Leaf.java
deleted file mode 100644
index 28efb94..0000000
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/Leaf.java
+++ /dev/null
@@ -1,21 +0,0 @@
-/*******************************************************************************
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- ******************************************************************************/
-package org.apache.drill.common.physical.pop.base;
-
-public interface Leaf extends PhysicalOperator{
-}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/PhysicalOperator.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/PhysicalOperator.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/PhysicalOperator.java
deleted file mode 100644
index d8d1b64..0000000
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/PhysicalOperator.java
+++ /dev/null
@@ -1,59 +0,0 @@
-/*******************************************************************************
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- ******************************************************************************/
-package org.apache.drill.common.physical.pop.base;
-
-import org.apache.drill.common.graph.GraphValue;
-import org.apache.drill.common.physical.OperatorCost;
-
-import com.fasterxml.jackson.annotation.JsonIdentityInfo;
-import com.fasterxml.jackson.annotation.JsonIgnore;
-import com.fasterxml.jackson.annotation.JsonInclude;
-import com.fasterxml.jackson.annotation.JsonInclude.Include;
-import com.fasterxml.jackson.annotation.JsonPropertyOrder;
-import com.fasterxml.jackson.annotation.JsonTypeInfo;
-import com.fasterxml.jackson.annotation.ObjectIdGenerators;
-
-@JsonInclude(Include.NON_NULL)
-@JsonPropertyOrder({ "@id" })
-@JsonIdentityInfo(generator = ObjectIdGenerators.IntSequenceGenerator.class, property = "@id")
-@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, include = JsonTypeInfo.As.PROPERTY, property = "pop")
-public interface PhysicalOperator extends GraphValue<PhysicalOperator> {
-
-  public OperatorCost getCost();
-
-  /**
-   * Describes whether or not a particular physical operator can actually be executed. Most physical operators can be
-   * executed. However, Exchange nodes cannot be executed. In order to be executed, they must be converted into their
-   * Exec sub components.
-   * 
-   * @return
-   */
-  @JsonIgnore
-  public boolean isExecutable();
-  
-  /**
-   * Provides capability to build a set of output based on traversing a query graph tree.
-   * @param physicalVisitor
-   * @return
-   */
-  public <T, X, E extends Throwable> T accept(PhysicalVisitor<T, X, E> physicalVisitor, X value) throws E;
-
-
-  
-  
-}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/PhysicalOperatorUtil.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/PhysicalOperatorUtil.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/PhysicalOperatorUtil.java
deleted file mode 100644
index fb1fdcd..0000000
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/PhysicalOperatorUtil.java
+++ /dev/null
@@ -1,34 +0,0 @@
-/*******************************************************************************
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- ******************************************************************************/
-package org.apache.drill.common.physical.pop.base;
-
-import org.apache.drill.common.config.CommonConstants;
-import org.apache.drill.common.config.DrillConfig;
-import org.apache.drill.common.util.PathScanner;
-
-public class PhysicalOperatorUtil {
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(PhysicalOperatorUtil.class);
-  
-  private PhysicalOperatorUtil(){}
-  
-  public synchronized static Class<?>[] getSubTypes(DrillConfig config){
-    Class<?>[] ops = PathScanner.scanForImplementationsArr(PhysicalOperator.class, config.getStringList(CommonConstants.PHYSICAL_OPERATOR_SCAN_PACKAGES));
-    logger.debug("Adding Physical Operator sub types: {}", ((Object) ops) );
-    return ops;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/PhysicalVisitor.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/PhysicalVisitor.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/PhysicalVisitor.java
deleted file mode 100644
index 2ecc6ce..0000000
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/PhysicalVisitor.java
+++ /dev/null
@@ -1,43 +0,0 @@
-/*******************************************************************************
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- ******************************************************************************/
-package org.apache.drill.common.physical.pop.base;
-
-import org.apache.drill.common.physical.pop.Filter;
-import org.apache.drill.common.physical.pop.PartitionToRandomExchange;
-import org.apache.drill.common.physical.pop.Project;
-import org.apache.drill.common.physical.pop.Sort;
-
-public interface PhysicalVisitor<RETURN, EXTRA, EXCEP extends Throwable> {
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(PhysicalVisitor.class);
-  
-  
-  public RETURN visitExchange(Exchange exchange, EXTRA value) throws EXCEP;
-  public RETURN visitScan(Scan<?> scan, EXTRA value) throws EXCEP;
-  public RETURN visitStore(Store store, EXTRA value) throws EXCEP;
-
-  public RETURN visitFilter(Filter filter, 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;
-  public RETURN visitReceiver(Receiver receiver, EXTRA value) throws EXCEP;
-  
-  public RETURN visitUnknown(PhysicalOperator op, EXTRA value) throws EXCEP;
-  
-  public RETURN visitPartitionToRandomExchange(PartitionToRandomExchange partitionToRandom, EXTRA value) throws EXCEP; 
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/Receiver.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/Receiver.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/Receiver.java
deleted file mode 100644
index db8f71f..0000000
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/Receiver.java
+++ /dev/null
@@ -1,38 +0,0 @@
-/*******************************************************************************
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- ******************************************************************************/
-package org.apache.drill.common.physical.pop.base;
-
-import java.util.List;
-
-import org.apache.drill.common.proto.CoordinationProtos.DrillbitEndpoint;
-
-public interface Receiver extends FragmentLeaf {
-  public abstract List<DrillbitEndpoint> getProvidingEndpoints();
-
-  /**
-   * Whether or not this receive supports out of order exchange. This provides a hint for the scheduling node on whether
-   * the receiver can start work if only a subset of all sending endpoints are currently providing data. A random
-   * receiver would supports this form of operation. A NWAY receiver would not.
-   * 
-   * @return True if this receiver supports working on a streaming/out of order input.
-   */
-  public abstract boolean supportsOutOfOrderExchange();
-  
-  
-  public int getSenderCount();
-}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/Root.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/Root.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/Root.java
deleted file mode 100644
index c4f9982..0000000
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/Root.java
+++ /dev/null
@@ -1,24 +0,0 @@
-/*******************************************************************************
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- ******************************************************************************/
-package org.apache.drill.common.physical.pop.base;
-
-/**
- * Marker interface describe the root of a query plan.
- */
-public interface Root extends PhysicalOperator{
-}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/Scan.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/Scan.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/Scan.java
deleted file mode 100644
index c7b45a8..0000000
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/Scan.java
+++ /dev/null
@@ -1,36 +0,0 @@
-/*******************************************************************************
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- ******************************************************************************/
-package org.apache.drill.common.physical.pop.base;
-
-import java.util.List;
-
-import org.apache.drill.common.physical.ReadEntry;
-import org.apache.drill.common.proto.CoordinationProtos.DrillbitEndpoint;
-
-import com.fasterxml.jackson.annotation.JsonProperty;
-
-public interface Scan<R extends ReadEntry> extends Leaf, HasAffinity{
-
-  @JsonProperty("entries")
-  public abstract List<R> getReadEntries();
-
-  public abstract void applyAssignments(List<DrillbitEndpoint> endpoints);
-
-  public abstract Scan<?> getSpecificScan(int minorFragmentId);
-
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/Sender.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/Sender.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/Sender.java
deleted file mode 100644
index 1859657..0000000
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/Sender.java
+++ /dev/null
@@ -1,29 +0,0 @@
-/*******************************************************************************
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- ******************************************************************************/
-package org.apache.drill.common.physical.pop.base;
-
-import java.util.List;
-
-import org.apache.drill.common.proto.CoordinationProtos.DrillbitEndpoint;
-
-
-
-
-public interface Sender extends FragmentRoot{
-  public abstract List<DrillbitEndpoint> getDestinations();
-}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/Store.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/Store.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/Store.java
deleted file mode 100644
index eec4a6c..0000000
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/Store.java
+++ /dev/null
@@ -1,30 +0,0 @@
-/*******************************************************************************
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- ******************************************************************************/
-package org.apache.drill.common.physical.pop.base;
-
-import java.util.List;
-
-import org.apache.drill.common.proto.CoordinationProtos.DrillbitEndpoint;
-
-public interface Store extends Root, HasAffinity{
-
-  public abstract void applyAssignments(List<DrillbitEndpoint> endpoints);
-  public abstract Store getSpecificStore(PhysicalOperator child, int minorFragmentId);
-  public abstract int getMaxWidth();
-  public abstract PhysicalOperator getChild();
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/common/src/main/protobuf/Coordination.proto
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/protobuf/Coordination.proto b/sandbox/prototype/common/src/main/protobuf/Coordination.proto
deleted file mode 100644
index f98d2c5..0000000
--- a/sandbox/prototype/common/src/main/protobuf/Coordination.proto
+++ /dev/null
@@ -1,26 +0,0 @@
-package exec;
-
-option java_package = "org.apache.drill.common.proto";
-option java_outer_classname = "CoordinationProtos";
-option optimize_for = LITE_RUNTIME;
-
-message DrillbitEndpoint{
-  optional string address = 1;
-  optional int32 user_port = 2;
-  optional int32 bit_port = 3;
-  optional Roles roles = 4;
-}
-
-message DrillServiceInstance{
-  optional string id = 1;
-  optional int64 registrationTimeUTC = 2;
-  optional DrillbitEndpoint endpoint = 3;
-}
-
-message Roles{
-	optional bool sql_query = 1 [default = true];
-	optional bool logical_plan = 2 [default = true];
-	optional bool physical_plan = 3 [default = true];
-	optional bool java_executor = 4 [default = true];
-	optional bool distributed_cache = 5 [default = true];
-}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/common/src/test/java/org/apache/drill/common/physical/MockScanPOP.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/test/java/org/apache/drill/common/physical/MockScanPOP.java b/sandbox/prototype/common/src/test/java/org/apache/drill/common/physical/MockScanPOP.java
deleted file mode 100644
index 1b042c5..0000000
--- a/sandbox/prototype/common/src/test/java/org/apache/drill/common/physical/MockScanPOP.java
+++ /dev/null
@@ -1,69 +0,0 @@
-/*******************************************************************************
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- ******************************************************************************/
-package org.apache.drill.common.physical;
-
-import java.util.Collections;
-import java.util.List;
-
-import org.apache.drill.common.physical.pop.base.AbstractScan;
-import org.apache.drill.common.physical.pop.base.Scan;
-import org.apache.drill.common.proto.CoordinationProtos.DrillbitEndpoint;
-
-import com.fasterxml.jackson.annotation.JsonCreator;
-import com.fasterxml.jackson.annotation.JsonProperty;
-import com.fasterxml.jackson.annotation.JsonTypeName;
-
-@JsonTypeName("mock-scan")
-public class MockScanPOP extends AbstractScan<MockScanPOP.MockScanEntry>{
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(MockScanPOP.class);
-  
-  private final String url;
-  
-  @JsonCreator
-  public MockScanPOP(@JsonProperty("url") String url, @JsonProperty("entries") List<MockScanEntry> readEntries) {
-    super(readEntries);
-    this.url = url;
-  }
-  
-  public String getUrl() {
-    return url;
-  }
-
-  public static class MockScanEntry implements ReadEntry{
-    public int id;
-  }
-
-  @Override
-  public List<EndpointAffinity> getOperatorAffinity() {
-    return Collections.emptyList();
-  }
-
-  @Override
-  public void applyAssignments(List<DrillbitEndpoint> endpoints) {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public Scan<?> getSpecificScan(int minorFragmentId) {
-    return this;
-  }
-  
-
-  
-  
-}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/common/src/test/java/org/apache/drill/common/physical/MockStorePOP.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/test/java/org/apache/drill/common/physical/MockStorePOP.java b/sandbox/prototype/common/src/test/java/org/apache/drill/common/physical/MockStorePOP.java
deleted file mode 100644
index f48c539..0000000
--- a/sandbox/prototype/common/src/test/java/org/apache/drill/common/physical/MockStorePOP.java
+++ /dev/null
@@ -1,62 +0,0 @@
-/*******************************************************************************
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- ******************************************************************************/
-package org.apache.drill.common.physical;
-
-import java.util.Collections;
-import java.util.List;
-
-import org.apache.drill.common.physical.pop.base.AbstractStore;
-import org.apache.drill.common.physical.pop.base.PhysicalOperator;
-import org.apache.drill.common.physical.pop.base.Store;
-import org.apache.drill.common.proto.CoordinationProtos.DrillbitEndpoint;
-
-import com.fasterxml.jackson.annotation.JsonCreator;
-import com.fasterxml.jackson.annotation.JsonIgnore;
-import com.fasterxml.jackson.annotation.JsonProperty;
-import com.fasterxml.jackson.annotation.JsonTypeName;
-
-@JsonTypeName("mock-store")
-public class MockStorePOP extends AbstractStore {
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(MockStorePOP.class);
-
-  @JsonCreator
-  public MockStorePOP(@JsonProperty("child") PhysicalOperator child) {
-    super(child);
-  }
-
-  public int getMaxWidth() {
-    return 1;
-  }
-
-  @Override
-  public List<EndpointAffinity> getOperatorAffinity() {
-    return Collections.emptyList();
-  }
-
-  @Override
-  public void applyAssignments(List<DrillbitEndpoint> endpoints) {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public Store getSpecificStore(PhysicalOperator child, int minorFragmentId) {
-    return this;
-  }
-
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/common/src/test/java/org/apache/drill/common/physical/ParsePhysicalPlan.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/test/java/org/apache/drill/common/physical/ParsePhysicalPlan.java b/sandbox/prototype/common/src/test/java/org/apache/drill/common/physical/ParsePhysicalPlan.java
deleted file mode 100644
index 0ad1f76..0000000
--- a/sandbox/prototype/common/src/test/java/org/apache/drill/common/physical/ParsePhysicalPlan.java
+++ /dev/null
@@ -1,41 +0,0 @@
-/*******************************************************************************
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- ******************************************************************************/
-package org.apache.drill.common.physical;
-
-import org.apache.drill.common.config.DrillConfig;
-import org.apache.drill.common.util.FileUtils;
-import org.junit.Test;
-
-import com.fasterxml.jackson.databind.ObjectReader;
-import com.fasterxml.jackson.databind.ObjectWriter;
-import com.google.common.base.Charsets;
-import com.google.common.io.Files;
-
-public class ParsePhysicalPlan {
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ParsePhysicalPlan.class);
-  
-  
-  @Test 
-  public void parseSimplePlan() throws Exception{
-    DrillConfig c = DrillConfig.create();
-    ObjectReader r = c.getMapper().reader(PhysicalPlan.class);
-    ObjectWriter writer = c.getMapper().writer();
-    PhysicalPlan plan = PhysicalPlan.parse(r, Files.toString(FileUtils.getResourceAsFile("/physical_test1.json"), Charsets.UTF_8));
-    System.out.println(plan.unparse(writer));
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/common/src/test/resources/drill-module.conf
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/test/resources/drill-module.conf b/sandbox/prototype/common/src/test/resources/drill-module.conf
index 86e828a..0e2c84e 100644
--- a/sandbox/prototype/common/src/test/resources/drill-module.conf
+++ b/sandbox/prototype/common/src/test/resources/drill-module.conf
@@ -1,2 +1 @@
 drill.logical.storage.packages += "org.apache.drill.storage"
-drill.physical.operator.packages += "org.apache.drill.common.physical.pop"
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/common/src/test/resources/physical_test1.json
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/test/resources/physical_test1.json b/sandbox/prototype/common/src/test/resources/physical_test1.json
deleted file mode 100644
index 16bc87a..0000000
--- a/sandbox/prototype/common/src/test/resources/physical_test1.json
+++ /dev/null
@@ -1,33 +0,0 @@
-{
-    head:{
-        type:"APACHE_DRILL_PHYSICAL",
-        version:"1",
-        generator:{
-            type:"manual"
-        }
-    },
-    graph:[
-        {
-            @id:1,
-            pop:"mock-scan",
-            url: "http://apache.org",
-            entries:[
-            	{id:1}
-            ],
-            cost: { disk: 1.0, memory: 1.0, cpu: 1.0, network: 1.0}
-        },
-        {
-            @id:2,
-            child: 1,
-            pop:"filter",
-            expr: "b > 5",
-            cost: { disk: 1.0, memory: 1.0, cpu: 1.0, network: 1.0}
-        },
-        {
-            @id: 3,
-            child: 2,
-            pop: "mock-store",
-            cost: { disk: 1.0, memory: 1.0, cpu: 1.0, network: 1.0}
-        }
-    ]
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/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 9766df7..f5ece33 100644
--- a/sandbox/prototype/exec/java-exec/pom.xml
+++ b/sandbox/prototype/exec/java-exec/pom.xml
@@ -56,7 +56,7 @@
 			<groupId>org.apache.drill</groupId>
 			<artifactId>common</artifactId>
 			<version>1.0-SNAPSHOT</version>
-			<classifier>test</classifier>
+			<classifier>tests</classifier>
 		</dependency>
 		<dependency>
 			<groupId>com.beust</groupId>
@@ -110,7 +110,7 @@
 		<dependency>
 			<groupId>io.netty</groupId>
 			<artifactId>netty-all</artifactId>
-			<version>4.0.0.CR1</version>
+			<version>4.0.0.CR2</version>
 		</dependency>
 		<dependency>
 			<groupId>com.google.protobuf</groupId>

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ByteReorder.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ByteReorder.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ByteReorder.java
deleted file mode 100644
index 82a8a85..0000000
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ByteReorder.java
+++ /dev/null
@@ -1,54 +0,0 @@
-/*******************************************************************************
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- ******************************************************************************/
-package org.apache.drill.exec;
-
-import java.util.Arrays;
-
-import com.google.common.base.Charsets;
-import com.google.common.primitives.Bytes;
-import com.google.common.primitives.UnsignedBytes;
-
-public class ByteReorder {
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ByteReorder.class);
-  
-  public static void main(String[] args){
-    String[] strings = {"hello", "goodbye", "my friend"};
-    byte[][] bytes = new byte[strings.length][];
-    for(int i =0; i < strings.length; i++){
-      bytes[i] = strings[i].getBytes(Charsets.UTF_8);
-    }
-    
-    for(int i =0; i < bytes.length; i++){
-      for(int v = 0; v < bytes[i].length; v++){
-        bytes[i][v] = (byte) ~bytes[i][v];
-      }
-    }
-    
-    Arrays.sort(bytes, UnsignedBytes.lexicographicalComparator());
-
-    for(int i =0; i < bytes.length; i++){
-      for(int v = 0; v < bytes[i].length; v++){
-        bytes[i][v] = (byte) ~bytes[i][v];
-      }
-    }
-
-    for(int i =0; i < bytes.length; i++){
-      System.out.println(new String(bytes[i]));
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/cache/DistributedCache.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/cache/DistributedCache.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/cache/DistributedCache.java
index 2928dbe..ba2c26b 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/cache/DistributedCache.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/cache/DistributedCache.java
@@ -20,8 +20,9 @@ package org.apache.drill.exec.cache;
 import java.io.Closeable;
 import java.util.List;
 
-import org.apache.drill.common.proto.CoordinationProtos.DrillbitEndpoint;
 import org.apache.drill.exec.exception.DrillbitStartupException;
+import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
+import org.apache.drill.exec.proto.ExecProtos.FragmentHandle;
 import org.apache.drill.exec.proto.ExecProtos.PlanFragment;
 import org.apache.drill.exec.proto.ExecProtos.WorkQueueStatus;
 
@@ -29,14 +30,11 @@ import org.apache.drill.exec.proto.ExecProtos.WorkQueueStatus;
 public interface DistributedCache extends Closeable{
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DistributedCache.class);
   
-  public void run(DrillbitEndpoint endpoint) throws DrillbitStartupException;
+  public void run() throws DrillbitStartupException;
   
-  public void saveOptimizedPlan(TemplatizedLogicalPlan logical, TemplatizedPhysicalPlan physical);
-  public TemplatizedPhysicalPlan getOptimizedPlan(TemplatizedLogicalPlan logical);
+//  public void updateLocalQueueLength(int length);
+//  public List<WorkQueueStatus> getQueueLengths(); 
   
-  public void updateLocalQueueLength(int length);
-  public List<WorkQueueStatus> getQueueLengths(); 
-  
-  public PlanFragment getFragment(long fragmentId);
+  public PlanFragment getFragment(FragmentHandle handle);
   public void storeFragment(PlanFragment fragment);
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/cache/HazelCache.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/cache/HazelCache.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/cache/HazelCache.java
index 943031d..f4fdbfa 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/cache/HazelCache.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/cache/HazelCache.java
@@ -17,20 +17,22 @@
  ******************************************************************************/
 package org.apache.drill.exec.cache;
 
-import java.io.DataInput;
-import java.io.DataOutput;
 import java.io.IOException;
 import java.util.List;
 
 import org.apache.drill.common.config.DrillConfig;
-import org.apache.drill.common.proto.CoordinationProtos.DrillbitEndpoint;
 import org.apache.drill.exec.ExecConstants;
+import org.apache.drill.exec.cache.ProtoBufImpl.HWorkQueueStatus;
+import org.apache.drill.exec.cache.ProtoBufImpl.HandlePlan;
+import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
+import org.apache.drill.exec.proto.ExecProtos.FragmentHandle;
 import org.apache.drill.exec.proto.ExecProtos.PlanFragment;
 import org.apache.drill.exec.proto.ExecProtos.WorkQueueStatus;
 
 import com.beust.jcommander.internal.Lists;
 import com.google.common.cache.Cache;
 import com.google.common.cache.CacheBuilder;
+import com.google.protobuf.InvalidProtocolBufferException;
 import com.hazelcast.config.Config;
 import com.hazelcast.core.Hazelcast;
 import com.hazelcast.core.HazelcastInstance;
@@ -38,39 +40,36 @@ import com.hazelcast.core.IMap;
 import com.hazelcast.core.ITopic;
 import com.hazelcast.core.Message;
 import com.hazelcast.core.MessageListener;
-import com.hazelcast.nio.DataSerializable;
 
 public class HazelCache implements DistributedCache {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(HazelCache.class);
 
   private final String instanceName;
   private HazelcastInstance instance;
-  private ITopic<WrappedWorkQueueStatus> workQueueLengths;
-  private DrillbitEndpoint endpoint;
+  private ITopic<HWorkQueueStatus> workQueueLengths;
+  private HandlePlan fragments;
   private Cache<WorkQueueStatus, Integer>  endpoints;
-  private IMap<TemplatizedLogicalPlan, TemplatizedPhysicalPlan> optimizedPlans;
   
   public HazelCache(DrillConfig config) {
     this.instanceName = config.getString(ExecConstants.SERVICE_NAME);
   }
 
-  private class Listener implements MessageListener<WrappedWorkQueueStatus>{
+  private class Listener implements MessageListener<HWorkQueueStatus>{
 
     @Override
-    public void onMessage(Message<WrappedWorkQueueStatus> wrapped) {
+    public void onMessage(Message<HWorkQueueStatus> wrapped) {
       logger.debug("Received new queue length message.");
-      endpoints.put(wrapped.getMessageObject().status, 0);
+      endpoints.put(wrapped.getMessageObject().get(), 0);
     }
     
   }
   
-  public void run(DrillbitEndpoint endpoint) {
+  public void run() {
     Config c = new Config();
     c.setInstanceName(instanceName);
     instance = getInstanceOrCreateNew(c);
     workQueueLengths = instance.getTopic("queue-length");
-    optimizedPlans = instance.getMap("plan-optimizations");
-    this.endpoint = endpoint;
+    fragments = new HandlePlan(instance);
     endpoints = CacheBuilder.newBuilder().maximumSize(2000).build();
     workQueueLengths.addMessageListener(new Listener());
   }
@@ -83,52 +82,16 @@ public class HazelCache implements DistributedCache {
     return Hazelcast.newHazelcastInstance(c);
   }
 
-  @Override
-  public void saveOptimizedPlan(TemplatizedLogicalPlan logical, TemplatizedPhysicalPlan physical) {
-    optimizedPlans.put(logical, physical);
-  }
-
-  @Override
-  public TemplatizedPhysicalPlan getOptimizedPlan(TemplatizedLogicalPlan logical) {
-    return optimizedPlans.get(logical);
-  }
-
-  @Override
-  public void updateLocalQueueLength(int length) {
-    workQueueLengths.publish(new WrappedWorkQueueStatus(WorkQueueStatus.newBuilder().setEndpoint(endpoint)
-        .setQueueLength(length).setReportTime(System.currentTimeMillis()).build()));
-  }
-
-  @Override
-  public List<WorkQueueStatus> getQueueLengths() {
-    return Lists.newArrayList(endpoints.asMap().keySet());
-  }
-
-  public class WrappedWorkQueueStatus implements DataSerializable {
-
-    public WorkQueueStatus status;
-
-    public WrappedWorkQueueStatus(WorkQueueStatus status) {
-      this.status = status;
-    }
-
-    @Override
-    public void readData(DataInput arg0) throws IOException {
-      int len = arg0.readShort();
-      byte[] b = new byte[len];
-      arg0.readFully(b);
-      this.status = WorkQueueStatus.parseFrom(b);
-    }
-
-    @Override
-    public void writeData(DataOutput arg0) throws IOException {
-      byte[] b = status.toByteArray();
-      if (b.length > Short.MAX_VALUE) throw new IOException("Unexpectedly long value.");
-      arg0.writeShort(b.length);
-      arg0.write(b);
-    }
-
-  }
+//  @Override
+//  public void updateLocalQueueLength(int length) {
+//    workQueueLengths.publish(new HWorkQueueStatus(WorkQueueStatus.newBuilder().setEndpoint(endpoint)
+//        .setQueueLength(length).setReportTime(System.currentTimeMillis()).build()));
+//  }
+//
+//  @Override
+//  public List<WorkQueueStatus> getQueueLengths() {
+//    return Lists.newArrayList(endpoints.asMap().keySet());
+//  }
 
   @Override
   public void close() throws IOException {
@@ -136,13 +99,13 @@ public class HazelCache implements DistributedCache {
   }
 
   @Override
-  public PlanFragment getFragment(long fragmentId) {
-    throw new UnsupportedOperationException();
+  public PlanFragment getFragment(FragmentHandle handle) {
+    return this.fragments.get(handle);
   }
 
   @Override
   public void storeFragment(PlanFragment fragment) {
-    throw new UnsupportedOperationException();
+    fragments.put(fragment.getHandle(), fragment);
   }
   
 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/cache/LocalCache.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/cache/LocalCache.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/cache/LocalCache.java
new file mode 100644
index 0000000..ddb2a02
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/cache/LocalCache.java
@@ -0,0 +1,55 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.cache;
+
+import java.io.IOException;
+import java.util.Map;
+
+import org.apache.drill.exec.exception.DrillbitStartupException;
+import org.apache.drill.exec.proto.ExecProtos.FragmentHandle;
+import org.apache.drill.exec.proto.ExecProtos.PlanFragment;
+
+import com.google.common.collect.Maps;
+
+public class LocalCache implements DistributedCache {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(LocalCache.class);
+
+  private volatile Map<FragmentHandle, PlanFragment> handles;
+  
+  @Override
+  public void close() throws IOException {
+    handles = null;
+  }
+
+  @Override
+  public void run() throws DrillbitStartupException {
+    handles = Maps.newConcurrentMap();
+  }
+
+  @Override
+  public PlanFragment getFragment(FragmentHandle handle) {
+    return handles.get(handle);
+  }
+
+  @Override
+  public void storeFragment(PlanFragment fragment) {
+    handles.put(fragment.getHandle(), fragment);
+  }
+  
+  
+}

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

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

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

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

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

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/client/DrillClient.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/client/DrillClient.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/client/DrillClient.java
index ee63213..bb7f77e 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/client/DrillClient.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/client/DrillClient.java
@@ -25,24 +25,34 @@ import io.netty.buffer.ByteBufAllocator;
 import io.netty.buffer.PooledByteBufAllocator;
 import io.netty.channel.nio.NioEventLoopGroup;
 
+import java.io.Closeable;
 import java.io.IOException;
 import java.util.Collection;
+import java.util.List;
+import java.util.Vector;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Future;
 
 import org.apache.drill.common.config.DrillConfig;
-import org.apache.drill.common.proto.CoordinationProtos.DrillbitEndpoint;
 import org.apache.drill.exec.coord.ClusterCoordinator;
 import org.apache.drill.exec.coord.ZKClusterCoordinator;
-import org.apache.drill.exec.proto.UserProtos.QueryHandle;
-import org.apache.drill.exec.rpc.DrillRpcFuture;
+import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
+import org.apache.drill.exec.proto.UserProtos.QueryType;
+import org.apache.drill.exec.proto.UserProtos.RpcType;
+import org.apache.drill.exec.proto.UserProtos.UserToBitHandshake;
 import org.apache.drill.exec.rpc.NamedThreadFactory;
 import org.apache.drill.exec.rpc.RpcException;
+import org.apache.drill.exec.rpc.user.QueryResultBatch;
 import org.apache.drill.exec.rpc.user.UserClient;
+import org.apache.drill.exec.rpc.user.UserResultsListener;
+import org.apache.drill.exec.rpc.user.UserRpcConfig;
 
 /**
  * Thin wrapper around a UserClient that handles connect/close and transforms String into ByteBuf
  */
-public class DrillClient {
-
+public class DrillClient implements Closeable{
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillClient.class);
+  
   DrillConfig config;
   private UserClient client;
   private ClusterCoordinator clusterCoordinator;
@@ -56,8 +66,17 @@ public class DrillClient {
   }
 
   public DrillClient(DrillConfig config) {
+    this(config, null);
+  }
+  
+  public DrillClient(DrillConfig config, ClusterCoordinator coordinator){
     this.config = config;
+    this.clusterCoordinator = coordinator;
   }
+  
+  
+  
+
 
   /**
    * Connects the client to a Drillbit server
@@ -65,9 +84,11 @@ public class DrillClient {
    * @throws IOException
    */
   public void connect() throws Exception {
-    this.clusterCoordinator = new ZKClusterCoordinator(this.config);
-    this.clusterCoordinator.start();
-    Thread.sleep(10000);
+    if(clusterCoordinator == null){
+      this.clusterCoordinator = new ZKClusterCoordinator(this.config);
+      this.clusterCoordinator.start(10000);
+    }
+    
     Collection<DrillbitEndpoint> endpoints = clusterCoordinator.getAvailableEndpoints();
     checkState(!endpoints.isEmpty(), "No DrillbitEndpoint can be found");
     // just use the first endpoint for now
@@ -75,7 +96,8 @@ public class DrillClient {
     ByteBufAllocator bb = new PooledByteBufAllocator(true);
     this.client = new UserClient(bb, new NioEventLoopGroup(1, new NamedThreadFactory("Client-")));
     try {
-      this.client.connectAsClient(endpoint.getAddress(), endpoint.getUserPort());
+      logger.debug("Connecting to server {}:{}", endpoint.getAddress(), endpoint.getUserPort());
+      this.client.connect(endpoint);
     } catch (InterruptedException e) {
       throw new IOException(e);
     }
@@ -97,8 +119,37 @@ public class DrillClient {
    * @return a handle for the query result
    * @throws RpcException
    */
-  public DrillRpcFuture<QueryHandle> submitPlan(String plan) throws RpcException {
-    return this.client.submitQuery(newBuilder().setMode(STREAM_FULL).setPlan(plan).build(), null);
+  public List<QueryResultBatch> runQuery(QueryType type, String plan) throws RpcException {
+    try {
+      ListHoldingResultsListener listener = new ListHoldingResultsListener();
+      Future<Void> f = client.submitQuery(newBuilder().setResultsMode(STREAM_FULL).setType(type).setPlan(plan).build(), listener);
+      f.get();
+      if(listener.ex != null){
+        throw listener.ex;
+      }else{
+        return listener.results;
+      }
+    } catch (InterruptedException | ExecutionException e) {
+      throw new RpcException(e);
+    }
+  }
+  
+  private class ListHoldingResultsListener extends UserResultsListener{
+    private RpcException ex;
+    private Vector<QueryResultBatch> results = new Vector<QueryResultBatch>();
+    
+    @Override
+    public void submissionFailed(RpcException ex) {
+      logger.debug("Submission failed.", ex);
+      this.ex = ex;
+    }
+
+    @Override
+    public void resultArrived(QueryResultBatch result) {
+      logger.debug("Result arrived.  Is Last Chunk: {}.  Full Result: {}", result.getHeader().getIsLastChunk(), result);
+      results.add(result);
+    }
+    
   }
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/coord/ClusterCoordinator.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/coord/ClusterCoordinator.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/coord/ClusterCoordinator.java
index d3580b5..7fb1f5b 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/coord/ClusterCoordinator.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/coord/ClusterCoordinator.java
@@ -20,7 +20,7 @@ package org.apache.drill.exec.coord;
 import java.io.Closeable;
 import java.util.Collection;
 
-import org.apache.drill.common.proto.CoordinationProtos.DrillbitEndpoint;
+import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
 
 /**
  * Pluggable interface built to manage cluster coordination. Allows Drillbit or DrillClient to register its capabilities
@@ -29,7 +29,12 @@ import org.apache.drill.common.proto.CoordinationProtos.DrillbitEndpoint;
 public abstract class ClusterCoordinator implements Closeable {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ClusterCoordinator.class);
 
-  public abstract void start() throws Exception;
+  /**
+   * Start the cluster coordinator.  Millis to wait is   
+   * @param millisToWait The maximum time to wait before throwing an exception if the cluster coordination service has not successfully started.  Use 0 to wait indefinitely.
+   * @throws Exception
+   */
+  public abstract void start(long millisToWait) throws Exception;
 
   public abstract RegistrationHandle register(DrillbitEndpoint data);
 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/coord/DrillServiceInstanceHelper.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/coord/DrillServiceInstanceHelper.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/coord/DrillServiceInstanceHelper.java
index ce0fb92..289aa3c 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/coord/DrillServiceInstanceHelper.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/coord/DrillServiceInstanceHelper.java
@@ -17,9 +17,9 @@
  ******************************************************************************/
 package org.apache.drill.exec.coord;
 
-import org.apache.drill.common.proto.CoordinationProtos.DrillServiceInstance;
-import org.apache.drill.common.proto.CoordinationProtos.DrillbitEndpoint;
 import org.apache.drill.exec.ExecConstants;
+import org.apache.drill.exec.proto.CoordinationProtos.DrillServiceInstance;
+import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
 
 import com.netflix.curator.x.discovery.ServiceInstance;
 import com.netflix.curator.x.discovery.ServiceInstanceBuilder;

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/coord/DrillbitEndpointSerDe.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/coord/DrillbitEndpointSerDe.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/coord/DrillbitEndpointSerDe.java
new file mode 100644
index 0000000..5886c2c
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/coord/DrillbitEndpointSerDe.java
@@ -0,0 +1,65 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.coord;
+
+import java.io.IOException;
+
+import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
+
+import com.fasterxml.jackson.core.JsonGenerationException;
+import com.fasterxml.jackson.core.JsonGenerator;
+import com.fasterxml.jackson.core.JsonParser;
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.DeserializationContext;
+import com.fasterxml.jackson.databind.SerializerProvider;
+import com.fasterxml.jackson.databind.deser.std.StdDeserializer;
+import com.fasterxml.jackson.databind.ser.std.StdSerializer;
+
+public class DrillbitEndpointSerDe {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillbitEndpointSerDe.class);
+  
+  public static class De extends StdDeserializer<DrillbitEndpoint> {
+
+    public De() {
+      super(DrillbitEndpoint.class);
+    }
+
+    @Override
+    public DrillbitEndpoint deserialize(JsonParser jp, DeserializationContext ctxt) throws IOException,
+        JsonProcessingException {
+      return DrillbitEndpoint.parseFrom(jp.getBinaryValue());
+    }
+    
+    
+  }
+  
+  
+  public static class Se extends StdSerializer<DrillbitEndpoint> {
+
+    public Se() {
+      super(DrillbitEndpoint.class);
+    }
+
+    @Override
+    public void serialize(DrillbitEndpoint value, JsonGenerator jgen, SerializerProvider provider) throws IOException,
+        JsonGenerationException {
+      jgen.writeBinary(value.toByteArray());
+    }
+    
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/coord/LocalClusterCoordinator.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/coord/LocalClusterCoordinator.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/coord/LocalClusterCoordinator.java
new file mode 100644
index 0000000..43a5430
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/coord/LocalClusterCoordinator.java
@@ -0,0 +1,95 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.coord;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.Map;
+import java.util.UUID;
+
+import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
+
+import com.google.common.collect.Maps;
+
+public class LocalClusterCoordinator extends ClusterCoordinator{
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(LocalClusterCoordinator.class);
+
+  private volatile Map<RegistrationHandle, DrillbitEndpoint> endpoints;
+  
+  @Override
+  public void close() throws IOException {
+    endpoints = null;
+  }
+
+  @Override
+  public void start(long millis) throws Exception {
+    logger.debug("Local Cluster Coordinator started.");
+    endpoints = Maps.newConcurrentMap();
+  }
+
+  @Override
+  public RegistrationHandle register(DrillbitEndpoint data) {
+    logger.debug("Endpoint registered {}.", data);
+    Handle h = new Handle();
+    endpoints.put(h, data);
+    return h;
+  }
+
+  @Override
+  public void unregister(RegistrationHandle handle) {
+    endpoints.remove(handle);
+  }
+
+  @Override
+  public Collection<DrillbitEndpoint> getAvailableEndpoints() {
+    return endpoints.values();
+  }
+  
+  
+  private class Handle implements RegistrationHandle{
+    UUID id = UUID.randomUUID();
+
+    @Override
+    public int hashCode() {
+      final int prime = 31;
+      int result = 1;
+      result = prime * result + getOuterType().hashCode();
+      result = prime * result + ((id == null) ? 0 : id.hashCode());
+      return result;
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+      if (this == obj) return true;
+      if (obj == null) return false;
+      if (getClass() != obj.getClass()) return false;
+      Handle other = (Handle) obj;
+      if (!getOuterType().equals(other.getOuterType())) return false;
+      if (id == null) {
+        if (other.id != null) return false;
+      } else if (!id.equals(other.id)) return false;
+      return true;
+    }
+
+    private LocalClusterCoordinator getOuterType() {
+      return LocalClusterCoordinator.this;
+    }
+    
+  }
+  
+}


[37/53] [abbrv] git commit: Initial argument validation implementation

Posted by ja...@apache.org.
Initial argument validation implementation


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

Branch: refs/heads/master
Commit: e68bba25a683cb9290322f7974b0743e8167b6bc
Parents: c941874
Author: Timothy Chen <tn...@gmail.com>
Authored: Wed Mar 13 00:15:04 2013 -0700
Committer: Jacques Nadeau <ja...@apache.org>
Committed: Mon Jul 15 13:32:08 2013 -0700

----------------------------------------------------------------------
 .../org/apache/drill/common/expression/Arg.java |   6 +-
 .../common/expression/ArgumentValidator.java    |   2 +-
 .../common/expression/ArgumentValidators.java   |  22 +--
 .../expression/BasicArgumentValidator.java      |   6 +-
 .../drill/common/expression/ErrorCollector.java |  30 ++--
 .../common/expression/ErrorCollectorImpl.java   |  83 +++++++++++
 .../expression/ExpressionValidationError.java   |   9 ++
 .../drill/common/expression/FieldReference.java |  22 ++-
 .../drill/common/expression/FunctionCall.java   |   5 +
 .../common/expression/FunctionDefinition.java   |   4 +
 .../drill/common/expression/IfExpression.java   |  36 +++--
 .../common/expression/LogicalExpression.java    | 103 +++++++-------
 .../expression/LogicalExpressionBase.java       |  45 +++---
 .../drill/common/expression/SchemaPath.java     |  15 +-
 .../common/expression/ValueExpressions.java     |  69 +++++++---
 .../drill/common/expression/types/AtomType.java |  67 ---------
 .../exec/record/ExpressionTreeMaterializer.java | 138 +++++++++++++++++++
 .../drill/exec/record/MaterializeVisitor.java   |   4 +
 .../drill/exec/record/MaterializedField.java    |  11 ++
 .../record/ExpressionTreeMaterializerTest.java  | 110 +++++++++++++++
 20 files changed, 584 insertions(+), 203 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e68bba25/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/Arg.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/Arg.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/Arg.java
index 7848cf7..de9057b 100644
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/Arg.java
+++ b/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/Arg.java
@@ -44,9 +44,9 @@ public class Arg {
     return name;
   }
   
-  public void confirmDataType(int argIndex, LogicalExpression e, ErrorCollector errors){
+  public void confirmDataType(String expr, int argIndex, LogicalExpression e, ErrorCollector errors){
     if(constantsOnly){
-      if(ConstantChecker.onlyIncludesConstants(e)) errors.addExpectedConstantValue(argIndex, name);
+      if(ConstantChecker.onlyIncludesConstants(e)) errors.addExpectedConstantValue(expr, argIndex, name);
     }
     DataType dt = e.getDataType();
     if(dt.isLateBind()){
@@ -57,7 +57,7 @@ public class Arg {
       }
       
       // didn't find an allowed type.
-      errors.addUnexpectedArgumentType(name, dt, allowedTypes, argIndex);
+      errors.addUnexpectedArgumentType(expr, name, dt, allowedTypes, argIndex);
       
     }
     

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e68bba25/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/ArgumentValidator.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/ArgumentValidator.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/ArgumentValidator.java
index 8c374b2..dc22045 100644
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/ArgumentValidator.java
+++ b/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/ArgumentValidator.java
@@ -23,6 +23,6 @@ import java.util.List;
  * Validates whether the set of arguments are acceptable
  */
 public interface ArgumentValidator {
-  public void validateArguments(List<LogicalExpression> expressions, ErrorCollector errors);
+  public void validateArguments(String expr, List<LogicalExpression> expressions, ErrorCollector errors);
   public String[] getArgumentNamesByPosition();
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e68bba25/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/ArgumentValidators.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/ArgumentValidators.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/ArgumentValidators.java
index 0ad1a64..25cb887 100644
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/ArgumentValidators.java
+++ b/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/ArgumentValidators.java
@@ -46,10 +46,10 @@ public class ArgumentValidators {
     }
 
     @Override
-    public void validateArguments(List<LogicalExpression> expressions, ErrorCollector errors) {
+    public void validateArguments(String expr, List<LogicalExpression> expressions, ErrorCollector errors) {
       // only need to check argument count since any type is allowed.
       if (!argumentCount.contains(expressions.size()))
-        errors.addUnexpectedArgumentCount(expressions.size(), argumentCount);
+        errors.addUnexpectedArgumentCount(expr, expressions.size(), argumentCount);
     }
 
     @Override
@@ -79,25 +79,26 @@ public class ArgumentValidators {
     }
 
     @Override
-    public void validateArguments(List<LogicalExpression> expressions, ErrorCollector errors) {
+    public void validateArguments(String expr, List<LogicalExpression> expressions, ErrorCollector errors) {
       int i = -1;
       DataType t = null;
       for (LogicalExpression le : expressions) {
         i++;
-        if (t == null) t = le.getDataType();
+          DataType dataType = le.getDataType();
+          if (t == null) t = dataType;
 
-        if (!predicate.apply(le.getDataType())) {
-          errors.addUnexpectedType(i, le.getDataType());
+        if (!predicate.apply(dataType)) {
+          errors.addUnexpectedType(expr, i, dataType);
           continue;
         }
 
-        if (allSame && t != le.getDataType()) {
-          errors.addUnexpectedType(i, le.getDataType());
+        if (allSame && t != DataType.LATEBIND && dataType != DataType.LATEBIND && t != dataType) {
+          errors.addUnexpectedType(expr, i, dataType);
         }
 
       }
       if (!argumentCount.contains(expressions.size()))
-        errors.addUnexpectedArgumentCount(expressions.size(), argumentCount);
+        errors.addUnexpectedArgumentCount(expr, expressions.size(), argumentCount);
     }
 
     @Override
@@ -119,7 +120,8 @@ public class ArgumentValidators {
     public static class ComparableChecker implements Predicate<DataType> {
 
       public boolean apply(DataType dt) {
-        return dt.getComparability().equals(Comparability.ORDERED);
+          Comparability comparability = dt.getComparability();
+          return comparability.equals(Comparability.ORDERED) || comparability.equals(Comparability.UNKNOWN);
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e68bba25/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/BasicArgumentValidator.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/BasicArgumentValidator.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/BasicArgumentValidator.java
index 89b015a..eed49d3 100644
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/BasicArgumentValidator.java
+++ b/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/BasicArgumentValidator.java
@@ -34,12 +34,12 @@ public class BasicArgumentValidator implements ArgumentValidator {
   }
 
   @Override
-  public void validateArguments(List<LogicalExpression> expressions, ErrorCollector errors) {
-    if (expressions.size() != args.length) errors.addUnexpectedArgumentCount(expressions.size(), args.length);
+  public void validateArguments(String expr, List<LogicalExpression> expressions, ErrorCollector errors) {
+    if (expressions.size() != args.length) errors.addUnexpectedArgumentCount(expr, expressions.size(), args.length);
 
     int i = 0;
     for (LogicalExpression e : expressions) {
-      args[i].confirmDataType(i, e, errors);
+      args[i].confirmDataType(expr, i, e, errors);
 
       i++;
     }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e68bba25/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/ErrorCollector.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/ErrorCollector.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/ErrorCollector.java
index a618ac6..21ecec4 100644
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/ErrorCollector.java
+++ b/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/ErrorCollector.java
@@ -6,9 +6,9 @@
  * to you under the Apache License, Version 2.0 (the
  * "License"); you may not use this file except in compliance
  * with the License.  You may obtain a copy of the License at
- * 
+ *
  * http://www.apache.org/licenses/LICENSE-2.0
- * 
+ *
  * Unless required by applicable law or agreed to in writing, software
  * distributed under the License is distributed on an "AS IS" BASIS,
  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
@@ -21,13 +21,23 @@ import org.apache.drill.common.expression.types.DataType;
 
 import com.google.common.collect.Range;
 
-public class ErrorCollector {
+public interface ErrorCollector {
 
-  public void addGeneralError(String s){};
-  public void addUnexpectedArgumentType(String name, DataType actual, DataType[] expected, int argumentIndex){}
-  public void addUnexpectedArgumentCount(int actual, Range<Integer> expected){}
-  public void addUnexpectedArgumentCount(int actual, int expected){}
-  public void addNonNumericType(DataType actual){};
-  public void addUnexpectedType(int index, DataType actual){};
-  public void addExpectedConstantValue(int actual, String s){};
+    public void addGeneralError(String expr, String s);
+
+    public void addUnexpectedArgumentType(String expr, String name, DataType actual, DataType[] expected, int argumentIndex);
+
+    public void addUnexpectedArgumentCount(String expr, int actual, Range<Integer> expected);
+
+    public void addUnexpectedArgumentCount(String expr, int actual, int expected);
+
+    public void addNonNumericType(String expr, DataType actual);
+
+    public void addUnexpectedType(String expr, int index, DataType actual);
+
+    public void addExpectedConstantValue(String expr, int actual, String s);
+
+    boolean hasErrors();
+
+    String toErrorString();
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e68bba25/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/ErrorCollectorImpl.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/ErrorCollectorImpl.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/ErrorCollectorImpl.java
new file mode 100644
index 0000000..cc90b82
--- /dev/null
+++ b/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/ErrorCollectorImpl.java
@@ -0,0 +1,83 @@
+package org.apache.drill.common.expression;
+
+import com.google.common.base.Joiner;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Range;
+import org.apache.drill.common.expression.types.DataType;
+
+import java.util.Arrays;
+import java.util.List;
+
+public class ErrorCollectorImpl implements ErrorCollector {
+    List<ExpressionValidationError> errors;
+
+    public ErrorCollectorImpl() {
+        errors = Lists.newArrayList();
+    }
+
+    private String addExpr(String expr, String message) {
+        return "Expression: [" + expr + "]. Error: " + message;
+    }
+
+    @Override
+    public void addGeneralError(String expr, String s) {
+        errors.add(new ExpressionValidationError(addExpr(expr, s)));
+    }
+
+    @Override
+    public void addUnexpectedArgumentType(String expr, String name, DataType actual, DataType[] expected, int argumentIndex) {
+        errors.add(
+                new ExpressionValidationError(
+                        addExpr(expr, String.format(
+                                "Unexpected argument type. Index :%d Name: %s, Type: %s, Expected type(s): %s",
+                                argumentIndex, name, actual, Arrays.toString(expected)
+                        ))
+                )
+        );
+    }
+
+    @Override
+    public void addUnexpectedArgumentCount(String expr, int actual, Range<Integer> expected) {
+        errors.add(new ExpressionValidationError(
+                addExpr(expr, String.format("Unexpected argument count. Actual argument count: %d, Expected range: %s", actual, expected))
+        ));
+    }
+
+    @Override
+    public void addUnexpectedArgumentCount(String expr, int actual, int expected) {
+        errors.add(new ExpressionValidationError(
+                addExpr(expr, String.format("Unexpected argument count. Actual argument count: %d, Expected count: %d", actual, expected))
+        ));
+    }
+
+    @Override
+    public void addNonNumericType(String expr, DataType actual) {
+        errors.add(new ExpressionValidationError(
+                addExpr(expr, String.format("Unexpected numeric type. Actual type: %s", actual))
+        ));
+    }
+
+    @Override
+    public void addUnexpectedType(String expr, int index, DataType actual) {
+        errors.add(new ExpressionValidationError(
+                addExpr(expr, String.format("Unexpected argument type. Actual type: %s, Index: %d", actual, index))
+        ));
+    }
+
+    @Override
+    public void addExpectedConstantValue(String expr, int actual, String s) {
+        errors.add(new ExpressionValidationError(
+                addExpr(expr, String.format("Unexpected constant value. Name: %s, Actual: %s", s, actual))
+        ));
+    }
+
+    @Override
+    public boolean hasErrors() {
+        return !errors.isEmpty();
+    }
+
+    @Override
+    public String toErrorString() {
+        return "\n" + Joiner.on("\n").join(errors);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e68bba25/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/ExpressionValidationError.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/ExpressionValidationError.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/ExpressionValidationError.java
index 3e47001..50024a3 100644
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/ExpressionValidationError.java
+++ b/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/ExpressionValidationError.java
@@ -18,5 +18,14 @@
 package org.apache.drill.common.expression;
 
 public class ExpressionValidationError {
+    String message;
 
+    public ExpressionValidationError(String message) {
+        this.message = message;
+    }
+
+    @Override
+    public String toString() {
+        return message;
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e68bba25/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/FieldReference.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/FieldReference.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/FieldReference.java
index 073e449..94800ba 100644
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/FieldReference.java
+++ b/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/FieldReference.java
@@ -6,9 +6,9 @@
  * to you under the Apache License, Version 2.0 (the
  * "License"); you may not use this file except in compliance
  * with the License.  You may obtain a copy of the License at
- * 
+ *
  * http://www.apache.org/licenses/LICENSE-2.0
- * 
+ *
  * Unless required by applicable law or agreed to in writing, software
  * distributed under the License is distributed on an "AS IS" BASIS,
  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
@@ -32,16 +32,32 @@ import com.fasterxml.jackson.databind.annotation.JsonDeserialize;
 import com.fasterxml.jackson.databind.annotation.JsonSerialize;
 import com.fasterxml.jackson.databind.deser.std.StdDeserializer;
 import com.fasterxml.jackson.databind.ser.std.StdSerializer;
+import org.apache.drill.common.expression.types.DataType;
 
 @JsonSerialize(using = Se.class)
 @JsonDeserialize(using = De.class)
 public class FieldReference extends SchemaPath {
+  DataType overrideType;
 
   public FieldReference(String value) {
     super(value);
   }
 
-  public static class De extends StdDeserializer<FieldReference> {
+  public FieldReference(String value, DataType dataType) {
+    super(value);
+    this.overrideType = dataType;
+  }
+
+    @Override
+    public DataType getDataType() {
+        if(overrideType == null) {
+            return super.getDataType();
+        } else {
+            return overrideType;
+        }
+    }
+
+    public static class De extends StdDeserializer<FieldReference> {
 
     public De() {
       super(FieldReference.class);

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e68bba25/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/FunctionCall.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/FunctionCall.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/FunctionCall.java
index 8c0b580..ee76a5c 100644
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/FunctionCall.java
+++ b/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/FunctionCall.java
@@ -86,4 +86,9 @@ public class FunctionCall extends LogicalExpressionBase implements Iterable<Logi
       sb.append(") ");
     }
   }
+
+    @Override
+    public void resolveAndValidate(String expr, ErrorCollector errors) {
+        func.getArgumentValidator().validateArguments(expr, args, errors);
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e68bba25/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/FunctionDefinition.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/FunctionDefinition.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/FunctionDefinition.java
index 8797a4c..9d21763 100644
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/FunctionDefinition.java
+++ b/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/FunctionDefinition.java
@@ -49,6 +49,10 @@ public class FunctionDefinition {
   public String[] getArgumentNames(){
     return argumentValidator.getArgumentNamesByPosition();
   }
+
+  public ArgumentValidator getArgumentValidator() {
+      return argumentValidator;
+  }
   
   public static FunctionDefinition simple(String name, ArgumentValidator argumentValidator, OutputTypeDeterminer outputType, String... registeredNames){
     return new FunctionDefinition(name, argumentValidator, outputType, false,  false, registeredNames);

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e68bba25/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 cdd4440..a553f4c 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
@@ -21,6 +21,7 @@ import java.util.ArrayList;
 import java.util.List;
 
 import org.apache.drill.common.expression.IfExpression.IfCondition;
+import org.apache.drill.common.expression.types.DataType;
 import org.apache.drill.common.expression.visitors.ExprVisitor;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -37,8 +38,7 @@ public class IfExpression extends LogicalExpressionBase implements Iterable<IfCo
 	private IfExpression(List<IfCondition> conditions, LogicalExpression elseExpression){
 		this.conditions = ImmutableList.copyOf(conditions);
 		this.elseExpression = elseExpression;
-	};
-	
+	}
 	
 	public static class IfCondition{
 		public final LogicalExpression condition;
@@ -64,12 +64,21 @@ public class IfExpression extends LogicalExpressionBase implements Iterable<IfCo
 		List<IfCondition> conditions = new ArrayList<IfCondition>();
 		private LogicalExpression elseExpression;
 		
-		public void addCondition(IfCondition condition){
+		public Builder addCondition(IfCondition condition){
 			conditions.add(condition);
+            return this;
 		}
+
+        public Builder addConditions(Iterable<IfCondition> conditions) {
+            for(IfCondition condition : conditions) {
+                addCondition(condition);
+            }
+            return this;
+        }
 		
-		public void setElse(LogicalExpression elseExpression) {
+		public Builder setElse(LogicalExpression elseExpression) {
 			this.elseExpression = elseExpression;
+            return this;
 		}
 		
 		public IfExpression build(){
@@ -77,11 +86,14 @@ public class IfExpression extends LogicalExpressionBase implements Iterable<IfCo
 		}
 		
 	}
-	
-	
-	
-	
-	@Override
+
+
+    @Override
+    public DataType getDataType() {
+        return DataType.BOOLEAN;
+    }
+
+    @Override
   public void addToString(StringBuilder sb) {
 	  sb.append(" ( ");
 	  for(int i =0; i < conditions.size(); i++){
@@ -97,8 +109,12 @@ public class IfExpression extends LogicalExpressionBase implements Iterable<IfCo
 	  sb.append(" ) ");
   }
 
+    @Override
+    public void resolveAndValidate(String expr, ErrorCollector errors) {
+    }
+
 
-  public static Builder newBuilder(){
+    public static Builder newBuilder(){
 		return new Builder();
 	}
 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e68bba25/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 f7632a6..3df33a0 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
@@ -6,9 +6,9 @@
  * to you under the Apache License, Version 2.0 (the
  * "License"); you may not use this file except in compliance
  * with the License.  You may obtain a copy of the License at
- * 
+ *
  * http://www.apache.org/licenses/LICENSE-2.0
- * 
+ *
  * Unless required by applicable law or agreed to in writing, software
  * distributed under the License is distributed on an "AS IS" BASIS,
  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
@@ -44,58 +44,65 @@ 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 {
-  static final Logger logger = LoggerFactory.getLogger(LogicalExpression.class);
-
-  public abstract DataType getDataType();
-  public void addToString(StringBuilder sb);
-  public void resolveAndValidate(ErrorCollector errors);
-  public <T> T accept(ExprVisitor<T> visitor);
-
-  public static class De extends StdDeserializer<LogicalExpression> {
-    DrillConfig config;
-    public De(DrillConfig config) {
-      super(LogicalExpression.class);
-      this.config = config;
-    }
+    static final Logger logger = LoggerFactory.getLogger(LogicalExpression.class);
 
-    @Override
-    public LogicalExpression deserialize(JsonParser jp, DeserializationContext ctxt) throws IOException,
-        JsonProcessingException {
-      String expr = jp.getText();
-
-      if (expr == null || expr.isEmpty())
-        return null;
-      try {
-        // logger.debug("Parsing expression string '{}'", expr);
-        ExprLexer lexer = new ExprLexer(new ANTLRStringStream(expr));
-
-        CommonTokenStream tokens = new CommonTokenStream(lexer);
-        ExprParser parser = new ExprParser(tokens);
-        parser.setRegistry(new FunctionRegistry(config));
-        parse_return ret = parser.parse();
-        // logger.debug("Found expression '{}'", ret.e);
-        return ret.e;
-      } catch (RecognitionException e) {
-        throw new RuntimeException(e);
-      }
-    }
+    public abstract DataType getDataType();
 
-  }
+    public void addToString(StringBuilder sb);
 
-  public static class Se extends StdSerializer<LogicalExpression> {
+    public void resolveAndValidate(String expr, ErrorCollector errors);
 
-    protected Se() {
-      super(LogicalExpression.class);
-    }
+    public <T> T accept(ExprVisitor<T> visitor);
+
+    public static class De extends StdDeserializer<LogicalExpression> {
+        DrillConfig config;
+        ErrorCollector errorCollector;
+
+        public De(DrillConfig config) {
+            super(LogicalExpression.class);
+            this.config = config;
+            this.errorCollector = config.getErrorCollector();
+        }
+
+        @Override
+        public LogicalExpression deserialize(JsonParser jp, DeserializationContext ctxt) throws IOException,
+                JsonProcessingException {
+            String expr = jp.getText();
+
+            if (expr == null || expr.isEmpty())
+                return null;
+            try {
+                // logger.debug("Parsing expression string '{}'", expr);
+                ExprLexer lexer = new ExprLexer(new ANTLRStringStream(expr));
+
+                CommonTokenStream tokens = new CommonTokenStream(lexer);
+                ExprParser parser = new ExprParser(tokens);
+                parser.setRegistry(new FunctionRegistry(config));
+                parse_return ret = parser.parse();
+                // logger.debug("Found expression '{}'", ret.e);
+                ret.e.resolveAndValidate(expr, errorCollector);
+                return ret.e;
+            } catch (RecognitionException e) {
+                throw new RuntimeException(e);
+            }
+        }
 
-    @Override
-    public void serialize(LogicalExpression value, JsonGenerator jgen, SerializerProvider provider) throws IOException,
-        JsonGenerationException {
-      StringBuilder sb = new StringBuilder();
-      value.addToString(sb);
-      jgen.writeString(sb.toString());
     }
 
-  }
+    public static class Se extends StdSerializer<LogicalExpression> {
+
+        protected Se() {
+            super(LogicalExpression.class);
+        }
+
+        @Override
+        public void serialize(LogicalExpression value, JsonGenerator jgen, SerializerProvider provider) throws IOException,
+                JsonGenerationException {
+            StringBuilder sb = new StringBuilder();
+            value.addToString(sb);
+            jgen.writeString(sb.toString());
+        }
+
+    }
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e68bba25/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/LogicalExpressionBase.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/LogicalExpressionBase.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/LogicalExpressionBase.java
index a907146..e973df7 100644
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/LogicalExpressionBase.java
+++ b/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/LogicalExpressionBase.java
@@ -6,9 +6,9 @@
  * to you under the Apache License, Version 2.0 (the
  * "License"); you may not use this file except in compliance
  * with the License.  You may obtain a copy of the License at
- * 
+ *
  * http://www.apache.org/licenses/LICENSE-2.0
- * 
+ *
  * Unless required by applicable law or agreed to in writing, software
  * distributed under the License is distributed on an "AS IS" BASIS,
  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
@@ -24,46 +24,33 @@ import com.fasterxml.jackson.annotation.JsonProperty;
 import com.fasterxml.jackson.annotation.JsonPropertyOrder;
 
 
-@JsonPropertyOrder({ "type" })
-public abstract class LogicalExpressionBase implements LogicalExpression{
+@JsonPropertyOrder({"type"})
+public abstract class LogicalExpressionBase implements LogicalExpression {
+
 
-	
 //	public static DataType getJointType(String parentName, LogicalExpression expr1, LogicalExpression expr2) throws ExpressionValidationException{
 //		DataType dt = DataType.getCombinedCast(expr1.getDataType(), expr2.getDataType());
 //		if(dt == null) throw new ExpressionValidationException();
 //		
 //		return dt;
 //	}
-	
 
-	protected void i(StringBuilder sb, int indent){
-		for(int i = 0; i < indent; i++){
-			sb.append("  ");
-		}
-	}
-	
+
+    protected void i(StringBuilder sb, int indent) {
+        for (int i = 0; i < indent; i++) {
+            sb.append("  ");
+        }
+    }
+
 //	@Override
 //	public <T> T accept(ExprVisitor<T> visitor) {
 //		return visitor.visit(this);
 //	}
 
-	@Override
-	public DataType getDataType() {
-		throw new UnsupportedOperationException();
-	}
-
-
-  @Override
-  public void resolveAndValidate(ErrorCollector errors) {
-  }
-
-
-  @JsonProperty("type")
-	public String getDescription(){
-		return this.getClass().getSimpleName();
-	}
-	
+    @JsonProperty("type")
+    public String getDescription() {
+        return this.getClass().getSimpleName();
+    }
 
-	
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e68bba25/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 004d812..dfae6fd 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
@@ -21,6 +21,7 @@ import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 
 import org.apache.drill.common.expression.ValueExpressions.CollisionBehavior;
+import org.apache.drill.common.expression.types.DataType;
 import org.apache.drill.common.expression.visitors.ExprVisitor;
 
 public class SchemaPath extends LogicalExpressionBase{
@@ -119,17 +120,25 @@ public class SchemaPath extends LogicalExpressionBase{
 	public CharSequence getPath(){
 	  return originalPath;
 	}
-	
-  @Override
+
+    @Override
+    public DataType getDataType() {
+        return DataType.LATEBIND;
+    }
+
+    @Override
   public void addToString(StringBuilder sb) {
     sb.append("'");
     sb.append(originalPath);
     sb.append("'");
   }
 
+    @Override
+    public void resolveAndValidate(String expr, ErrorCollector errors) {
+    }
 
 
-  @Override
+    @Override
   public String toString() {
     return "SchemaPath [rootSegment=" + rootSegment + "]";
   }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e68bba25/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 d572715..f07f2a7 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,6 +17,7 @@
  ******************************************************************************/
 package org.apache.drill.common.expression;
 
+import org.apache.drill.common.expression.types.DataType;
 import org.apache.drill.common.expression.visitors.ExprVisitor;
 
 
@@ -44,8 +45,7 @@ public class ValueExpressions {
     
   }
   
-	protected static abstract class ValueExpression<V> extends
-			LogicalExpressionBase {
+	protected static abstract class ValueExpression<V> extends LogicalExpressionBase {
 		public final V value;
 
 		protected ValueExpression(String value) {
@@ -67,12 +67,22 @@ public class ValueExpressions {
 			return Boolean.parseBoolean(s);
 		}
 
-    @Override
+        @Override
+        public DataType getDataType() {
+            return DataType.BOOLEAN;
+        }
+
+        @Override
     public void addToString(StringBuilder sb) {
       sb.append(value.toString());
     }
 
-    @Override
+        @Override
+        public void resolveAndValidate(String expr, ErrorCollector errors) {
+        }
+
+
+        @Override
     public <T> T accept(ExprVisitor<T> visitor) {
       return visitor.visitBoolean(this);
     }
@@ -92,13 +102,22 @@ public class ValueExpressions {
 	    public double getDouble(){
 	      return d;
 	    }
-	    
-	    @Override
+
+         @Override
+         public DataType getDataType() {
+             return DataType.FLOAT32;
+         }
+
+         @Override
 	    public void addToString(StringBuilder sb) {
 	      sb.append(d);
 	    }
-	    
-	    @Override
+
+         @Override
+         public void resolveAndValidate(String expr, ErrorCollector errors) {
+         }
+
+         @Override
 	    public <T> T accept(ExprVisitor<T> visitor) {
 	      return visitor.visitDoubleExpression(this);
 	    }
@@ -113,13 +132,22 @@ public class ValueExpressions {
 		public long getLong(){
 		  return l;
 		}
-		
-    @Override
+
+        @Override
+        public DataType getDataType() {
+            return DataType.INT64;
+        }
+
+        @Override
     public void addToString(StringBuilder sb) {
       sb.append(l);
     }
-    
-    @Override
+
+        @Override
+        public void resolveAndValidate(String expr, ErrorCollector errors) {
+        }
+
+        @Override
     public <T> T accept(ExprVisitor<T> visitor) {
       return visitor.visitLongExpression(this);
     }
@@ -134,15 +162,24 @@ public class ValueExpressions {
 		protected String parseValue(String s) {
 			return s;
 		}
-		
-    @Override
+
+        @Override
+        public DataType getDataType() {
+            return DataType.NVARCHAR;
+        }
+
+        @Override
     public void addToString(StringBuilder sb) {
       sb.append("\"");
       sb.append(value.toString());
       sb.append("\"");
     }
-    
-    @Override
+
+        @Override
+        public void resolveAndValidate(String expr, ErrorCollector errors) {
+        }
+
+        @Override
     public <T> T accept(ExprVisitor<T> visitor) {
       return visitor.visitQuotedString(this);
     }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e68bba25/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/types/AtomType.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/types/AtomType.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/types/AtomType.java
deleted file mode 100644
index f21ddb9..0000000
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/types/AtomType.java
+++ /dev/null
@@ -1,67 +0,0 @@
-/*******************************************************************************
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- ******************************************************************************/
-package org.apache.drill.common.expression.types;
-
-
-public class AtomType extends DataType {
-  private String name;
-  private Comparability comparability;
-  private boolean isNumericType;
-  
-  public AtomType(String name, Comparability comparability, boolean isNumericType) {
-    super();
-    this.name = name;
-    this.comparability = comparability;
-    this.isNumericType = isNumericType;
-  }
-
-  
-  public boolean isNumericType() {
-    return isNumericType;
-  }
-
-
-  @Override
-  public String getName() {
-    return name;
-  }
-
-  @Override
-  public boolean isLateBind() {
-    return false;
-  }
-
-  @Override
-  public boolean hasChildType() {
-    return false;
-  }
-
-  @Override
-  public DataType getChildType() {
-    return null;
-  }
-
-  @Override
-  public Comparability getComparability() {
-    return comparability;
-  }
-  
-  
-  
-  
-}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e68bba25/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/ExpressionTreeMaterializer.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/ExpressionTreeMaterializer.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/ExpressionTreeMaterializer.java
new file mode 100644
index 0000000..391aec5
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/ExpressionTreeMaterializer.java
@@ -0,0 +1,138 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+
+package org.apache.drill.exec.record;
+
+import com.google.common.collect.Lists;
+import org.apache.drill.common.expression.*;
+import org.apache.drill.common.expression.types.DataType;
+import org.apache.drill.common.expression.visitors.ExprVisitor;
+
+import java.util.List;
+
+public class ExpressionTreeMaterializer {
+    public LogicalExpression Materialize(LogicalExpression expr, BatchSchema schema, ErrorCollector errorCollector) {
+        return expr.accept(new MaterializeVisitor(schema, errorCollector));
+    }
+
+    private class MaterializeVisitor implements ExprVisitor<LogicalExpression> {
+        private final ErrorCollector errorCollector;
+        private final BatchSchema schema;
+        private boolean isModified; // Flag to track if children is changed
+
+        public MaterializeVisitor(BatchSchema schema, ErrorCollector errorCollector) {
+            this.schema = schema;
+            this.errorCollector = errorCollector;
+            isModified = false;
+        }
+
+        private LogicalExpression validateNewExpr(LogicalExpression newExpr) {
+            StringBuilder stringBuilder = new StringBuilder();
+            newExpr.addToString(stringBuilder);
+            newExpr.resolveAndValidate(stringBuilder.toString(), errorCollector);
+            return newExpr;
+        }
+
+        @Override
+        public LogicalExpression visitFunctionCall(FunctionCall call) {
+            List<LogicalExpression> args = Lists.newArrayList(call.iterator());
+            boolean hasChanged = false;
+            for (int i = 0; i < args.size(); ++i) {
+                LogicalExpression newExpr = args.get(i).accept(this);
+                if (isModified) {
+                    hasChanged = true;
+                    args.set(i, newExpr);
+                    isModified = false;
+                }
+            }
+
+            if(hasChanged) {
+                isModified = true;
+                return validateNewExpr(new FunctionCall(call.getDefinition(), args));
+            }
+
+            return call;
+        }
+
+        @Override
+        public LogicalExpression visitIfExpression(IfExpression ifExpr) {
+            List<IfExpression.IfCondition> conditions = Lists.newArrayList(ifExpr.iterator());
+            boolean hasChanged = false;
+            LogicalExpression newElseExpr = null;
+            if(ifExpr.elseExpression != null) {
+                newElseExpr = ifExpr.elseExpression.accept(this);
+                hasChanged = isModified;
+            }
+
+            isModified = false;
+
+            for(int i = 0; i < conditions.size(); ++i) {
+                IfExpression.IfCondition condition = conditions.get(i);
+
+                LogicalExpression newCondition = condition.condition.accept(this);
+                boolean modified = isModified;
+                isModified = false;
+                LogicalExpression newExpr = condition.expression.accept(this);
+                if(modified || isModified) {
+                    conditions.set(i, new IfExpression.IfCondition(newCondition, newExpr));
+                    hasChanged = true;
+                    isModified = false;
+                }
+            }
+
+            if(hasChanged) {
+                isModified = true;
+                return validateNewExpr(IfExpression.newBuilder().setElse(newElseExpr).addConditions(conditions).build());
+            }
+
+            return ifExpr;
+        }
+
+        @Override
+        public LogicalExpression visitSchemaPath(SchemaPath path) {
+            for (MaterializedField field : schema) {
+                if (field.getType() != DataType.LATEBIND && field.matches(path)) {
+                    isModified = true;
+                    return validateNewExpr(new FieldReference(path.getPath().toString(), field.getType()));
+                }
+            }
+
+            return path;
+        }
+
+        @Override
+        public LogicalExpression visitLongExpression(ValueExpressions.LongExpression intExpr) {
+            return intExpr;
+        }
+
+        @Override
+        public LogicalExpression visitDoubleExpression(ValueExpressions.DoubleExpression dExpr) {
+            return dExpr;
+        }
+
+        @Override
+        public LogicalExpression visitBoolean(ValueExpressions.BooleanExpression e) {
+            return e;
+        }
+
+        @Override
+        public LogicalExpression visitQuotedString(ValueExpressions.QuotedString e) {
+            return e;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e68bba25/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/MaterializeVisitor.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/MaterializeVisitor.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/MaterializeVisitor.java
new file mode 100644
index 0000000..e28778f
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/MaterializeVisitor.java
@@ -0,0 +1,4 @@
+package org.apache.drill.exec.record;
+
+public interface MaterializeVisitor {
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e68bba25/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/MaterializedField.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/MaterializedField.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/MaterializedField.java
index 05fb576..b692a93 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/MaterializedField.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/MaterializedField.java
@@ -142,6 +142,17 @@ public class MaterializedField implements Comparable<MaterializedField> {
     // we've reviewed all path segments. confirm that we don't have any extra name parts.
     return !iter.hasNext();
 
+  private void check(String name, Object val1, Object expected) throws SchemaChangeException{
+    if(expected.equals(val1)) return;
+    throw new SchemaChangeException("Expected and actual field definitions don't match. Actual %s: %s, expected %s: %s", name, val1, name, expected);
+  }
+  
+  public void checkMaterialization(MaterializedField expected) throws SchemaChangeException{
+    if(this.type == expected.type || expected.type == DataType.LATEBIND) throw new SchemaChangeException("Expected and actual field definitions don't match. Actual DataType: %s, expected DataTypes: %s", this.type, expected.type);
+    if(expected.valueClass != null) check("valueClass", this.valueClass, expected.valueClass);
+    check("fieldId", this.fieldId, expected.fieldId);
+    check("nullability", this.nullable, expected.nullable);
+    check("valueMode", this.mode, expected.mode);
   }
 
   // private void check(String name, Object val1, Object expected) throws SchemaChangeException{

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e68bba25/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
new file mode 100644
index 0000000..ab68ea2
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/record/ExpressionTreeMaterializerTest.java
@@ -0,0 +1,110 @@
+package org.apache.drill.exec.record;
+
+import com.beust.jcommander.internal.Lists;
+import com.google.common.collect.Range;
+import org.apache.drill.common.expression.*;
+import org.apache.drill.common.expression.types.DataType;
+import org.apache.drill.common.physical.RecordField;
+import org.apache.drill.exec.exception.SchemaChangeException;
+import org.junit.Test;
+
+import java.util.List;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+public class ExpressionTreeMaterializerTest {
+    @Test
+    public void testMaterializingConstantTree() throws SchemaChangeException {
+        ExpressionTreeMaterializer tm = new ExpressionTreeMaterializer();
+        ErrorCollector ec = new ErrorCollectorImpl();
+        BatchSchema schema = new BatchSchema.BatchSchemaBuilder().buildAndClear();
+        LogicalExpression expr = tm.Materialize(new ValueExpressions.LongExpression(1L), schema, ec);
+        assertTrue(expr instanceof ValueExpressions.LongExpression);
+        assertEquals(1L, ValueExpressions.LongExpression.class.cast(expr).getLong());
+        assertFalse(ec.hasErrors());
+    }
+
+    @Test
+    public void testMaterializingLateboundField() throws SchemaChangeException {
+        ExpressionTreeMaterializer tm = new ExpressionTreeMaterializer();
+        ErrorCollector ec = new ErrorCollectorImpl();
+        BatchSchema.BatchSchemaBuilder builder = new BatchSchema.BatchSchemaBuilder();
+        builder.addTypedField((short) 2, DataType.INT64, false, RecordField.ValueMode.RLE, Long.class);
+        LogicalExpression expr = tm.Materialize(new FieldReference("test"), builder.buildAndClear(), ec);
+        assertEquals(DataType.INT64, expr.getDataType());
+        assertFalse(ec.hasErrors());
+    }
+
+    @Test
+    public void testMaterializingLateboundTree() throws SchemaChangeException {
+        ExpressionTreeMaterializer tm = new ExpressionTreeMaterializer();
+        ErrorCollector ec = new ErrorCollectorImpl();
+        BatchSchema.BatchSchemaBuilder builder = new BatchSchema.BatchSchemaBuilder();
+        builder.addTypedField((short) 2, DataType.INT64, false, RecordField.ValueMode.RLE, Long.class);
+        LogicalExpression expr = new IfExpression.Builder().addCondition(
+                new IfExpression.IfCondition(new FieldReference("test"),
+                        new IfExpression.Builder().addCondition(new IfExpression.IfCondition(new ValueExpressions.LongExpression(1L), new FieldReference("test1"))).build()
+                )
+        ).build();
+        LogicalExpression newExpr = tm.Materialize(expr, builder.buildAndClear(), ec);
+        assertTrue(newExpr instanceof IfExpression);
+        IfExpression newIfExpr = (IfExpression) newExpr;
+        assertEquals(1, newIfExpr.conditions.size());
+        IfExpression.IfCondition ifCondition = newIfExpr.conditions.get(0);
+        assertEquals(DataType.INT64, ifCondition.condition.getDataType());
+        assertTrue(ifCondition.expression instanceof IfExpression);
+        newIfExpr = (IfExpression) ifCondition.expression;
+        assertEquals(1, newIfExpr.conditions.size());
+        ifCondition = newIfExpr.conditions.get(0);
+        assertEquals(DataType.INT64, ifCondition.expression.getDataType());
+        assertEquals(1L, ((ValueExpressions.LongExpression) ifCondition.condition).getLong());
+        assertFalse(ec.hasErrors());
+    }
+
+    @Test
+    public void testMaterializingLateboundTreeValidated() throws SchemaChangeException {
+        ExpressionTreeMaterializer tm = new ExpressionTreeMaterializer();
+        ErrorCollector ec = new ErrorCollector() {
+            boolean errorFound = false;
+            @Override
+            public void addGeneralError(String expr, String s) {errorFound = true;}
+            @Override
+            public void addUnexpectedArgumentType(String expr, String name, DataType actual, DataType[] expected, int argumentIndex) {}
+            @Override
+            public void addUnexpectedArgumentCount(String expr, int actual, Range<Integer> expected) {}
+            @Override
+            public void addUnexpectedArgumentCount(String expr, int actual, int expected) {}
+            @Override
+            public void addNonNumericType(String expr, DataType actual) {}
+            @Override
+            public void addUnexpectedType(String expr, int index, DataType actual) {}
+            @Override
+            public void addExpectedConstantValue(String expr, int actual, String s) {}
+            @Override
+            public boolean hasErrors() { return errorFound; }
+            @Override
+            public String toErrorString() { return ""; }
+        };
+        BatchSchema.BatchSchemaBuilder builder = new BatchSchema.BatchSchemaBuilder();
+        builder.addTypedField((short) 2, DataType.INT64, false, RecordField.ValueMode.RLE, Long.class);
+        LogicalExpression expr = new FunctionCall(FunctionDefinition.simple("testFunc", new ArgumentValidator() {
+            @Override
+            public void validateArguments(String expr, List<LogicalExpression> expressions, ErrorCollector errors) {
+                errors.addGeneralError(expr, "Error!");
+            }
+
+            @Override
+            public String[] getArgumentNamesByPosition() {
+                return new String[0];
+            }
+        }, OutputTypeDeterminer.FIXED_BOOLEAN), Lists.newArrayList((LogicalExpression) new FieldReference("test")));
+        LogicalExpression newExpr = tm.Materialize(expr, builder.buildAndClear(), ec);
+        assertTrue(newExpr instanceof FunctionCall);
+        FunctionCall funcExpr = (FunctionCall) newExpr;
+        assertEquals(1, funcExpr.args.size());
+        assertEquals(DataType.INT64, funcExpr.args.get(0).getDataType());
+        assertTrue(ec.hasErrors());
+    }
+}


[14/53] [abbrv] Update typing system. Update RPC system. Add Fragmenting Implementation. Working single node. Distributed failing due to threading issues.

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/ExecPlanner.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/ExecPlanner.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/ExecPlanner.java
index 7b76810..94904a1 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/ExecPlanner.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/ExecPlanner.java
@@ -17,10 +17,10 @@
  ******************************************************************************/
 package org.apache.drill.exec.planner;
 
-import org.apache.drill.common.physical.PhysicalPlan;
-import org.apache.drill.exec.exception.FragmentSetupException;
-import org.apache.drill.exec.foreman.QueryWorkUnit;
+import org.apache.drill.common.exceptions.ExecutionSetupException;
 import org.apache.drill.exec.ops.QueryContext;
+import org.apache.drill.exec.physical.PhysicalPlan;
+import org.apache.drill.exec.work.QueryWorkUnit;
 
 
 /**
@@ -30,5 +30,5 @@ import org.apache.drill.exec.ops.QueryContext;
 public interface ExecPlanner {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ExecPlanner.class);
   
-  public QueryWorkUnit getWorkUnit(QueryContext context, PhysicalPlan plan, int maxWidth) throws FragmentSetupException;
+  public QueryWorkUnit getWorkUnit(QueryContext context, PhysicalPlan plan, int maxWidth) throws ExecutionSetupException;
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/FragmentMaterializer.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/FragmentMaterializer.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/FragmentMaterializer.java
deleted file mode 100644
index 51b0691..0000000
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/FragmentMaterializer.java
+++ /dev/null
@@ -1,86 +0,0 @@
-/*******************************************************************************
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- ******************************************************************************/
-package org.apache.drill.exec.planner;
-
-import org.apache.drill.common.physical.pop.base.AbstractPhysicalVisitor;
-import org.apache.drill.common.physical.pop.base.AbstractStore;
-import org.apache.drill.common.physical.pop.base.Exchange;
-import org.apache.drill.common.physical.pop.base.PhysicalOperator;
-import org.apache.drill.common.physical.pop.base.Scan;
-import org.apache.drill.common.physical.pop.base.Store;
-import org.apache.drill.exec.exception.FragmentSetupException;
-
-public class FragmentMaterializer extends AbstractPhysicalVisitor<PhysicalOperator, FragmentMaterializer.IndexedFragmentNode, FragmentSetupException>{
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(FragmentMaterializer.class);
-
-  
-  @Override
-  public PhysicalOperator visitExchange(Exchange exchange, IndexedFragmentNode iNode) throws FragmentSetupException {
-    if(exchange == iNode.getNode().getSendingExchange()){
-      
-      // this is a sending exchange.
-      PhysicalOperator child = exchange.getChild();
-      return exchange.getSender(iNode.getMinorFragmentId(), child);
-      
-    }else{
-      // receiving exchange.
-      return exchange.getReceiver(iNode.getMinorFragmentId());
-    }
-  }
-
-  @Override
-  public PhysicalOperator visitScan(Scan<?> scan, IndexedFragmentNode iNode) throws FragmentSetupException {
-    return scan.getSpecificScan(iNode.getMinorFragmentId());
-  }
-
-  @Override
-  public PhysicalOperator visitStore(Store store, IndexedFragmentNode iNode) throws FragmentSetupException {
-    PhysicalOperator child = store.getChild();
-    return store.getSpecificStore(child, iNode.getMinorFragmentId());
-  }
-
-  @Override
-  public PhysicalOperator visitUnknown(PhysicalOperator op, IndexedFragmentNode iNode) throws FragmentSetupException {
-    return op;
-  }
-  
-  public static class IndexedFragmentNode{
-    final FragmentWrapper info;
-    final int minorFragmentId;
-    
-    public IndexedFragmentNode(int minorFragmentId, FragmentWrapper info) {
-      super();
-      this.info = info;
-      this.minorFragmentId = minorFragmentId;
-    }
-
-    public FragmentNode getNode() {
-      return info.getNode();
-    }
-
-    public int getMinorFragmentId() {
-      return minorFragmentId;
-    }
-
-    public FragmentWrapper getInfo() {
-      return info;
-    }
-    
-  }
-  
-}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/FragmentNode.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/FragmentNode.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/FragmentNode.java
deleted file mode 100644
index f53240e..0000000
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/FragmentNode.java
+++ /dev/null
@@ -1,138 +0,0 @@
-/*******************************************************************************
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- ******************************************************************************/
-package org.apache.drill.exec.planner;
-
-import java.util.Iterator;
-import java.util.List;
-
-import org.apache.drill.common.physical.pop.base.Exchange;
-import org.apache.drill.common.physical.pop.base.PhysicalOperator;
-import org.apache.drill.common.proto.CoordinationProtos.DrillbitEndpoint;
-import org.apache.drill.exec.exception.FragmentSetupException;
-
-import com.google.common.collect.Lists;
-
-public class FragmentNode implements Iterable<FragmentNode.ExchangeFragmentPair>{
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(FragmentNode.class);
-  
-  private PhysicalOperator root;
-  private Exchange sendingExchange;
-  private final List<ExchangeFragmentPair> receivingExchangePairs = Lists.newLinkedList();
-  private FragmentStats stats = new FragmentStats();
-  
-  public void addOperator(PhysicalOperator o){
-    if(root == null){
-      root = o;
-    }
-  }
-  
-  public void addSendExchange(Exchange e) throws FragmentSetupException{
-    if(sendingExchange != null) throw new FragmentSetupException("Fragment was trying to add a second SendExchange.  ");
-    sendingExchange = e;
-  }
-  
-  public void addReceiveExchange(Exchange e, FragmentNode fragment){
-    this.receivingExchangePairs.add(new ExchangeFragmentPair(e, fragment));
-  }
-
-  @Override
-  public Iterator<ExchangeFragmentPair> iterator() {
-    return this.receivingExchangePairs.iterator();
-  }
-
-  public List<ExchangeFragmentPair> getReceivingExchangePairs() {
-    return receivingExchangePairs;
-  }
-
-  public PhysicalOperator getRoot() {
-    return root;
-  }
-
-  public Exchange getSendingExchange() {
-    return sendingExchange;
-  }
-
-  public <T, V> T accept(FragmentVisitor<T, V> visitor, V extra){
-    return visitor.visit(this, extra);
-  }
-  
-  public FragmentStats getStats(){
-    return stats;
-  }
-  
-  public class ExchangeFragmentPair {
-    private Exchange exchange;
-    private FragmentNode node;
-    public ExchangeFragmentPair(Exchange exchange, FragmentNode node) {
-      super();
-      this.exchange = exchange;
-      this.node = node;
-    }
-    public Exchange getExchange() {
-      return exchange;
-    }
-    public FragmentNode getNode() {
-      return node;
-    }
-    @Override
-    public int hashCode() {
-      final int prime = 31;
-      int result = 1;
-      result = prime * result + FragmentNode.this.hashCode();
-      result = prime * result + ((exchange == null) ? 0 : exchange.hashCode());
-      result = prime * result + ((node == null) ? 0 : node.hashCode());
-      return result;
-    }
-    
-  }
-
-  @Override
-  public int hashCode() {
-    final int prime = 31;
-    int result = 1;
-    result = prime * result + ((receivingExchangePairs == null) ? 0 : receivingExchangePairs.hashCode());
-    result = prime * result + ((root == null) ? 0 : root.hashCode());
-    result = prime * result + ((sendingExchange == null) ? 0 : sendingExchange.hashCode());
-    result = prime * result + ((stats == null) ? 0 : stats.hashCode());
-    return result;
-  }
-
-  @Override
-  public boolean equals(Object obj) {
-    if (this == obj) return true;
-    if (obj == null) return false;
-    if (getClass() != obj.getClass()) return false;
-    FragmentNode other = (FragmentNode) obj;
-    if (receivingExchangePairs == null) {
-      if (other.receivingExchangePairs != null) return false;
-    } else if (!receivingExchangePairs.equals(other.receivingExchangePairs)) return false;
-    if (root == null) {
-      if (other.root != null) return false;
-    } else if (!root.equals(other.root)) return false;
-    if (sendingExchange == null) {
-      if (other.sendingExchange != null) return false;
-    } else if (!sendingExchange.equals(other.sendingExchange)) return false;
-    if (stats == null) {
-      if (other.stats != null) return false;
-    } else if (!stats.equals(other.stats)) return false;
-    return true;
-  }
-
-  
- 
-}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/FragmentPlanningSet.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/FragmentPlanningSet.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/FragmentPlanningSet.java
deleted file mode 100644
index 3f7c3a9..0000000
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/FragmentPlanningSet.java
+++ /dev/null
@@ -1,61 +0,0 @@
-/*******************************************************************************
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- ******************************************************************************/
-package org.apache.drill.exec.planner;
-
-import java.util.Iterator;
-import java.util.Map;
-
-import org.apache.drill.common.proto.CoordinationProtos.DrillbitEndpoint;
-import org.apache.drill.exec.ops.QueryContext;
-
-public class FragmentPlanningSet implements Iterable<FragmentWrapper>{
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(FragmentPlanningSet.class);
-  
-  private Map<FragmentNode, FragmentWrapper> fragmentMap;
-  private int majorFragmentIdIndex = 0;
-  private QueryContext context;
-  
-  public FragmentPlanningSet(QueryContext context){
-    this.context = context;
-  }
-  
-  public void setStats(FragmentNode node, FragmentStats stats){
-    get(node).setStats(stats);
-  }
-
-  public void addAffinity(FragmentNode n, DrillbitEndpoint endpoint, float affinity){
-    get(n).addEndpointAffinity(endpoint, affinity);
-  }
-  
-  public void setWidth(FragmentNode n, int width){
-    get(n).setWidth(width);
-  }
-  
-  private FragmentWrapper get(FragmentNode node){
-    FragmentWrapper info = fragmentMap.get(node);
-    if(info == null) info = new FragmentWrapper(node, majorFragmentIdIndex++);
-    return info;
-  }
-
-  @Override
-  public Iterator<FragmentWrapper> iterator() {
-    return this.fragmentMap.values().iterator();
-  }
-  
-  
-}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/FragmentRunnable.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/FragmentRunnable.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/FragmentRunnable.java
deleted file mode 100644
index d551aa4..0000000
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/FragmentRunnable.java
+++ /dev/null
@@ -1,124 +0,0 @@
-/*******************************************************************************
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- ******************************************************************************/
-package org.apache.drill.exec.planner;
-
-import io.netty.buffer.ByteBuf;
-
-import java.util.concurrent.atomic.AtomicInteger;
-
-import org.apache.drill.exec.exception.FragmentSetupException;
-import org.apache.drill.exec.foreman.CancelableQuery;
-import org.apache.drill.exec.foreman.StatusProvider;
-import org.apache.drill.exec.metrics.SingleThreadNestedCounter;
-import org.apache.drill.exec.ops.FragmentContext;
-import org.apache.drill.exec.ops.FragmentConverter;
-import org.apache.drill.exec.ops.FragmentRoot;
-import org.apache.drill.exec.proto.ExecProtos.FragmentStatus;
-import org.apache.drill.exec.proto.ExecProtos.FragmentStatus.FragmentState;
-import org.apache.drill.exec.proto.ExecProtos.PlanFragment;
-import org.apache.drill.exec.server.DrillbitContext;
-
-import com.yammer.metrics.MetricRegistry;
-import com.yammer.metrics.Timer;
-
-/**
- * Responsible for running a single fragment on a single Drillbit. Listens/responds to status request and cancellation
- * messages.
- */
-public class FragmentRunnable implements Runnable, CancelableQuery, StatusProvider {
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(FragmentRunnable.class);
-
-  private final AtomicInteger state = new AtomicInteger(FragmentState.AWAITING_ALLOCATION_VALUE);
-  private final FragmentRoot root;
-  private final FragmentContext context;
-
-  public FragmentRunnable(DrillbitContext dbContext, long fragmentId) throws FragmentSetupException {
-    PlanFragment fragment = dbContext.getCache().getFragment(fragmentId);
-    if (fragment == null) throw new FragmentSetupException(String.format("The provided fragment id [%d] was unknown.", fragmentId));
-    this.context = new FragmentContext(dbContext, fragment);
-    this.root = FragmentConverter.getFragment(this.context);
-  }
-
-  @Override
-  public FragmentStatus getStatus() {
-    return FragmentStatus.newBuilder() //
-        .setBatchesCompleted(context.batchesCompleted.get()) //
-        .setDataProcessed(context.dataProcessed.get()) //
-        .setMemoryUse(context.getAllocator().getAllocatedMemory()) //
-        .build();
-  }
-
-  @Override
-  public boolean cancel(long queryId) {
-    if (context.getFragment().getQueryId() == queryId) {
-      state.set(FragmentState.CANCELLED_VALUE);
-      return true;
-    }
-    return false;
-  }
-
-  private void fail(Throwable cause){
-    context.fail(cause);
-    state.set(FragmentState.FAILED_VALUE);
-  }
-  
-  @Override
-  public void run() {
-    if(!updateState(FragmentState.AWAITING_ALLOCATION, FragmentState.RUNNING, false)){
-      fail(new RuntimeException(String.format("Run was called when fragment was in %s state.  FragmentRunnables should only be started when they are currently in awaiting allocation state.", FragmentState.valueOf(state.get()))));
-      return;
-    }
-    
-    Timer.Context t = context.fragmentTime.time();
-    
-    // setup the query.
-    try{
-      root.setup();
-    }catch(FragmentSetupException e){
-      
-      context.fail(e);
-      return;
-    }
-    
-    // run the query.
-    try{
-      while(state.get() == FragmentState.RUNNING_VALUE){
-        if(!root.next()){
-          updateState(FragmentState.RUNNING, FragmentState.FINISHED, false);
-        }
-      }
-      t.stop();
-    }catch(Exception ex){
-      fail(ex);
-    }
-    
-  }
-
-  private boolean updateState(FragmentState current, FragmentState update, boolean exceptionOnFailure) {
-    boolean success = state.compareAndSet(current.getNumber(), update.getNumber());
-    if (!success && exceptionOnFailure) {
-      context.fail(new RuntimeException(String.format(
-          "State was different than expected.  Attempting to update state from %s to %s however current state was %s.",
-          current.name(), update.name(), FragmentState.valueOf(state.get()))));
-      return false;
-    }
-
-    return true;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/FragmentScheduler.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/FragmentScheduler.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/FragmentScheduler.java
deleted file mode 100644
index 168072a..0000000
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/FragmentScheduler.java
+++ /dev/null
@@ -1,32 +0,0 @@
-/*******************************************************************************
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- ******************************************************************************/
-package org.apache.drill.exec.planner;
-
-import org.apache.drill.exec.foreman.ResourceRequest;
-import org.apache.drill.exec.foreman.ResourceRequest.ResourceAllocation;
-
-import com.google.common.util.concurrent.ListenableFutureTask;
-
-public class FragmentScheduler {
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(FragmentScheduler.class);
-  
-  public void getRunningResources(ResourceRequest resources, Runnable listener){
-    // request the resource.
-    
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/FragmentStats.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/FragmentStats.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/FragmentStats.java
deleted file mode 100644
index 512b5d0..0000000
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/FragmentStats.java
+++ /dev/null
@@ -1,63 +0,0 @@
-/*******************************************************************************
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- ******************************************************************************/
-package org.apache.drill.exec.planner;
-
-import org.apache.drill.common.physical.OperatorCost;
-
-public class FragmentStats {
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(FragmentStats.class);
-  
-  private int maxWidth = Integer.MAX_VALUE;
-  private float networkCost; 
-  private float diskCost;
-  private float memoryCost;
-  private float cpuCost;
-  
-  public void addMaxWidth(int width){
-    maxWidth = Math.min(maxWidth, width);
-  }
-  
-  public void addCost(OperatorCost cost){
-    networkCost += cost.getNetwork();
-    diskCost += cost.getDisk();
-    memoryCost += cost.getMemory();
-    cpuCost += cost.getCpu();
-  }
-
-  public int getMaxWidth() {
-    return maxWidth;
-  }
-
-  public float getNetworkCost() {
-    return networkCost;
-  }
-
-  public float getDiskCost() {
-    return diskCost;
-  }
-
-  public float getMemoryCost() {
-    return memoryCost;
-  }
-
-  public float getCpuCost() {
-    return cpuCost;
-  }
-  
-  
-}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/FragmentStatsCollector.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/FragmentStatsCollector.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/FragmentStatsCollector.java
deleted file mode 100644
index a0dcde3..0000000
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/FragmentStatsCollector.java
+++ /dev/null
@@ -1,109 +0,0 @@
-/*******************************************************************************
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- ******************************************************************************/
-package org.apache.drill.exec.planner;
-
-import org.apache.drill.common.physical.pop.base.AbstractPhysicalVisitor;
-import org.apache.drill.common.physical.pop.base.Exchange;
-import org.apache.drill.common.physical.pop.base.PhysicalOperator;
-import org.apache.drill.common.physical.pop.base.Scan;
-import org.apache.drill.common.physical.pop.base.Store;
-import org.apache.drill.exec.planner.FragmentNode.ExchangeFragmentPair;
-
-import com.google.common.base.Preconditions;
-
-public class FragmentStatsCollector implements FragmentVisitor<Void, FragmentStats> {
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(FragmentStatsCollector.class);
-
-  //private HashMap<FragmentNode, FragmentStats> nodeStats = Maps.newHashMap();
-  private final StatsCollector opCollector = new StatsCollector();
-  private final FragmentPlanningSet planningSet;
-  
-  public FragmentStatsCollector(FragmentPlanningSet planningSet){
-    this.planningSet = planningSet;
-  }
-  
-  @Override
-  public Void visit(FragmentNode n, FragmentStats stats) {
-    Preconditions.checkNotNull(stats);
-    Preconditions.checkNotNull(n);
-
-    n.getRoot().accept(opCollector, stats);
-
-    // sending exchange.
-    Exchange sending = n.getSendingExchange();
-    if (sending != null) {
-      stats.addCost(sending.getAggregateSendCost());
-      stats.addMaxWidth(sending.getMaxSendWidth());
-    }
-
-    // receivers...
-    for (ExchangeFragmentPair child : n) {
-      // add exchange receive cost.
-      Exchange receivingExchange = child.getExchange();
-      stats.addCost(receivingExchange.getAggregateReceiveCost());
-
-      FragmentStats childStats = new FragmentStats();
-      FragmentNode childNode = child.getNode();
-      childNode.accept(this, childStats);
-    }
-    
-    // store the stats for later use.
-    planningSet.setStats(n, stats);
-    
-    return null;
-  }
-
-  public void collectStats(FragmentNode rootFragment) {
-    FragmentStats s = new FragmentStats();
-    rootFragment.accept(this, s);
-  }
-
-  private class StatsCollector extends AbstractPhysicalVisitor<Void, FragmentStats, RuntimeException> {
-
-    @Override
-    public Void visitExchange(Exchange exchange, FragmentStats stats) throws RuntimeException {
-      // don't do anything here since we'll add the exchange costs elsewhere. We also don't want navigate across
-      // exchanges since they are separate fragments.
-      return null;
-    }
-
-    @Override
-    public Void visitScan(Scan<?> scan, FragmentStats stats) {
-      stats.addMaxWidth(scan.getReadEntries().size());
-      return super.visitScan(scan, stats);
-    }
-
-    @Override
-    public Void visitStore(Store store, FragmentStats stats) {
-      stats.addMaxWidth(store.getMaxWidth());
-      return super.visitStore(store, stats);
-    }
-
-    @Override
-    public Void visitUnknown(PhysicalOperator op, FragmentStats stats) {
-      stats.addCost(op.getCost());
-      for (PhysicalOperator child : op) {
-        child.accept(this, stats);
-      }
-      return null;
-    }
-
-  }
-
-  
-}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/FragmentVisitor.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/FragmentVisitor.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/FragmentVisitor.java
deleted file mode 100644
index 12d2b9f..0000000
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/FragmentVisitor.java
+++ /dev/null
@@ -1,22 +0,0 @@
-/*******************************************************************************
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- ******************************************************************************/
-package org.apache.drill.exec.planner;
-
-public interface FragmentVisitor<T, V> {
-  public T visit(FragmentNode n, V extra);
-}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/FragmentWrapper.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/FragmentWrapper.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/FragmentWrapper.java
deleted file mode 100644
index a1e4f81..0000000
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/FragmentWrapper.java
+++ /dev/null
@@ -1,127 +0,0 @@
-/*******************************************************************************
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- ******************************************************************************/
-package org.apache.drill.exec.planner;
-
-import java.util.Collection;
-import java.util.Collections;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.ThreadLocalRandom;
-
-import org.apache.commons.lang.NotImplementedException;
-import org.apache.drill.common.physical.EndpointAffinity;
-import org.apache.drill.common.proto.CoordinationProtos.DrillbitEndpoint;
-import org.apache.drill.exec.planner.FragmentNode.ExchangeFragmentPair;
-
-import com.google.common.base.Preconditions;
-import com.google.common.collect.Lists;
-import com.google.common.collect.Maps;
-
-public class FragmentWrapper {
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(FragmentWrapper.class);
-
-  private final FragmentNode node;
-  private final int majorFragmentId;
-  private int width = -1;
-  private FragmentStats stats;
-  private boolean endpointsAssigned;
-  private Map<DrillbitEndpoint, EndpointAffinity> endpointAffinity = Maps.newHashMap();
-
-  // a list of assigned endpoints. Technically, there could repeated endpoints in this list if we'd like to assign the
-  // same fragment multiple times to the same endpoint.
-  private List<DrillbitEndpoint> endpoints = Lists.newLinkedList();
-
-  public FragmentWrapper(FragmentNode node, int majorFragmentId) {
-    this.majorFragmentId = majorFragmentId;
-    this.node = node;
-  }
-
-  public FragmentStats getStats() {
-    return stats;
-  }
-
-  public void setStats(FragmentStats stats) {
-    this.stats = stats;
-  }
-
-  public void addEndpointAffinity(DrillbitEndpoint endpoint, float affinity) {
-    Preconditions.checkState(!endpointsAssigned);
-    EndpointAffinity ea = endpointAffinity.get(endpoint);
-    if (ea == null) {
-      ea = new EndpointAffinity(endpoint);
-      endpointAffinity.put(endpoint, ea);
-    }
-
-    ea.addAffinity(affinity);
-    endpointAffinity.put(endpoint, ea);
-  }
-
-  public int getMajorFragmentId() {
-    return majorFragmentId;
-  }
-
-  public int getWidth() {
-    return width;
-  }
-
-  public void setWidth(int width) {
-    Preconditions.checkState(width == -1);
-    this.width = width;
-  }
-
-  public FragmentNode getNode() {
-    return node;
-  }
-
-  public void assignEndpoints(Collection<DrillbitEndpoint> allPossible) {
-    Preconditions.checkState(!endpointsAssigned);
-
-    endpointsAssigned = true;
-    
-    List<EndpointAffinity> values = Lists.newArrayList();
-    values.addAll(endpointAffinity.values());
-    
-    if(values.size() == 0){
-      final int div = allPossible.size();
-      int start = ThreadLocalRandom.current().nextInt(div);
-      // round robin with random start.
-      for(int i = start; i < start + width; i++){
-        endpoints.add(values.get(i % div).getEndpoint());
-      }
-    }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());
-      }
-    }
-
-    node.getSendingExchange().setupSenders(endpoints);
-    for (ExchangeFragmentPair e : node.getReceivingExchangePairs()) {
-      e.getExchange().setupReceivers(endpoints);
-    }
-  }
-
-  public DrillbitEndpoint getAssignedEndpoint(int minorFragmentId) {
-    Preconditions.checkState(endpointsAssigned);
-    return this.endpoints.get(minorFragmentId);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/FragmentingPhysicalVisitor.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/FragmentingPhysicalVisitor.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/FragmentingPhysicalVisitor.java
deleted file mode 100644
index 5f67617..0000000
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/FragmentingPhysicalVisitor.java
+++ /dev/null
@@ -1,71 +0,0 @@
-/*******************************************************************************
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- ******************************************************************************/
-package org.apache.drill.exec.planner;
-
-import org.apache.drill.common.physical.pop.base.AbstractPhysicalVisitor;
-import org.apache.drill.common.physical.pop.base.Exchange;
-import org.apache.drill.common.physical.pop.base.PhysicalOperator;
-import org.apache.drill.exec.exception.FragmentSetupException;
-
-/**
- * Responsible for breaking a plan into its constituent Fragments.
- */
-public class FragmentingPhysicalVisitor extends AbstractPhysicalVisitor<FragmentNode, FragmentNode, FragmentSetupException> {
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(FragmentingPhysicalVisitor.class);
-
-  private FragmentNode rootFragment = new FragmentNode();
-  
-  public FragmentingPhysicalVisitor(){
-  }
-  
-  
-  @Override
-  public FragmentNode visitExchange(Exchange exchange, FragmentNode value) throws FragmentSetupException {
-//    logger.debug("Visiting Exchange {}", exchange);
-    if(value == null) throw new FragmentSetupException("The simple fragmenter was called without a FragmentBuilder value.  This will only happen if the initial call to SimpleFragmenter is by a Exchange node.  This should never happen since an Exchange node should never be the root node of a plan.");
-    FragmentNode next = getNextBuilder();
-    value.addReceiveExchange(exchange, next);
-    next.addSendExchange(exchange);
-    exchange.getChild().accept(this, getNextBuilder());
-    return value;
-  }
-  
-  @Override
-  public FragmentNode visitUnknown(PhysicalOperator op, FragmentNode value)  throws FragmentSetupException{
-//    logger.debug("Visiting Other {}", op);
-    value = ensureBuilder(value);
-    value.addOperator(op);
-    for(PhysicalOperator child : op){
-      child.accept(this, value);
-    }
-    return value;
-  }
-  
-  private FragmentNode ensureBuilder(FragmentNode value) throws FragmentSetupException{
-    if(value != null){
-      return value;
-    }else{
-      return rootFragment;
-    }
-  }
-  
-  public FragmentNode getNextBuilder(){
-    return new FragmentNode();
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/MaterializedFragment.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/MaterializedFragment.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/MaterializedFragment.java
deleted file mode 100644
index d1c85cb..0000000
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/MaterializedFragment.java
+++ /dev/null
@@ -1,69 +0,0 @@
-/*******************************************************************************
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- ******************************************************************************/
-package org.apache.drill.exec.planner;
-
-import org.apache.drill.common.physical.OperatorCost;
-import org.apache.drill.common.physical.pop.base.PhysicalOperator;
-import org.apache.drill.common.proto.CoordinationProtos.DrillbitEndpoint;
-
-public class MaterializedFragment {
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(MaterializedFragment.class);
-
-  final PhysicalOperator root;
-  final DrillbitEndpoint endpoint;
-  final long queryId;
-  final int majorFragmentId;
-  final int minorFragmentId;
-  final OperatorCost cost;
-
-  public MaterializedFragment(PhysicalOperator root, DrillbitEndpoint endpoint, long queryId, int majorFragmentId,
-      int minorFragmentId, OperatorCost cost) {
-    super();
-    this.root = root;
-    this.endpoint = endpoint;
-    this.queryId = queryId;
-    this.majorFragmentId = majorFragmentId;
-    this.minorFragmentId = minorFragmentId;
-    this.cost = cost;
-  }
-
-  public PhysicalOperator getRoot() {
-    return root;
-  }
-
-  public DrillbitEndpoint getEndpoint() {
-    return endpoint;
-  }
-
-  public long getQueryId() {
-    return queryId;
-  }
-
-  public int getMajorFragmentId() {
-    return majorFragmentId;
-  }
-
-  public int getMinorFragmentId() {
-    return minorFragmentId;
-  }
-
-  public OperatorCost getCost() {
-    return cost;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/PhysicalPlanReader.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/PhysicalPlanReader.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/PhysicalPlanReader.java
index ff31cd9..3c2df61 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/PhysicalPlanReader.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/PhysicalPlanReader.java
@@ -19,29 +19,74 @@ package org.apache.drill.exec.planner;
 
 import java.io.IOException;
 
-import org.apache.drill.common.physical.PhysicalPlan;
-import org.apache.drill.common.proto.CoordinationProtos.DrillbitEndpoint;
-import org.apache.drill.exec.server.DrillbitContext;
-import org.apache.drill.exec.store.StorageEngineRegistry;
+import org.apache.drill.common.config.DrillConfig;
+import org.apache.drill.common.logical.LogicalPlan;
+import org.apache.drill.exec.coord.DrillbitEndpointSerDe;
+import org.apache.drill.exec.physical.PhysicalPlan;
+import org.apache.drill.exec.physical.base.FragmentLeaf;
+import org.apache.drill.exec.physical.base.FragmentRoot;
+import org.apache.drill.exec.physical.base.PhysicalOperator;
+import org.apache.drill.exec.physical.base.PhysicalOperatorUtil;
+import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
+import org.apache.drill.exec.proto.SchemaDefProtos.MajorType;
+import org.apache.drill.exec.record.MajorTypeSerDe;
 
 import com.fasterxml.jackson.core.JsonProcessingException;
 import com.fasterxml.jackson.databind.InjectableValues;
 import com.fasterxml.jackson.databind.ObjectMapper;
 import com.fasterxml.jackson.databind.ObjectReader;
+import com.fasterxml.jackson.databind.module.SimpleModule;
 
 public class PhysicalPlanReader {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(PhysicalPlanReader.class);
 
-  private final ObjectReader reader;
+  private final ObjectReader physicalPlanReader;
+  private final ObjectMapper mapper;
+  private final ObjectReader operatorReader;
+  private final ObjectReader logicalPlanReader;
 
-  public PhysicalPlanReader(ObjectMapper mapper, DrillbitEndpoint endpoint) {
+  public PhysicalPlanReader(DrillConfig config, ObjectMapper mapper, final DrillbitEndpoint endpoint) {
+
+    // Endpoint serializer/deserializer.
+    SimpleModule deserModule = new SimpleModule("PhysicalOperatorModule") //
+        .addSerializer(DrillbitEndpoint.class, new DrillbitEndpointSerDe.Se()) //
+        .addDeserializer(DrillbitEndpoint.class, new DrillbitEndpointSerDe.De()) //
+        .addSerializer(MajorType.class, new MajorTypeSerDe.Se())
+        .addDeserializer(MajorType.class, new MajorTypeSerDe.De());
+        
+        
+    mapper.registerModule(deserModule);
+    mapper.registerSubtypes(PhysicalOperatorUtil.getSubTypes(config));
     InjectableValues injectables = new InjectableValues.Std() //
         .addValue(DrillbitEndpoint.class, endpoint); //
-    this.reader = mapper.reader(PhysicalPlan.class).with(injectables);
+
+    this.mapper = mapper;
+    this.physicalPlanReader = mapper.reader(PhysicalPlan.class).with(injectables);
+    this.operatorReader = mapper.reader(PhysicalOperator.class).with(injectables);
+    this.logicalPlanReader = mapper.reader(LogicalPlan.class).with(injectables);
+  }
+
+  public String writeJson(PhysicalOperator op) throws JsonProcessingException{
+    return mapper.writeValueAsString(op);
+  }
+  
+  public PhysicalPlan readPhysicalPlan(String json) throws JsonProcessingException, IOException {
+    logger.debug("Reading physical plan {}", json);
+    return physicalPlanReader.readValue(json);
   }
 
-  public PhysicalPlan read(String json) throws JsonProcessingException, IOException {
-    return reader.readValue(json);
+  public FragmentRoot readFragmentOperator(String json) throws JsonProcessingException, IOException {
+    logger.debug("Attempting to read {}", json);
+    PhysicalOperator op = operatorReader.readValue(json);
+    if(op instanceof FragmentLeaf){
+      return (FragmentRoot) op;
+    }else{
+      throw new UnsupportedOperationException(String.format("The provided json fragment doesn't have a FragmentRoot as its root operator.  The operator was %s.", op.getClass().getCanonicalName()));
+    }
   }
 
+  public LogicalPlan readLogicalPlan(String json) throws JsonProcessingException, IOException{
+    logger.debug("Reading logical plan {}", json);
+    return logicalPlanReader.readValue(json);
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/ScanFinder.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/ScanFinder.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/ScanFinder.java
deleted file mode 100644
index ff81d90..0000000
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/ScanFinder.java
+++ /dev/null
@@ -1,54 +0,0 @@
-/*******************************************************************************
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- ******************************************************************************/
-package org.apache.drill.exec.planner;
-
-import org.apache.drill.common.physical.pop.base.AbstractPhysicalVisitor;
-import org.apache.drill.common.physical.pop.base.Exchange;
-import org.apache.drill.common.physical.pop.base.PhysicalOperator;
-import org.apache.drill.common.physical.pop.base.Store;
-
-public class ScanFinder extends AbstractPhysicalVisitor<Boolean, Void, RuntimeException> {
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ScanFinder.class);
-
-  private static final ScanFinder finder = new ScanFinder();
-  
-  private ScanFinder(){}
-  
-  @Override
-  public Boolean visitExchange(Exchange exchange, Void value) throws RuntimeException {
-    return false;
-  }
-
-  @Override
-  public Boolean visitStore(Store store, Void value) throws RuntimeException {
-    return true;
-  }
-
-  @Override
-  public Boolean visitUnknown(PhysicalOperator op, Void value) throws RuntimeException {
-    for(PhysicalOperator child : op){
-      if(child.accept(this,  null)) return true;
-    }
-    return false;
-  }
-  
-  public static boolean containsScan(PhysicalOperator op){
-    return op.accept(finder, null);
-  }
-  
-}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/SimpleExecPlanner.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/SimpleExecPlanner.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/SimpleExecPlanner.java
index d1c3add..824b2e2 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/SimpleExecPlanner.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/SimpleExecPlanner.java
@@ -17,14 +17,16 @@
  ******************************************************************************/
 package org.apache.drill.exec.planner;
 
-import java.util.List;
-
-import org.apache.drill.common.physical.PhysicalPlan;
-import org.apache.drill.common.physical.pop.base.PhysicalOperator;
-import org.apache.drill.exec.exception.FragmentSetupException;
-import org.apache.drill.exec.foreman.QueryWorkUnit;
+import org.apache.drill.common.exceptions.ExecutionSetupException;
 import org.apache.drill.exec.ops.QueryContext;
-import org.apache.drill.exec.proto.ExecProtos.PlanFragment;
+import org.apache.drill.exec.physical.PhysicalPlan;
+import org.apache.drill.exec.physical.base.PhysicalOperator;
+import org.apache.drill.exec.planner.fragment.Fragment;
+import org.apache.drill.exec.planner.fragment.MakeFragmentsVisitor;
+import org.apache.drill.exec.planner.fragment.PlanningSet;
+import org.apache.drill.exec.planner.fragment.SimpleParallelizer;
+import org.apache.drill.exec.planner.fragment.StatsCollector;
+import org.apache.drill.exec.work.QueryWorkUnit;
 
 /**
  * Parallelization is based on available nodes with source or target data.  Nodes that are "overloaded" are excluded from execution.
@@ -32,22 +34,20 @@ import org.apache.drill.exec.proto.ExecProtos.PlanFragment;
 public class SimpleExecPlanner implements ExecPlanner{
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(SimpleExecPlanner.class);
   
-  private FragmentingPhysicalVisitor fragmenter = new FragmentingPhysicalVisitor();
+  private MakeFragmentsVisitor fragmenter = new MakeFragmentsVisitor();
   private SimpleParallelizer parallelizer = new SimpleParallelizer();
 
   @Override
-  public QueryWorkUnit getWorkUnit(QueryContext context, PhysicalPlan plan, int maxWidth) throws FragmentSetupException {
+  public QueryWorkUnit getWorkUnit(QueryContext context, PhysicalPlan plan, int maxWidth) throws ExecutionSetupException {
     
     // get the root physical operator and split the plan into sub fragments.
     PhysicalOperator root = plan.getSortedOperators(false).iterator().next();
-    FragmentNode fragmentRoot = root.accept(fragmenter, null);
+    Fragment fragmentRoot = root.accept(fragmenter, null);
     
     // generate a planning set and collect stats.
-    FragmentPlanningSet planningSet = new FragmentPlanningSet(context);
-    FragmentStatsCollector statsCollector = new FragmentStatsCollector(planningSet);
-    statsCollector.collectStats(fragmentRoot);
+    PlanningSet planningSet = StatsCollector.collectStats(fragmentRoot);
     
-    return parallelizer.getFragments(context, fragmentRoot, planningSet, maxWidth);
+    return parallelizer.getFragments(context.getCurrentEndpoint(), context.getQueryId(), context.getActiveEndpoints(), context.getPlanReader(), fragmentRoot, planningSet, maxWidth);
     
     
   }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/SimpleParallelizer.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/SimpleParallelizer.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/SimpleParallelizer.java
deleted file mode 100644
index a52abaa..0000000
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/SimpleParallelizer.java
+++ /dev/null
@@ -1,147 +0,0 @@
-/*******************************************************************************
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- ******************************************************************************/
-package org.apache.drill.exec.planner;
-
-import java.util.Collection;
-import java.util.List;
-
-import org.apache.drill.common.physical.pop.base.PhysicalOperator;
-import org.apache.drill.common.proto.CoordinationProtos.DrillbitEndpoint;
-import org.apache.drill.exec.exception.FragmentSetupException;
-import org.apache.drill.exec.foreman.QueryWorkUnit;
-import org.apache.drill.exec.ops.QueryContext;
-import org.apache.drill.exec.planner.FragmentMaterializer.IndexedFragmentNode;
-import org.apache.drill.exec.proto.ExecProtos.PlanFragment;
-
-import com.fasterxml.jackson.core.JsonProcessingException;
-import com.fasterxml.jackson.databind.ObjectMapper;
-import com.google.common.collect.Lists;
-
-public class SimpleParallelizer {
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(SimpleParallelizer.class);
-
-  private final FragmentMaterializer materializer = new FragmentMaterializer();
-
-  /**
-   * Generate a set of assigned fragments based on the provided planningSet. Do not allow parallelization stages to go
-   * beyond the global max width.
-   * 
-   * @param context
-   *          The current QueryContext.
-   * @param planningSet
-   *          The set of queries with collected statistics that we'll work with.
-   * @param globalMaxWidth
-   *          The maximum level or paralellization any stage of the query can do. Note that while this might be the
-   *          number of active Drillbits, realistically, this could be well beyond that number of we want to do things
-   *          like speed results return.
-   * @return The list of generatoe PlanFragment protobuf objects to be assigned out to the individual nodes.
-   * @throws FragmentSetupException
-   */
-  public QueryWorkUnit getFragments(QueryContext context, FragmentNode rootNode, FragmentPlanningSet planningSet,
-      int globalMaxWidth) throws FragmentSetupException {
-    assignEndpoints(context.getActiveEndpoints(), planningSet, globalMaxWidth);
-    return generateWorkUnit(context.getQueryId(), context.getMapper(), rootNode, planningSet);
-  }
-
-  private QueryWorkUnit generateWorkUnit(long queryId, ObjectMapper mapper, FragmentNode rootNode,
-      FragmentPlanningSet planningSet) throws FragmentSetupException {
-
-    List<PlanFragment> fragments = Lists.newArrayList();
-
-    PlanFragment rootFragment = null;
-
-    // now we generate all the individual plan fragments and associated assignments. Note, we need all endpoints
-    // assigned before we can materialize, so we start a new loop here rather than utilizing the previous one.
-    for (FragmentWrapper info : planningSet) {
-
-      FragmentNode node = info.getNode();
-      FragmentStats stats = node.getStats();
-      PhysicalOperator abstractRoot = node.getRoot();
-      boolean isRootNode = rootNode == node;
-
-      if (isRootNode && info.getWidth() != 1)
-        throw new FragmentSetupException(
-            String
-                .format(
-                    "Failure while trying to setup fragment.  The root fragment must always have parallelization one.  In the current case, the width was set to %d.",
-                    info.getWidth()));
-      // a fragment is self driven if it doesn't rely on any other exchanges.
-      boolean selfDriven = node.getReceivingExchangePairs().size() == 0;
-
-      // Create a minorFragment for each major fragment.
-      for (int minorFragmentId = 0; minorFragmentId < info.getWidth(); minorFragmentId++) {
-        IndexedFragmentNode iNode = new IndexedFragmentNode(minorFragmentId, info);
-        PhysicalOperator root = abstractRoot.accept(materializer, iNode);
-
-        // get plan as JSON
-        String plan;
-        try {
-          plan = mapper.writeValueAsString(root);
-        } catch (JsonProcessingException e) {
-          throw new FragmentSetupException("Failure while trying to convert fragment into json.", e);
-        }
-
-        PlanFragment fragment = PlanFragment.newBuilder() //
-            .setCpuCost(stats.getCpuCost()) //
-            .setDiskCost(stats.getDiskCost()) //
-            .setMemoryCost(stats.getMemoryCost()) //
-            .setNetworkCost(stats.getNetworkCost()) //
-            .setFragmentJson(plan) //
-            .setMinorFragmentId(minorFragmentId) //
-            .setMajorFragmentId(info.getMajorFragmentId()).setQueryId(queryId) //
-            .setAssignment(info.getAssignedEndpoint(minorFragmentId)).setSelfDriven(selfDriven).build();
-
-        if (isRootNode) {
-          rootFragment = fragment;
-        } else {
-          fragments.add(fragment);
-        }
-      }
-    }
-
-    return new QueryWorkUnit(rootFragment, fragments);
-
-  }
-
-  private void assignEndpoints(Collection<DrillbitEndpoint> allNodes, FragmentPlanningSet planningSet,
-      int globalMaxWidth) {
-    // First we determine the amount of parallelization for a fragment. This will be between 1 and maxWidth based on
-    // cost. (Later could also be based on cluster operation.) then we decide endpoints based on affinity (later this
-    // could be based on endpoint load)
-    for (FragmentWrapper info : planningSet) {
-
-      FragmentStats stats = info.getStats();
-
-      // figure out width.
-      int width = Math.min(stats.getMaxWidth(), globalMaxWidth);
-      float diskCost = stats.getDiskCost();
-
-      // TODO: right now we'll just assume that each task is cost 1 so we'll set the breadth at the lesser of the number
-      // of tasks or the maximum width of the fragment.
-      if (diskCost < width) {
-        width = (int) diskCost;
-      }
-
-      if (width < 1) width = 1;
-      info.setWidth(width);
-
-      // figure out endpoint assignments. also informs the exchanges about their respective endpoints.
-      info.assignEndpoints(allNodes);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/Fragment.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/Fragment.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/Fragment.java
new file mode 100644
index 0000000..3284086
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/Fragment.java
@@ -0,0 +1,150 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.planner.fragment;
+
+import java.util.Iterator;
+import java.util.List;
+
+import org.apache.drill.exec.exception.FragmentSetupException;
+import org.apache.drill.exec.physical.base.Exchange;
+import org.apache.drill.exec.physical.base.PhysicalOperator;
+import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
+
+import com.google.common.collect.Lists;
+
+public class Fragment implements Iterable<Fragment.ExchangeFragmentPair>{
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(Fragment.class);
+  
+  private PhysicalOperator root;
+  private Exchange sendingExchange;
+  private final List<ExchangeFragmentPair> receivingExchangePairs = Lists.newLinkedList();
+  private Stats stats = new Stats();
+  
+  public void addOperator(PhysicalOperator o){
+    if(root == null){
+      root = o;
+    }
+  }
+  
+  public void addSendExchange(Exchange e) throws FragmentSetupException{
+    if(sendingExchange != null) throw new FragmentSetupException("Fragment was trying to add a second SendExchange.  ");
+    addOperator(e);
+    sendingExchange = e;
+  }
+  
+  public void addReceiveExchange(Exchange e, Fragment fragment){
+    this.receivingExchangePairs.add(new ExchangeFragmentPair(e, fragment));
+  }
+
+  @Override
+  public Iterator<ExchangeFragmentPair> iterator() {
+    return this.receivingExchangePairs.iterator();
+  }
+
+  public List<ExchangeFragmentPair> getReceivingExchangePairs() {
+    return receivingExchangePairs;
+  }
+
+  public PhysicalOperator getRoot() {
+    return root;
+  }
+
+  public Exchange getSendingExchange() {
+    return sendingExchange;
+  }
+
+//  public <T, V> T accept(FragmentVisitor<T, V> visitor, V extra){
+//    return visitor.visit(this, extra);
+//  }
+  
+  public Stats getStats(){
+    return stats;
+  }
+  
+  public class ExchangeFragmentPair {
+    private Exchange exchange;
+    private Fragment node;
+    public ExchangeFragmentPair(Exchange exchange, Fragment node) {
+      super();
+      this.exchange = exchange;
+      this.node = node;
+    }
+    public Exchange getExchange() {
+      return exchange;
+    }
+    public Fragment getNode() {
+      return node;
+    }
+    @Override
+    public int hashCode() {
+      final int prime = 31;
+      int result = 1;
+      result = prime * result + ((exchange == null) ? 0 : exchange.hashCode());
+      result = prime * result + ((node == null) ? 0 : node.hashCode());
+      return result;
+    }
+    @Override
+    public String toString() {
+      return "ExchangeFragmentPair [exchange=" + exchange + "]";
+    }
+    
+    
+    
+  }
+
+  @Override
+  public int hashCode() {
+    final int prime = 31;
+    int result = 1;
+    result = prime * result + ((receivingExchangePairs == null) ? 0 : receivingExchangePairs.hashCode());
+    result = prime * result + ((root == null) ? 0 : root.hashCode());
+    result = prime * result + ((sendingExchange == null) ? 0 : sendingExchange.hashCode());
+    result = prime * result + ((stats == null) ? 0 : stats.hashCode());
+    return result;
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (this == obj) return true;
+    if (obj == null) return false;
+    if (getClass() != obj.getClass()) return false;
+    Fragment other = (Fragment) obj;
+    if (receivingExchangePairs == null) {
+      if (other.receivingExchangePairs != null) return false;
+    } else if (!receivingExchangePairs.equals(other.receivingExchangePairs)) return false;
+    if (root == null) {
+      if (other.root != null) return false;
+    } else if (!root.equals(other.root)) return false;
+    if (sendingExchange == null) {
+      if (other.sendingExchange != null) return false;
+    } else if (!sendingExchange.equals(other.sendingExchange)) return false;
+    if (stats == null) {
+      if (other.stats != null) return false;
+    } else if (!stats.equals(other.stats)) return false;
+    return true;
+  }
+
+  @Override
+  public String toString() {
+    return "FragmentNode [root=" + root + ", sendingExchange=" + sendingExchange + ", receivingExchangePairs="
+        + receivingExchangePairs + ", stats=" + stats + "]";
+  }
+
+  
+ 
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/FragmentVisitor.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/FragmentVisitor.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/FragmentVisitor.java
new file mode 100644
index 0000000..0cba2ec
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/FragmentVisitor.java
@@ -0,0 +1,23 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.planner.fragment;
+
+
+public interface FragmentVisitor<T, V> {
+  public T visit(Fragment n, V extra);
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/MakeFragmentsVisitor.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/MakeFragmentsVisitor.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/MakeFragmentsVisitor.java
new file mode 100644
index 0000000..4188435
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/MakeFragmentsVisitor.java
@@ -0,0 +1,69 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.planner.fragment;
+
+import org.apache.drill.exec.exception.FragmentSetupException;
+import org.apache.drill.exec.physical.base.AbstractPhysicalVisitor;
+import org.apache.drill.exec.physical.base.Exchange;
+import org.apache.drill.exec.physical.base.PhysicalOperator;
+
+/**
+ * Responsible for breaking a plan into its constituent Fragments.
+ */
+public class MakeFragmentsVisitor extends AbstractPhysicalVisitor<Fragment, Fragment, FragmentSetupException> {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(MakeFragmentsVisitor.class);
+
+  
+  public MakeFragmentsVisitor(){
+  }
+  
+  @Override
+  public Fragment visitExchange(Exchange exchange, Fragment value) throws FragmentSetupException {
+//    logger.debug("Visiting Exchange {}", exchange);
+    if(value == null) throw new FragmentSetupException("The simple fragmenter was called without a FragmentBuilder value.  This will only happen if the initial call to SimpleFragmenter is by a Exchange node.  This should never happen since an Exchange node should never be the root node of a plan.");
+    Fragment next = getNextBuilder();
+    value.addReceiveExchange(exchange, next);
+    next.addSendExchange(exchange);
+    exchange.getChild().accept(this, next);
+    return value;
+  }
+  
+  @Override
+  public Fragment visitOp(PhysicalOperator op, Fragment value)  throws FragmentSetupException{
+//    logger.debug("Visiting Other {}", op);
+    value = ensureBuilder(value);
+    value.addOperator(op);
+    for(PhysicalOperator child : op){
+      child.accept(this, value);
+    }
+    return value;
+  }
+  
+  private Fragment ensureBuilder(Fragment value) throws FragmentSetupException{
+    if(value != null){
+      return value;
+    }else{
+      return getNextBuilder();
+    }
+  }
+  
+  public Fragment getNextBuilder(){
+    return new Fragment();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/MaterializedFragment.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/MaterializedFragment.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/MaterializedFragment.java
new file mode 100644
index 0000000..e292cd2
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/MaterializedFragment.java
@@ -0,0 +1,69 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.planner.fragment;
+
+import org.apache.drill.exec.physical.OperatorCost;
+import org.apache.drill.exec.physical.base.PhysicalOperator;
+import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
+
+public class MaterializedFragment {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(MaterializedFragment.class);
+
+  final PhysicalOperator root;
+  final DrillbitEndpoint endpoint;
+  final long queryId;
+  final int majorFragmentId;
+  final int minorFragmentId;
+  final OperatorCost cost;
+
+  public MaterializedFragment(PhysicalOperator root, DrillbitEndpoint endpoint, long queryId, int majorFragmentId,
+      int minorFragmentId, OperatorCost cost) {
+    super();
+    this.root = root;
+    this.endpoint = endpoint;
+    this.queryId = queryId;
+    this.majorFragmentId = majorFragmentId;
+    this.minorFragmentId = minorFragmentId;
+    this.cost = cost;
+  }
+
+  public PhysicalOperator getRoot() {
+    return root;
+  }
+
+  public DrillbitEndpoint getEndpoint() {
+    return endpoint;
+  }
+
+  public long getQueryId() {
+    return queryId;
+  }
+
+  public int getMajorFragmentId() {
+    return majorFragmentId;
+  }
+
+  public int getMinorFragmentId() {
+    return minorFragmentId;
+  }
+
+  public OperatorCost getCost() {
+    return cost;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/Materializer.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/Materializer.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/Materializer.java
new file mode 100644
index 0000000..9fee586
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/Materializer.java
@@ -0,0 +1,107 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.planner.fragment;
+
+import java.util.List;
+
+import org.apache.drill.common.exceptions.ExecutionSetupException;
+import org.apache.drill.common.exceptions.PhysicalOperatorSetupException;
+import org.apache.drill.exec.exception.FragmentSetupException;
+import org.apache.drill.exec.physical.base.AbstractPhysicalVisitor;
+import org.apache.drill.exec.physical.base.Exchange;
+import org.apache.drill.exec.physical.base.PhysicalOperator;
+import org.apache.drill.exec.physical.base.Scan;
+import org.apache.drill.exec.physical.base.Store;
+
+import com.google.common.collect.Lists;
+
+public class Materializer extends AbstractPhysicalVisitor<PhysicalOperator, Materializer.IndexedFragmentNode, ExecutionSetupException>{
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(Materializer.class);
+
+  
+  @Override
+  public PhysicalOperator visitExchange(Exchange exchange, IndexedFragmentNode iNode) throws ExecutionSetupException {
+    if(exchange == iNode.getNode().getSendingExchange()){
+      
+      // this is a sending exchange.
+      PhysicalOperator child = exchange.getChild().accept(this, iNode);
+      PhysicalOperator materializedSender = exchange.getSender(iNode.getMinorFragmentId(), child);
+      logger.debug("Visit sending exchange, materialized {} with child {}.", materializedSender, child);
+      return materializedSender;
+      
+    }else{
+      // receiving exchange.
+      PhysicalOperator materializedReceiver = exchange.getReceiver(iNode.getMinorFragmentId());
+      logger.debug("Visit receiving exchange, materialized receiver: {}.", materializedReceiver);
+      return materializedReceiver;
+    }
+  }
+
+  @Override
+  public PhysicalOperator visitScan(Scan<?> scan, IndexedFragmentNode iNode) throws ExecutionSetupException {
+    return scan.getSpecificScan(iNode.getMinorFragmentId());
+  }
+
+  @Override
+  public PhysicalOperator visitStore(Store store, IndexedFragmentNode iNode) throws ExecutionSetupException {
+    PhysicalOperator child = store.getChild().accept(this, iNode);
+    
+    try {
+      PhysicalOperator o = store.getSpecificStore(child, iNode.getMinorFragmentId());
+      logger.debug("New materialized store node {} with child {}", o, child);
+      return o;
+    } catch (PhysicalOperatorSetupException e) {
+      throw new FragmentSetupException("Failure while generating a specific Store materialization.");
+    }
+  }
+
+  @Override
+  public PhysicalOperator visitOp(PhysicalOperator op, IndexedFragmentNode iNode) throws ExecutionSetupException {
+    logger.debug("Visiting catch all: {}", op);
+    List<PhysicalOperator> children = Lists.newArrayList();
+    for(PhysicalOperator child : op){
+      children.add(child.accept(this, iNode));
+    }
+    return op.getNewWithChildren(children);
+  }
+  
+  public static class IndexedFragmentNode{
+    final Wrapper info;
+    final int minorFragmentId;
+    
+    public IndexedFragmentNode(int minorFragmentId, Wrapper info) {
+      super();
+      this.info = info;
+      this.minorFragmentId = minorFragmentId;
+    }
+
+    public Fragment getNode() {
+      return info.getNode();
+    }
+
+    public int getMinorFragmentId() {
+      return minorFragmentId;
+    }
+
+    public Wrapper getInfo() {
+      return info;
+    }
+    
+  }
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/PlanningSet.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/PlanningSet.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/PlanningSet.java
new file mode 100644
index 0000000..fb04255
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/PlanningSet.java
@@ -0,0 +1,66 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.planner.fragment;
+
+import java.util.Iterator;
+import java.util.Map;
+
+import org.apache.drill.exec.ops.QueryContext;
+import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
+
+import com.google.common.collect.Maps;
+
+public class PlanningSet implements Iterable<Wrapper>{
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(PlanningSet.class);
+  
+  private Map<Fragment, Wrapper> fragmentMap = Maps.newHashMap();
+  private int majorFragmentIdIndex = 0;
+  
+  PlanningSet(){
+  }
+
+  public void addAffinity(Fragment n, DrillbitEndpoint endpoint, float affinity){
+    get(n).addEndpointAffinity(endpoint, affinity);
+  }
+  
+  public void setWidth(Fragment n, int width){
+    get(n).setWidth(width);
+  }
+  
+  Wrapper get(Fragment node){
+    Wrapper wrapper = fragmentMap.get(node);
+    if(wrapper == null){
+      wrapper = new Wrapper(node, majorFragmentIdIndex++);
+      fragmentMap.put(node,  wrapper);
+    }
+    return wrapper;
+  }
+
+  @Override
+  public Iterator<Wrapper> iterator() {
+    return this.fragmentMap.values().iterator();
+  }
+
+  @Override
+  public String toString() {
+    return "FragmentPlanningSet:\n" + fragmentMap.values() + "]";
+  }
+  
+  
+  
+}


[06/53] [abbrv] git commit: WIP fragmentation, physical plan, byte compiling, some vector work

Posted by ja...@apache.org.
WIP fragmentation, physical plan, byte compiling, some vector work


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

Branch: refs/heads/master
Commit: f0be80dcdaf22e2bb0a428b202c43d03ed063eb6
Parents: 5ede21f
Author: Jacques Nadeau <ja...@apache.org>
Authored: Mon Apr 22 23:04:58 2013 -0700
Committer: Jacques Nadeau <ja...@apache.org>
Committed: Tue Apr 30 15:01:38 2013 -0700

----------------------------------------------------------------------
 sandbox/prototype/common/pom.xml                |   5 +-
 .../org/apache/drill/common/PlanProperties.java |   4 +
 .../apache/drill/common/config/DrillConfig.java |   6 +-
 .../apache/drill/common/defs/PartitionDef.java  |  14 +-
 .../drill/common/expression/types/DataType.java |   4 +
 .../drill/common/physical/EndpointAffinity.java |  60 +++++
 .../drill/common/physical/OperatorCost.java     |  61 +++++
 .../apache/drill/common/physical/POPConfig.java |  24 --
 .../apache/drill/common/physical/POPCost.java   |  34 ---
 .../drill/common/physical/PhysicalPlan.java     |  37 +--
 .../apache/drill/common/physical/SetSpec.java   |  36 ---
 .../apache/drill/common/physical/StitchDef.java |  48 ----
 .../drill/common/physical/pop/ExchangePOP.java  |  54 ----
 .../common/physical/pop/FieldCombinePOP.java    |  28 ---
 .../common/physical/pop/FieldSubdividePOP.java  |  22 --
 .../drill/common/physical/pop/Filter.java       |  52 ++++
 .../drill/common/physical/pop/POPBase.java      |  65 -----
 .../physical/pop/PartitionToRandomExchange.java |  92 +++++++
 .../common/physical/pop/PhysicalOperator.java   |  35 ---
 .../drill/common/physical/pop/Project.java      |  53 ++++
 .../drill/common/physical/pop/ProjectPOP.java   |  53 ----
 .../common/physical/pop/QuickNWaySortPOP.java   |  50 ----
 .../drill/common/physical/pop/ScanPOP.java      |  55 -----
 .../drill/common/physical/pop/Screen.java       |  77 ++++++
 .../common/physical/pop/SingleChildPOP.java     |  41 ---
 .../drill/common/physical/pop/SinkPOP.java      |  22 --
 .../apache/drill/common/physical/pop/Sort.java  |  57 +++++
 .../drill/common/physical/pop/SortPOP.java      |  54 ----
 .../drill/common/physical/pop/SourcePOP.java    |  22 --
 .../drill/common/physical/pop/StorePOP.java     |  61 -----
 .../common/physical/pop/base/AbstractBase.java  |  56 +++++
 .../physical/pop/base/AbstractExchange.java     |  68 +++++
 .../pop/base/AbstractPhysicalVisitor.java       |  80 ++++++
 .../physical/pop/base/AbstractReceiver.java     |  32 +++
 .../common/physical/pop/base/AbstractScan.java  |  62 +++++
 .../physical/pop/base/AbstractSender.java       |  29 +++
 .../physical/pop/base/AbstractSingle.java       |  48 ++++
 .../common/physical/pop/base/AbstractStore.java |  42 ++++
 .../common/physical/pop/base/Exchange.java      |  69 ++++++
 .../common/physical/pop/base/ExchangeCost.java  |  55 +++++
 .../common/physical/pop/base/FragmentLeaf.java  |  25 ++
 .../common/physical/pop/base/FragmentRoot.java  |  25 ++
 .../common/physical/pop/base/HasAffinity.java   |  26 ++
 .../drill/common/physical/pop/base/Leaf.java    |  21 ++
 .../physical/pop/base/PhysicalOperator.java     |  59 +++++
 .../physical/pop/base/PhysicalOperatorUtil.java |  34 +++
 .../physical/pop/base/PhysicalVisitor.java      |  43 ++++
 .../common/physical/pop/base/Receiver.java      |  38 +++
 .../drill/common/physical/pop/base/Root.java    |  24 ++
 .../drill/common/physical/pop/base/Scan.java    |  36 +++
 .../drill/common/physical/pop/base/Sender.java  |  29 +++
 .../drill/common/physical/pop/base/Store.java   |  30 +++
 .../drill/common/physical/props/OrderProp.java  |  45 ----
 .../common/physical/props/PartitionProp.java    |  36 ---
 .../common/physical/props/PhysicalProp.java     |  24 --
 .../common/physical/props/SegmentProp.java      |  42 ----
 .../common/src/main/protobuf/Coordination.proto |  26 ++
 .../drill/common/physical/MockScanPOP.java      |  30 ++-
 .../drill/common/physical/MockStorePOP.java     |  46 ++--
 .../common/physical/ParsePhysicalPlan.java      |   8 +-
 .../apache/drill/common/physical/ParsePlan.java |  36 ---
 .../common/src/test/resources/drill-module.conf |   2 +-
 .../src/test/resources/dsort-physical.json      |  66 -----
 .../src/test/resources/physical_test1.json      |  33 +++
 sandbox/prototype/exec/java-exec/pom.xml        |  35 ++-
 .../org/apache/drill/exec/BufferAllocator.java  |  52 ----
 .../java/org/apache/drill/exec/ByteReorder.java |  54 ++++
 .../drill/exec/DirectBufferAllocator.java       |  47 ----
 .../org/apache/drill/exec/ExecConstants.java    |   1 +
 .../drill/exec/cache/DistributedCache.java      |   8 +-
 .../org/apache/drill/exec/cache/HazelCache.java |  15 +-
 .../apache/drill/exec/client/DrillClient.java   |  20 +-
 .../drill/exec/compile/ClassBodyBuilder.java    | 247 +++++++++++++++++++
 .../drill/exec/compile/ClassCompiler.java       |  29 +++
 .../drill/exec/compile/ClassTransformer.java    | 210 ++++++++++++++++
 .../drill/exec/compile/JDKClassCompiler.java    | 177 +++++++++++++
 .../drill/exec/compile/JaninoClassCompiler.java |  62 +++++
 .../drill/exec/compile/QueryClassLoader.java    |  80 ++++++
 .../exec/compile/TemplateClassDefinition.java   |  58 +++++
 .../drill/exec/coord/ClusterCoordinator.java    |   4 +-
 .../exec/coord/DrillServiceInstanceHelper.java  |   4 +-
 .../drill/exec/coord/ZKClusterCoordinator.java  |  21 +-
 .../exception/ClassTransformationException.java |  47 ++++
 .../exec/exception/FragmentSetupException.java  |  42 ++++
 .../drill/exec/foreman/CancelableQuery.java     |  22 ++
 .../drill/exec/foreman/ExecutionPlanner.java    |  24 ++
 .../org/apache/drill/exec/foreman/Foreman.java  |  39 +++
 .../drill/exec/foreman/QueryWorkUnit.java       |  54 ++++
 .../drill/exec/foreman/ResourceRequest.java     |  30 +++
 .../drill/exec/foreman/StatusProvider.java      |  24 ++
 .../drill/exec/memory/BufferAllocator.java      |  58 +++++
 .../exec/memory/DirectBufferAllocator.java      |  58 +++++
 .../exec/metrics/SingleThreadNestedCounter.java |  55 +++++
 .../apache/drill/exec/ops/BatchIterator.java    |  32 ---
 .../ops/FilteringRecordBatchTransformer.java    |  58 +++++
 .../apache/drill/exec/ops/FragmentContext.java  |  47 +++-
 .../drill/exec/ops/FragmentConverter.java       |  30 +++
 .../org/apache/drill/exec/ops/FragmentRoot.java |  37 +++
 .../apache/drill/exec/ops/OperatorFactory.java  |  22 ++
 .../org/apache/drill/exec/ops/QueryContext.java |  51 ++++
 .../org/apache/drill/exec/ops/QueryOutcome.java |  22 --
 .../org/apache/drill/exec/ops/ScanBatch.java    |   2 +-
 .../drill/exec/ops/StreamingRecordBatch.java    |  25 ++
 .../exec/ops/exchange/ExchangeRecordBatch.java  |  22 ++
 .../exec/ops/exchange/PartitioningSender.java   |  23 --
 .../drill/exec/ops/exchange/RandomReceiver.java |  24 --
 .../exec/ops/filter/FilterRecordBatch.java      | 109 ++++++++
 .../exec/ops/filter/SelectionVectorUpdater.java |  80 ++++++
 .../apache/drill/exec/planner/ExecPlanner.java  |   9 +-
 .../exec/planner/FragmentMaterializer.java      |  86 +++++++
 .../apache/drill/exec/planner/FragmentNode.java | 138 +++++++++++
 .../drill/exec/planner/FragmentPlanningSet.java |  61 +++++
 .../drill/exec/planner/FragmentRunnable.java    | 124 ++++++++++
 .../drill/exec/planner/FragmentScheduler.java   |  32 +++
 .../drill/exec/planner/FragmentStats.java       |  63 +++++
 .../exec/planner/FragmentStatsCollector.java    | 109 ++++++++
 .../drill/exec/planner/FragmentVisitor.java     |  22 ++
 .../drill/exec/planner/FragmentWrapper.java     | 127 ++++++++++
 .../planner/FragmentingPhysicalVisitor.java     |  71 ++++++
 .../exec/planner/MaterializedFragment.java      |  69 ++++++
 .../drill/exec/planner/PhysicalPlanReader.java  |  47 ++++
 .../apache/drill/exec/planner/ScanFinder.java   |  54 ++++
 .../drill/exec/planner/SimpleExecPlanner.java   |  54 ++++
 .../drill/exec/planner/SimpleParallelizer.java  | 147 +++++++++++
 .../exec/pop/receiver/NWayOrderingReceiver.java |  52 ++++
 .../drill/exec/pop/receiver/RandomReceiver.java |  55 +++++
 .../exec/pop/sender/HashPartitionSender.java    |  49 ++++
 .../apache/drill/exec/record/BatchSchema.java   |  29 ++-
 .../drill/exec/record/MaterializedField.java    |   4 +
 .../record/vector/AbstractFixedValueVector.java |   2 +-
 .../exec/record/vector/BaseValueVector.java     |   6 +-
 .../drill/exec/record/vector/BitVector.java     | 104 +++++---
 .../drill/exec/record/vector/ByteVector.java    |   2 +-
 .../drill/exec/record/vector/Int16Vector.java   |  52 ++++
 .../drill/exec/record/vector/Int32Vector.java   |   4 +-
 .../exec/record/vector/NullableInt32Vector.java |  47 ++++
 .../exec/record/vector/NullableValueVector.java |  18 +-
 .../exec/record/vector/SelectionVector.java     |  31 +++
 .../drill/exec/record/vector/UInt16Vector.java  |  51 ++++
 .../drill/exec/record/vector/ValueVector.java   |   1 +
 .../exec/record/vector/VariableVector.java      |   2 +-
 .../org/apache/drill/exec/rpc/bit/BitCom.java   |  26 +-
 .../drill/exec/rpc/bit/BitComHandler.java       |   2 +-
 .../apache/drill/exec/rpc/bit/BitComImpl.java   |  24 +-
 .../apache/drill/exec/rpc/bit/SendProgress.java |  22 ++
 .../apache/drill/exec/rpc/user/UserClient.java  |   4 +
 .../apache/drill/exec/rpc/user/UserServer.java  |   2 +-
 .../org/apache/drill/exec/server/Drillbit.java  |  11 +-
 .../drill/exec/server/DrillbitContext.java      |  27 +-
 .../drill/exec/store/AbstractStorageEngine.java |  83 +++++++
 .../apache/drill/exec/store/StorageEngine.java  |  32 ++-
 .../drill/exec/store/StorageEngineRegistry.java |   4 +-
 .../src/main/protobuf/Coordination.proto        |  32 ---
 .../src/main/protobuf/ExecutionProtos.proto     |  72 ++++--
 .../java-exec/src/main/protobuf/SchemaDef.proto |  13 +-
 .../java-exec/src/test/java/BBOutputStream.java |  38 ---
 .../src/test/java/CompressingBytesColumn.java   |  46 ----
 .../java-exec/src/test/java/ExternalSort.java   |  21 --
 .../src/test/java/GenerateExternalSortData.java | 124 ----------
 .../exec/compile/ExampleExternalInterface.java  |  23 ++
 .../exec/compile/ExampleInternalInterface.java  |  24 ++
 .../drill/exec/compile/ExampleTemplate.java     |  30 +++
 .../exec/compile/TestClassCompilationTypes.java |  67 +++++
 .../exec/compile/TestClassTransformation.java   |  53 ++++
 .../apache/drill/exec/pop/CheckFragmenter.java  |  86 +++++++
 .../drill/exec/pop/CheckInjectionValue.java     |  61 +++++
 .../drill/exec/rpc/user/RunRemoteQuery.java     |  41 +++
 .../apache/drill/exec/server/StartDrillbit.java |   4 +-
 .../drill/exec/store/MockRecordConfig.java      |  46 ++++
 .../drill/exec/store/MockRecordReader.java      | 108 ++++++++
 .../drill/exec/store/MockStorageEngine.java     |  54 ++++
 .../src/test/resources/drill-module.conf        |   7 +-
 .../src/test/resources/physical_screen.json     |  25 ++
 .../test/resources/physical_simpleexchange.json |  41 +++
 .../prototype/exec/java-exec/src/test/sh/runbit |   2 +-
 sandbox/prototype/pom.xml                       |  16 +-
 176 files changed, 6097 insertions(+), 1608 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/common/pom.xml
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/pom.xml b/sandbox/prototype/common/pom.xml
index 9702dbe..3a15c05 100644
--- a/sandbox/prototype/common/pom.xml
+++ b/sandbox/prototype/common/pom.xml
@@ -8,7 +8,7 @@
 		<groupId>org.apache.drill</groupId>
 		<version>1.0-SNAPSHOT</version>
 	</parent>
-	
+
 	<artifactId>common</artifactId>
 	<packaging>jar</packaging>
 	<name>common</name>
@@ -18,7 +18,7 @@
 		<dependency>
 			<groupId>com.google.protobuf</groupId>
 			<artifactId>protobuf-java</artifactId>
-			<version>2.4.1</version>
+			<version>2.5.0</version>
 		</dependency>
 
 		<dependency>
@@ -67,6 +67,7 @@
 		</dependency>
 
 
+
 	</dependencies>
 
 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/common/src/main/java/org/apache/drill/common/PlanProperties.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/PlanProperties.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/PlanProperties.java
index 57d367a..c532e18 100644
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/PlanProperties.java
+++ b/sandbox/prototype/common/src/main/java/org/apache/drill/common/PlanProperties.java
@@ -17,6 +17,9 @@
  ******************************************************************************/
 package org.apache.drill.common;
 
+import com.fasterxml.jackson.annotation.JsonInclude;
+import com.fasterxml.jackson.annotation.JsonInclude.Include;
+
 
 public class PlanProperties {
   public static enum PlanType {APACHE_DRILL_LOGICAL, APACHE_DRILL_PHYSICAL}
@@ -25,6 +28,7 @@ public class PlanProperties {
   public int version;
 	public Generator generator = new Generator();
 	
+	@JsonInclude(Include.NON_NULL)
 	public static class Generator{
 		public String type;
 		public String info;

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/common/src/main/java/org/apache/drill/common/config/DrillConfig.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/config/DrillConfig.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/config/DrillConfig.java
index 5750aaf..2b8f45d 100644
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/config/DrillConfig.java
+++ b/sandbox/prototype/common/src/main/java/org/apache/drill/common/config/DrillConfig.java
@@ -23,18 +23,18 @@ import java.util.List;
 import java.util.Queue;
 import java.util.concurrent.CopyOnWriteArrayList;
 
-import com.google.common.annotations.VisibleForTesting;
 import org.apache.drill.common.exceptions.DrillConfigurationException;
 import org.apache.drill.common.expression.LogicalExpression;
 import org.apache.drill.common.logical.StorageEngineConfigBase;
 import org.apache.drill.common.logical.data.LogicalOperatorBase;
-import org.apache.drill.common.physical.pop.POPBase;
+import org.apache.drill.common.physical.pop.base.PhysicalOperatorUtil;
 import org.apache.drill.common.util.PathScanner;
 
 import com.fasterxml.jackson.core.JsonParser.Feature;
 import com.fasterxml.jackson.databind.ObjectMapper;
 import com.fasterxml.jackson.databind.SerializationFeature;
 import com.fasterxml.jackson.databind.module.SimpleModule;
+import com.google.common.annotations.VisibleForTesting;
 import com.typesafe.config.Config;
 import com.typesafe.config.ConfigFactory;
 
@@ -56,7 +56,7 @@ public final class DrillConfig extends NestedConfig{
     mapper.configure(Feature.ALLOW_UNQUOTED_FIELD_NAMES, true);
     mapper.configure(Feature.ALLOW_COMMENTS, true);
     mapper.registerSubtypes(LogicalOperatorBase.getSubTypes(this));
-    mapper.registerSubtypes(POPBase.getSubTypes(this));
+    mapper.registerSubtypes(PhysicalOperatorUtil.getSubTypes(this));
     mapper.registerSubtypes(StorageEngineConfigBase.getSubTypes(this));
     
   };

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/common/src/main/java/org/apache/drill/common/defs/PartitionDef.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/defs/PartitionDef.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/defs/PartitionDef.java
index 181c327..45298df 100644
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/defs/PartitionDef.java
+++ b/sandbox/prototype/common/src/main/java/org/apache/drill/common/defs/PartitionDef.java
@@ -26,13 +26,13 @@ public class PartitionDef {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(PartitionDef.class);
 
   private final PartitionType partitionType;
-  private final LogicalExpression[] expressions;
+  private final LogicalExpression expr;
   private final LogicalExpression[] starts;
   
   @JsonCreator
-  public PartitionDef(@JsonProperty("mode") PartitionType partitionType, @JsonProperty("exprs") LogicalExpression[] expressions, @JsonProperty("starts") LogicalExpression[] starts) {
+  public PartitionDef(@JsonProperty("mode") PartitionType partitionType, @JsonProperty("expr") LogicalExpression expr, @JsonProperty("starts") LogicalExpression[] starts) {
     this.partitionType = partitionType;
-    this.expressions = expressions;
+    this.expr = expr;
     this.starts = starts;
   }
 
@@ -41,9 +41,9 @@ public class PartitionDef {
     return partitionType;
   }
 
-  @JsonProperty("exprs")
-  public LogicalExpression[] getExpressions() {
-    return expressions;
+  @JsonProperty("expr")
+  public LogicalExpression getExpr() {
+    return expr;
   }
 
   @JsonProperty("starts")
@@ -53,6 +53,6 @@ public class PartitionDef {
   
 
   public static enum PartitionType{ 
-    RANDOM, HASH, RANGE;
+    DUPLICATE, RANDOM, HASH, RANGE;
   };
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/types/DataType.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/types/DataType.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/types/DataType.java
index 60d26dc..25b82a7 100644
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/types/DataType.java
+++ b/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/types/DataType.java
@@ -51,6 +51,8 @@ public abstract class DataType {
   public abstract Comparability getComparability();
   public abstract boolean isNumericType();
   
+  
+  
   public static final DataType LATEBIND = new LateBindType();
   public static final DataType BOOLEAN = new AtomType("BOOLEAN", Comparability.EQUAL, false);
   public static final DataType BYTES = new AtomType("BYTES", Comparability.ORDERED, false);
@@ -61,6 +63,8 @@ public abstract class DataType {
   public static final DataType FLOAT64 = new AtomType("FLOAT64", Comparability.ORDERED, true);
   public static final DataType INT64 = new AtomType("INT64", Comparability.ORDERED, true);
   public static final DataType INT32 = new AtomType("INT32", Comparability.ORDERED, true);
+  public static final DataType INT16 = new AtomType("INT16", Comparability.ORDERED, true);
+  public static final DataType UINT16 = new AtomType("UINT16", Comparability.ORDERED, true);
 //  public static final DataType INT16 = new AtomType("int16", Comparability.ORDERED, true);
 //  public static final DataType BIG_INTEGER = new AtomType("bigint", Comparability.ORDERED, true);
 //  public static final DataType BIG_DECIMAL = new AtomType("bigdecimal", Comparability.ORDERED, true);

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/EndpointAffinity.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/EndpointAffinity.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/EndpointAffinity.java
new file mode 100644
index 0000000..9ccf430
--- /dev/null
+++ b/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/EndpointAffinity.java
@@ -0,0 +1,60 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.common.physical;
+
+import org.apache.drill.common.proto.CoordinationProtos.DrillbitEndpoint;
+
+
+public class EndpointAffinity implements Comparable<EndpointAffinity>{
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(EndpointAffinity.class);
+  
+  private DrillbitEndpoint endpoint;
+  private float affinity = 0.0f;
+  
+  public EndpointAffinity(DrillbitEndpoint endpoint) {
+    super();
+    this.endpoint = endpoint;
+  }
+  
+  public EndpointAffinity(DrillbitEndpoint endpoint, float affinity) {
+    super();
+    this.endpoint = endpoint;
+    this.affinity = affinity;
+  }
+
+  public DrillbitEndpoint getEndpoint() {
+    return endpoint;
+  }
+  public void setEndpoint(DrillbitEndpoint endpoint) {
+    this.endpoint = endpoint;
+  }
+  public float getAffinity() {
+    return affinity;
+  }
+  
+  @Override
+  public int compareTo(EndpointAffinity o) {
+    return Float.compare(affinity, o.affinity);
+  }
+  
+  public void addAffinity(float f){
+    affinity += f;
+  }
+  
+  
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/OperatorCost.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/OperatorCost.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/OperatorCost.java
new file mode 100644
index 0000000..fadfff0
--- /dev/null
+++ b/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/OperatorCost.java
@@ -0,0 +1,61 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.common.physical;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+
+public class OperatorCost {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(OperatorCost.class);
+  
+  private final float network; 
+  private final float disk;
+  private final float memory;
+  private final float cpu;
+  
+  @JsonCreator
+  public OperatorCost(@JsonProperty("network") float network, @JsonProperty("disk") float disk, @JsonProperty("memory") float memory, @JsonProperty("cpu") float cpu) {
+    super();
+    this.network = network;
+    this.disk = disk;
+    this.memory = memory;
+    this.cpu = cpu;
+  }
+
+  public float getNetwork() {
+    return network;
+  }
+
+  public float getDisk() {
+    return disk;
+  }
+
+  public float getMemory() {
+    return memory;
+  }
+
+  public float getCpu() {
+    return cpu;
+  }
+  
+  public static OperatorCost combine(OperatorCost c1, OperatorCost c2){
+    return new OperatorCost(c1.network + c2.network, c1.disk + c2.disk, c1.memory + c2.memory, c1.cpu + c2.cpu);
+  }
+
+  
+}

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

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/POPCost.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/POPCost.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/POPCost.java
deleted file mode 100644
index b2ee440..0000000
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/POPCost.java
+++ /dev/null
@@ -1,34 +0,0 @@
-/*******************************************************************************
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- ******************************************************************************/
-package org.apache.drill.common.physical;
-
-public class POPCost {
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(POPCost.class);
-  
-  long outputRecordCount;
-  long outputRecordSize;
-  
-  
-  public POPCost(long outputRecordCount, long outputRecordSize) {
-    super();
-    this.outputRecordCount = outputRecordCount;
-    this.outputRecordSize = outputRecordSize;
-  }
-
-  
-}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/PhysicalPlan.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/PhysicalPlan.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/PhysicalPlan.java
index b81ca42..e83dac7 100644
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/PhysicalPlan.java
+++ b/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/PhysicalPlan.java
@@ -19,22 +19,22 @@ package org.apache.drill.common.physical;
 
 import java.io.IOException;
 import java.util.List;
-import java.util.Map;
 
 import org.apache.drill.common.PlanProperties;
 import org.apache.drill.common.config.DrillConfig;
 import org.apache.drill.common.graph.Graph;
 import org.apache.drill.common.graph.GraphAlgos;
-import org.apache.drill.common.logical.StorageEngineConfig;
-import org.apache.drill.common.physical.pop.PhysicalOperator;
-import org.apache.drill.common.physical.pop.SinkPOP;
-import org.apache.drill.common.physical.pop.SourcePOP;
+import org.apache.drill.common.physical.pop.base.Leaf;
+import org.apache.drill.common.physical.pop.base.PhysicalOperator;
+import org.apache.drill.common.physical.pop.base.Root;
 
 import com.fasterxml.jackson.annotation.JsonCreator;
 import com.fasterxml.jackson.annotation.JsonProperty;
 import com.fasterxml.jackson.annotation.JsonPropertyOrder;
 import com.fasterxml.jackson.core.JsonProcessingException;
 import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.ObjectReader;
+import com.fasterxml.jackson.databind.ObjectWriter;
 import com.google.common.collect.Lists;
 
 @JsonPropertyOrder({ "head", "graph" })
@@ -42,19 +42,29 @@ public class PhysicalPlan {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(PhysicalPlan.class);
   
   PlanProperties properties;
-  Graph<PhysicalOperator, SinkPOP, SourcePOP> graph;
+  
+  Graph<PhysicalOperator, Root, Leaf> graph;
   
   @JsonCreator
   public PhysicalPlan(@JsonProperty("head") PlanProperties properties, @JsonProperty("graph") List<PhysicalOperator> operators){
     this.properties = properties;
-    this.graph = Graph.newGraph(operators, SinkPOP.class, SourcePOP.class);
+    this.graph = Graph.newGraph(operators, Root.class, Leaf.class);
   }
   
   @JsonProperty("graph")
   public List<PhysicalOperator> getSortedOperators(){
-    List<PhysicalOperator> list = GraphAlgos.TopoSorter.sort(graph);
     // reverse the list so that nested references are flattened rather than nested.
-    return Lists.reverse(list);
+    return getSortedOperators(true);
+  }
+  
+  public List<PhysicalOperator> getSortedOperators(boolean reverse){
+    List<PhysicalOperator> list = GraphAlgos.TopoSorter.sort(graph);
+    if(reverse){
+      return Lists.reverse(list);
+    }else{
+      return list;
+    }
+    
   }
 
 
@@ -64,10 +74,9 @@ public class PhysicalPlan {
   }
 
   /** Parses a physical plan. */
-  public static PhysicalPlan parse(DrillConfig config, String planString) {
-    ObjectMapper mapper = config.getMapper();
+  public static PhysicalPlan parse(ObjectReader reader, String planString) {
     try {
-      PhysicalPlan plan = mapper.readValue(planString, PhysicalPlan.class);
+      PhysicalPlan plan = reader.readValue(planString);
       return plan;
     } catch (IOException e) {
       throw new RuntimeException(e);
@@ -75,9 +84,9 @@ public class PhysicalPlan {
   }
 
   /** Converts a physical plan to a string. (Opposite of {@link #parse}.) */
-  public String unparse(DrillConfig config) {
+  public String unparse(ObjectWriter writer) {
     try {
-      return config.getMapper().writeValueAsString(this);
+      return writer.writeValueAsString(this);
     } catch (JsonProcessingException e) {
       throw new RuntimeException(e);
     }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/SetSpec.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/SetSpec.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/SetSpec.java
deleted file mode 100644
index 5250dbb..0000000
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/SetSpec.java
+++ /dev/null
@@ -1,36 +0,0 @@
-/*******************************************************************************
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- ******************************************************************************/
-package org.apache.drill.common.physical;
-
-import java.util.List;
-
-import org.apache.drill.common.expression.types.DataType;
-import org.apache.drill.common.physical.props.PhysicalProp;
-
-public class SetSpec {
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(SetSpec.class);
-
-  private List<Field> fields;
-  private List<PhysicalProp> traits;
-
-  public class Field {
-    public String name;
-    public DataType type;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/StitchDef.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/StitchDef.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/StitchDef.java
deleted file mode 100644
index d9a7d33..0000000
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/StitchDef.java
+++ /dev/null
@@ -1,48 +0,0 @@
-/*******************************************************************************
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- ******************************************************************************/
-package org.apache.drill.common.physical;
-
-import org.apache.drill.common.expression.LogicalExpression;
-
-import com.fasterxml.jackson.annotation.JsonCreator;
-import com.fasterxml.jackson.annotation.JsonProperty;
-
-public class StitchDef {
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(StitchDef.class);
-  
-  public static enum StitchMode {RANDOM, NWAY, BLOCK} 
-  
-  private StitchMode mode;
-  private LogicalExpression[] exprs;
-  
-  @JsonCreator 
-  public StitchDef(@JsonProperty("pattern") StitchMode mode, @JsonProperty("exprs") LogicalExpression[] exprs) {
-    super();
-    this.mode = mode;
-    this.exprs = exprs;
-  }
-
-  public StitchMode getMode() {
-    return mode;
-  }
-
-  public LogicalExpression[] getExprs() {
-    return exprs;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/ExchangePOP.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/ExchangePOP.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/ExchangePOP.java
deleted file mode 100644
index 757f03b..0000000
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/ExchangePOP.java
+++ /dev/null
@@ -1,54 +0,0 @@
-/*******************************************************************************
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- ******************************************************************************/
-package org.apache.drill.common.physical.pop;
-
-import java.util.Iterator;
-
-import org.apache.drill.common.defs.PartitionDef;
-import org.apache.drill.common.physical.FieldSet;
-import org.apache.drill.common.physical.StitchDef;
-
-import com.fasterxml.jackson.annotation.JsonCreator;
-import com.fasterxml.jackson.annotation.JsonProperty;
-import com.fasterxml.jackson.annotation.JsonTypeName;
-
-@JsonTypeName("exchange")
-public class ExchangePOP extends SingleChildPOP{
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ExchangePOP.class);
-  
-  private PartitionDef partition;
-  private StitchDef stitch;
-  
-  @JsonCreator
-  public ExchangePOP(@JsonProperty("fields") FieldSet fields, @JsonProperty("partition") PartitionDef partition, @JsonProperty("stitch") StitchDef stitch) {
-    super(fields);
-    this.partition = partition;
-    this.stitch = stitch;
-  }
-  
-  public PartitionDef getPartition() {
-    return partition;
-  }
-
-  public StitchDef getStitch() {
-    return stitch;
-  }
-
-
-  
-}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/FieldCombinePOP.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/FieldCombinePOP.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/FieldCombinePOP.java
deleted file mode 100644
index ac7e036..0000000
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/FieldCombinePOP.java
+++ /dev/null
@@ -1,28 +0,0 @@
-/*******************************************************************************
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- ******************************************************************************/
-package org.apache.drill.common.physical.pop;
-
-/**
- * Creates a complex field out of two or more component fields
- */
-public class FieldCombinePOP {
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(FieldCombinePOP.class);
-  
-  // fieldsInSortOrder
-  private int[] fieldIds; 
-}

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

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/Filter.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/Filter.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/Filter.java
new file mode 100644
index 0000000..2c86d99
--- /dev/null
+++ b/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/Filter.java
@@ -0,0 +1,52 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.common.physical.pop;
+
+import org.apache.drill.common.expression.LogicalExpression;
+import org.apache.drill.common.physical.pop.base.AbstractSingle;
+import org.apache.drill.common.physical.pop.base.PhysicalOperator;
+import org.apache.drill.common.physical.pop.base.PhysicalVisitor;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.annotation.JsonTypeName;
+
+@JsonTypeName("filter")
+public class Filter extends AbstractSingle {
+
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(Filter.class);
+
+  private final LogicalExpression expr;
+  
+  @JsonCreator
+  public Filter(@JsonProperty("child") PhysicalOperator child, @JsonProperty("expr") LogicalExpression expr) {
+    super(child);
+    this.expr = expr;
+  }
+
+  public LogicalExpression getExpr() {
+    return expr;
+  }
+
+  @Override
+  public <T, X, E extends Throwable> T accept(PhysicalVisitor<T, X, E> physicalVisitor, X value) throws E{
+    return physicalVisitor.visitFilter(this, value);
+  }
+
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/POPBase.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/POPBase.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/POPBase.java
deleted file mode 100644
index 5d44e2a..0000000
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/POPBase.java
+++ /dev/null
@@ -1,65 +0,0 @@
-/*******************************************************************************
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- ******************************************************************************/
-package org.apache.drill.common.physical.pop;
-
-import org.apache.drill.common.config.CommonConstants;
-import org.apache.drill.common.config.DrillConfig;
-import org.apache.drill.common.graph.GraphVisitor;
-import org.apache.drill.common.physical.FieldSet;
-import org.apache.drill.common.physical.POPCost;
-import org.apache.drill.common.util.PathScanner;
-
-import com.fasterxml.jackson.annotation.JsonProperty;
-
-public abstract class POPBase implements PhysicalOperator{
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(POPBase.class);
-  
-  private FieldSet fieldSet;
-  
-  
-  public POPBase(FieldSet fieldSet){
-    this.fieldSet = fieldSet;
-  }
-  
-  public synchronized static Class<?>[] getSubTypes(DrillConfig config){
-    Class<?>[] ops = PathScanner.scanForImplementationsArr(PhysicalOperator.class, config.getStringList(CommonConstants.PHYSICAL_OPERATOR_SCAN_PACKAGES));
-    logger.debug("Adding Physical Operator sub types: {}", ((Object) ops) );
-    return ops;
-  }
-  
-  @JsonProperty("fields")
-  public FieldSet getFieldSet(){
-    return fieldSet;
-  }
-
-  @Override
-  public void accept(GraphVisitor<PhysicalOperator> visitor) {
-    visitor.enter(this);
-    if(this.iterator() == null) throw new IllegalArgumentException("Null iterator for pop." + this);
-    for(PhysicalOperator o : this){
-      o.accept(visitor);  
-    }
-    visitor.leave(this);
-  }
-
-  @Override
-  public POPCost getCost() {
-    return null;
-  }
-  
-}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/PartitionToRandomExchange.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/PartitionToRandomExchange.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/PartitionToRandomExchange.java
new file mode 100644
index 0000000..0289780
--- /dev/null
+++ b/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/PartitionToRandomExchange.java
@@ -0,0 +1,92 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.common.physical.pop;
+
+import java.util.List;
+
+import org.apache.drill.common.defs.PartitionDef;
+import org.apache.drill.common.expression.LogicalExpression;
+import org.apache.drill.common.physical.OperatorCost;
+import org.apache.drill.common.physical.pop.base.AbstractExchange;
+import org.apache.drill.common.physical.pop.base.ExchangeCost;
+import org.apache.drill.common.physical.pop.base.PhysicalOperator;
+import org.apache.drill.common.physical.pop.base.PhysicalVisitor;
+import org.apache.drill.common.physical.pop.base.Receiver;
+import org.apache.drill.common.physical.pop.base.Sender;
+import org.apache.drill.common.proto.CoordinationProtos.DrillbitEndpoint;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.annotation.JsonTypeName;
+
+@JsonTypeName("partition-to-random-exchange")
+public class PartitionToRandomExchange extends AbstractExchange{
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(PartitionToRandomExchange.class);
+
+  private final PartitionDef partition;
+  private final int maxWidth;
+  
+  @JsonCreator
+  public PartitionToRandomExchange(@JsonProperty("child") PhysicalOperator child, @JsonProperty("partition") PartitionDef partition, @JsonProperty("cost") ExchangeCost cost) {
+    super(child, cost);
+    this.partition = partition;
+    
+    LogicalExpression[] parts = partition.getStarts();
+    if(parts != null && parts.length > 0){
+      this.maxWidth = parts.length+1;
+    }else{
+      this.maxWidth = Integer.MAX_VALUE;
+    }
+  }
+
+  public PartitionDef getPartition() {
+    return partition;
+  }
+
+  @Override
+  public <T, X, E extends Throwable> T accept(PhysicalVisitor<T, X, E> physicalVisitor, X value) throws E{
+    return physicalVisitor.visitExchange(this,  value);
+  }
+
+  @Override
+  public int getMaxSendWidth() {
+    return maxWidth;
+  }
+
+  @Override
+  public void setupSenders(List<DrillbitEndpoint> senderLocations) {
+  }
+
+  @Override
+  public void setupReceivers(List<DrillbitEndpoint> receiverLocations) {
+  }
+
+  @Override
+  public Sender getSender(int minorFragmentId, PhysicalOperator child) {
+    return null;
+  }
+
+  @Override
+  public Receiver getReceiver(int minorFragmentId) {
+    return null;
+  }
+
+
+  
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/PhysicalOperator.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/PhysicalOperator.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/PhysicalOperator.java
deleted file mode 100644
index 0a8927a..0000000
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/PhysicalOperator.java
+++ /dev/null
@@ -1,35 +0,0 @@
-/*******************************************************************************
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- ******************************************************************************/
-package org.apache.drill.common.physical.pop;
-
-import org.apache.drill.common.graph.GraphValue;
-import org.apache.drill.common.physical.FieldSet;
-import org.apache.drill.common.physical.POPCost;
-
-import com.fasterxml.jackson.annotation.JsonIdentityInfo;
-import com.fasterxml.jackson.annotation.JsonPropertyOrder;
-import com.fasterxml.jackson.annotation.JsonTypeInfo;
-import com.fasterxml.jackson.annotation.ObjectIdGenerators;
-
-@JsonPropertyOrder({"@id"})
-@JsonIdentityInfo(generator=ObjectIdGenerators.IntSequenceGenerator.class, property="@id")
-@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, include = JsonTypeInfo.As.PROPERTY, property="pop")
-public interface PhysicalOperator extends GraphValue<PhysicalOperator>{
-  public FieldSet getFieldSet();
-  public POPCost getCost();
-}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/Project.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/Project.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/Project.java
new file mode 100644
index 0000000..7cff28d
--- /dev/null
+++ b/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/Project.java
@@ -0,0 +1,53 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.common.physical.pop;
+
+import java.util.Iterator;
+import java.util.List;
+
+import org.apache.drill.common.logical.data.NamedExpression;
+import org.apache.drill.common.physical.pop.base.AbstractSingle;
+import org.apache.drill.common.physical.pop.base.PhysicalOperator;
+import org.apache.drill.common.physical.pop.base.PhysicalVisitor;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.annotation.JsonTypeName;
+
+@JsonTypeName("project")
+public class Project extends AbstractSingle{
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(Project.class);
+
+  private final List<NamedExpression> exprs;
+  
+  @JsonCreator
+  public Project(@JsonProperty("exprs") List<NamedExpression> exprs, @JsonProperty("child") PhysicalOperator child) {
+    super(child);
+    this.exprs = exprs;
+  }
+
+  public List<NamedExpression> getExprs() {
+    return exprs;
+  }
+
+  
+  @Override
+  public <T, X, E extends Throwable> T accept(PhysicalVisitor<T, X, E> physicalVisitor, X value) throws E{
+    return physicalVisitor.visitProject(this, value);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/ProjectPOP.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/ProjectPOP.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/ProjectPOP.java
deleted file mode 100644
index bd481d4..0000000
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/ProjectPOP.java
+++ /dev/null
@@ -1,53 +0,0 @@
-/*******************************************************************************
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- ******************************************************************************/
-package org.apache.drill.common.physical.pop;
-
-import java.util.List;
-
-import org.apache.drill.common.defs.PartitionDef;
-import org.apache.drill.common.expression.LogicalExpression;
-import org.apache.drill.common.physical.FieldSet;
-import org.apache.drill.common.physical.StitchDef;
-
-import com.fasterxml.jackson.annotation.JsonCreator;
-import com.fasterxml.jackson.annotation.JsonProperty;
-import com.fasterxml.jackson.annotation.JsonTypeName;
-
-@JsonTypeName("project")
-public class ProjectPOP extends SingleChildPOP{
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ProjectPOP.class);
-  
-  private List<Integer> fieldIds;
-  private List<LogicalExpression> exprs;
-  
-  @JsonCreator
-  public ProjectPOP(@JsonProperty("output") FieldSet fields, @JsonProperty("fields") List<Integer> fieldIds, @JsonProperty("exprs") List<LogicalExpression> exprs) {
-    super(fields);
-    this.fieldIds = fieldIds;
-    this.exprs = exprs;
-  }
-
-  public List<Integer> getFields() {
-    return fieldIds;
-  }
-
-  public List<LogicalExpression> getExprs() {
-    return exprs;
-  }
-    
-}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/QuickNWaySortPOP.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/QuickNWaySortPOP.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/QuickNWaySortPOP.java
deleted file mode 100644
index f7fcdb0..0000000
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/QuickNWaySortPOP.java
+++ /dev/null
@@ -1,50 +0,0 @@
-/*******************************************************************************
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- ******************************************************************************/
-package org.apache.drill.common.physical.pop;
-
-import java.util.Iterator;
-import java.util.List;
-
-import org.apache.drill.common.defs.OrderDef;
-import org.apache.drill.common.physical.FieldSet;
-
-import com.fasterxml.jackson.annotation.JsonCreator;
-import com.fasterxml.jackson.annotation.JsonProperty;
-import com.fasterxml.jackson.annotation.JsonTypeName;
-
-@JsonTypeName("quicknwaysort")
-public class QuickNWaySortPOP extends SingleChildPOP{
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(QuickNWaySortPOP.class);
-  
-  private List<OrderDef> orderings;
-
-  @JsonCreator
-  public QuickNWaySortPOP(@JsonProperty("fields") FieldSet fieldSet, @JsonProperty("orderings") List<OrderDef> orderings) {
-    super(fieldSet);
-    this.orderings = orderings;
-  }
-
-  @JsonProperty("orderings")
-  public List<OrderDef> getOrderings() {
-    return orderings;
-  }
-
-
-  
-  
-}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/ScanPOP.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/ScanPOP.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/ScanPOP.java
deleted file mode 100644
index 2aaf8fa..0000000
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/ScanPOP.java
+++ /dev/null
@@ -1,55 +0,0 @@
-/*******************************************************************************
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- ******************************************************************************/
-package org.apache.drill.common.physical.pop;
-
-import java.util.Iterator;
-import java.util.List;
-
-import org.apache.drill.common.JSONOptions;
-import org.apache.drill.common.config.DrillConfig;
-import org.apache.drill.common.physical.FieldSet;
-import org.apache.drill.common.physical.ReadEntry;
-
-import com.fasterxml.jackson.annotation.JsonCreator;
-import com.fasterxml.jackson.annotation.JsonProperty;
-import com.fasterxml.jackson.annotation.JsonTypeName;
-import com.google.common.collect.Iterators;
-import com.google.common.collect.Lists;
-
-public abstract class ScanPOP<T extends ReadEntry> extends POPBase implements SourcePOP{
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ScanPOP.class);
-  
-  private List<T> readEntries;
-  
-  public ScanPOP(List<T> readEntries, FieldSet fieldSet) {
-    super(fieldSet);
-    this.readEntries = readEntries;
-  }
-
-  @JsonProperty("entries")
-  public List<T> getReadEntries() {
-    return readEntries;
-  }
-  
-  @Override
-  public Iterator<PhysicalOperator> iterator() {
-    return Iterators.emptyIterator();
-  }
-
-  
-}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/Screen.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/Screen.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/Screen.java
new file mode 100644
index 0000000..fdbd8f1
--- /dev/null
+++ b/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/Screen.java
@@ -0,0 +1,77 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.common.physical.pop;
+
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.drill.common.physical.EndpointAffinity;
+import org.apache.drill.common.physical.pop.base.AbstractStore;
+import org.apache.drill.common.physical.pop.base.PhysicalOperator;
+import org.apache.drill.common.physical.pop.base.Store;
+import org.apache.drill.common.proto.CoordinationProtos.DrillbitEndpoint;
+
+import com.fasterxml.jackson.annotation.JacksonInject;
+import com.fasterxml.jackson.annotation.JsonIgnore;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.annotation.JsonTypeName;
+
+@JsonTypeName("screen")
+public class Screen extends AbstractStore {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(Screen.class);
+
+  private final DrillbitEndpoint endpoint;
+
+  public Screen(@JsonProperty("child") PhysicalOperator child, @JacksonInject DrillbitEndpoint endpoint) {
+    super(child);
+    this.endpoint = endpoint;
+  }
+
+  @Override
+  public List<EndpointAffinity> getOperatorAffinity() {
+    return Collections.singletonList(new EndpointAffinity(endpoint, 1000));
+  }
+
+  @Override
+  public int getMaxWidth() {
+    return 1;
+  }
+
+  @Override
+  public void applyAssignments(List<DrillbitEndpoint> endpoints) {
+    // we actually don't have to do anything since nothing should have changed. we'll check just check that things
+    // didn't get screwed up.
+    if (endpoints.size() != 1)
+      throw new UnsupportedOperationException("A Screen operator can only be assigned to a single node.");
+    DrillbitEndpoint endpoint = endpoints.iterator().next();
+    if (this.endpoint != endpoint)
+      throw new UnsupportedOperationException("A Screen operator can only be assigned to its home node.");
+
+  }
+
+  @Override
+  public Store getSpecificStore(PhysicalOperator child, int minorFragmentId) {
+    return new Screen(child, endpoint);
+  }
+
+  @JsonIgnore
+  public DrillbitEndpoint getEndpoint() {
+    return endpoint;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/SingleChildPOP.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/SingleChildPOP.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/SingleChildPOP.java
deleted file mode 100644
index cf0c08b..0000000
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/SingleChildPOP.java
+++ /dev/null
@@ -1,41 +0,0 @@
-/*******************************************************************************
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- ******************************************************************************/
-package org.apache.drill.common.physical.pop;
-
-import java.util.Iterator;
-
-import org.apache.drill.common.physical.FieldSet;
-
-import com.google.common.collect.Iterators;
-
-public abstract class SingleChildPOP extends POPBase{
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(SingleChildPOP.class);
-  
-  public PhysicalOperator child;
-
-  public SingleChildPOP(FieldSet fieldSet) {
-    super(fieldSet);
-  }
-
-  @Override
-  public Iterator<PhysicalOperator> iterator() {
-    return Iterators.singletonIterator(child);
-  }
-  
-  
-}

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

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/Sort.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/Sort.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/Sort.java
new file mode 100644
index 0000000..b4d802d
--- /dev/null
+++ b/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/Sort.java
@@ -0,0 +1,57 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.common.physical.pop;
+
+import org.apache.drill.common.expression.LogicalExpression;
+import org.apache.drill.common.physical.pop.base.AbstractSingle;
+import org.apache.drill.common.physical.pop.base.PhysicalOperator;
+import org.apache.drill.common.physical.pop.base.PhysicalVisitor;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.annotation.JsonTypeName;
+
+@JsonTypeName("sort")
+public class Sort extends AbstractSingle{
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(Sort.class);
+  
+  private final LogicalExpression expr;
+  private boolean reverse = false;
+  
+  @JsonCreator
+  public Sort(@JsonProperty("child") PhysicalOperator child, @JsonProperty("expr") LogicalExpression expr, @JsonProperty("reverse") boolean reverse) {
+    super(child);
+    this.expr = expr;
+    this.reverse = reverse;
+  }
+  
+  public LogicalExpression getExpr() {
+    return expr;
+  }
+
+  public boolean getReverse() {
+    return reverse;
+  }
+
+  @Override
+  public <T, X, E extends Throwable> T accept(PhysicalVisitor<T, X, E> physicalVisitor, X value) throws E{
+    return physicalVisitor.visitSort(this, value);
+  }
+    
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/SortPOP.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/SortPOP.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/SortPOP.java
deleted file mode 100644
index 4d0263b..0000000
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/SortPOP.java
+++ /dev/null
@@ -1,54 +0,0 @@
-/*******************************************************************************
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- ******************************************************************************/
-package org.apache.drill.common.physical.pop;
-
-import java.util.List;
-
-import org.apache.drill.common.defs.PartitionDef;
-import org.apache.drill.common.expression.LogicalExpression;
-import org.apache.drill.common.physical.FieldSet;
-import org.apache.drill.common.physical.StitchDef;
-
-import com.fasterxml.jackson.annotation.JsonCreator;
-import com.fasterxml.jackson.annotation.JsonProperty;
-import com.fasterxml.jackson.annotation.JsonTypeName;
-
-@JsonTypeName("sort")
-public class SortPOP extends SingleChildPOP{
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(SortPOP.class);
-  
-  private int field;
-  private boolean reverse = false;
-  
-  @JsonCreator
-  public SortPOP(@JsonProperty("output") FieldSet fields, @JsonProperty("field") int field, @JsonProperty("reverse") boolean reverse) {
-    super(fields);
-    this.field = field;
-    this.reverse = reverse;
-  }
-
-  public int getField() {
-    return field;
-  }
-
-  public boolean getReverse() {
-    return reverse;
-  }
-    
-  
-}

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

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/StorePOP.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/StorePOP.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/StorePOP.java
deleted file mode 100644
index 2b8e075..0000000
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/StorePOP.java
+++ /dev/null
@@ -1,61 +0,0 @@
-/*******************************************************************************
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- ******************************************************************************/
-package org.apache.drill.common.physical.pop;
-
-import java.util.List;
-
-import org.apache.drill.common.JSONOptions;
-import org.apache.drill.common.defs.PartitionDef;
-import org.apache.drill.common.physical.FieldSet;
-import org.apache.drill.common.physical.WriteEntry;
-
-import com.fasterxml.jackson.annotation.JsonCreator;
-import com.fasterxml.jackson.annotation.JsonProperty;
-import com.fasterxml.jackson.annotation.JsonTypeName;
-
-public abstract class StorePOP<T extends WriteEntry> extends SingleChildPOP implements SinkPOP{
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(StorePOP.class);
-
-  public static enum StoreMode {SYSTEM_CHOICE, PREDEFINED_PARTITIONS};
-  
-  private StoreMode mode;
-  private PartitionDef partition;
-  private List<T> entries;
-  
-  @JsonCreator
-  public StorePOP(FieldSet fieldSet, StoreMode mode, PartitionDef partition, List<T> entries) {
-    super(fieldSet);
-    this.mode = mode;
-    this.partition = partition;
-    this.entries = entries;
-  }
-
-  public StoreMode getMode() {
-    return mode;
-  }
-
-  public PartitionDef getPartition() {
-    return partition;
-  }
-
-  public List<T> getEntries(){
-    return entries;
-  }
-  
-  
-}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/AbstractBase.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/AbstractBase.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/AbstractBase.java
new file mode 100644
index 0000000..5d3584c
--- /dev/null
+++ b/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/AbstractBase.java
@@ -0,0 +1,56 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.common.physical.pop.base;
+
+import org.apache.drill.common.graph.GraphVisitor;
+import org.apache.drill.common.physical.OperatorCost;
+
+public abstract class AbstractBase implements PhysicalOperator{
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(AbstractBase.class);
+
+  private OperatorCost cost;
+  
+  @Override
+  public void accept(GraphVisitor<PhysicalOperator> visitor) {
+    visitor.enter(this);
+    if(this.iterator() == null) throw new IllegalArgumentException("Null iterator for pop." + this);
+    for(PhysicalOperator o : this){
+      o.accept(visitor);  
+    }
+    visitor.leave(this);
+  }
+  
+  @Override
+  public boolean isExecutable() {
+    return true;
+  }
+
+  @Override
+  public OperatorCost getCost() {
+    return cost;
+  }
+  
+  // should be used only for the purposes of json...
+  void setCost(OperatorCost cost){
+    this.cost = cost;
+  }
+
+
+  
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/AbstractExchange.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/AbstractExchange.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/AbstractExchange.java
new file mode 100644
index 0000000..1f60c53
--- /dev/null
+++ b/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/AbstractExchange.java
@@ -0,0 +1,68 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.common.physical.pop.base;
+
+import java.util.List;
+
+import org.apache.drill.common.physical.OperatorCost;
+import org.apache.drill.common.proto.CoordinationProtos.DrillbitEndpoint;
+
+import com.fasterxml.jackson.annotation.JsonIgnore;
+import com.fasterxml.jackson.annotation.JsonProperty;
+
+public abstract class AbstractExchange extends AbstractSingle implements Exchange {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(AbstractExchange.class);
+
+  private final ExchangeCost cost;
+
+  public AbstractExchange(PhysicalOperator child, ExchangeCost cost) {
+    super(child);
+    this.cost = cost;
+  }
+
+  /**
+   * Exchanges are not executable. The Execution layer first has to set their parallelization and convert them into
+   * something executable
+   */
+  @Override
+  public boolean isExecutable() {
+    return false;
+  }
+
+  @Override
+  public OperatorCost getAggregateSendCost() {
+    return cost.getSend();
+  }
+
+  @Override
+  public OperatorCost getAggregateReceiveCost() {
+    return cost.getReceive();
+  }
+
+  @Override
+  public ExchangeCost getExchangeCost() {
+    return cost;
+  }
+
+  @JsonIgnore
+  @Override
+  public OperatorCost getCost() {
+    return cost.getCombinedCost();
+  }
+
+}


[13/53] [abbrv] Update typing system. Update RPC system. Add Fragmenting Implementation. Working single node. Distributed failing due to threading issues.

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/SimpleParallelizer.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/SimpleParallelizer.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/SimpleParallelizer.java
new file mode 100644
index 0000000..fc03a23
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/SimpleParallelizer.java
@@ -0,0 +1,163 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.planner.fragment;
+
+import java.util.Collection;
+import java.util.List;
+
+import org.apache.drill.common.exceptions.ExecutionSetupException;
+import org.apache.drill.common.exceptions.PhysicalOperatorSetupException;
+import org.apache.drill.exec.exception.FragmentSetupException;
+import org.apache.drill.exec.physical.base.FragmentRoot;
+import org.apache.drill.exec.physical.base.PhysicalOperator;
+import org.apache.drill.exec.planner.PhysicalPlanReader;
+import org.apache.drill.exec.planner.fragment.Materializer.IndexedFragmentNode;
+import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
+import org.apache.drill.exec.proto.ExecProtos.FragmentHandle;
+import org.apache.drill.exec.proto.ExecProtos.PlanFragment;
+import org.apache.drill.exec.proto.UserBitShared.QueryId;
+import org.apache.drill.exec.work.QueryWorkUnit;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+
+public class SimpleParallelizer {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(SimpleParallelizer.class);
+
+  private final Materializer materializer = new Materializer();
+
+  /**
+   * Generate a set of assigned fragments based on the provided planningSet. Do not allow parallelization stages to go
+   * beyond the global max width.
+   * 
+   * @param context
+   *          The current QueryContext.
+   * @param planningSet
+   *          The set of queries with collected statistics that we'll work with.
+   * @param globalMaxWidth
+   *          The maximum level or paralellization any stage of the query can do. Note that while this might be the
+   *          number of active Drillbits, realistically, this could be well beyond that number of we want to do things
+   *          like speed results return.
+   * @return The list of generatoe PlanFragment protobuf objects to be assigned out to the individual nodes.
+   * @throws FragmentSetupException
+   */
+  public QueryWorkUnit getFragments(DrillbitEndpoint foremanNode, QueryId queryId, Collection<DrillbitEndpoint> activeEndpoints, PhysicalPlanReader reader, Fragment rootNode, PlanningSet planningSet,
+      int globalMaxWidth) throws ExecutionSetupException {
+    assignEndpoints(activeEndpoints, planningSet, globalMaxWidth);
+    return generateWorkUnit(foremanNode, queryId, reader, rootNode, planningSet);
+  }
+
+  private QueryWorkUnit generateWorkUnit(DrillbitEndpoint foremanNode, QueryId queryId, PhysicalPlanReader reader, Fragment rootNode,
+      PlanningSet planningSet) throws ExecutionSetupException {
+
+    List<PlanFragment> fragments = Lists.newArrayList();
+
+    PlanFragment rootFragment = null;
+    FragmentRoot rootOperator = null;
+
+    // now we generate all the individual plan fragments and associated assignments. Note, we need all endpoints
+    // assigned before we can materialize, so we start a new loop here rather than utilizing the previous one.
+    for (Wrapper wrapper : planningSet) {
+      Fragment node = wrapper.getNode();
+      Stats stats = node.getStats();
+      final PhysicalOperator physicalOperatorRoot = node.getRoot();
+      boolean isRootNode = rootNode == node;
+
+      if (isRootNode && wrapper.getWidth() != 1)
+        throw new FragmentSetupException(
+            String.format(
+                    "Failure while trying to setup fragment.  The root fragment must always have parallelization one.  In the current case, the width was set to %d.",
+                    wrapper.getWidth()));
+      // a fragment is self driven if it doesn't rely on any other exchanges.
+      boolean isLeafFragment = node.getReceivingExchangePairs().size() == 0;
+
+      // Create a minorFragment for each major fragment.
+      for (int minorFragmentId = 0; minorFragmentId < wrapper.getWidth(); minorFragmentId++) {
+        IndexedFragmentNode iNode = new IndexedFragmentNode(minorFragmentId, wrapper);
+        PhysicalOperator op = physicalOperatorRoot.accept(materializer, iNode);
+        Preconditions.checkArgument(op instanceof FragmentRoot);
+        FragmentRoot root = (FragmentRoot) op; 
+
+        // get plan as JSON
+        String plan;
+        try {
+          plan = reader.writeJson(root);
+        } catch (JsonProcessingException e) {
+          throw new FragmentSetupException("Failure while trying to convert fragment into json.", e);
+        }
+        
+        FragmentHandle handle = FragmentHandle //
+            .newBuilder() //
+            .setMajorFragmentId(wrapper.getMajorFragmentId()) //
+            .setMinorFragmentId(minorFragmentId) //
+            .setQueryId(queryId) //
+            .build();
+        PlanFragment fragment = PlanFragment.newBuilder() //
+            .setCpuCost(stats.getCpuCost()) //
+            .setDiskCost(stats.getDiskCost()) //
+            .setForeman(foremanNode) //
+            .setMemoryCost(stats.getMemoryCost()) //
+            .setNetworkCost(stats.getNetworkCost()) //
+            .setFragmentJson(plan) //
+            .setHandle(handle) //
+            .setAssignment(wrapper.getAssignedEndpoint(minorFragmentId)) //
+            .setLeafFragment(isLeafFragment) //
+            .build();
+
+        if (isRootNode) {
+          rootFragment = fragment;
+          rootOperator = root;
+        } else {
+          fragments.add(fragment);
+        }
+      }
+    }
+
+    return new QueryWorkUnit(rootOperator, rootFragment, fragments);
+
+  }
+
+  private void assignEndpoints(Collection<DrillbitEndpoint> allNodes, PlanningSet planningSet,
+      int globalMaxWidth) throws PhysicalOperatorSetupException {
+    // First we determine the amount of parallelization for a fragment. This will be between 1 and maxWidth based on
+    // cost. (Later could also be based on cluster operation.) then we decide endpoints based on affinity (later this
+    // could be based on endpoint load)
+    for (Wrapper wrapper : planningSet) {
+
+      Stats stats = wrapper.getStats();
+
+      // figure out width.
+      int width = Math.min(stats.getMaxWidth(), globalMaxWidth);
+      float diskCost = stats.getDiskCost();
+      logger.debug("Frag max width: {} and diskCost: {}", stats.getMaxWidth(), diskCost);
+
+      // TODO: right now we'll just assume that each task is cost 1 so we'll set the breadth at the lesser of the number
+      // of tasks or the maximum width of the fragment.
+      if (diskCost < width) {
+        width = (int) diskCost;
+      }
+
+      if (width < 1) width = 1;
+      logger.debug("Setting width {} on fragment {}", width, wrapper);
+      wrapper.setWidth(width);
+      // figure out endpoint assignments. also informs the exchanges about their respective endpoints.
+      wrapper.assignEndpoints(allNodes);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/Stats.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/Stats.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/Stats.java
new file mode 100644
index 0000000..729b2f1
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/Stats.java
@@ -0,0 +1,70 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.planner.fragment;
+
+import org.apache.drill.exec.physical.OperatorCost;
+
+public class Stats {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(Stats.class);
+  
+  private int maxWidth = Integer.MAX_VALUE;
+  private float networkCost; 
+  private float diskCost;
+  private float memoryCost;
+  private float cpuCost;
+  
+  public void addMaxWidth(int maxWidth){
+    this.maxWidth = Math.min(this.maxWidth, maxWidth);
+  }
+  
+  public void addCost(OperatorCost cost){
+    networkCost += cost.getNetwork();
+    diskCost += cost.getDisk();
+    memoryCost += cost.getMemory();
+    cpuCost += cost.getCpu();
+  }
+
+  public int getMaxWidth() {
+    return maxWidth;
+  }
+
+  public float getNetworkCost() {
+    return networkCost;
+  }
+
+  public float getDiskCost() {
+    return diskCost;
+  }
+
+  public float getMemoryCost() {
+    return memoryCost;
+  }
+
+  public float getCpuCost() {
+    return cpuCost;
+  }
+
+  @Override
+  public String toString() {
+    return "FragmentStats [maxWidth=" + maxWidth + ", networkCost=" + networkCost + ", diskCost=" + diskCost
+        + ", memoryCost=" + memoryCost + ", cpuCost=" + cpuCost + "]";
+  }
+  
+  
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/StatsCollector.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/StatsCollector.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/StatsCollector.java
new file mode 100644
index 0000000..d53a78c
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/StatsCollector.java
@@ -0,0 +1,106 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.planner.fragment;
+
+import org.apache.drill.exec.physical.base.Exchange;
+import org.apache.drill.exec.physical.base.HasAffinity;
+import org.apache.drill.exec.physical.base.PhysicalOperator;
+import org.apache.drill.exec.physical.base.Scan;
+import org.apache.drill.exec.physical.base.Store;
+import org.apache.drill.exec.planner.AbstractOpWrapperVisitor;
+import org.apache.drill.exec.planner.fragment.Fragment.ExchangeFragmentPair;
+
+import com.google.common.base.Preconditions;
+
+public class StatsCollector {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(StatsCollector.class);
+
+  private final static OpStatsCollector opStatCollector = new OpStatsCollector();
+
+  private StatsCollector() {
+  };
+
+  private static void visit(PlanningSet planningSet, Fragment n) {
+    Preconditions.checkNotNull(planningSet);
+    Preconditions.checkNotNull(n);
+
+    Wrapper wrapper = planningSet.get(n);
+    n.getRoot().accept(opStatCollector, wrapper);
+    logger.debug("Set stats to {}", wrapper.getStats());
+    // receivers...
+    for (ExchangeFragmentPair child : n) {
+      // get the fragment node that feeds this node.
+      Fragment childNode = child.getNode();
+      visit(planningSet, childNode);
+    }
+
+  }
+
+  public static PlanningSet collectStats(Fragment rootFragment) {
+    PlanningSet fps = new PlanningSet();
+    visit(fps, rootFragment);
+    return fps;
+  }
+
+  private static class OpStatsCollector extends AbstractOpWrapperVisitor<Void, RuntimeException> {
+
+    @Override
+    public Void visitSendingExchange(Exchange exchange, Wrapper wrapper) throws RuntimeException {
+      Stats stats = wrapper.getStats();
+      stats.addCost(exchange.getAggregateSendCost());
+      stats.addMaxWidth(exchange.getMaxSendWidth());
+      return super.visitSendingExchange(exchange, wrapper);
+    }
+
+    @Override
+    public Void visitReceivingExchange(Exchange exchange, Wrapper wrapper) throws RuntimeException {
+      wrapper.getStats().addCost(exchange.getAggregateReceiveCost());
+      // no traversal since it would cross fragment boundary.
+      return null;
+    }
+
+    @Override
+    public Void visitScan(Scan<?> scan, Wrapper wrapper) {
+      Stats stats = wrapper.getStats();      
+      stats.addMaxWidth(scan.getReadEntries().size());
+      return super.visitScan(scan, wrapper);
+    }
+
+    @Override
+    public Void visitStore(Store store, Wrapper wrapper) {
+      Stats stats = wrapper.getStats();
+      stats.addMaxWidth(store.getMaxWidth());
+      return super.visitStore(store, wrapper);
+    }
+
+    @Override
+    public Void visitOp(PhysicalOperator op, Wrapper wrapper) {
+      if(op instanceof HasAffinity){
+        wrapper.addEndpointAffinity(((HasAffinity)op).getOperatorAffinity());
+      }
+      Stats stats = wrapper.getStats();
+      stats.addCost(op.getCost());
+      for (PhysicalOperator child : op) {
+        child.accept(this, wrapper);
+      }
+      return null;
+    }
+
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/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
new file mode 100644
index 0000000..0dfcb62
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/Wrapper.java
@@ -0,0 +1,186 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.planner.fragment;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ThreadLocalRandom;
+
+import org.apache.commons.lang.NotImplementedException;
+import org.apache.drill.common.exceptions.PhysicalOperatorSetupException;
+import org.apache.drill.exec.physical.EndpointAffinity;
+import org.apache.drill.exec.physical.base.AbstractPhysicalVisitor;
+import org.apache.drill.exec.physical.base.Exchange;
+import org.apache.drill.exec.physical.base.PhysicalOperator;
+import org.apache.drill.exec.physical.base.Scan;
+import org.apache.drill.exec.physical.base.Store;
+import org.apache.drill.exec.planner.fragment.Fragment.ExchangeFragmentPair;
+import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+
+/**
+ * A wrapping class that allows us to add additional information to each fragment node for planning purposes.
+ */
+public class Wrapper {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(Wrapper.class);
+
+  private final Fragment node;
+  private final int majorFragmentId;
+  private int width = -1;
+  private final Stats stats;
+  private boolean endpointsAssigned;
+  private Map<DrillbitEndpoint, EndpointAffinity> endpointAffinity = Maps.newHashMap();
+
+  // a list of assigned endpoints. Technically, there could repeated endpoints in this list if we'd like to assign the
+  // same fragment multiple times to the same endpoint.
+  private List<DrillbitEndpoint> endpoints = Lists.newLinkedList();
+
+  public Wrapper(Fragment node, int majorFragmentId) {
+    this.majorFragmentId = majorFragmentId;
+    this.node = node;
+    this.stats = new Stats();
+  }
+
+  public Stats getStats() {
+    return stats;
+  }
+
+  public void addEndpointAffinity(List<EndpointAffinity> affinities){
+    Preconditions.checkState(!endpointsAssigned);
+    for(EndpointAffinity ea : affinities){
+      addEndpointAffinity(ea.getEndpoint(), ea.getAffinity());
+    }
+  }
+  
+  public void addEndpointAffinity(DrillbitEndpoint endpoint, float affinity) {
+    Preconditions.checkState(!endpointsAssigned);
+    EndpointAffinity ea = endpointAffinity.get(endpoint);
+    if (ea == null) {
+      ea = new EndpointAffinity(endpoint);
+      endpointAffinity.put(endpoint, ea);
+    }
+
+    ea.addAffinity(affinity);
+  }
+
+  public int getMajorFragmentId() {
+    return majorFragmentId;
+  }
+
+  public int getWidth() {
+    return width;
+  }
+
+  public void setWidth(int width) {
+    Preconditions.checkState(this.width == -1);
+    this.width = width;
+  }
+
+  public Fragment getNode() {
+    return node;
+  }
+
+  private class AssignEndpointsToScanAndStore extends AbstractPhysicalVisitor<Void, List<DrillbitEndpoint>, PhysicalOperatorSetupException>{
+
+    
+    @Override
+    public Void visitExchange(Exchange exchange, List<DrillbitEndpoint> value) throws PhysicalOperatorSetupException {
+      if(exchange == node.getSendingExchange()){
+        return visitOp(exchange, value);
+      }
+      // stop on receiver exchange.
+      return null;
+    }
+
+    @Override
+    public Void visitScan(Scan<?> scan, List<DrillbitEndpoint> value) throws PhysicalOperatorSetupException {
+      scan.applyAssignments(value);
+      return super.visitScan(scan, value);
+    }
+
+    @Override
+    public Void visitStore(Store store, List<DrillbitEndpoint> value) throws PhysicalOperatorSetupException {
+      store.applyAssignments(value);
+      return super.visitStore(store, value);
+    }
+
+    @Override
+    public Void visitOp(PhysicalOperator op, List<DrillbitEndpoint> value) throws PhysicalOperatorSetupException {
+      return visitChildren(op, value);
+    }
+    
+  }
+  
+  public void assignEndpoints(Collection<DrillbitEndpoint> allPossible) throws PhysicalOperatorSetupException {
+    Preconditions.checkState(!endpointsAssigned);
+
+    endpointsAssigned = true;
+
+    List<EndpointAffinity> values = Lists.newArrayList();
+    values.addAll(endpointAffinity.values());
+
+    if (values.size() == 0) {
+      List<DrillbitEndpoint> all = Lists.newArrayList(allPossible);
+      final int div = allPossible.size();
+      int start = ThreadLocalRandom.current().nextInt(div);
+      // round robin with random start.
+      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());
+      }
+    }
+
+    // Set scan and store endpoints.
+    AssignEndpointsToScanAndStore visitor = new AssignEndpointsToScanAndStore();
+    node.getRoot().accept(visitor, endpoints);
+    
+    // Set the endpoints for this (one at most) sending exchange.
+    if (node.getSendingExchange() != null) {
+      node.getSendingExchange().setupSenders(majorFragmentId, endpoints);
+    }
+
+    // Set the endpoints for each incoming exchange within this fragment.
+    for (ExchangeFragmentPair e : node.getReceivingExchangePairs()) {
+      e.getExchange().setupReceivers(majorFragmentId, endpoints);
+    }
+  }
+
+  @Override
+  public String toString() {
+    return "FragmentWrapper [majorFragmentId=" + majorFragmentId + ", width=" + width + ", stats=" + stats + "]";
+  }
+
+  public DrillbitEndpoint getAssignedEndpoint(int minorFragmentId) {
+    Preconditions.checkState(endpointsAssigned);
+    return this.endpoints.get(minorFragmentId);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/pop/receiver/NWayOrderingReceiver.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/pop/receiver/NWayOrderingReceiver.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/pop/receiver/NWayOrderingReceiver.java
deleted file mode 100644
index 562d109..0000000
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/pop/receiver/NWayOrderingReceiver.java
+++ /dev/null
@@ -1,52 +0,0 @@
-/*******************************************************************************
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- ******************************************************************************/
-package org.apache.drill.exec.pop.receiver;
-
-import java.util.List;
-
-import org.apache.drill.common.physical.pop.base.AbstractReceiver;
-import org.apache.drill.common.physical.pop.base.PhysicalVisitor;
-import org.apache.drill.common.proto.CoordinationProtos.DrillbitEndpoint;
-
-import com.fasterxml.jackson.annotation.JsonTypeName;
-
-@JsonTypeName("nway-ordering-receiver")
-public class NWayOrderingReceiver extends AbstractReceiver{
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(NWayOrderingReceiver.class);
-
-  @Override
-  public List<DrillbitEndpoint> getProvidingEndpoints() {
-    return null;
-  }
-
-  @Override
-  public boolean supportsOutOfOrderExchange() {
-    return false;
-  }
-
-  @Override
-  public int getSenderCount() {
-    return 0;
-  }
-
-  @Override
-  public <T, X, E extends Throwable> T accept(PhysicalVisitor<T, X, E> physicalVisitor, X value) throws E {
-    return null;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/pop/receiver/RandomReceiver.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/pop/receiver/RandomReceiver.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/pop/receiver/RandomReceiver.java
deleted file mode 100644
index 487c645..0000000
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/pop/receiver/RandomReceiver.java
+++ /dev/null
@@ -1,55 +0,0 @@
-/*******************************************************************************
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- ******************************************************************************/
-package org.apache.drill.exec.pop.receiver;
-
-import java.util.Iterator;
-import java.util.List;
-
-import org.apache.drill.common.physical.pop.base.AbstractReceiver;
-import org.apache.drill.common.physical.pop.base.PhysicalOperator;
-import org.apache.drill.common.physical.pop.base.PhysicalVisitor;
-import org.apache.drill.common.proto.CoordinationProtos.DrillbitEndpoint;
-
-import com.fasterxml.jackson.annotation.JsonTypeName;
-
-@JsonTypeName("receiver-random")
-public class RandomReceiver extends AbstractReceiver{
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(RandomReceiver.class);
-
-  @Override
-  public List<DrillbitEndpoint> getProvidingEndpoints() {
-    return null;
-  }
-
-  @Override
-  public boolean supportsOutOfOrderExchange() {
-    return false;
-  }
-
-  @Override
-  public int getSenderCount() {
-    return 0;
-  }
-
-  @Override
-  public <T, X, E extends Throwable> T accept(PhysicalVisitor<T, X, E> physicalVisitor, X value) throws E {
-    return null;
-  }
-
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/pop/sender/HashPartitionSender.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/pop/sender/HashPartitionSender.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/pop/sender/HashPartitionSender.java
deleted file mode 100644
index b0fb51c..0000000
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/pop/sender/HashPartitionSender.java
+++ /dev/null
@@ -1,49 +0,0 @@
-/*******************************************************************************
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- ******************************************************************************/
-package org.apache.drill.exec.pop.sender;
-
-import java.util.List;
-
-import org.apache.drill.common.physical.pop.base.AbstractSender;
-import org.apache.drill.common.physical.pop.base.PhysicalOperator;
-import org.apache.drill.common.physical.pop.base.PhysicalVisitor;
-import org.apache.drill.common.proto.CoordinationProtos.DrillbitEndpoint;
-
-import com.fasterxml.jackson.annotation.JsonTypeName;
-
-@JsonTypeName("hash-partition-sender")
-public class HashPartitionSender extends AbstractSender {
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(HashPartitionSender.class);
-
-  public HashPartitionSender(PhysicalOperator child) {
-    super(child);
-  }
-
-
-  @Override
-  public List<DrillbitEndpoint> getDestinations() {
-    return null;
-  }
-
-  @Override
-  public <T, X, E extends Throwable> T accept(PhysicalVisitor<T, X, E> physicalVisitor, X value) throws E {
-    return null;
-  }
-
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/BatchSchema.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/BatchSchema.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/BatchSchema.java
index d3e4b23..05b1cc7 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/BatchSchema.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/BatchSchema.java
@@ -17,25 +17,19 @@
  ******************************************************************************/
 package org.apache.drill.exec.record;
 
-import java.util.Collections;
 import java.util.Iterator;
 import java.util.List;
 
-import org.apache.drill.common.expression.types.DataType;
-import org.apache.drill.common.physical.RecordField.ValueMode;
-import org.apache.drill.exec.exception.SchemaChangeException;
 import org.apache.drill.exec.record.vector.ValueVector;
 
-import com.carrotsearch.hppc.IntObjectOpenHashMap;
-import com.google.common.collect.Lists;
 
-public class BatchSchema implements Iterable<MaterializedField>{
+public class BatchSchema implements Iterable<MaterializedField> {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(BatchSchema.class);
-  
+
   private final List<MaterializedField> fields;
-  private final boolean hasSelectionVector;
-  
-  private BatchSchema(boolean hasSelectionVector, List<MaterializedField> fields) {
+  final boolean hasSelectionVector;
+
+  BatchSchema(boolean hasSelectionVector, List<MaterializedField> fields) {
     this.fields = fields;
     this.hasSelectionVector = hasSelectionVector;
   }
@@ -45,88 +39,16 @@ public class BatchSchema implements Iterable<MaterializedField>{
     return fields.iterator();
   }
 
-  public void addAnyField(short fieldId, boolean nullable, ValueMode mode){
-    addTypedField(fieldId, DataType.LATEBIND, nullable, mode, Void.class);
+  public static SchemaBuilder newBuilder() {
+    return new SchemaBuilder();
   }
-  
-  public void addTypedField(short fieldId, DataType type, boolean nullable, ValueMode mode, Class<?> valueClass){
-    fields.add(new MaterializedField(fieldId, type, nullable, mode, valueClass));
+
+  @Override
+  public String toString() {
+    return "BatchSchema [fields=" + fields + ", hasSelectionVector=" + hasSelectionVector + "]";
   }
+
   
   
-  /**
-   * Builder to build BatchSchema.  Can have a supporting expected object.  If the expected Schema object is defined, the builder will always check that this schema is a equal or more materialized version of the current schema.
-   */
-  public class BatchSchemaBuilder{
-    private IntObjectOpenHashMap<MaterializedField> fields = new IntObjectOpenHashMap<MaterializedField>();
-    private IntObjectOpenHashMap<MaterializedField> expectedFields = new IntObjectOpenHashMap<MaterializedField>();
-    
-    private boolean hasSelectionVector;
-    
-    public BatchSchemaBuilder(BatchSchema expected){
-      for(MaterializedField f: expected){
-        expectedFields.put(f.getFieldId(), f);
-      }
-      hasSelectionVector = expected.hasSelectionVector;
-    }
-    
-    public BatchSchemaBuilder(){
-    }
-    
-    
-    /**
-     * Add a field where we don't have type information.  In this case, DataType will be set to LATEBIND and valueClass will be set to null.
-     * @param fieldId The desired fieldId.  Should be unique for this BatchSchema.
-     * @param nullable Whether this field supports nullability.
-     * @param mode
-     * @throws SchemaChangeException
-     */
-    public void addLateBindField(short fieldId, boolean nullable, ValueMode mode) throws SchemaChangeException{
-      addTypedField(fieldId, DataType.LATEBIND, nullable, mode, Void.class);
-    }
-    
-    public void setSelectionVector(boolean hasSelectionVector){
-      this.hasSelectionVector = hasSelectionVector;
-    }
-    
-    private void setTypedField(short fieldId, DataType type, boolean nullable, ValueMode mode, Class<?> valueClass) throws SchemaChangeException{
-      MaterializedField f = new MaterializedField(fieldId, type, nullable, mode, valueClass);
-      if(expectedFields != null){
-        if(!expectedFields.containsKey(f.getFieldId())) throw new SchemaChangeException(String.format("You attempted to add a field for Id An attempt was made to add a duplicate fieldId to the schema.  The offending fieldId was %d", fieldId));
-        f.checkMaterialization(expectedFields.lget());
-      }
-      fields.put(f.getFieldId(), f);
-    }
-    
-    public void addTypedField(short fieldId, DataType type, boolean nullable, ValueMode mode, Class<?> valueClass) throws SchemaChangeException{
-      if(fields.containsKey(fieldId)) throw new SchemaChangeException(String.format("An attempt was made to add a duplicate fieldId to the schema.  The offending fieldId was %d", fieldId));
-      setTypedField(fieldId, type, nullable, mode, valueClass);
-    }
-    
-    public void replaceTypedField(short fieldId, DataType type, boolean nullable, ValueMode mode, Class<?> valueClass) throws SchemaChangeException{
-      if(!fields.containsKey(fieldId)) throw new SchemaChangeException(String.format("An attempt was made to replace a field in the schema, however the schema does not currently contain that field id.  The offending fieldId was %d", fieldId));
-      setTypedField(fieldId, type, nullable, mode, valueClass);
-    }
-    
-//    public void addVector(ValueVector<?> v){
-//      
-//    }
-//    
-//    public void replaceVector(ValueVector<?> oldVector, ValueVector<?> newVector){
-//      
-//    }
-    
-    
-    public BatchSchema buildAndClear() throws SchemaChangeException{
-      // check if any fields are unaccounted for.
-      
-      List<MaterializedField> fieldList = Lists.newArrayList();
-      for(MaterializedField f : fields.values){
-        if(f != null) fieldList.add(f);
-      }
-      Collections.sort(fieldList);
-      return new BatchSchema(this.hasSelectionVector, fieldList);
-    }
-  }
   
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/FragmentWritableBatch.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/FragmentWritableBatch.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/FragmentWritableBatch.java
new file mode 100644
index 0000000..c19065d
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/FragmentWritableBatch.java
@@ -0,0 +1,59 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.record;
+
+import io.netty.buffer.ByteBuf;
+
+import org.apache.drill.exec.proto.ExecProtos.FragmentHandle;
+import org.apache.drill.exec.proto.ExecProtos.FragmentRecordBatch;
+import org.apache.drill.exec.proto.UserBitShared.QueryId;
+
+public class FragmentWritableBatch{
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(FragmentWritableBatch.class);
+  
+  private final ByteBuf[] buffers;
+  private final FragmentRecordBatch header;
+  
+  public FragmentWritableBatch(boolean isLast, QueryId queryId, int sendMajorFragmentId, int sendMinorFragmentId, int receiveMajorFragmentId, int receiveMinorFragmentId, WritableBatch batch){
+    this.buffers = batch.getBuffers();
+    FragmentHandle handle = FragmentHandle //
+        .newBuilder() //
+        .setMajorFragmentId(receiveMajorFragmentId) //
+        .setMinorFragmentId(receiveMinorFragmentId) //
+        .setQueryId(queryId) //
+        .build();
+    this.header = FragmentRecordBatch //
+        .newBuilder() //
+        .setIsLastBatch(isLast) //
+        .setDef(batch.getDef()) //
+        .setHandle(handle) //
+        .setSendingMajorFragmentId(sendMajorFragmentId) //
+        .setSendingMinorFragmentId(sendMinorFragmentId) //
+        .build();
+  }
+
+  public ByteBuf[] getBuffers(){
+    return buffers;
+  }
+
+  public FragmentRecordBatch getHeader() {
+    return header;
+  }
+  
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/InvalidValueAccessor.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/InvalidValueAccessor.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/InvalidValueAccessor.java
index 403c7a3..d820e0e 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/InvalidValueAccessor.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/InvalidValueAccessor.java
@@ -17,7 +17,7 @@
  ******************************************************************************/
 package org.apache.drill.exec.record;
 
-import org.apache.drill.exec.exception.ExecutionSetupException;
+import org.apache.drill.common.exceptions.ExecutionSetupException;
 
 public class InvalidValueAccessor extends ExecutionSetupException{
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(InvalidValueAccessor.class);

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/MajorTypeSerDe.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/MajorTypeSerDe.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/MajorTypeSerDe.java
new file mode 100644
index 0000000..718396e
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/MajorTypeSerDe.java
@@ -0,0 +1,116 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.record;
+
+import java.io.IOException;
+
+import org.apache.drill.exec.proto.SchemaDefProtos.DataMode;
+import org.apache.drill.exec.proto.SchemaDefProtos.MajorType;
+import org.apache.drill.exec.proto.SchemaDefProtos.MinorType;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonIgnore;
+import com.fasterxml.jackson.annotation.JsonInclude;
+import com.fasterxml.jackson.annotation.JsonInclude.Include;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.core.JsonGenerationException;
+import com.fasterxml.jackson.core.JsonGenerator;
+import com.fasterxml.jackson.core.JsonParser;
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.DeserializationContext;
+import com.fasterxml.jackson.databind.SerializerProvider;
+import com.fasterxml.jackson.databind.deser.std.StdDeserializer;
+import com.fasterxml.jackson.databind.ser.std.StdSerializer;
+
+public class MajorTypeSerDe {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(MajorTypeSerDe.class);
+  
+  
+  public static class De extends StdDeserializer<MajorType> {
+
+    public De() {
+      super(MajorType.class);
+    }
+
+    @Override
+    public MajorType deserialize(JsonParser jp, DeserializationContext ctxt) throws IOException,
+        JsonProcessingException {
+      return jp.readValueAs(MajorTypeHolder.class).getMajorType();
+    }
+    
+    
+  }
+  
+  
+  public static class Se extends StdSerializer<MajorType> {
+
+    public Se() {
+      super(MajorType.class);
+    }
+
+    @Override
+    public void serialize(MajorType value, JsonGenerator jgen, SerializerProvider provider) throws IOException,
+        JsonGenerationException {
+      MajorTypeHolder holder = MajorTypeHolder.get(value);
+      jgen.writeObject(holder);
+    }
+    
+  }
+  
+  @JsonInclude(Include.NON_NULL)
+  public static class MajorTypeHolder{
+    @JsonProperty("type") public MinorType minorType;
+    public DataMode mode;
+    public Integer width;
+    public Integer precision;
+    public Integer scale;
+    
+    @JsonCreator
+    public MajorTypeHolder(@JsonProperty("type") MinorType minorType, @JsonProperty("mode") DataMode mode, @JsonProperty("width") Integer width, @JsonProperty("precision") Integer precision, @JsonProperty("scale") Integer scale) {
+      super();
+      this.minorType = minorType;
+      this.mode = mode;
+      this.width = width;
+      this.precision = precision;
+      this.scale = scale;
+    }
+
+    private MajorTypeHolder(){}
+    
+    @JsonIgnore
+    public MajorType getMajorType(){
+      MajorType.Builder b = MajorType.newBuilder();
+      b.setMode(mode);
+      b.setMinorType(minorType);
+      if(precision != null) b.setPrecision(precision);
+      if(width != null) b.setWidth(width);
+      if(scale != null) b.setScale(scale);
+      return b.build();
+    }
+    
+    public static MajorTypeHolder get(MajorType mt){
+      MajorTypeHolder h = new MajorTypeHolder();
+      h.minorType = mt.getMinorType();
+      h.mode = mt.getMode();
+      if(mt.hasPrecision()) h.precision = mt.getPrecision();
+      if(mt.hasScale()) h.scale = mt.getScale();
+      if(mt.hasWidth()) h.width = mt.getWidth();
+      return h;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/MaterializedField.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/MaterializedField.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/MaterializedField.java
index 2e941a2..09427ef 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/MaterializedField.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/MaterializedField.java
@@ -17,67 +17,152 @@
  ******************************************************************************/
 package org.apache.drill.exec.record;
 
-import org.apache.drill.common.expression.types.DataType;
-import org.apache.drill.common.physical.RecordField.ValueMode;
-import org.apache.drill.exec.exception.SchemaChangeException;
-
-public class MaterializedField implements Comparable<MaterializedField>{
-  private int fieldId;
-  private DataType type;
-  private boolean nullable;
-  private ValueMode mode;
-  private Class<?> valueClass;
+import java.util.Iterator;
+import java.util.List;
+
+import org.apache.drill.common.expression.PathSegment;
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.exec.physical.RecordField.ValueMode;
+import org.apache.drill.exec.proto.SchemaDefProtos.DataMode;
+import org.apache.drill.exec.proto.SchemaDefProtos.FieldDef;
+import org.apache.drill.exec.proto.SchemaDefProtos.MajorType;
+import org.apache.drill.exec.proto.SchemaDefProtos.NamePart;
+import org.apache.drill.exec.proto.SchemaDefProtos.NamePart.Type;
+import org.apache.drill.exec.record.vector.TypeHelper;
+
+public class MaterializedField implements Comparable<MaterializedField> {
+  private final FieldDef def;
+
+  public MaterializedField(FieldDef def) {
+    this.def = def;
+  }
+  
+  public static MaterializedField create(FieldDef def){
+    return new MaterializedField(def);
+  }
+  
+  public static MaterializedField create(SchemaPath path, int fieldId, int parentId, MajorType type) {
+    FieldDef.Builder b = FieldDef.newBuilder();
+    b.setFieldId(fieldId);
+    b.setMajorType(type);
+    addSchemaPathToFieldDef(path, b);
+    b.setParentId(parentId);
+    return create(b.build());
+  }
+
+  private static void addSchemaPathToFieldDef(SchemaPath path, FieldDef.Builder builder) {
+    for (PathSegment p = path.getRootSegment();; p = p.getChild()) {
+      NamePart.Builder b = NamePart.newBuilder();
+      if (p.isArray()) {
+        b.setType(Type.ARRAY);
+      } else {
+        b.setName(p.getNameSegment().getPath().toString());
+        b.setType(Type.NAME);
+      }
+      builder.addName(b.build());
+      if(p.isLastPath()) break;
+    }
+  }
+
+  public FieldDef getDef() {
+    return def;
+  }
   
-  public MaterializedField(int fieldId, DataType type, boolean nullable, ValueMode mode, Class<?> valueClass) {
-    super();
-    this.fieldId = fieldId;
-    this.type = type;
-    this.nullable = nullable;
-    this.mode = mode;
-    this.valueClass = valueClass;
+  public String getName(){
+    StringBuilder sb = new StringBuilder();
+    boolean first = true;
+    for(NamePart np : def.getNameList()){
+      if(np.getType() == Type.ARRAY){
+        sb.append("[]");
+      }else{
+        if(first){
+          first = false;
+        }else{
+          sb.append(".");
+        }
+        sb.append(np.getName());
+        
+      }
+    }
+    return sb.toString();
+  }
+
+  public int getWidth() {
+    return def.getMajorType().getWidth();
   }
 
   public int getFieldId() {
-    return fieldId;
+    return def.getFieldId();
   }
 
-  public DataType getType() {
-    return type;
+  public MajorType getType() {
+    return def.getMajorType();
   }
 
   public boolean isNullable() {
-    return nullable;
+    return def.getMajorType().getMode() == DataMode.OPTIONAL;
   }
 
-  public ValueMode getMode() {
-    return mode;
+  public DataMode getDataMode() {
+    return def.getMajorType().getMode();
   }
 
   public Class<?> getValueClass() {
-    return valueClass;
+    return TypeHelper.getValueVectorClass(getType().getMinorType(), getDataMode());
   }
 
-  private void check(String name, Object val1, Object expected) throws SchemaChangeException{
-    if(expected.equals(val1)) return;
-    throw new SchemaChangeException("Expected and actual field definitions don't match. Actual %s: %s, expected %s: %s", name, val1, name, expected);
-  }
-  
-  public void checkMaterialization(MaterializedField expected) throws SchemaChangeException{
-    if(this.type == expected.type || expected.type == DataType.LATEBIND) throw new SchemaChangeException("Expected and actual field definitions don't match. Actual DataType: %s, expected DataTypes: %s", this.type, expected.type);
-    if(expected.valueClass != null) check("valueClass", this.valueClass, expected.valueClass);
-    check("fieldId", this.fieldId, expected.fieldId);
-    check("nullability", this.nullable, expected.nullable);
-    check("valueMode", this.mode, expected.mode);
-  }
+  public boolean matches(SchemaPath path) {
+    Iterator<NamePart> iter = def.getNameList().iterator();
+    
+    for (PathSegment p = path.getRootSegment();; p = p.getChild()) {
+      if(p == null) break;
+      if (!iter.hasNext()) return false;
+      NamePart n = iter.next();
+
+      if (p.isArray()) {
+        if (n.getType() == Type.ARRAY) continue;
+        return false;
+      } else {
+        if (p.getNameSegment().getPath().equals(n.getName())) continue;
+        return false;
+      }
+      
+    }
+    // we've reviewed all path segments. confirm that we don't have any extra name parts.
+    return !iter.hasNext();
 
-  public MaterializedField getNullableVersion(Class<?> valueClass){
-    return new MaterializedField(fieldId, type, true, mode, valueClass);
   }
-  
+
+  // private void check(String name, Object val1, Object expected) throws SchemaChangeException{
+  // if(expected.equals(val1)) return;
+  // throw new
+  // SchemaChangeException("Expected and actual field definitions don't match. Actual %s: %s, expected %s: %s", name,
+  // val1, name, expected);
+  // }
+
+  // public void checkMaterialization(MaterializedField expected) throws SchemaChangeException{
+  // if(this.type == expected.type || expected.type == DataType.LATEBIND) throw new
+  // SchemaChangeException("Expected and actual field definitions don't match. Actual DataType: %s, expected DataTypes: %s",
+  // this.type, expected.type);
+  // if(expected.valueClass != null) check("valueClass", this.valueClass, expected.valueClass);
+  // check("fieldId", this.fieldId, expected.fieldId);
+  // check("nullability", this.nullable, expected.nullable);
+  // check("valueMode", this.mode, expected.mode);
+  // }
+  //
+  // public MaterializedField getNullableVersion(Class<?> valueClass){
+  // return new MaterializedField(path, fieldId, type, true, mode, valueClass);
+  // }
+
   @Override
   public int compareTo(MaterializedField o) {
-    return Integer.compare(this.fieldId, o.fieldId);
+    return Integer.compare(this.getFieldId(), o.getFieldId());
   }
-  
+
+  @Override
+  public String toString() {
+    return "MaterializedField [" + def.toString() + "]";
+  }
+
   
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/RawFragmentBatch.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/RawFragmentBatch.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/RawFragmentBatch.java
new file mode 100644
index 0000000..c244cea
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/RawFragmentBatch.java
@@ -0,0 +1,44 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.record;
+
+import io.netty.buffer.ByteBuf;
+
+import org.apache.drill.exec.proto.ExecProtos.FragmentRecordBatch;
+
+public class RawFragmentBatch {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(RawFragmentBatch.class);
+
+  final FragmentRecordBatch header;
+  final ByteBuf body;
+
+  public RawFragmentBatch(FragmentRecordBatch header, ByteBuf body) {
+    super();
+    this.header = header;
+    this.body = body;
+  }
+
+  public FragmentRecordBatch getHeader() {
+    return header;
+  }
+
+  public ByteBuf getBody() {
+    return body;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/RawFragmentBatchProvider.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/RawFragmentBatchProvider.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/RawFragmentBatchProvider.java
new file mode 100644
index 0000000..08b0e11
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/RawFragmentBatchProvider.java
@@ -0,0 +1,27 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.record;
+
+import org.apache.drill.exec.ops.FragmentContext;
+
+public interface RawFragmentBatchProvider {
+  
+  public RawFragmentBatch getNext();
+  public void kill(FragmentContext context);
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatch.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatch.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatch.java
index eca62bb..3e4ded2 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatch.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatch.java
@@ -34,8 +34,9 @@ public interface RecordBatch {
     NONE, // No more records were found.
     OK, // A new range of records have been provided.
     OK_NEW_SCHEMA, // A full collection of records
-    STOP // Informs parent nodes that the query has terminated. In this case, a consumer can consume their QueryContext
+    STOP, // Informs parent nodes that the query has terminated. In this case, a consumer can consume their QueryContext
          // to understand the current state of things.
+    NOT_YET // used by batches that haven't received incoming data yet.
   }
 
   /**
@@ -81,5 +82,11 @@ public interface RecordBatch {
    * @return An IterOutcome describing the result of the iteration.
    */
   public IterOutcome next();
+  
+  /**
+   * Get a writable version of this batch.  Takes over owernship of existing buffers.
+   * @return
+   */
+  public WritableBatch getWritableBatch();
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatchLoader.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatchLoader.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatchLoader.java
new file mode 100644
index 0000000..d990198
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatchLoader.java
@@ -0,0 +1,143 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.record;
+
+import io.netty.buffer.ByteBuf;
+
+import java.util.Iterator;
+import java.util.List;
+
+import org.apache.drill.exec.exception.SchemaChangeException;
+import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.proto.SchemaDefProtos.FieldDef;
+import org.apache.drill.exec.proto.UserBitShared.FieldMetadata;
+import org.apache.drill.exec.proto.UserBitShared.RecordBatchDef;
+import org.apache.drill.exec.record.vector.TypeHelper;
+import org.apache.drill.exec.record.vector.ValueVector;
+
+import com.carrotsearch.hppc.IntObjectOpenHashMap;
+import com.carrotsearch.hppc.cursors.IntObjectCursor;
+
+public class RecordBatchLoader implements Iterable<IntObjectCursor<ValueVector<?>>>{
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(RecordBatchLoader.class);
+
+  private IntObjectOpenHashMap<ValueVector<?>> vectors = new IntObjectOpenHashMap<ValueVector<?>>();
+  private final BufferAllocator allocator;
+  private int recordCount; 
+  private BatchSchema schema;
+  
+  public RecordBatchLoader(BufferAllocator allocator) {
+    super();
+    this.allocator = allocator;
+  }
+
+  /**
+   * Load a record batch from a single buffer.
+   * 
+   * @param def
+   *          The definition for the record batch.
+   * @param buf
+   *          The buffer that holds the data ssociated with the record batch
+   * @return Whether or not the schema changed since the previous load.
+   * @throws SchemaChangeException 
+   */
+  public boolean load(RecordBatchDef def, ByteBuf buf) throws SchemaChangeException {
+//    logger.debug("Loading record batch with def {} and data {}", def, buf);
+    this.recordCount = def.getRecordCount();
+    boolean schemaChanged = false;
+    
+    IntObjectOpenHashMap<ValueVector<?>> newVectors = new IntObjectOpenHashMap<ValueVector<?>>();
+
+    List<FieldMetadata> fields = def.getFieldList();
+    
+    int bufOffset = 0;
+    for (FieldMetadata fmd : fields) {
+      FieldDef fieldDef = fmd.getDef();
+      ValueVector<?> v = vectors.remove(fieldDef.getFieldId());
+      if (v != null) {
+        if (v.getField().getDef().equals(fieldDef)) {
+          v.setTo(fmd, buf.slice(bufOffset, fmd.getBufferLength()));
+          newVectors.put(fieldDef.getFieldId(), v);
+          continue;
+        } else {
+          v.close();
+          v = null;
+        }
+      }
+      // if we arrive here, either the metadata didn't match, or we didn't find a vector.
+      schemaChanged = true;
+      MaterializedField m = new MaterializedField(fieldDef);
+      v = TypeHelper.getNewVector(m, allocator);
+      v.setTo(fmd, buf.slice(bufOffset, fmd.getBufferLength()));
+      newVectors.put(fieldDef.getFieldId(), v);
+    }
+    
+    if(!vectors.isEmpty()){
+      schemaChanged = true;
+      for(IntObjectCursor<ValueVector<?>> cursor : newVectors){
+        cursor.value.close();
+      }
+      
+    }
+    
+    if(schemaChanged){
+      // rebuild the schema.
+      SchemaBuilder b = BatchSchema.newBuilder();
+      for(IntObjectCursor<ValueVector<?>> cursor : newVectors){
+        b.addField(cursor.value.getField());
+      }
+      b.setSelectionVector(false);
+      this.schema = b.build();
+    }
+    vectors = newVectors;
+    return schemaChanged;
+
+  }
+
+  @SuppressWarnings("unchecked")
+  public <T extends ValueVector<T>> T getValueVector(int fieldId, Class<T> clazz) throws InvalidValueAccessor {
+    ValueVector<?> v = vectors.get(fieldId);
+    assert v != null;
+    if (v.getClass() != clazz)
+      throw new InvalidValueAccessor(String.format(
+          "Failure while reading vector.  Expected vector class of %s but was holding vector class %s.",
+          clazz.getCanonicalName(), v.getClass().getCanonicalName()));
+    return (T) v;
+  }
+
+  public int getRecordCount() {
+    return recordCount;
+  }
+
+
+  public WritableBatch getWritableBatch(){
+    return WritableBatch.get(recordCount, vectors);
+  }
+
+  @Override
+  public Iterator<IntObjectCursor<ValueVector<?>>> iterator() {
+    return this.vectors.iterator();
+  }
+
+  public BatchSchema getSchema(){
+    return schema;
+  }
+
+  
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/SchemaBuilder.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/SchemaBuilder.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/SchemaBuilder.java
new file mode 100644
index 0000000..1e25b1a
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/SchemaBuilder.java
@@ -0,0 +1,127 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.record;
+
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.drill.common.expression.types.DataType;
+import org.apache.drill.exec.exception.SchemaChangeException;
+import org.apache.drill.exec.physical.RecordField.ValueMode;
+
+import com.carrotsearch.hppc.IntObjectOpenHashMap;
+import com.carrotsearch.hppc.cursors.ObjectCursor;
+import com.google.common.collect.Lists;
+
+/**
+ * A reusable builder that supports the creation of BatchSchemas. Can have a supporting expected object. If the expected Schema object is defined, the
+ * builder will always check that this schema is a equal or more materialized version of the current schema.
+ */
+public class SchemaBuilder {
+  private IntObjectOpenHashMap<MaterializedField> fields = new IntObjectOpenHashMap<MaterializedField>();
+  private IntObjectOpenHashMap<MaterializedField> expectedFields = new IntObjectOpenHashMap<MaterializedField>();
+
+  private boolean hasSelectionVector;
+
+  public SchemaBuilder(BatchSchema expected) {
+    for (MaterializedField f : expected) {
+      expectedFields.put(f.getFieldId(), f);
+    }
+    hasSelectionVector = expected.hasSelectionVector;
+  }
+
+  SchemaBuilder() {
+  }
+
+  /**
+   * Add a field where we don't have type information. In this case, DataType will be set to LATEBIND and valueClass
+   * will be set to null.
+   * 
+   * @param fieldId
+   *          The desired fieldId. Should be unique for this BatchSchema.
+   * @param nullable
+   *          Whether this field supports nullability.
+   * @param mode
+   * @throws SchemaChangeException
+   */
+//  public void addLateBindField(short fieldId, boolean nullable, ValueMode mode) throws SchemaChangeException {
+//    addTypedField(fieldId, DataType.LATEBIND, nullable, mode, Void.class);
+//  }
+
+  public void setSelectionVector(boolean hasSelectionVector) {
+    this.hasSelectionVector = hasSelectionVector;
+  }
+
+  
+//  private void setTypedField(short fieldId, DataType type, boolean nullable, ValueMode mode, Class<?> valueClass)
+//      throws SchemaChangeException {
+//    MaterializedField f = new MaterializedField(fieldId, type, nullable, mode, valueClass);
+//    if (expectedFields != null) {
+//      if (!expectedFields.containsKey(f.getFieldId()))
+//        throw new SchemaChangeException(
+//            String
+//                .format(
+//                    "You attempted to add a field for Id An attempt was made to add a duplicate fieldId to the schema.  The offending fieldId was %d",
+//                    fieldId));
+//      f.checkMaterialization(expectedFields.lget());
+//    }
+//    fields.put(f.getFieldId(), f);
+//  }
+  
+  public void addField(MaterializedField f){
+    fields.put(f.getFieldId(), f);
+  }
+
+//  public void addTypedField(short fieldId, DataType type, boolean nullable, ValueMode mode, Class<?> valueClass)
+//      throws SchemaChangeException {
+//    if (fields.containsKey(fieldId))
+//      throw new SchemaChangeException(String.format(
+//          "An attempt was made to add a duplicate fieldId to the schema.  The offending fieldId was %d", fieldId));
+//    setTypedField(fieldId, type, nullable, mode, valueClass);
+//  }
+//
+//  public void replaceTypedField(short fieldId, DataType type, boolean nullable, ValueMode mode, Class<?> valueClass)
+//      throws SchemaChangeException {
+//    if (!fields.containsKey(fieldId))
+//      throw new SchemaChangeException(
+//          String.format("An attempt was made to replace a field in the schema, however the schema does " +
+//          		"not currently contain that field id.  The offending fieldId was %d", fieldId));
+//    setTypedField(fieldId, type, nullable, mode, valueClass);
+//  }
+  
+  public void removeField(short fieldId) throws SchemaChangeException{
+    MaterializedField f = fields.remove(fieldId);
+    if(f == null) throw new SchemaChangeException("You attempted to remove an nonexistent field.");
+  }
+
+  /**
+   * Generate a new BatchSchema object based on the current state of the builder.
+   * @return
+   * @throws SchemaChangeException
+   */
+  public BatchSchema build() throws SchemaChangeException {
+    // check if any fields are unaccounted for.
+
+    List<MaterializedField> fieldList = Lists.newArrayList();
+    for (ObjectCursor<MaterializedField> f : fields.values()) {
+      if (f != null) fieldList.add(f.value);
+    }
+    Collections.sort(fieldList);
+    return new BatchSchema(this.hasSelectionVector, fieldList);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/WritableBatch.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/WritableBatch.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/WritableBatch.java
new file mode 100644
index 0000000..788c731
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/WritableBatch.java
@@ -0,0 +1,108 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.record;
+
+import io.netty.buffer.ByteBuf;
+
+import java.util.List;
+
+import org.apache.drill.exec.proto.UserBitShared.FieldMetadata;
+import org.apache.drill.exec.proto.UserBitShared.RecordBatchDef;
+import org.apache.drill.exec.record.vector.ValueVector;
+
+import com.carrotsearch.hppc.IntObjectOpenHashMap;
+import com.carrotsearch.hppc.procedures.IntObjectProcedure;
+import com.google.common.collect.Lists;
+
+/**
+ * A specialized version of record batch that can moves out buffers and preps them for writing. 
+ */
+public class WritableBatch {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(WritableBatch.class);
+  
+  private final RecordBatchDef def;
+  private final ByteBuf[] buffers;
+  
+  public WritableBatch(RecordBatchDef def, List<ByteBuf> buffers) {
+    logger.debug("Created new writable batch with def {} and buffers {}", def, buffers);
+    this.def = def;
+    this.buffers = buffers.toArray(new ByteBuf[buffers.size()]);
+  }
+  
+  public WritableBatch(RecordBatchDef def, ByteBuf[] buffers) {
+    super();
+    this.def = def;
+    this.buffers = buffers;
+  }
+  
+  
+  public RecordBatchDef getDef(){
+    return def;
+  }
+  public ByteBuf[] getBuffers(){
+    return buffers;
+  }
+  
+//  public static WritableBatch get(ValueVector<?>[] vectors){
+//    WritableCreator c = new WritableCreator();
+//    for(int i =0; i < vectors.length; i++){
+//      c.apply(i, vectors[i]);
+//    }
+//    return c.get();
+//  }
+//  
+  
+  public static WritableBatch get(int recordCount, IntObjectOpenHashMap<ValueVector<?>> fields){
+    WritableCreator creator = new WritableCreator(recordCount);
+    fields.forEach(creator);
+    return creator.get();
+    
+  }
+  
+  private static class WritableCreator implements IntObjectProcedure<ValueVector<?>>{
+    
+    List<ByteBuf> buffers = Lists.newArrayList();
+    List<FieldMetadata> metadata = Lists.newArrayList();
+    private int recordCount;
+    
+
+    public WritableCreator(int recordCount) {
+      super();
+      this.recordCount = recordCount;
+    }
+    
+    @Override
+    public void apply(int key, ValueVector<?> value) {
+      metadata.add(value.getMetadata());
+      for(ByteBuf b : value.getBuffers()){
+        buffers.add(b);
+        b.retain();
+      }
+      // allocate new buffer to release hold on old buffer.
+      value.allocateNew(value.capacity());
+    }
+
+
+    public WritableBatch get(){
+      RecordBatchDef batchDef = RecordBatchDef.newBuilder().addAllField(metadata).setRecordCount(recordCount).build();
+      WritableBatch b = new WritableBatch(batchDef, buffers);
+      return b;
+    }
+    
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/AbstractFixedValueVector.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/AbstractFixedValueVector.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/AbstractFixedValueVector.java
index 912e02d..b32f067 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/AbstractFixedValueVector.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/AbstractFixedValueVector.java
@@ -20,6 +20,8 @@ package org.apache.drill.exec.record.vector;
 import io.netty.buffer.ByteBuf;
 
 import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.proto.UserBitShared.FieldMetadata;
+import org.apache.drill.exec.record.MaterializedField;
 
 /**
  * Abstract class that fixed value vectors are derived from.
@@ -27,12 +29,12 @@ import org.apache.drill.exec.memory.BufferAllocator;
 abstract class AbstractFixedValueVector<T extends AbstractFixedValueVector<T>> extends BaseValueVector<T> {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(AbstractFixedValueVector.class);
 
-  private final int widthInBits;
+  protected final int widthInBits;
 
   protected int longWords = 0;
-
-  public AbstractFixedValueVector(int fieldId, BufferAllocator allocator, int widthInBits) {
-    super(fieldId, allocator);
+  
+  public AbstractFixedValueVector(MaterializedField field, BufferAllocator allocator, int widthInBits) {
+    super(field, allocator);
     this.widthInBits = widthInBits;
   }
   
@@ -56,5 +58,16 @@ abstract class AbstractFixedValueVector<T extends AbstractFixedValueVector<T>> e
     longWords = 0;
   }
 
+  @Override
+  public void setRecordCount(int recordCount) {
+    this.data.writerIndex(recordCount*(widthInBits/8));
+    super.setRecordCount(recordCount);
+  }
+
+
+
+
+
+  
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/BaseValueVector.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/BaseValueVector.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/BaseValueVector.java
index 8d524b2..b001add 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/BaseValueVector.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/BaseValueVector.java
@@ -19,20 +19,25 @@ package org.apache.drill.exec.record.vector;
 
 import io.netty.buffer.ByteBuf;
 
+import java.util.Random;
+
 import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.proto.UserBitShared.FieldMetadata;
 import org.apache.drill.exec.record.DeadBuf;
+import org.apache.drill.exec.record.MaterializedField;
 
 public abstract class BaseValueVector<T extends BaseValueVector<T>> implements ValueVector<T>{
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(BaseValueVector.class);
   
   protected final BufferAllocator allocator;
   protected ByteBuf data = DeadBuf.DEAD_BUFFER;
-  protected int valueCount = 0;
-  protected final int fieldId;
+  protected int maxValueCount = 0;
+  protected final MaterializedField field;
+  private int recordCount;
   
-  public BaseValueVector(int fieldId, BufferAllocator allocator) {
+  public BaseValueVector(MaterializedField field, BufferAllocator allocator) {
     this.allocator = allocator;
-    this.fieldId = fieldId;
+    this.field = field;
   }
 
   public final void allocateNew(int valueCount){
@@ -42,35 +47,42 @@ public abstract class BaseValueVector<T extends BaseValueVector<T>> implements V
     resetAllocation(valueCount, newBuf);
   }
 
-  protected abstract int getAllocationSize(int valueCount);
+  protected abstract int getAllocationSize(int maxValueCount);
   protected abstract void childResetAllocation(int valueCount, ByteBuf buf);
   protected abstract void childCloneMetadata(T other);
   protected abstract void childClear();
   
-  protected final void resetAllocation(int valueCount, ByteBuf buf){
+  /**
+   * Update the current buffer allocation utilize the provided allocation.
+   * @param valueCount
+   * @param buf
+   */
+  protected final void resetAllocation(int maxValueCount, ByteBuf buf){
     clear();
-    this.valueCount = valueCount;
+    buf.retain();
+    this.maxValueCount = maxValueCount;
     this.data = buf;
-    childResetAllocation(valueCount, buf);
+    childResetAllocation(maxValueCount, buf);
   }
   
   public final void cloneMetadata(T other){
-    other.valueCount = this.valueCount;
+    other.maxValueCount = this.maxValueCount;
   }
   
+  
   @Override
   public final void cloneInto(T vector) {
-    vector.allocateNew(valueCount);
+    vector.allocateNew(maxValueCount);
     data.writeBytes(vector.data);
     cloneMetadata(vector);
-    childResetAllocation(valueCount, vector.data);
+    childResetAllocation(maxValueCount, vector.data);
   }
   
   @Override
   public final void transferTo(T vector) {
     vector.data = this.data;
     cloneMetadata(vector);
-    childResetAllocation(valueCount, data);
+    childResetAllocation(maxValueCount, data);
     clear();
   }
 
@@ -78,7 +90,7 @@ public abstract class BaseValueVector<T extends BaseValueVector<T>> implements V
     if(this.data != DeadBuf.DEAD_BUFFER){
       this.data.release();
       this.data = DeadBuf.DEAD_BUFFER;
-      this.valueCount = 0;
+      this.maxValueCount = 0;
     }
     childClear();
   }
@@ -88,8 +100,8 @@ public abstract class BaseValueVector<T extends BaseValueVector<T>> implements V
    * 
    * @return
    */
-  public int size() {
-    return valueCount;
+  public int capacity() {
+    return maxValueCount;
   }
   
   @Override
@@ -98,8 +110,48 @@ public abstract class BaseValueVector<T extends BaseValueVector<T>> implements V
   }
 
   @Override
-  public ByteBuf getBuffer() {
-    return data;
+  public ByteBuf[] getBuffers() {
+    return new ByteBuf[]{data};
+  }
+  
+  public MaterializedField getField(){
+    return field;
+  }
+  
+  
+  public int getRecordCount() {
+    return recordCount;
+  }
+
+  public void setRecordCount(int recordCount) {
+    this.recordCount = recordCount;
+  }
+
+  @Override
+  public FieldMetadata getMetadata() {
+    int len = 0;
+    for(ByteBuf b : getBuffers()){
+      len += b.writerIndex();
+    }
+    return FieldMetadata.newBuilder().setDef(getField().getDef()).setValueCount(getRecordCount()).setBufferLength(len).build();
+  }
+  
+  @Override
+  public void setTo(FieldMetadata metadata, ByteBuf data) {
+//    logger.debug("Updating value vector to {}, {}", metadata, data);
+    clear();
+    resetAllocation(metadata.getValueCount(), data);
+  }
+
+  @Override
+  public void randomizeData() {
+    if(this.data != DeadBuf.DEAD_BUFFER){
+      Random r = new Random();
+      for(int i =0; i < data.capacity()-8; i+=8){
+        data.setLong(i, r.nextLong());
+      }
+    }
+    
   }
   
   

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/Bit.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/Bit.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/Bit.java
new file mode 100644
index 0000000..533e3bd
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/Bit.java
@@ -0,0 +1,168 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.record.vector;
+
+import org.apache.drill.common.expression.types.DataType;
+import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.physical.RecordField.ValueMode;
+import org.apache.drill.exec.record.MaterializedField;
+import org.apache.hadoop.io.SequenceFile;
+
+/**
+ * Describes a vector which holds a number of true/false values.
+ */
+public class Bit extends AbstractFixedValueVector<Bit> {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(Bit.class);
+
+  public Bit(MaterializedField field, BufferAllocator allocator) {
+    super(field, allocator, 1);
+  }
+
+  
+//  /** Returns true or false for the specified bit index.
+//   * The index should be less than the OpenBitSet size
+//   */
+//  public boolean get(int index) {
+//    assert index >= 0 && index < this.valueCount;
+//    int i = index >> 3;               // div 8
+//    // signed shift will keep a negative index and force an
+//    // array-index-out-of-bounds-exception, removing the need for an explicit check.
+//    int bit = index & 0x3f;           // mod 64
+//    long bitmask = 1L << bit;
+//    return (data.getLong(i) & bitmask) != 0;
+//  }
+  
+  public int getBit(int index) {
+    
+    assert index >= 0 && index < this.maxValueCount;
+    int i = 8*(index >> 6); // div 8
+    int bit = index & 0x3f; // mod 64
+    return ((int) (data.getLong(i) >>> bit)) & 0x01;
+  }
+  
+  /** Sets the bit at the specified index.
+   * The index should be less than the OpenBitSet size.
+   */
+   public void set(int index) {
+     assert index >= 0 && index < this.maxValueCount;
+     int wordNum = index >> 3;   
+     int bit = index & 0x3f;
+     long bitmask = 1L << bit;
+     data.setLong(wordNum, data.getLong(wordNum) | bitmask);
+   }
+   
+   public void clear(int index) {
+     assert index >= 0 && index < this.maxValueCount;
+     int wordNum = index >> 3;
+     int bit = index & 0x03f;
+     long bitmask = 1L << bit;
+     data.setLong(wordNum, data.getLong(wordNum) & ~bitmask);
+   }
+   
+   
+   
+   /** Clears a range of bits.  Clearing past the end does not change the size of the set.
+   *
+   * @param startBitIndex lower index
+   * @param lastBitIndex one-past the last bit to clear
+   */
+  private void clear2(int startBitIndex, int lastBitIndex) {
+    if (lastBitIndex <= startBitIndex) return;
+
+    int firstWordStart = (startBitIndex>>3);
+    if (firstWordStart >= this.longWords) return;
+
+    // since endIndex is one past the end, this is index of the last
+    // word to be changed.
+    int lastWordStart   = ((lastBitIndex-1)>>3);
+
+    long startmask = -1L << startBitIndex;
+    long endmask = -1L >>> -lastBitIndex;  // 64-(endIndex&0x3f) is the same as -endIndex due to wrap
+
+    // invert masks since we are clearing
+    startmask = ~startmask;
+    endmask = ~endmask;
+
+    if (firstWordStart == lastWordStart) {
+      data.setLong(firstWordStart,  data.getLong(firstWordStart) & (startmask | endmask));
+      return;
+    }
+    data.setLong(firstWordStart,  data.getLong(firstWordStart) & startmask);
+
+    int middle = Math.min(this.longWords, lastWordStart);
+    
+    for(int i =firstWordStart+8; i < middle; i += 8){
+      data.setLong(i, 0L);
+    }
+    if (lastWordStart < this.longWords) {
+      data.setLong(lastWordStart,  data.getLong(lastWordStart) & endmask);
+    }
+  }
+  
+  public void setAllFalse(){
+    clear(0, maxValueCount);
+  }
+
+  
+  public void clear(int startIndex, int endIndex) {
+    if (endIndex <= startIndex) return;
+
+    int startWord = (startIndex >> 6);
+    if (startWord >= longWords) return;
+
+    // since endIndex is one past the end, this is index of the last
+    // word to be changed.
+    int endWord = ((endIndex - 1) >> 6);
+
+    long startmask = -1L << startIndex;
+    long endmask = -1L >>> -endIndex; // 64-(endIndex&0x3f) is the same as -endIndex due to wrap
+
+    // invert masks since we are clearing
+    startmask = ~startmask;
+    endmask = ~endmask;
+    
+    int startWordPos = startWord * 8;
+    if (startWord == endWord) {
+      data.setLong(startWordPos, data.getLong(startWordPos) & (startmask | endmask));
+      return;
+    }
+
+    int endWordPos = endWord * 8;
+
+    data.setLong(startWordPos, data.getLong(startWordPos) & startmask);
+
+    int middle = Math.min(longWords, endWord)*8;
+    
+    
+    for(int i =startWordPos+8; i < middle; i += 8){
+      data.setLong(i, 0L);
+    }
+    
+    if (endWordPos < startWordPos) {
+      data.setLong(endWordPos, data.getLong(endWordPos) & endmask);
+    }
+  }
+
+
+  @Override
+  public Object getObject(int index) {
+    return this.getBit(index);
+  }
+  
+  
+}


[28/53] [abbrv] git commit: DRILL-75 and DRILL-76

Posted by ja...@apache.org.
DRILL-75 and DRILL-76


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

Branch: refs/heads/master
Commit: 01278ae56f0ac3d16ff5e05722055fee1332cd9a
Parents: 719c8be
Author: Jacques Nadeau <ja...@apache.org>
Authored: Mon Jun 17 09:10:15 2013 -0700
Committer: Jacques Nadeau <ja...@apache.org>
Committed: Mon Jun 17 09:10:15 2013 -0700

----------------------------------------------------------------------
 .../apache/drill/common/logical/data/Scan.java  |   4 +-
 .../org/apache/drill/exec/record/DeadBuf.java   |   2 +-
 .../record/vector/AbstractFixedValueVector.java |   8 +-
 .../exec/record/vector/BaseValueVector.java     |   5 +-
 .../apache/drill/exec/record/vector/Fixed4.java |  12 +-
 .../drill/exec/record/vector/NullableBit.java   |  20 +
 .../exec/record/vector/NullableFixed4.java      |  17 +-
 .../exec/record/vector/NullableValueVector.java |  12 +-
 .../exec/record/vector/NullableVarLen4.java     |  21 +
 .../drill/exec/record/vector/TypeHelper.java    |  33 +-
 .../drill/exec/record/vector/VarLen1.java       |   6 +-
 .../drill/exec/record/vector/VarLen2.java       |   6 +-
 .../drill/exec/record/vector/VarLen4.java       |  18 +-
 .../exec/record/vector/VariableVector.java      |  33 +-
 .../drill/exec/rpc/InboundRpcMessage.java       |   2 +-
 .../apache/drill/exec/schema/DiffSchema.java    |   1 +
 .../org/apache/drill/exec/schema/Field.java     |  74 +---
 .../apache/drill/exec/schema/IdGenerator.java   |  25 +-
 .../apache/drill/exec/schema/ListSchema.java    |   5 +-
 .../apache/drill/exec/schema/NamedField.java    |  21 +-
 .../apache/drill/exec/schema/OrderedField.java  |  19 +-
 .../drill/exec/schema/SchemaRecorder.java       | 122 ------
 .../exec/schema/json/jackson/JacksonHelper.java |  62 ++-
 .../schema/json/jackson/PhysicalOperator.java   |  36 --
 .../json/jackson/PhysicalOperatorIterator.java  |  31 --
 .../exec/schema/json/jackson/ScanJson.java      | 203 ---------
 .../transform/ProtobufSchemaTransformer.java    | 109 -----
 .../schema/transform/SchemaTransformer.java     |  30 --
 .../drill/exec/server/DrillbitContext.java      |   4 +-
 .../exec/store/BatchExceededException.java      |   9 +
 .../drill/exec/store/JSONRecordReader.java      | 423 +++++++++++++++++++
 .../apache/drill/exec/store/RecordReader.java   |   4 -
 .../apache/drill/exec/store/VectorHolder.java   |  59 +++
 .../drill/exec/store/JSONRecordReaderTest.java  | 290 +++++++++++++
 .../src/test/resources/scan_json_test_1.json    |  10 +
 .../src/test/resources/scan_json_test_2.json    |  21 +
 .../src/test/resources/scan_json_test_3.json    |  18 +
 .../src/test/resources/scan_json_test_4.json    |  14 +
 .../src/test/resources/scan_json_test_5.json    |  21 +
 .../src/test/resources/scan_json_test_6.json    |  14 +
 .../exec/ref/src/test/resources/donuts.json     |   7 +-
 .../ref/src/test/resources/simple_plan.json     |   1 +
 sandbox/prototype/pom.xml                       |  18 +-
 43 files changed, 1162 insertions(+), 688 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/01278ae5/sandbox/prototype/common/src/main/java/org/apache/drill/common/logical/data/Scan.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/logical/data/Scan.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/logical/data/Scan.java
index c8d396b..7933e40 100644
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/logical/data/Scan.java
+++ b/sandbox/prototype/common/src/main/java/org/apache/drill/common/logical/data/Scan.java
@@ -6,9 +6,9 @@
  * to you under the Apache License, Version 2.0 (the
  * "License"); you may not use this file except in compliance
  * with the License.  You may obtain a copy of the License at
- * 
+ *
  * http://www.apache.org/licenses/LICENSE-2.0
- * 
+ *
  * Unless required by applicable law or agreed to in writing, software
  * distributed under the License is distributed on an "AS IS" BASIS,
  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/01278ae5/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/DeadBuf.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/DeadBuf.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/DeadBuf.java
index dafb68c..0c2adcf 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/DeadBuf.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/DeadBuf.java
@@ -34,7 +34,7 @@ import io.netty.buffer.ByteBufIndexFinder;
 public class DeadBuf implements ByteBuf {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DeadBuf.class);
   
-  private static final String ERROR_MESSAGE = "Attemped to access a DeadBuf.  This would happen if you attempted to interact with a buffer that has been moved or not yet initialized.";
+  private static final String ERROR_MESSAGE = "Attemped to access a DeadBuf. This would happen if you attempted to interact with a buffer that has been moved or not yet initialized.";
   
   public static final DeadBuf DEAD_BUFFER = new DeadBuf();
 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/01278ae5/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/AbstractFixedValueVector.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/AbstractFixedValueVector.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/AbstractFixedValueVector.java
index b32f067..8748285 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/AbstractFixedValueVector.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/AbstractFixedValueVector.java
@@ -37,8 +37,12 @@ abstract class AbstractFixedValueVector<T extends AbstractFixedValueVector<T>> e
     super(field, allocator);
     this.widthInBits = widthInBits;
   }
-  
-  @Override
+
+    public int getWidthInBits() {
+        return widthInBits;
+    }
+
+    @Override
   protected int getAllocationSize(int valueCount) {
     return (int) Math.ceil(valueCount*widthInBits*1.0/8);
   }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/01278ae5/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/BaseValueVector.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/BaseValueVector.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/BaseValueVector.java
index b001add..69cd628 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/BaseValueVector.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/BaseValueVector.java
@@ -42,8 +42,7 @@ public abstract class BaseValueVector<T extends BaseValueVector<T>> implements V
 
   public final void allocateNew(int valueCount){
     int allocationSize = getAllocationSize(valueCount);
-    ByteBuf newBuf =  allocator.buffer(allocationSize);
-    newBuf.retain();
+    ByteBuf newBuf = allocator.buffer(allocationSize);
     resetAllocation(valueCount, newBuf);
   }
 
@@ -54,7 +53,7 @@ public abstract class BaseValueVector<T extends BaseValueVector<T>> implements V
   
   /**
    * Update the current buffer allocation utilize the provided allocation.
-   * @param valueCount
+   * @param maxValueCount
    * @param buf
    */
   protected final void resetAllocation(int maxValueCount, ByteBuf buf){

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/01278ae5/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/Fixed4.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/Fixed4.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/Fixed4.java
index 650029d..ff54784 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/Fixed4.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/Fixed4.java
@@ -18,6 +18,7 @@
 package org.apache.drill.exec.record.vector;
 
 import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.proto.SchemaDefProtos;
 import org.apache.drill.exec.record.MaterializedField;
 
 public class Fixed4 extends AbstractFixedValueVector<Fixed4>{
@@ -38,18 +39,21 @@ public class Fixed4 extends AbstractFixedValueVector<Fixed4>{
   }
   
   public final void setFloat4(int index, float value){
-    index*=8;
+    index*=4;
     data.setFloat(index, value);
   }
   
   public final float getFloat4(int index){
-    index*=8;
+    index*=4;
     return data.getFloat(index);
   }
   
   @Override
   public Object getObject(int index) {
-    return getInt(index);
+    if (field != null && field.getType().getMinorType() == SchemaDefProtos.MinorType.FLOAT4) {
+      return getFloat4(index);
+    } else {
+      return getInt(index);
+    }
   }
-  
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/01278ae5/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/NullableBit.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/NullableBit.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/NullableBit.java
new file mode 100644
index 0000000..ac8c6b6
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/NullableBit.java
@@ -0,0 +1,20 @@
+package org.apache.drill.exec.record.vector;
+
+import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.record.MaterializedField;
+
+public class NullableBit extends NullableValueVector<NullableBit, Bit>{
+    public NullableBit(MaterializedField field, BufferAllocator allocator) {
+        super(field, allocator);
+    }
+
+    @Override
+    protected Bit getNewValueVector(BufferAllocator allocator) {
+        return new Bit(this.field, allocator);
+    }
+
+    public void set(int index) {
+        setNotNull(index);
+        value.set(index);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/01278ae5/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/NullableFixed4.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/NullableFixed4.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/NullableFixed4.java
index cc18538..d09b04a 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/NullableFixed4.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/NullableFixed4.java
@@ -6,9 +6,9 @@
  * to you under the Apache License, Version 2.0 (the
  * "License"); you may not use this file except in compliance
  * with the License.  You may obtain a copy of the License at
- * 
+ *
  * http://www.apache.org/licenses/LICENSE-2.0
- * 
+ *
  * Unless required by applicable law or agreed to in writing, software
  * distributed under the License is distributed on an "AS IS" BASIS,
  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
@@ -24,14 +24,21 @@ public final class NullableFixed4 extends NullableValueVector<NullableFixed4, Fi
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(NullableFixed4.class);
 
   public NullableFixed4(MaterializedField field, BufferAllocator allocator) {
-    super(field, allocator, NullableFixed4.class);
+    super(field, allocator);
   }
 
   @Override
   protected Fixed4 getNewValueVector(BufferAllocator allocator) {
-    return new Fixed4(null, allocator);
+    return new Fixed4(this.field, allocator);
   }
 
+  public void setInt(int index, int newVal) {
+      setNotNull(index);
+      value.setInt(index, newVal);
+  }
 
-  
+  public void setFloat4(int index, float val) {
+      setNotNull(index);
+      value.setFloat4(index, val);
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/01278ae5/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/NullableValueVector.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/NullableValueVector.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/NullableValueVector.java
index 692ab87..8c3d7f6 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/NullableValueVector.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/NullableValueVector.java
@@ -32,7 +32,7 @@ abstract class NullableValueVector<T extends NullableValueVector<T, E>, E extend
   protected Bit bits;
   protected E value;
 
-  public NullableValueVector(MaterializedField field, BufferAllocator allocator, Class<T> valueClass) {
+  public NullableValueVector(MaterializedField field, BufferAllocator allocator) {
     super(field, allocator);
     bits = new Bit(null, allocator);
     value = getNewValueVector(allocator);
@@ -40,6 +40,14 @@ abstract class NullableValueVector<T extends NullableValueVector<T, E>, E extend
   
   protected abstract E getNewValueVector(BufferAllocator allocator);
 
+  public void setNull(int index) {
+      bits.set(index);
+  }
+
+  public void setNotNull(int index) {
+      bits.clear(index);
+  }
+
   public int isNull(int index){
     return bits.getBit(index);
   }
@@ -89,7 +97,7 @@ abstract class NullableValueVector<T extends NullableValueVector<T, E>, E extend
 
   @Override
   public Object getObject(int index) {
-    if(isNull(index) == 0){
+    if(isNull(index) == 1){
       return null;
     }else{
       return value.getObject(index);

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/01278ae5/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/NullableVarLen4.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/NullableVarLen4.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/NullableVarLen4.java
new file mode 100644
index 0000000..f5a2770
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/NullableVarLen4.java
@@ -0,0 +1,21 @@
+package org.apache.drill.exec.record.vector;
+
+import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.record.MaterializedField;
+
+public class NullableVarLen4 extends NullableValueVector<NullableVarLen4, VarLen4> {
+
+    public NullableVarLen4(MaterializedField field, BufferAllocator allocator) {
+        super(field, allocator);
+    }
+
+    @Override
+    protected VarLen4 getNewValueVector(BufferAllocator allocator) {
+        return new VarLen4(field, allocator);
+    }
+
+    public void setBytes(int index, byte[] bytes) {
+        setNotNull(index);
+        value.setBytes(index, bytes);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/01278ae5/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/TypeHelper.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/TypeHelper.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/TypeHelper.java
index 8e89c41..543a222 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/TypeHelper.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/TypeHelper.java
@@ -6,9 +6,9 @@
  * to you under the Apache License, Version 2.0 (the
  * "License"); you may not use this file except in compliance
  * with the License.  You may obtain a copy of the License at
- * 
+ *
  * http://www.apache.org/licenses/LICENSE-2.0
- * 
+ *
  * Unless required by applicable law or agreed to in writing, software
  * distributed under the License is distributed on an "AS IS" BASIS,
  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
@@ -25,11 +25,11 @@ import org.apache.drill.exec.record.MaterializedField;
 
 public class TypeHelper {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TypeHelper.class);
-  
+
   private static final int WIDTH_ESTIMATE_1 = 10;
   private static final int WIDTH_ESTIMATE_2 = 50000;
   private static final int WIDTH_ESTIMATE_4 = 1024*1024;
-  
+
   public static int getSize(MajorType major){
     switch(major.getMinorType()){
     case TINYINT: return 1;
@@ -65,11 +65,11 @@ public class TypeHelper {
     case PROTO2: return 2 + WIDTH_ESTIMATE_2;
     case PROTO4: return 4 + WIDTH_ESTIMATE_4;
     case MSGPACK2: return 2 + WIDTH_ESTIMATE_2;
-    case MSGPACK4: return 4 + WIDTH_ESTIMATE_4;    
+    case MSGPACK4: return 4 + WIDTH_ESTIMATE_4;
     }
     return 4;
   }
-  
+
   public static Class<?> getValueVectorClass(MinorType type, DataMode mode){
     switch(mode){
     case OPTIONAL:
@@ -189,12 +189,12 @@ public class TypeHelper {
       break;
     default:
       break;
-    
+
     }
     throw new UnsupportedOperationException();
   }
-  
-  
+
+
   public static ValueVector<?> getNewVector(MaterializedField field, BufferAllocator allocator){
     MajorType type = field.getType();
     switch(type.getMode()){
@@ -233,18 +233,27 @@ public class TypeHelper {
       case PROTO2: return new VarLen2(field, allocator);
       case PROTO4: return new VarLen4(field, allocator);
       case MSGPACK2: return new VarLen2(field, allocator);
-      case MSGPACK4: return new VarLen4(field, allocator);      
+      case MSGPACK4: return new VarLen4(field, allocator);
       }
       break;
     case REPEATED:
+        switch(type.getMinorType()) {
+            case MAP: return new RepeatMap(field, allocator);
+        }
       break;
     case OPTIONAL:
+        switch(type.getMinorType()) {
+            case BOOLEAN: return new NullableBit(field, allocator);
+            case INT: return new NullableFixed4(field, allocator);
+            case FLOAT4: return new NullableFixed4(field, allocator);
+            case VARCHAR4: return new NullableVarLen4(field, allocator);
+        }
       break;
     default:
       break;
-    
+
     }
-    throw new UnsupportedOperationException();
+    throw new UnsupportedOperationException(type.getMinorType() + " type is not supported. Mode: " + type.getMode());
   }
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/01278ae5/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/VarLen1.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/VarLen1.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/VarLen1.java
index d87029d..6c2b39e 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/VarLen1.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/VarLen1.java
@@ -24,13 +24,13 @@ public class VarLen1 extends VariableVector<VarLen1, Fixed1>{
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(VarLen1.class);
 
   public VarLen1(MaterializedField field, BufferAllocator allocator) {
-    super(field, allocator);
+    super(field, allocator, 1);
   }
 
   @Override
   protected Fixed1 getNewLengthVector(BufferAllocator allocator) {
     return new Fixed1(null, allocator);
   }
-  
-  
+
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/01278ae5/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/VarLen2.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/VarLen2.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/VarLen2.java
index ebd440a..3bcdd94 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/VarLen2.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/VarLen2.java
@@ -24,13 +24,13 @@ public class VarLen2 extends VariableVector<VarLen2, Fixed2>{
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(VarLen2.class);
 
   public VarLen2(MaterializedField field, BufferAllocator allocator) {
-    super(field, allocator);
+    super(field, allocator, 2);
   }
 
   @Override
   protected Fixed2 getNewLengthVector(BufferAllocator allocator) {
     return new Fixed2(null, allocator);
   }
-  
-  
+
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/01278ae5/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/VarLen4.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/VarLen4.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/VarLen4.java
index b3cd712..71ed7a1 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/VarLen4.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/VarLen4.java
@@ -20,17 +20,29 @@ package org.apache.drill.exec.record.vector;
 import org.apache.drill.exec.memory.BufferAllocator;
 import org.apache.drill.exec.record.MaterializedField;
 
+import static com.google.common.base.Preconditions.checkArgument;
+
 public class VarLen4 extends VariableVector<VarLen4, Fixed4>{
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(VarLen4.class);
 
   public VarLen4(MaterializedField field, BufferAllocator allocator) {
-    super(field, allocator);
+    super(field, allocator, 4);
   }
 
   @Override
   protected Fixed4 getNewLengthVector(BufferAllocator allocator) {
     return new Fixed4(null, allocator);
   }
-  
-  
+
+  public void setBytes(int index, byte[] bytes) {
+      checkArgument(index >= 0);
+      if(index == 0) {
+          lengthVector.setInt(0, bytes.length);
+          data.setBytes(0, bytes);
+      } else {
+          int previousOffset = lengthVector.getInt(index - 1);
+          lengthVector.setInt(index, previousOffset + bytes.length);
+          data.setBytes(previousOffset, bytes);
+      }
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/01278ae5/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/VariableVector.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/VariableVector.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/VariableVector.java
index 4247f14..dc8fec4 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/VariableVector.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/VariableVector.java
@@ -23,6 +23,9 @@ import org.apache.drill.exec.memory.BufferAllocator;
 import org.apache.drill.exec.record.DeadBuf;
 import org.apache.drill.exec.record.MaterializedField;
 
+import static com.google.common.base.Preconditions.checkArgument;
+import static com.google.common.base.Preconditions.checkState;
+
 /** 
  * A vector of variable length bytes.  Constructed as a vector of lengths or positions and a vector of values.  Random access is only possible if the variable vector stores positions as opposed to lengths.
  */
@@ -31,12 +34,12 @@ public abstract class VariableVector<T extends VariableVector<T, E>, E extends B
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(VariableVector.class);
   
   protected final E lengthVector;
-  private ByteBuf values = DeadBuf.DEAD_BUFFER;
   protected int expectedValueLength;
   
-  public VariableVector(MaterializedField field, BufferAllocator allocator) {
+  public VariableVector(MaterializedField field, BufferAllocator allocator, int expectedValueLength) {
     super(field, allocator);
     this.lengthVector = getNewLengthVector(allocator);
+    this.expectedValueLength = expectedValueLength;
   }
   
   protected abstract E getNewLengthVector(BufferAllocator allocator);
@@ -50,7 +53,7 @@ public abstract class VariableVector<T extends VariableVector<T, E>, E extends B
   protected void childResetAllocation(int valueCount, ByteBuf buf) {
     int firstSize = lengthVector.getAllocationSize(valueCount);
     lengthVector.resetAllocation(valueCount, buf.slice(0, firstSize));
-    values = buf.slice(firstSize, expectedValueLength * valueCount);
+    data = buf.slice(firstSize, expectedValueLength * valueCount);
   }
 
   @Override
@@ -62,16 +65,15 @@ public abstract class VariableVector<T extends VariableVector<T, E>, E extends B
   @Override
   protected void childClear() {
     lengthVector.clear();
-    if(values != DeadBuf.DEAD_BUFFER){
-      values.release();
-      values = DeadBuf.DEAD_BUFFER;
+    if(data != DeadBuf.DEAD_BUFFER){
+      data.release();
+      data = DeadBuf.DEAD_BUFFER;
     }
   }
 
-  
   @Override
   public ByteBuf[] getBuffers() {
-    return new ByteBuf[]{lengthVector.data, values};
+    return new ByteBuf[]{lengthVector.data, data};
   }
 
   @Override
@@ -81,13 +83,20 @@ public abstract class VariableVector<T extends VariableVector<T, E>, E extends B
   }  
   
   public void setTotalBytes(int totalBytes){
-    values.writerIndex(totalBytes);
+    data.writerIndex(totalBytes);
   }
 
   @Override
   public Object getObject(int index) {
-    return null;
+      checkArgument(index >= 0);
+      int startIdx = 0;
+      if(index > 0) {
+          startIdx = (int) lengthVector.getObject(index - 1);
+      }
+      int size = (int) lengthVector.getObject(index) - startIdx;
+      checkState(size >= 0);
+      byte[] dst = new byte[size];
+      data.getBytes(startIdx, dst, 0, size);
+      return dst;
   }
-  
-  
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/01278ae5/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/InboundRpcMessage.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/InboundRpcMessage.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/InboundRpcMessage.java
index be1ff6b..650a8ea 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/InboundRpcMessage.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/InboundRpcMessage.java
@@ -6,7 +6,7 @@
  * to you under the Apache License, Version 2.0 (the
  * "License"); you may not use this file except in compliance
  * with the License.  You may obtain a copy of the License at
- * 
+ *
  * http://www.apache.org/licenses/LICENSE-2.0
  * 
  * Unless required by applicable law or agreed to in writing, software

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/01278ae5/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/schema/DiffSchema.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/schema/DiffSchema.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/schema/DiffSchema.java
index 016e097..b654a92 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/schema/DiffSchema.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/schema/DiffSchema.java
@@ -19,6 +19,7 @@
 package org.apache.drill.exec.schema;
 
 import com.google.common.collect.Lists;
+import org.apache.drill.exec.physical.impl.OutputMutator;
 
 import java.util.List;
 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/01278ae5/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/schema/Field.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/schema/Field.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/schema/Field.java
index e19c099..eef0634 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/schema/Field.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/schema/Field.java
@@ -20,46 +20,47 @@ package org.apache.drill.exec.schema;
 
 import com.google.common.base.Objects;
 import com.google.common.base.Strings;
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.proto.SchemaDefProtos;
+import org.apache.drill.exec.record.MaterializedField;
+import org.apache.drill.exec.record.vector.*;
+import org.apache.drill.exec.store.BatchExceededException;
+import org.apache.drill.exec.store.VectorHolder;
+
+import java.nio.charset.Charset;
 
 import static com.google.common.base.Preconditions.checkArgument;
 import static com.google.common.base.Preconditions.checkState;
+import static org.apache.drill.exec.proto.SchemaDefProtos.*;
 
 public abstract class Field {
-    final FieldType fieldType;
-    int fieldId;
-    String prefixFieldName;
-    String fieldName;
+    final MajorType fieldType;
+    final int parentFieldId;
+    final int fieldId;
+    final String prefixFieldName;
     RecordSchema schema;
     RecordSchema parentSchema;
     boolean read;
 
-
-    public Field(RecordSchema parentSchema, IdGenerator<Integer> generator, FieldType fieldType, String prefixFieldName) {
+    public Field(RecordSchema parentSchema, int parentFieldId, IdGenerator<Integer> generator, MajorType type, String prefixFieldName) {
         this.fieldId = generator.getNextId();
-        this.fieldType = fieldType;
+        fieldType = type;
         this.prefixFieldName = prefixFieldName;
         this.parentSchema = parentSchema;
+        this.parentFieldId = parentFieldId;
     }
 
-    public Field assignSchema(RecordSchema newSchema) {
-        checkState(schema == null, "Schema already assigned to field: %s", fieldName);
-        checkState(fieldType.isEmbedSchema(), "Schema cannot be assigned to non-embedded types: %s", fieldType);
-        schema = newSchema;
-        return this;
-    }
+    public abstract String getFieldName();
 
     public String getFullFieldName() {
-        return Strings.isNullOrEmpty(prefixFieldName) ? fieldName : prefixFieldName + "." + fieldName;
+        return Strings.isNullOrEmpty(prefixFieldName) ? getFieldName() : prefixFieldName + "." + getFieldName();
     }
 
     public int getFieldId() {
         return fieldId;
     }
 
-    public String getFieldName() {
-        return fieldName;
-    }
-
     public void setRead(boolean read) {
         this.read = read;
     }
@@ -78,18 +79,10 @@ public abstract class Field {
         return addAttributesToHelper(getAttributesStringHelper()).toString();
     }
 
-    public RecordSchema getParentSchema() {
-        return parentSchema;
-    }
-
     public RecordSchema getAssignedSchema() {
         return schema;
     }
 
-    public FieldType getFieldType() {
-        return fieldType;
-    }
-
     public void assignSchemaIfNull(RecordSchema newSchema) {
         if (!hasSchema()) {
             schema = newSchema;
@@ -104,32 +97,7 @@ public abstract class Field {
         return schema != null;
     }
 
-    public static enum FieldType {
-        INTEGER(1),
-        FLOAT(2),
-        BOOLEAN(3),
-        STRING(4),
-        ARRAY(5, true),
-        MAP(6, true);
-
-        byte value;
-        boolean embedSchema;
-
-        FieldType(int value, boolean embedSchema) {
-            this.value = (byte) value;
-            this.embedSchema = embedSchema;
-        }
-
-        FieldType(int value) {
-            this(value, false);
-        }
-
-        public byte value() {
-            return value;
-        }
-
-        public boolean isEmbedSchema() {
-            return embedSchema;
-        }
+    public MajorType getFieldType() {
+        return fieldType;
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/01278ae5/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/schema/IdGenerator.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/schema/IdGenerator.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/schema/IdGenerator.java
index 728e8e1..d5de4ab 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/schema/IdGenerator.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/schema/IdGenerator.java
@@ -1,12 +1,23 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+
 package org.apache.drill.exec.schema;
 
-/**
- * Created with IntelliJ IDEA.
- * User: tnachen
- * Date: 1/2/13
- * Time: 10:50 PM
- * To change this template use File | Settings | File Templates.
- */
 public interface IdGenerator<T> {
     public T getNextId();
     public void reset();

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/01278ae5/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/schema/ListSchema.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/schema/ListSchema.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/schema/ListSchema.java
index efdc8fd..e5bd1a4 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/schema/ListSchema.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/schema/ListSchema.java
@@ -21,6 +21,7 @@ package org.apache.drill.exec.schema;
 import com.google.common.base.Predicate;
 import com.google.common.collect.Iterables;
 import com.google.common.collect.Lists;
+import org.apache.drill.exec.proto.SchemaDefProtos;
 
 import java.util.List;
 
@@ -36,7 +37,7 @@ public class ListSchema implements RecordSchema {
 
     @Override
     public void addField(Field field) {
-        if (field.fieldType.isEmbedSchema() || fields.isEmpty() || !isSingleTyped() ||
+        if (field.getFieldType().getMode() == SchemaDefProtos.DataMode.REPEATED || fields.isEmpty() || !isSingleTyped() ||
                 !Iterables.getOnlyElement(fields).equals(field.getFieldType())) {
             fields.add(field);
         }
@@ -96,7 +97,7 @@ public class ListSchema implements RecordSchema {
                 if (!field.isRead()) {
                     removedFields.add(field);
                     return true;
-                } else if(field.hasSchema()) {
+                } else if (field.hasSchema()) {
                     Iterables.addAll(removedFields, field.getAssignedSchema().removeUnreadFields());
                 }
 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/01278ae5/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/schema/NamedField.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/schema/NamedField.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/schema/NamedField.java
index aa0d6aa..b975ad7 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/schema/NamedField.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/schema/NamedField.java
@@ -19,18 +19,27 @@
 package org.apache.drill.exec.schema;
 
 import com.google.common.base.Objects;
+import org.apache.drill.exec.proto.SchemaDefProtos;
+import org.apache.drill.exec.schema.json.jackson.JacksonHelper;
 
 public class NamedField extends Field {
-    final FieldType keyType;
+    final SchemaDefProtos.MajorType keyType;
+    String fieldName;
 
-    public NamedField(RecordSchema parentSchema, IdGenerator<Integer> generator, String prefixFieldName, String fieldName, Field.FieldType fieldType) {
-        this(parentSchema, generator, prefixFieldName, fieldName, fieldType, FieldType.STRING);
+    public NamedField(RecordSchema parentSchema, int parentFieldId, IdGenerator<Integer> generator, String prefixFieldName, String fieldName, SchemaDefProtos.MajorType fieldType) {
+        this(parentSchema, parentFieldId, generator, prefixFieldName, fieldName, fieldType, JacksonHelper.STRING_TYPE);
     }
 
-    public NamedField(RecordSchema parentSchema, IdGenerator<Integer> generator, String prefixFieldName, String fieldName, Field.FieldType fieldType, FieldType keyType) {
-        super(parentSchema, generator, fieldType, prefixFieldName);
+    public NamedField(RecordSchema parentSchema,
+                      int parentFieldId,
+                      IdGenerator<Integer> generator,
+                      String prefixFieldName,
+                      String fieldName,
+                      SchemaDefProtos.MajorType fieldType,
+                      SchemaDefProtos.MajorType keyType) {
+        super(parentSchema, parentFieldId, generator, fieldType, prefixFieldName);
         this.fieldName = fieldName;
-        this.keyType = FieldType.STRING;
+        this.keyType = keyType;
     }
 
     public String getFieldName() {

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/01278ae5/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/schema/OrderedField.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/schema/OrderedField.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/schema/OrderedField.java
index 67fd2fa..5f514af 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/schema/OrderedField.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/schema/OrderedField.java
@@ -19,11 +19,24 @@
 package org.apache.drill.exec.schema;
 
 import com.google.common.base.Objects;
+import org.apache.drill.exec.proto.SchemaDefProtos;
 
 public class OrderedField extends Field {
-    public OrderedField(RecordSchema parentSchema, IdGenerator<Integer> generator, FieldType fieldType, String prefixFieldName, int index) {
-        super(parentSchema, generator, fieldType, prefixFieldName);
-        this.fieldName = "[" + index + "]";
+    private final int index;
+
+    public OrderedField(RecordSchema parentSchema,
+                        int parentFieldId,
+                        IdGenerator<Integer> generator,
+                        SchemaDefProtos.MajorType type,
+                        String prefixFieldName,
+                        int index) {
+        super(parentSchema, parentFieldId, generator, type, prefixFieldName);
+        this.index = index;
+    }
+
+    @Override
+    public String getFieldName() {
+        return "[" + index + "]";
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/01278ae5/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/schema/SchemaRecorder.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/schema/SchemaRecorder.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/schema/SchemaRecorder.java
deleted file mode 100644
index 54a4e0e..0000000
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/schema/SchemaRecorder.java
+++ /dev/null
@@ -1,122 +0,0 @@
-/*******************************************************************************
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- ******************************************************************************/
-
-package org.apache.drill.exec.schema;
-
-import com.fasterxml.jackson.core.JsonParser;
-import com.google.common.collect.Iterables;
-import com.google.common.collect.Lists;
-import org.apache.drill.exec.schema.json.jackson.JacksonHelper;
-import org.apache.drill.exec.schema.json.jackson.ScanJson;
-
-import java.io.IOException;
-import java.util.List;
-
-public class SchemaRecorder {
-    DiffSchema diffSchema;
-    RecordSchema currentSchema;
-    List<Field> removedFields;
-
-    public SchemaRecorder() {
-        currentSchema = new ObjectSchema();
-        diffSchema = new DiffSchema();
-        removedFields = Lists.newArrayList();
-    }
-
-    public RecordSchema getCurrentSchema() {
-        return currentSchema;
-    }
-
-    public void recordData(ScanJson.ReadType currentReadType, ScanJson.ReadType readType, JsonParser parser, IdGenerator generator, DataRecord record, Field.FieldType fieldType, String prefixFieldName, String fieldName, int index) throws IOException {
-        Field field = currentSchema.getField(fieldName, index);
-
-        if (field == null || field.getFieldType() != fieldType) {
-            if (field != null) {
-                removeStaleField(index, field);
-            }
-            field = currentReadType.createField(currentSchema, generator, prefixFieldName, fieldName, fieldType, index);
-            field.setRead(true);
-            diffSchema.recordNewField(field);
-            currentSchema.addField(field);
-        } else {
-            field.setRead(true);
-        }
-
-        if (readType != null) {
-            RecordSchema origSchema = currentSchema;
-            if (field != null) {
-                currentSchema = field.getAssignedSchema();
-            }
-
-            RecordSchema newSchema = readType.createSchema();
-            field.assignSchemaIfNull(newSchema);
-            setCurrentSchemaIfNull(newSchema);
-            readType.readRecord(parser, generator, this, record, field.getFullFieldName());
-
-            currentSchema = origSchema;
-        } else {
-            RecordSchema schema = field.getParentSchema();
-            record.addData(field.getFieldId(), JacksonHelper.getValueFromFieldType(parser, fieldType), schema != null && schema instanceof ListSchema);
-        }
-    }
-
-    private void removeStaleField(int index, Field field) {
-        if (field.hasSchema()) {
-            removeChildFields(field);
-        }
-        removedFields.add(field);
-        currentSchema.removeField(field, index);
-    }
-
-    private void removeChildFields(Field field) {
-        RecordSchema schema = field.getAssignedSchema();
-        if(schema == null) { return; }
-        for (Field childField : schema.getFields()) {
-            removedFields.add(childField);
-            if (childField.hasSchema()) {
-                removeChildFields(childField);
-            }
-        }
-    }
-
-    public boolean hasDiffs() {
-        return diffSchema.hasDiffFields();
-    }
-
-    public DiffSchema getDiffSchema() {
-        return hasDiffs() ? diffSchema : null;
-    }
-
-    public void setCurrentSchemaIfNull(RecordSchema newSchema) {
-        if (currentSchema == null) {
-            currentSchema = newSchema;
-        }
-    }
-
-    public void reset() {
-        currentSchema.resetMarkedFields();
-        diffSchema.reset();
-        removedFields.clear();
-    }
-
-    public void addMissingFields() {
-        for (Field field : Iterables.concat(currentSchema.removeUnreadFields(), removedFields)) {
-            diffSchema.addRemovedField(field);
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/01278ae5/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/schema/json/jackson/JacksonHelper.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/schema/json/jackson/JacksonHelper.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/schema/json/jackson/JacksonHelper.java
index 0643710..64a9d58 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/schema/json/jackson/JacksonHelper.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/schema/json/jackson/JacksonHelper.java
@@ -20,42 +20,58 @@ package org.apache.drill.exec.schema.json.jackson;
 
 import com.fasterxml.jackson.core.JsonParser;
 import com.fasterxml.jackson.core.JsonToken;
-import com.google.common.collect.Maps;
-
-import org.apache.drill.exec.schema.Field;
+import org.apache.drill.exec.proto.SchemaDefProtos;
 
 import java.io.IOException;
-import java.util.EnumMap;
 
-import static com.google.common.base.Preconditions.checkNotNull;
+import static org.apache.drill.exec.proto.SchemaDefProtos.MajorType;
+import static org.apache.drill.exec.proto.SchemaDefProtos.MinorType;
 
 public class JacksonHelper {
-    private static final EnumMap<JsonToken, Field.FieldType> TYPE_LOOKUP = Maps.newEnumMap(JsonToken.class);
-
-    static {
-        TYPE_LOOKUP.put(JsonToken.VALUE_STRING, Field.FieldType.STRING);
-        TYPE_LOOKUP.put(JsonToken.VALUE_FALSE, Field.FieldType.BOOLEAN);
-        TYPE_LOOKUP.put(JsonToken.VALUE_TRUE, Field.FieldType.BOOLEAN);
-        TYPE_LOOKUP.put(JsonToken.START_ARRAY, Field.FieldType.ARRAY);
-        TYPE_LOOKUP.put(JsonToken.START_OBJECT, Field.FieldType.MAP);
-        TYPE_LOOKUP.put(JsonToken.VALUE_NUMBER_INT, Field.FieldType.INTEGER);
-        TYPE_LOOKUP.put(JsonToken.VALUE_NUMBER_FLOAT, Field.FieldType.FLOAT);
-    }
 
-    public static Field.FieldType getFieldType(JsonToken token) {
-        return TYPE_LOOKUP.get(token);
+    public static final MajorType STRING_TYPE = MajorType.newBuilder().setMinorType(MinorType.VARCHAR4).setMode(SchemaDefProtos.DataMode.OPTIONAL).build();
+    public static final MajorType BOOLEAN_TYPE = MajorType.newBuilder().setMinorType(MinorType.BOOLEAN).setMode(SchemaDefProtos.DataMode.OPTIONAL).build();
+    public static final MajorType ARRAY_TYPE = MajorType.newBuilder().setMinorType(MinorType.LATE).setMode(SchemaDefProtos.DataMode.REPEATED).build();
+    public static final MajorType MAP_TYPE = MajorType.newBuilder().setMinorType(MinorType.MAP).setMode(SchemaDefProtos.DataMode.REPEATED).build();
+    public static final MajorType INT_TYPE = MajorType.newBuilder().setMinorType(MinorType.INT).setMode(SchemaDefProtos.DataMode.OPTIONAL).build();
+    public static final MajorType FLOAT_TYPE = MajorType.newBuilder().setMinorType(MinorType.FLOAT4).setMode(SchemaDefProtos.DataMode.OPTIONAL).build();
+    public static final MajorType NULL_TYPE = MajorType.newBuilder().setMinorType(MinorType.LATE).setMode(SchemaDefProtos.DataMode.OPTIONAL).build();
+
+    public static MajorType getFieldType(JsonToken token) {
+        switch(token) {
+            case VALUE_STRING:
+                return STRING_TYPE;
+            case VALUE_FALSE:
+                return BOOLEAN_TYPE;
+            case VALUE_TRUE:
+                return BOOLEAN_TYPE;
+            case START_ARRAY:
+                return ARRAY_TYPE;
+            case START_OBJECT:
+                return MAP_TYPE;
+            case VALUE_NUMBER_INT:
+                return INT_TYPE;
+            case VALUE_NUMBER_FLOAT:
+                return FLOAT_TYPE;
+            case VALUE_NULL:
+                return NULL_TYPE;
+        }
+
+        throw new UnsupportedOperationException("Unsupported Jackson type: " + token);
     }
 
-    public static Object getValueFromFieldType(JsonParser parser, Field.FieldType fieldType) throws IOException {
-        switch(fieldType) {
-            case INTEGER:
+    public static Object getValueFromFieldType(JsonParser parser, MinorType fieldType) throws IOException {
+        switch (fieldType) {
+            case INT:
                 return parser.getIntValue();
-            case STRING:
+            case VARCHAR4:
                 return parser.getValueAsString();
-            case FLOAT:
+            case FLOAT4:
                 return parser.getFloatValue();
             case BOOLEAN:
                 return parser.getBooleanValue();
+            case LATE:
+                return null;
             default:
                 throw new RuntimeException("Unexpected Field type to return value: " + fieldType.toString());
         }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/01278ae5/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/schema/json/jackson/PhysicalOperator.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/schema/json/jackson/PhysicalOperator.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/schema/json/jackson/PhysicalOperator.java
deleted file mode 100644
index e450ee9..0000000
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/schema/json/jackson/PhysicalOperator.java
+++ /dev/null
@@ -1,36 +0,0 @@
-/*******************************************************************************
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- ******************************************************************************/
-
-package org.apache.drill.exec.schema.json.jackson;
-
-import com.google.common.collect.Lists;
-
-import org.apache.drill.exec.schema.Record;
-
-import java.io.IOException;
-import java.util.List;
-
-public abstract class PhysicalOperator {
-    List<PhysicalOperatorIterator> parents;
-
-    public PhysicalOperator(PhysicalOperatorIterator... parents) {
-        this.parents = Lists.newArrayList(parents);
-    }
-
-    public abstract PhysicalOperatorIterator getIterator();
-}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/01278ae5/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/schema/json/jackson/PhysicalOperatorIterator.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/schema/json/jackson/PhysicalOperatorIterator.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/schema/json/jackson/PhysicalOperatorIterator.java
deleted file mode 100644
index bf4053e..0000000
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/schema/json/jackson/PhysicalOperatorIterator.java
+++ /dev/null
@@ -1,31 +0,0 @@
-/*******************************************************************************
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- ******************************************************************************/
-
-package org.apache.drill.exec.schema.json.jackson;
-
-import org.apache.drill.exec.schema.Record;
-
-import java.io.IOException;
-
-public interface PhysicalOperatorIterator{
-    public enum NextOutcome {NONE_LEFT, INCREMENTED_SCHEMA_UNCHANGED, INCREMENTED_SCHEMA_CHANGED}
-    public Record getRecord();
-    public NextOutcome next() throws IOException;
-}
-
-

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/01278ae5/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/schema/json/jackson/ScanJson.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/schema/json/jackson/ScanJson.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/schema/json/jackson/ScanJson.java
deleted file mode 100644
index a1c30e9..0000000
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/schema/json/jackson/ScanJson.java
+++ /dev/null
@@ -1,203 +0,0 @@
-/*******************************************************************************
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- ******************************************************************************/
-
-package org.apache.drill.exec.schema.json.jackson;
-
-import com.fasterxml.jackson.core.JsonFactory;
-import com.fasterxml.jackson.core.JsonParser;
-import com.fasterxml.jackson.core.JsonToken;
-import com.google.common.base.Charsets;
-import com.google.common.collect.Maps;
-import com.google.common.io.Files;
-import com.google.common.io.InputSupplier;
-import com.google.common.io.Resources;
-
-import org.apache.drill.exec.schema.*;
-
-import java.io.*;
-import java.util.Map;
-
-public class ScanJson extends PhysicalOperator {
-    private ScanJsonIterator iterator;
-
-    private static final Map<JsonToken, ReadType> READ_TYPES = Maps.newHashMap();
-
-    static {
-        READ_TYPES.put(JsonToken.START_ARRAY, ReadType.ARRAY);
-        READ_TYPES.put(JsonToken.START_OBJECT, ReadType.OBJECT);
-    }
-
-    public ScanJson(String inputName) throws IOException {
-        super();
-        this.iterator = new ScanJsonIterator(inputName);
-    }
-
-    @Override
-    public PhysicalOperatorIterator getIterator() {
-        return iterator;
-    }
-
-    class ScanJsonIterator implements PhysicalOperatorIterator {
-        private JsonParser parser;
-        private SchemaRecorder recorder;
-        private BackedRecord record;
-        private IdGenerator generator;
-
-        private ScanJsonIterator(String inputName) throws IOException {
-            InputSupplier<InputStreamReader> input;
-            if (inputName.startsWith("resource:")) {
-                input = Resources.newReaderSupplier(Resources.getResource(inputName.substring(inputName.indexOf(':') + 1)), Charsets.UTF_8);
-            } else {
-                input = Files.newReaderSupplier(new File(inputName), Charsets.UTF_8);
-            }
-
-            JsonFactory factory = new JsonFactory();
-            parser = factory.createJsonParser(input.getInput());
-            parser.nextToken(); // Read to the first START_OBJECT token
-            recorder = new SchemaRecorder();
-            generator = new SchemaIdGenerator();
-        }
-
-        @Override
-        public Record getRecord() {
-            return record;
-        }
-
-        @Override
-        public NextOutcome next() throws IOException {
-            if (parser.isClosed() || !parser.hasCurrentToken()) {
-                return NextOutcome.NONE_LEFT;
-            }
-
-            recorder.reset();
-
-            DataRecord dataRecord = new DataRecord();
-            ReadType.OBJECT.readRecord(parser, generator, recorder, dataRecord, null);
-
-            parser.nextToken(); // Read to START_OBJECT token
-
-            if (!parser.hasCurrentToken()) {
-                parser.close();
-            }
-
-            recorder.addMissingFields();
-            if (record == null) {
-                record = new BackedRecord(recorder.getDiffSchema(), dataRecord);
-            } else {
-                record.setBackend(recorder.getDiffSchema(), dataRecord);
-            }
-            return recorder.hasDiffs() ? NextOutcome.INCREMENTED_SCHEMA_CHANGED : NextOutcome.INCREMENTED_SCHEMA_UNCHANGED;
-        }
-
-        public RecordSchema getCurrentSchema() {
-            return recorder.getCurrentSchema();
-        }
-    }
-
-    public static enum ReadType {
-        ARRAY(JsonToken.END_ARRAY) {
-            @Override
-            public Field createField(RecordSchema parentSchema, IdGenerator<Integer> generator, String prefixFieldName, String fieldName, Field.FieldType fieldType, int index) {
-                return new OrderedField(parentSchema, generator, fieldType, prefixFieldName, index);
-            }
-
-            @Override
-            public RecordSchema createSchema() throws IOException {
-                return new ListSchema();
-            }
-        },
-        OBJECT(JsonToken.END_OBJECT) {
-            @Override
-            public Field createField(RecordSchema parentSchema, IdGenerator<Integer> generator, String prefixFieldName, String fieldName, Field.FieldType fieldType, int index) {
-                return new NamedField(parentSchema, generator, prefixFieldName, fieldName, fieldType);
-            }
-
-            @Override
-            public RecordSchema createSchema() throws IOException {
-                return new ObjectSchema();
-            }
-        };
-
-        private final JsonToken endObject;
-
-        ReadType(JsonToken endObject) {
-            this.endObject = endObject;
-        }
-
-        public JsonToken getEndObject() {
-            return endObject;
-        }
-
-        public void readRecord(JsonParser parser, IdGenerator generator, SchemaRecorder recorder, DataRecord record, String prefixFieldName) throws IOException {
-            JsonToken token = parser.nextToken();
-            JsonToken endObject = getEndObject();
-            int index = 0;
-            while (token != endObject) {
-                if (token == JsonToken.FIELD_NAME) {
-                    token = parser.nextToken();
-                    continue;
-                }
-
-                String fieldName = parser.getCurrentName();
-                Field.FieldType fieldType = JacksonHelper.getFieldType(token);
-                ReadType readType = READ_TYPES.get(token);
-                if (fieldType != null) { // Including nulls
-                    recorder.recordData(this, readType, parser, generator, record, fieldType, prefixFieldName, fieldName, index);
-                }
-                token = parser.nextToken();
-                ++index;
-            }
-        }
-
-        public abstract RecordSchema createSchema() throws IOException;
-
-        public abstract Field createField(RecordSchema parentSchema, IdGenerator<Integer> generator, String prefixFieldName, String fieldName, Field.FieldType fieldType, int index);
-    }
-
-    public static void main(String[] args) throws IOException {
-        if (args.length != 1) {
-            System.err.println("Requires json path: ScanJson <json_path>");
-            return;
-        }
-
-        String jsonPath = args[0];
-
-        System.out.println("Reading json input...");
-        ScanJson sj = new ScanJson(jsonPath);
-        ScanJsonIterator iterator = (ScanJsonIterator) sj.getIterator();
-        long count = 0;
-
-        while (iterator.next() != PhysicalOperatorIterator.NextOutcome.NONE_LEFT) {
-            Record record = iterator.getRecord();
-            System.out.println("Record " + ++count);
-            System.out.println("Schema: ");
-            System.out.println(iterator.getCurrentSchema().toSchemaString());
-            System.out.println();
-            System.out.println("Changes since last record: ");
-            System.out.println();
-            System.out.println(record.getSchemaChanges());
-            System.out.println();
-        }
-    }
-}
-
-
-
-
-
-

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/01278ae5/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/schema/transform/ProtobufSchemaTransformer.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/schema/transform/ProtobufSchemaTransformer.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/schema/transform/ProtobufSchemaTransformer.java
deleted file mode 100644
index a81a9d9..0000000
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/schema/transform/ProtobufSchemaTransformer.java
+++ /dev/null
@@ -1,109 +0,0 @@
-/*******************************************************************************
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- ******************************************************************************/
-
-package org.apache.drill.exec.schema.transform;
-
-import com.google.common.base.Function;
-import com.google.common.collect.Iterables;
-import com.google.common.collect.Lists;
-import com.google.common.collect.Maps;
-import com.google.common.io.ByteArrayDataOutput;
-import com.google.common.io.ByteStreams;
-import com.google.protobuf.DescriptorProtos;
-
-import org.apache.drill.exec.schema.Field;
-import org.apache.drill.exec.schema.ListSchema;
-import org.apache.drill.exec.schema.ObjectSchema;
-import org.apache.drill.exec.schema.RecordSchema;
-
-import java.util.EnumMap;
-import java.util.List;
-import java.util.Map;
-
-import static com.google.common.base.Preconditions.checkArgument;
-import static com.google.common.base.Preconditions.checkNotNull;
-
-public class ProtobufSchemaTransformer implements SchemaTransformer<DescriptorProtos.DescriptorProto> {
-    private static final Map<Field.FieldType, Function<Field, Object>> FIELD_MAP = Maps.newEnumMap(Field.FieldType.class);
-    private static final Map<Field.FieldType, DescriptorProtos.FieldDescriptorProto.Type> TYPE_MAP = Maps.newEnumMap(Field.FieldType.class);
-    private int fieldIndex = 0;
-    public static final String LIST_SCHEMA_NAME = "_EmbeddedList"; //Read from config?
-
-    static {
-        TYPE_MAP.put(Field.FieldType.BOOLEAN, DescriptorProtos.FieldDescriptorProto.Type.TYPE_BOOL);
-        TYPE_MAP.put(Field.FieldType.STRING, DescriptorProtos.FieldDescriptorProto.Type.TYPE_STRING);
-        TYPE_MAP.put(Field.FieldType.FLOAT, DescriptorProtos.FieldDescriptorProto.Type.TYPE_FLOAT);
-        TYPE_MAP.put(Field.FieldType.INTEGER, DescriptorProtos.FieldDescriptorProto.Type.TYPE_INT32);
-    }
-
-    private DescriptorProtos.DescriptorProto.Builder transformSchema(String name, DescriptorProtos.DescriptorProto.Builder parentBuilder, RecordSchema schema) {
-        if (schema instanceof ObjectSchema) {
-            return addObjectSchema(name, parentBuilder, ObjectSchema.class.cast(schema));
-        } else if (schema instanceof ListSchema) {
-            return addListSchema(name, ListSchema.class.cast(schema));
-        } else {
-            throw new RuntimeException("Unknown schema passed to transformer: " + schema);
-        }
-    }
-
-    public DescriptorProtos.DescriptorProto transformSchema(String name, RecordSchema schema) {
-        return transformSchema(name, null, schema).build();
-    }
-
-    private DescriptorProtos.DescriptorProto.Builder addListSchema(String name, ListSchema schema) {
-        DescriptorProtos.DescriptorProto.Builder builder = DescriptorProtos.DescriptorProto.newBuilder().setName(name);
-        DescriptorProtos.FieldDescriptorProto.Builder builderForValue = DescriptorProtos.FieldDescriptorProto.newBuilder();
-        builderForValue.setTypeName(LIST_SCHEMA_NAME).setType(DescriptorProtos.FieldDescriptorProto.Type.TYPE_MESSAGE);
-        builder.addField(builderForValue);
-        return builder;
-    }
-
-    private DescriptorProtos.DescriptorProto.Builder addObjectSchema(String name,
-                                                                     DescriptorProtos.DescriptorProto.Builder parentBuilder,
-                                                                     ObjectSchema schema) {
-        DescriptorProtos.DescriptorProto.Builder builder = DescriptorProtos.DescriptorProto.newBuilder().setName(name);
-        for (Field field : schema.getFields()) {
-            DescriptorProtos.FieldDescriptorProto.Builder builderForValue = DescriptorProtos.FieldDescriptorProto.newBuilder();
-            String fieldName = field.getFieldName();
-            builderForValue.setName(fieldName).setLabel(DescriptorProtos.FieldDescriptorProto.Label.LABEL_OPTIONAL).setNumber(++fieldIndex);
-            if (field.hasSchema()) {
-                RecordSchema innerSchema = field.getAssignedSchema();
-                if (innerSchema instanceof ObjectSchema) {
-                    addObjectSchema(fieldName, builder, (ObjectSchema) innerSchema);
-                    DescriptorProtos.DescriptorProto innerProto = Iterables.getLast(builder.getNestedTypeList());
-                    builderForValue.setTypeName(innerProto.getName()).setType(DescriptorProtos.FieldDescriptorProto.Type.TYPE_MESSAGE);
-                } else if (innerSchema instanceof ListSchema) {
-                    builderForValue.setTypeName(LIST_SCHEMA_NAME).setType(DescriptorProtos.FieldDescriptorProto.Type.TYPE_MESSAGE);
-                }
-            } else {
-                builderForValue.setType(getProtoType(field.getFieldType()));
-            }
-            builder.addField(builderForValue);
-        }
-
-        if (parentBuilder != null) {
-            parentBuilder.addNestedType(builder);
-        }
-
-        return builder;
-    }
-
-    private DescriptorProtos.FieldDescriptorProto.Type getProtoType(Field.FieldType fieldType) {
-        return checkNotNull(TYPE_MAP.get(fieldType));
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/01278ae5/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/schema/transform/SchemaTransformer.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/schema/transform/SchemaTransformer.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/schema/transform/SchemaTransformer.java
deleted file mode 100644
index 54d851f..0000000
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/schema/transform/SchemaTransformer.java
+++ /dev/null
@@ -1,30 +0,0 @@
-/*******************************************************************************
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- ******************************************************************************/
-
-package org.apache.drill.exec.schema.transform;
-
-import org.apache.drill.exec.schema.Field;
-import org.apache.drill.exec.schema.ListSchema;
-import org.apache.drill.exec.schema.ObjectSchema;
-import org.apache.drill.exec.schema.RecordSchema;
-
-import java.util.List;
-
-public interface SchemaTransformer<T> {
-    public T transformSchema(String name, RecordSchema schema);
-}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/01278ae5/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/server/DrillbitContext.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/server/DrillbitContext.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/server/DrillbitContext.java
index d5aaab2..c933594 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/server/DrillbitContext.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/server/DrillbitContext.java
@@ -51,10 +51,8 @@ public class DrillbitContext {
     super();
     Preconditions.checkNotNull(endpoint);
     Preconditions.checkNotNull(context);
-    Preconditions.checkNotNull(context);
-    Preconditions.checkNotNull(com);
     Preconditions.checkNotNull(com);
-    
+
     this.context = context;
     this.coord = coord;
     this.com = com;

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/01278ae5/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/BatchExceededException.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/BatchExceededException.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/BatchExceededException.java
new file mode 100644
index 0000000..d47caf7
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/BatchExceededException.java
@@ -0,0 +1,9 @@
+package org.apache.drill.exec.store;
+
+import org.apache.drill.common.exceptions.DrillRuntimeException;
+
+public class BatchExceededException extends DrillRuntimeException {
+    public BatchExceededException(int capacity, int attempted) {
+        super("Batch exceeded in size. Capacity: " + capacity + ", Attempted: " + attempted);
+    }
+}


[27/53] [abbrv] DRILL-75 and DRILL-76

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/01278ae5/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/JSONRecordReader.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/JSONRecordReader.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/JSONRecordReader.java
new file mode 100644
index 0000000..42d1be5
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/JSONRecordReader.java
@@ -0,0 +1,423 @@
+package org.apache.drill.exec.store;
+
+import com.carrotsearch.hppc.IntObjectOpenHashMap;
+import com.carrotsearch.hppc.cursors.IntObjectCursor;
+import com.carrotsearch.hppc.cursors.ObjectCursor;
+import com.fasterxml.jackson.core.JsonFactory;
+import com.fasterxml.jackson.core.JsonParser;
+import com.fasterxml.jackson.core.JsonToken;
+import com.google.common.base.Charsets;
+import com.google.common.collect.Iterables;
+import com.google.common.collect.Lists;
+import com.google.common.io.Files;
+import com.google.common.io.InputSupplier;
+import com.google.common.io.Resources;
+import org.apache.drill.common.exceptions.DrillRuntimeException;
+import org.apache.drill.common.exceptions.ExecutionSetupException;
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.exec.exception.SchemaChangeException;
+import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.physical.impl.OutputMutator;
+import org.apache.drill.exec.proto.SchemaDefProtos;
+import org.apache.drill.exec.record.MaterializedField;
+import org.apache.drill.exec.record.vector.*;
+import org.apache.drill.exec.schema.*;
+import org.apache.drill.exec.schema.json.jackson.JacksonHelper;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.nio.charset.Charset;
+import java.util.List;
+
+import static com.fasterxml.jackson.core.JsonToken.*;
+
+public class JSONRecordReader implements RecordReader {
+    static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(JSONRecordReader.class);
+    private static final int DEFAULT_LENGTH = 256 * 1024; // 256kb
+    public static final Charset UTF_8 = Charset.forName("UTF-8");
+
+    private final String inputPath;
+
+    private final IntObjectOpenHashMap<VectorHolder> valueVectorMap;
+
+    private JsonParser parser;
+    private SchemaIdGenerator generator;
+    private DiffSchema diffSchema;
+    private RecordSchema currentSchema;
+    private List<Field> removedFields;
+    private OutputMutator outputMutator;
+    private BufferAllocator allocator;
+    private int batchSize;
+
+    public JSONRecordReader(FragmentContext fragmentContext, String inputPath, int batchSize) {
+        this.inputPath = inputPath;
+        this.allocator = fragmentContext.getAllocator();
+        this.batchSize = batchSize;
+        valueVectorMap = new IntObjectOpenHashMap<>();
+    }
+
+    public JSONRecordReader(FragmentContext fragmentContext, String inputPath) {
+        this(fragmentContext, inputPath, DEFAULT_LENGTH);
+    }
+
+    private JsonParser getParser() {
+        return parser;
+    }
+
+    @Override
+    public void setup(OutputMutator output) throws ExecutionSetupException {
+        outputMutator = output;
+        currentSchema = new ObjectSchema();
+        diffSchema = new DiffSchema();
+        removedFields = Lists.newArrayList();
+
+        try {
+            InputSupplier<InputStreamReader> input;
+            if (inputPath.startsWith("resource:")) {
+                input = Resources.newReaderSupplier(Resources.getResource(inputPath.substring(9)), Charsets.UTF_8);
+            } else {
+                input = Files.newReaderSupplier(new File(inputPath), Charsets.UTF_8);
+            }
+
+            JsonFactory factory = new JsonFactory();
+            parser = factory.createJsonParser(input.getInput());
+            parser.nextToken(); // Read to the first START_OBJECT token
+            generator = new SchemaIdGenerator();
+        } catch (IOException e) {
+            throw new ExecutionSetupException(e);
+        }
+    }
+
+    @Override
+    public int next() {
+        if (parser.isClosed() || !parser.hasCurrentToken()) {
+            return 0;
+        }
+
+        resetBatch();
+
+        int nextRowIndex = 0;
+
+        try {
+            while (ReadType.OBJECT.readRecord(null, this, null, nextRowIndex++)) {
+                parser.nextToken(); // Read to START_OBJECT token
+
+                if (!parser.hasCurrentToken()) {
+                    parser.close();
+                    break;
+                }
+            }
+
+            parser.nextToken();
+
+            if (!parser.hasCurrentToken()) {
+                parser.close();
+            }
+
+            // Garbage collect fields never referenced in this batch
+            for (Field field : Iterables.concat(currentSchema.removeUnreadFields(), removedFields)) {
+                diffSchema.addRemovedField(field);
+                outputMutator.removeField(field.getFieldId());
+            }
+
+        } catch (IOException | SchemaChangeException e) {
+            logger.error("Error reading next in Json reader", e);
+        }
+        return nextRowIndex;
+    }
+
+    private void resetBatch() {
+        for (ObjectCursor<VectorHolder> holder : valueVectorMap.values()) {
+            holder.value.reset();
+        }
+
+        currentSchema.resetMarkedFields();
+        diffSchema.reset();
+        removedFields.clear();
+    }
+
+    @Override
+    public void cleanup() {
+        try {
+            parser.close();
+        } catch (IOException e) {
+            logger.warn("Error closing Json parser", e);
+        }
+    }
+
+    private SchemaIdGenerator getGenerator() {
+        return generator;
+    }
+
+    private RecordSchema getCurrentSchema() {
+        return currentSchema;
+    }
+
+    private void setCurrentSchema(RecordSchema schema) {
+        currentSchema = schema;
+    }
+
+    private List<Field> getRemovedFields() {
+        return removedFields;
+    }
+
+    private DiffSchema getDiffSchema() {
+        return diffSchema;
+    }
+
+    public BufferAllocator getAllocator() {
+        return allocator;
+    }
+
+    public OutputMutator getOutputMutator() {
+        return outputMutator;
+    }
+
+    public static enum ReadType {
+        ARRAY(END_ARRAY) {
+            @Override
+            public Field createField(RecordSchema parentSchema, int parentFieldId, IdGenerator<Integer> generator, String prefixFieldName, String fieldName, SchemaDefProtos.MajorType fieldType, int index) {
+                return new OrderedField(parentSchema, parentFieldId, generator, fieldType, prefixFieldName, index);
+            }
+
+            @Override
+            public RecordSchema createSchema() throws IOException {
+                return new ListSchema();
+            }
+        },
+        OBJECT(END_OBJECT) {
+            @Override
+            public Field createField(RecordSchema parentSchema,
+                                     int parentFieldId,
+                                     IdGenerator<Integer> generator,
+                                     String prefixFieldName,
+                                     String fieldName,
+                                     SchemaDefProtos.MajorType fieldType,
+                                     int index) {
+                return new NamedField(parentSchema, parentFieldId, generator, prefixFieldName, fieldName, fieldType);
+            }
+
+            @Override
+            public RecordSchema createSchema() throws IOException {
+                return new ObjectSchema();
+            }
+        };
+
+        private final JsonToken endObject;
+
+        ReadType(JsonToken endObject) {
+            this.endObject = endObject;
+        }
+
+        public JsonToken getEndObject() {
+            return endObject;
+        }
+
+        public boolean readRecord(Field parentField,
+                                  JSONRecordReader reader,
+                                  String prefixFieldName,
+                                  int rowIndex) throws IOException, SchemaChangeException {
+            JsonParser parser = reader.getParser();
+            JsonToken token = parser.nextToken();
+            JsonToken endObject = getEndObject();
+            int colIndex = 0;
+            boolean isFull = false;
+            while (token != endObject) {
+                if (token == FIELD_NAME) {
+                    token = parser.nextToken();
+                    continue;
+                }
+
+                String fieldName = parser.getCurrentName();
+                SchemaDefProtos.MajorType fieldType = JacksonHelper.getFieldType(token);
+                ReadType readType = null;
+                switch (token) {
+                    case START_ARRAY:
+                        readType = ReadType.ARRAY;
+                        break;
+                    case START_OBJECT:
+                        readType = ReadType.OBJECT;
+                        break;
+                }
+                if (fieldType != null) { // Including nulls
+                    isFull = isFull ||
+                            !recordData(
+                                    parentField,
+                                    readType,
+                                    reader,
+                                    fieldType,
+                                    prefixFieldName,
+                                    fieldName,
+                                    rowIndex, colIndex);
+                }
+                token = parser.nextToken();
+                colIndex += 1;
+            }
+            return !isFull;
+        }
+
+        private void removeChildFields(List<Field> removedFields, Field field) {
+            RecordSchema schema = field.getAssignedSchema();
+            if (schema == null) {
+                return;
+            }
+            for (Field childField : schema.getFields()) {
+                removedFields.add(childField);
+                if (childField.hasSchema()) {
+                    removeChildFields(removedFields, childField);
+                }
+            }
+        }
+
+        private boolean recordData(Field parentField,
+                                   JSONRecordReader.ReadType readType,
+                                   JSONRecordReader reader,
+                                   SchemaDefProtos.MajorType fieldType,
+                                   String prefixFieldName,
+                                   String fieldName,
+                                   int rowIndex,
+                                   int colIndex) throws IOException, SchemaChangeException {
+            RecordSchema currentSchema = reader.getCurrentSchema();
+            Field field = currentSchema.getField(fieldName, colIndex);
+            boolean isFieldFound = field != null;
+            List<Field> removedFields = reader.getRemovedFields();
+            int parentFieldId = parentField == null ? 0 : parentField.getFieldId();
+            if (!isFieldFound || !field.getFieldType().equals(fieldType)) {
+                if (isFieldFound) {
+                    if (field.hasSchema()) {
+                        removeChildFields(removedFields, field);
+                    }
+                    removedFields.add(field);
+                    currentSchema.removeField(field, colIndex);
+                }
+
+                field = createField(
+                        currentSchema,
+                        parentFieldId,
+                        reader.getGenerator(),
+                        prefixFieldName,
+                        fieldName,
+                        fieldType,
+                        colIndex
+                );
+
+                reader.recordNewField(field);
+                currentSchema.addField(field);
+            }
+
+            field.setRead(true);
+
+            VectorHolder holder = getOrCreateVectorHolder(reader, field, parentFieldId);
+            if (readType != null) {
+                RecordSchema fieldSchema = field.getAssignedSchema();
+                reader.setCurrentSchema(fieldSchema);
+
+                RecordSchema newSchema = readType.createSchema();
+                field.assignSchemaIfNull(newSchema);
+
+                if (fieldSchema == null) reader.setCurrentSchema(newSchema);
+                readType.readRecord(field, reader, field.getFullFieldName(), rowIndex);
+
+                reader.setCurrentSchema(currentSchema);
+            } else {
+                return addValueToVector(
+                        rowIndex,
+                        holder,
+                        reader.getAllocator(),
+                        JacksonHelper.getValueFromFieldType(
+                                reader.getParser(),
+                                fieldType.getMinorType()
+                        ),
+                        fieldType.getMinorType()
+                );
+            }
+
+            return true;
+        }
+
+        private static <T> boolean addValueToVector(int index, VectorHolder holder, BufferAllocator allocator, T val, SchemaDefProtos.MinorType minorType) {
+            switch (minorType) {
+                case INT: {
+                    holder.incAndCheckLength(32);
+                    NullableFixed4 fixed4 = (NullableFixed4) holder.getValueVector();
+                    if (val == null) {
+                        fixed4.setNull(index);
+                    } else {
+                        fixed4.setInt(index, (Integer) val);
+                    }
+                    return holder.hasEnoughSpace(32);
+                }
+                case FLOAT4: {
+                    holder.incAndCheckLength(32);
+                    NullableFixed4 fixed4 = (NullableFixed4) holder.getValueVector();
+                    if (val == null) {
+                        fixed4.setNull(index);
+                    } else {
+                        fixed4.setFloat4(index, (Float) val);
+                    }
+                    return holder.hasEnoughSpace(32);
+                }
+                case VARCHAR4: {
+                    if (val == null) {
+                        ((NullableVarLen4) holder.getValueVector()).setNull(index);
+                        return (index + 1) * 4 <= holder.getLength();
+                    } else {
+                        byte[] bytes = ((String) val).getBytes(UTF_8);
+                        int length = bytes.length * 8;
+                        holder.incAndCheckLength(length);
+                        NullableVarLen4 varLen4 = (NullableVarLen4) holder.getValueVector();
+                        varLen4.setBytes(index, bytes);
+                        return holder.hasEnoughSpace(length);
+                    }
+                }
+                case BOOLEAN: {
+                    holder.incAndCheckLength(1);
+                    NullableBit bit = (NullableBit) holder.getValueVector();
+                    if (val == null) {
+                        bit.setNull(index);
+                    } else if ((Boolean) val) {
+                        bit.set(index);
+                    }
+                    return holder.hasEnoughSpace(1);
+                }
+                default:
+                    throw new DrillRuntimeException("Type not supported to add value. Type: " + minorType);
+            }
+        }
+
+        private VectorHolder getOrCreateVectorHolder(JSONRecordReader reader, Field field, int parentFieldId) throws SchemaChangeException {
+            return reader.getOrCreateVectorHolder(field, parentFieldId);
+        }
+
+        public abstract RecordSchema createSchema() throws IOException;
+
+        public abstract Field createField(RecordSchema parentSchema,
+                                          int parentFieldId,
+                                          IdGenerator<Integer> generator,
+                                          String prefixFieldName,
+                                          String fieldName,
+                                          SchemaDefProtos.MajorType fieldType,
+                                          int index);
+    }
+
+    private void recordNewField(Field field) {
+        diffSchema.recordNewField(field);
+    }
+
+    private VectorHolder getOrCreateVectorHolder(Field field, int parentFieldId) throws SchemaChangeException {
+        if (!valueVectorMap.containsKey(field.getFieldId())) {
+            SchemaDefProtos.MajorType type = field.getFieldType();
+            int fieldId = field.getFieldId();
+            MaterializedField f = MaterializedField.create(new SchemaPath(field.getFieldName()), fieldId, parentFieldId, type);
+            ValueVector<?> v = TypeHelper.getNewVector(f, allocator);
+            v.allocateNew(batchSize);
+            VectorHolder holder = new VectorHolder(batchSize, v);
+            valueVectorMap.put(fieldId, holder);
+            outputMutator.addField(fieldId, v);
+            return holder;
+        }
+        return valueVectorMap.lget();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/01278ae5/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/RecordReader.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/RecordReader.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/RecordReader.java
index 67c84ed..f0d2daf 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/RecordReader.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/RecordReader.java
@@ -25,10 +25,6 @@ public interface RecordReader {
   /**
    * Configure the RecordReader with the provided schema and the record batch that should be written to.
    * 
-   * @param knownSchema
-   *          The set of fields that should be written to as well as the expected types for those fields. In the case
-   *          that RecordReader has a known schema and the expectedSchema does not match the actual schema, a
-   *          ExceptionSetupException will be thrown.
    * @param output
    *          The place where output for a particular scan should be written. The record reader is responsible for
    *          mutating the set of schema values for that particular record.

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/01278ae5/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/VectorHolder.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/VectorHolder.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/VectorHolder.java
new file mode 100644
index 0000000..59590d2
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/VectorHolder.java
@@ -0,0 +1,59 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+
+package org.apache.drill.exec.store;
+
+import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.record.MaterializedField;
+import org.apache.drill.exec.record.vector.TypeHelper;
+import org.apache.drill.exec.record.vector.ValueVector;
+
+public class VectorHolder {
+    private int length;
+    private ValueVector vector;
+    private int currentLength;
+
+    VectorHolder(int length, ValueVector<?> vector) {
+        this.length = length;
+        this.vector = vector;
+    }
+
+    public ValueVector getValueVector() {
+        return vector;
+    }
+
+    public void incAndCheckLength(int newLength) {
+        if (!hasEnoughSpace(newLength)) {
+            throw new BatchExceededException(length, currentLength + newLength);
+        }
+        currentLength += newLength;
+    }
+
+    public boolean hasEnoughSpace(int newLength) {
+        return length >= currentLength + newLength;
+    }
+
+    public int getLength() {
+        return length;
+    }
+
+    public void reset() {
+        currentLength = 0;
+        vector.allocateNew(length);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/01278ae5/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/JSONRecordReaderTest.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/JSONRecordReaderTest.java b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/JSONRecordReaderTest.java
new file mode 100644
index 0000000..a29e9f0
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/JSONRecordReaderTest.java
@@ -0,0 +1,290 @@
+package org.apache.drill.exec.store;
+
+import com.beust.jcommander.internal.Lists;
+import mockit.Expectations;
+import mockit.Injectable;
+import org.apache.drill.common.exceptions.ExecutionSetupException;
+import org.apache.drill.exec.exception.SchemaChangeException;
+import org.apache.drill.exec.memory.DirectBufferAllocator;
+import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.physical.impl.OutputMutator;
+import org.apache.drill.exec.proto.SchemaDefProtos;
+import org.apache.drill.exec.proto.UserBitShared;
+import org.apache.drill.exec.record.vector.ValueVector;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.nio.charset.Charset;
+import java.util.Arrays;
+import java.util.List;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+public class JSONRecordReaderTest {
+    private static final Charset UTF_8 = Charset.forName("UTF-8");
+
+    private String getResource(String resourceName) {
+        return "resource:" + resourceName;
+    }
+
+    class MockOutputMutator implements OutputMutator {
+        List<Integer> removedFields = Lists.newArrayList();
+        List<ValueVector> addFields = Lists.newArrayList();
+
+        @Override
+        public void removeField(int fieldId) throws SchemaChangeException {
+            removedFields.add(fieldId);
+        }
+
+        @Override
+        public void addField(int fieldId, ValueVector<?> vector) throws SchemaChangeException {
+            addFields.add(vector);
+        }
+
+        @Override
+        public void setNewSchema() throws SchemaChangeException {
+        }
+
+        List<Integer> getRemovedFields() {
+            return removedFields;
+        }
+
+        List<ValueVector> getAddFields() {
+            return addFields;
+        }
+    }
+
+    private <T> void assertField(ValueVector valueVector, int index, SchemaDefProtos.MinorType expectedMinorType, T value, String name) {
+        assertField(valueVector, index, expectedMinorType, value, name, 0);
+    }
+
+    private <T> void assertField(ValueVector valueVector, int index, SchemaDefProtos.MinorType expectedMinorType, T value, String name, int parentFieldId) {
+        UserBitShared.FieldMetadata metadata = valueVector.getMetadata();
+        SchemaDefProtos.FieldDef def = metadata.getDef();
+        assertEquals(expectedMinorType, def.getMajorType().getMinorType());
+        assertEquals(name, def.getNameList().get(0).getName());
+        assertEquals(parentFieldId, def.getParentId());
+
+        if(expectedMinorType == SchemaDefProtos.MinorType.MAP) {
+            return;
+        }
+
+        T val = (T) valueVector.getObject(index);
+        if (val instanceof byte[]) {
+            assertTrue(Arrays.equals((byte[]) value, (byte[]) val));
+        } else {
+            assertEquals(value, val);
+        }
+    }
+
+    @Test
+    public void testSameSchemaInSameBatch(@Injectable final FragmentContext context) throws IOException, ExecutionSetupException {
+        new Expectations() {
+            {
+                context.getAllocator();
+                returns(new DirectBufferAllocator());
+            }
+        };
+        JSONRecordReader jr = new JSONRecordReader(context, getResource("scan_json_test_1.json"));
+
+        MockOutputMutator mutator = new MockOutputMutator();
+        List<ValueVector> addFields = mutator.getAddFields();
+        jr.setup(mutator);
+        assertEquals(2, jr.next());
+        assertEquals(3, addFields.size());
+        assertField(addFields.get(0), 0, SchemaDefProtos.MinorType.INT, 123, "test");
+        assertField(addFields.get(1), 0, SchemaDefProtos.MinorType.BOOLEAN, 1, "b");
+        assertField(addFields.get(2), 0, SchemaDefProtos.MinorType.VARCHAR4, "hi!".getBytes(UTF_8), "c");
+        assertField(addFields.get(0), 1, SchemaDefProtos.MinorType.INT, 1234, "test");
+        assertField(addFields.get(1), 1, SchemaDefProtos.MinorType.BOOLEAN, 0, "b");
+        assertField(addFields.get(2), 1, SchemaDefProtos.MinorType.VARCHAR4, "drill!".getBytes(UTF_8), "c");
+
+        assertEquals(0, jr.next());
+        assertTrue(mutator.getRemovedFields().isEmpty());
+    }
+
+    @Test
+    public void testChangedSchemaInSameBatch(@Injectable final FragmentContext context) throws IOException, ExecutionSetupException {
+        new Expectations() {
+            {
+                context.getAllocator();
+                returns(new DirectBufferAllocator());
+            }
+        };
+
+        JSONRecordReader jr = new JSONRecordReader(context, getResource("scan_json_test_2.json"));
+        MockOutputMutator mutator = new MockOutputMutator();
+        List<ValueVector> addFields = mutator.getAddFields();
+
+        jr.setup(mutator);
+        assertEquals(3, jr.next());
+        assertEquals(7, addFields.size());
+        assertField(addFields.get(0), 0, SchemaDefProtos.MinorType.INT, 123, "test");
+        assertField(addFields.get(1), 0, SchemaDefProtos.MinorType.INT, 1, "b");
+        assertField(addFields.get(2), 0, SchemaDefProtos.MinorType.FLOAT4, (float) 2.15, "c");
+        assertField(addFields.get(3), 0, SchemaDefProtos.MinorType.BOOLEAN, 1, "bool");
+        assertField(addFields.get(4), 0, SchemaDefProtos.MinorType.VARCHAR4, "test1".getBytes(UTF_8), "str1");
+
+        assertField(addFields.get(0), 1, SchemaDefProtos.MinorType.INT, 1234, "test");
+        assertField(addFields.get(1), 1, SchemaDefProtos.MinorType.INT, 3, "b");
+        assertField(addFields.get(3), 1, SchemaDefProtos.MinorType.BOOLEAN, 0, "bool");
+        assertField(addFields.get(4), 1, SchemaDefProtos.MinorType.VARCHAR4, "test2".getBytes(UTF_8), "str1");
+        assertField(addFields.get(5), 1, SchemaDefProtos.MinorType.INT, 4, "d");
+
+        assertField(addFields.get(0), 2, SchemaDefProtos.MinorType.INT, 12345, "test");
+        assertField(addFields.get(2), 2, SchemaDefProtos.MinorType.FLOAT4, (float) 5.16, "c");
+        assertField(addFields.get(3), 2, SchemaDefProtos.MinorType.BOOLEAN, 1, "bool");
+        assertField(addFields.get(5), 2, SchemaDefProtos.MinorType.INT, 6, "d");
+        assertField(addFields.get(6), 2, SchemaDefProtos.MinorType.VARCHAR4, "test3".getBytes(UTF_8), "str2");
+        assertTrue(mutator.getRemovedFields().isEmpty());
+        assertEquals(0, jr.next());
+    }
+
+    @Test
+    public void testChangedSchemaInTwoBatches(@Injectable final FragmentContext context) throws IOException, ExecutionSetupException {
+        new Expectations() {
+            {
+                context.getAllocator();
+                returns(new DirectBufferAllocator());
+            }
+        };
+
+        JSONRecordReader jr = new JSONRecordReader(context, getResource("scan_json_test_2.json"), 64); // batch only fits 1 int
+        MockOutputMutator mutator = new MockOutputMutator();
+        List<ValueVector> addFields = mutator.getAddFields();
+        List<Integer> removedFields = mutator.getRemovedFields();
+
+        jr.setup(mutator);
+        assertEquals(1, jr.next());
+        assertEquals(5, addFields.size());
+        assertField(addFields.get(0), 0, SchemaDefProtos.MinorType.INT, 123, "test");
+        assertField(addFields.get(1), 0, SchemaDefProtos.MinorType.INT, 1, "b");
+        assertField(addFields.get(2), 0, SchemaDefProtos.MinorType.FLOAT4, (float) 2.15, "c");
+        assertField(addFields.get(3), 0, SchemaDefProtos.MinorType.BOOLEAN, 1, "bool");
+        assertField(addFields.get(4), 0, SchemaDefProtos.MinorType.VARCHAR4, "test1".getBytes(UTF_8), "str1");
+        assertTrue(removedFields.isEmpty());
+        assertEquals(1, jr.next());
+        assertEquals(6, addFields.size());
+        assertField(addFields.get(0), 0, SchemaDefProtos.MinorType.INT, 1234, "test");
+        assertField(addFields.get(1), 0, SchemaDefProtos.MinorType.INT, 3, "b");
+        assertField(addFields.get(3), 0, SchemaDefProtos.MinorType.BOOLEAN, 0, "bool");
+        assertField(addFields.get(4), 0, SchemaDefProtos.MinorType.VARCHAR4, "test2".getBytes(UTF_8), "str1");
+        assertField(addFields.get(5), 0, SchemaDefProtos.MinorType.INT, 4, "d");
+        assertEquals(1, removedFields.size());
+        assertEquals(3, (int) removedFields.get(0));
+        removedFields.clear();
+        assertEquals(1, jr.next());
+        assertEquals(8, addFields.size()); // The reappearing of field 'c' is also included
+        assertField(addFields.get(0), 0, SchemaDefProtos.MinorType.INT, 12345, "test");
+        assertField(addFields.get(3), 0, SchemaDefProtos.MinorType.BOOLEAN, 1, "bool");
+        assertField(addFields.get(5), 0, SchemaDefProtos.MinorType.INT, 6, "d");
+        assertField(addFields.get(6), 0, SchemaDefProtos.MinorType.FLOAT4, (float) 5.16, "c");
+        assertField(addFields.get(7), 0, SchemaDefProtos.MinorType.VARCHAR4, "test3".getBytes(UTF_8), "str2");
+        assertEquals(2, removedFields.size());
+        assertTrue(removedFields.contains(5));
+        assertTrue(removedFields.contains(2));
+        assertEquals(0, jr.next());
+    }
+
+    @Test
+    public void testNestedFieldInSameBatch(@Injectable final FragmentContext context) throws ExecutionSetupException {
+        new Expectations() {
+            {
+                context.getAllocator();
+                returns(new DirectBufferAllocator());
+            }
+        };
+
+        JSONRecordReader jr = new JSONRecordReader(context, getResource("scan_json_test_3.json"));
+
+        MockOutputMutator mutator = new MockOutputMutator();
+        List<ValueVector> addFields = mutator.getAddFields();
+        jr.setup(mutator);
+        assertEquals(2, jr.next());
+        assertEquals(5, addFields.size());
+        assertField(addFields.get(0), 0, SchemaDefProtos.MinorType.INT, 123, "test");
+        assertField(addFields.get(1), 0, SchemaDefProtos.MinorType.MAP, null, "a");
+        assertField(addFields.get(2), 0, SchemaDefProtos.MinorType.VARCHAR4, "test".getBytes(UTF_8), "b", 2);
+        assertField(addFields.get(3), 0, SchemaDefProtos.MinorType.MAP, null, "a", 2);
+        assertField(addFields.get(4), 0, SchemaDefProtos.MinorType.BOOLEAN, 1, "d", 4);
+        assertField(addFields.get(0), 1, SchemaDefProtos.MinorType.INT, 1234, "test");
+        assertField(addFields.get(1), 1, SchemaDefProtos.MinorType.MAP, null, "a");
+        assertField(addFields.get(2), 1, SchemaDefProtos.MinorType.VARCHAR4, "test2".getBytes(UTF_8), "b", 2);
+        assertField(addFields.get(3), 1, SchemaDefProtos.MinorType.MAP, null, "a", 2);
+        assertField(addFields.get(4), 1, SchemaDefProtos.MinorType.BOOLEAN, 0, "d", 4);
+
+        assertEquals(0, jr.next());
+        assertTrue(mutator.getRemovedFields().isEmpty());
+    }
+
+    /*
+
+    @Test
+    public void testScanJsonRemovedOneField() throws IOException {
+        ScanJson sj = new ScanJson(getResource("scan_json_test_3.json"));
+        PhysicalOperatorIterator iterator = sj.getIterator();
+        expectSchemaChanged(iterator);
+        DiffSchema diffSchema = expectSchemaChanged(iterator).getSchemaChanges();
+        assertEquals(0, diffSchema.getAddedFields().size());
+        assertEquals(1, diffSchema.getRemovedFields().size());
+        assertEquals(PhysicalOperatorIterator.NextOutcome.NONE_LEFT, iterator.next());
+    }
+
+    @Test
+    public void testScanJsonAddOneRemoveOne() throws IOException {
+        ScanJson sj = new ScanJson(getResource("scan_json_test_4.json"));
+        PhysicalOperatorIterator iterator = sj.getIterator();
+        expectSchemaChanged(iterator);
+        DiffSchema diffSchema = expectSchemaChanged(iterator).getSchemaChanges();
+        assertEquals(1, diffSchema.getAddedFields().size());
+        assertEquals(1, diffSchema.getRemovedFields().size());
+        assertEquals(PhysicalOperatorIterator.NextOutcome.NONE_LEFT, iterator.next());
+    }
+
+    @Test
+    public void testScanJsonCycleAdditions() throws IOException {
+        ScanJson sj = new ScanJson(getResource("scan_json_test_5.json"));
+        PhysicalOperatorIterator iterator = sj.getIterator();
+        expectSchemaChanged(iterator);
+        DiffSchema diffSchema = expectSchemaChanged(iterator).getSchemaChanges();
+        assertEquals(1, diffSchema.getAddedFields().size());
+        assertEquals(1, diffSchema.getRemovedFields().size());
+        diffSchema = expectSchemaChanged(iterator).getSchemaChanges();
+        assertEquals(1, diffSchema.getAddedFields().size());
+        assertEquals(Field.FieldType.FLOAT, diffSchema.getAddedFields().get(0).getFieldType());
+        assertEquals("test2", diffSchema.getAddedFields().get(0).getFieldName());
+        assertEquals(1, diffSchema.getRemovedFields().size());
+        assertEquals(Field.FieldType.BOOLEAN, diffSchema.getRemovedFields().get(0).getFieldType());
+        assertEquals("test3", diffSchema.getRemovedFields().get(0).getFieldName());
+        assertEquals(PhysicalOperatorIterator.NextOutcome.NONE_LEFT, iterator.next());
+    }
+
+    @Test
+    public void testScanJsonModifiedOneFieldType() throws IOException {
+        ScanJson sj = new ScanJson(getResource("scan_json_test_6.json"));
+        PhysicalOperatorIterator iterator = sj.getIterator();
+        expectSchemaChanged(iterator);
+        DiffSchema diffSchema = expectSchemaChanged(iterator).getSchemaChanges();
+        List<Field> addedFields = diffSchema.getAddedFields();
+        assertEquals(4, addedFields.size());
+        List<Field> removedFields = diffSchema.getRemovedFields();
+        assertEquals(4, removedFields.size());
+        assertFieldExists("test", Field.FieldType.STRING, addedFields);
+        assertFieldExists("test2", Field.FieldType.BOOLEAN, addedFields);
+        assertFieldExists("b", Field.FieldType.ARRAY, addedFields);
+        assertFieldExists("[0]", Field.FieldType.INTEGER, addedFields);
+        assertFieldExists("test", Field.FieldType.INTEGER, removedFields);
+        assertFieldExists("test2", Field.FieldType.ARRAY, removedFields);
+        assertFieldExists("b", Field.FieldType.INTEGER, removedFields);
+        assertFieldExists("[0]", Field.FieldType.INTEGER, removedFields);
+        assertEquals(PhysicalOperatorIterator.NextOutcome.NONE_LEFT, iterator.next());
+    }
+
+    private void expectSchemaChanged(PhysicalOperatorIterator iterator) throws IOException {
+    }
+
+    private void expectDataRecord(PhysicalOperatorIterator iterator) throws IOException {
+    }
+*/
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/01278ae5/sandbox/prototype/exec/java-exec/src/test/resources/scan_json_test_1.json
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/resources/scan_json_test_1.json b/sandbox/prototype/exec/java-exec/src/test/resources/scan_json_test_1.json
new file mode 100644
index 0000000..2ab53dd
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/test/resources/scan_json_test_1.json
@@ -0,0 +1,10 @@
+{
+    "test": 123,
+    "b": true,
+    "c": "hi!"
+}
+{
+    "test": 1234,
+    "b": false,
+    "c": "drill!"
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/01278ae5/sandbox/prototype/exec/java-exec/src/test/resources/scan_json_test_2.json
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/resources/scan_json_test_2.json b/sandbox/prototype/exec/java-exec/src/test/resources/scan_json_test_2.json
new file mode 100644
index 0000000..ce46bbf
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/test/resources/scan_json_test_2.json
@@ -0,0 +1,21 @@
+{
+    "test": 123,
+    "b": 1,
+    "c": 2.15,
+    "bool": true,
+    "str1": "test1"
+}
+{
+    "test": 1234,
+    "b": 3,
+    "d": 4,
+    "bool": false,
+    "str1": "test2"
+}
+{
+    "test": 12345,
+    "c": 5.16,
+    "d": 6,
+    "bool": true,
+    "str2": "test3"
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/01278ae5/sandbox/prototype/exec/java-exec/src/test/resources/scan_json_test_3.json
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/resources/scan_json_test_3.json b/sandbox/prototype/exec/java-exec/src/test/resources/scan_json_test_3.json
new file mode 100644
index 0000000..cb7897e
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/test/resources/scan_json_test_3.json
@@ -0,0 +1,18 @@
+{
+    "test": 123,
+    "a": {
+    	 "b": "test",
+    	 "a": {
+    	    "d": true
+    	 }
+    }
+}
+{
+    "test": 1234,
+    "a": {
+    	 "b": "test2",
+    	 "a": {
+    	    "d": false
+    	 }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/01278ae5/sandbox/prototype/exec/java-exec/src/test/resources/scan_json_test_4.json
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/resources/scan_json_test_4.json b/sandbox/prototype/exec/java-exec/src/test/resources/scan_json_test_4.json
new file mode 100644
index 0000000..0fb3202
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/test/resources/scan_json_test_4.json
@@ -0,0 +1,14 @@
+{
+    "test": 123,
+    "test2": [1,2,3],
+    "a": {
+    	 "b": 1
+    }
+}
+{
+    "test": 1234,
+    "test3": false,
+    "a": {
+    	 "b": 2
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/01278ae5/sandbox/prototype/exec/java-exec/src/test/resources/scan_json_test_5.json
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/resources/scan_json_test_5.json b/sandbox/prototype/exec/java-exec/src/test/resources/scan_json_test_5.json
new file mode 100644
index 0000000..ae1aaf2
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/test/resources/scan_json_test_5.json
@@ -0,0 +1,21 @@
+{
+    "test": 123,
+    "test2": [1,2,3],
+    "a": {
+    	 "b": 1
+    }
+}
+{
+    "test": 1234,
+    "test3": false,
+    "a": {
+    	 "b": 2
+    }
+}
+{
+    "test": 1234,
+    "test2": 1.5,
+    "a": {
+    	 "b": 2
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/01278ae5/sandbox/prototype/exec/java-exec/src/test/resources/scan_json_test_6.json
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/resources/scan_json_test_6.json b/sandbox/prototype/exec/java-exec/src/test/resources/scan_json_test_6.json
new file mode 100644
index 0000000..68b53d4
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/test/resources/scan_json_test_6.json
@@ -0,0 +1,14 @@
+{
+    "test": 123,
+    "test2": [1,2,3],
+    "a": {
+    	 "b": 1
+    }
+}
+{
+    "test": "abc",
+    "test2": false,
+    "a": {
+    	 "b": [1,2,3,4]
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/01278ae5/sandbox/prototype/exec/ref/src/test/resources/donuts.json
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/ref/src/test/resources/donuts.json b/sandbox/prototype/exec/ref/src/test/resources/donuts.json
index 2d98b59..d16ef46 100644
--- a/sandbox/prototype/exec/ref/src/test/resources/donuts.json
+++ b/sandbox/prototype/exec/ref/src/test/resources/donuts.json
@@ -1,11 +1,11 @@
-
-	{
+{
 		"id": "0001",
 		"type": "donut",
 		"name": "Cake",
 		"ppu": 0.55,
 		"sales": 35,
 
+
 		"batters":
 			{
 				"batter":
@@ -56,6 +56,7 @@
 		"ppu": 0.55,
 		"sales": 300,
 
+
 		"batters":
 			{
 				"batter":
@@ -79,6 +80,7 @@
 		"ppu": 0.69,
 		"sales": 14,
 
+
 		"batters":
 			{
 				"batter":
@@ -115,6 +117,7 @@
 		"ppu": 1.00,
 		"sales": 700,
 
+
 		"batters":
 			{
 				"batter":

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/01278ae5/sandbox/prototype/exec/ref/src/test/resources/simple_plan.json
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/ref/src/test/resources/simple_plan.json b/sandbox/prototype/exec/ref/src/test/resources/simple_plan.json
index 0f38476..c0837e6 100644
--- a/sandbox/prototype/exec/ref/src/test/resources/simple_plan.json
+++ b/sandbox/prototype/exec/ref/src/test/resources/simple_plan.json
@@ -78,3 +78,4 @@
     }
   ]
 }
+

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/01278ae5/sandbox/prototype/pom.xml
----------------------------------------------------------------------
diff --git a/sandbox/prototype/pom.xml b/sandbox/prototype/pom.xml
index 25f156d..105016e 100644
--- a/sandbox/prototype/pom.xml
+++ b/sandbox/prototype/pom.xml
@@ -28,7 +28,17 @@
                         <url>http://conjars.org/repo</url>
                         <layout>default</layout>
                 </repository>
-        </repositories>
+            <repository>
+                  <id>sonatype-nexus-snapshots</id>
+                  <url>https://oss.sonatype.org/content/repositories/snapshots</url>
+                  <releases>
+                        <enabled>false</enabled>
+                  </releases>
+                  <snapshots>
+                        <enabled>true</enabled>
+                  </snapshots>
+             </repository>
+       </repositories>
 
 	<properties>
 		<dep.junit.version>4.11</dep.junit.version>
@@ -123,6 +133,12 @@
 
 		<!-- Test Dependencies -->
 		<dependency>
+		  <groupId>com.googlecode.jmockit</groupId>
+		  <artifactId>jmockit</artifactId>
+		  <version>1.2</version>
+		  <scope>test</scope>
+		</dependency>
+		<dependency>
 			<groupId>junit</groupId>
 			<artifactId>junit</artifactId>
 			<version>${dep.junit.version}</version>


[11/53] [abbrv] Update typing system. Update RPC system. Add Fragmenting Implementation. Working single node. Distributed failing due to threading issues.

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/BasicServer.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/BasicServer.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/BasicServer.java
index bc942ac..52bb0a2 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/BasicServer.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/BasicServer.java
@@ -17,9 +17,9 @@
  ******************************************************************************/
 package org.apache.drill.exec.rpc;
 
-import com.google.protobuf.Internal.EnumLite;
 import io.netty.bootstrap.ServerBootstrap;
 import io.netty.buffer.ByteBufAllocator;
+import io.netty.channel.Channel;
 import io.netty.channel.ChannelInitializer;
 import io.netty.channel.ChannelOption;
 import io.netty.channel.EventLoopGroup;
@@ -27,23 +27,32 @@ import io.netty.channel.socket.SocketChannel;
 import io.netty.channel.socket.nio.NioServerSocketChannel;
 import io.netty.handler.logging.LogLevel;
 import io.netty.handler.logging.LoggingHandler;
-import org.apache.drill.exec.exception.DrillbitStartupException;
 
 import java.io.IOException;
 import java.net.BindException;
 
+import org.apache.drill.exec.exception.DrillbitStartupException;
+import org.apache.drill.exec.proto.GeneralRPCProtos.RpcMode;
+
+import com.google.protobuf.Internal.EnumLite;
+import com.google.protobuf.MessageLite;
+import com.google.protobuf.Parser;
+
 /**
  * A server is bound to a port and is responsible for responding to various type of requests. In some cases, the inbound
  * requests will generate more than one outbound request.
  */
-public abstract class BasicServer<T extends EnumLite> extends RpcBus<T>{
+public abstract class BasicServer<T extends EnumLite, C extends RemoteConnection> extends RpcBus<T, C>{
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(BasicServer.class);
 
   private ServerBootstrap b;
   private volatile boolean connect = false;
+  private final EventLoopGroup eventLoopGroup;
 
-  public BasicServer(ByteBufAllocator alloc, EventLoopGroup eventLoopGroup) {
-
+  public BasicServer(RpcConfig rpcMapping, ByteBufAllocator alloc, EventLoopGroup eventLoopGroup) {
+    super(rpcMapping);
+    this.eventLoopGroup = eventLoopGroup;
+    
     b = new ServerBootstrap() //
         .channel(NioServerSocketChannel.class) //
         .option(ChannelOption.SO_BACKLOG, 100) //
@@ -56,17 +65,19 @@ public abstract class BasicServer<T extends EnumLite> extends RpcBus<T>{
           @Override
           protected void initChannel(SocketChannel ch) throws Exception {
             
-            ch.closeFuture().addListener(getCloseHandler(ch));
+            C connection = initRemoteConnection(ch);
+            ch.closeFuture().addListener(getCloseHandler(connection));
 
             ch.pipeline().addLast( //
                 new ZeroCopyProtobufLengthDecoder(), //
-                new RpcDecoder(), //
-                new RpcEncoder(), //
-                new InboundHandler(ch), //
+                new RpcDecoder(rpcConfig.getName()), //
+                new RpcEncoder(rpcConfig.getName()), //
+                getHandshakeHandler(),
+                new InboundHandler(connection), //
                 new RpcExceptionHandler() //
                 );            
-            channel = ch;
             connect = true;
+            
           }
         });
   }
@@ -76,12 +87,34 @@ public abstract class BasicServer<T extends EnumLite> extends RpcBus<T>{
     return false;
   }
 
+  
+  protected abstract ServerHandshakeHandler<?> getHandshakeHandler();
 
+  protected static abstract class ServerHandshakeHandler<T extends MessageLite> extends AbstractHandshakeHandler<T> {
+
+    public ServerHandshakeHandler(EnumLite handshakeType, Parser<T> parser) {
+      super(handshakeType, parser);
+    }
+
+    @Override
+    protected final void consumeHandshake(Channel c, T inbound) throws Exception {
+      OutboundRpcMessage msg = new OutboundRpcMessage(RpcMode.RESPONSE, this.handshakeType, coordinationId, getHandshakeResponse(inbound));
+      c.write(msg);
+    }
+    
+    public abstract MessageLite getHandshakeResponse(T inbound) throws Exception;
+    
+
+      
+    
+  }
+  
+  
   public int bind(final int initialPort) throws InterruptedException, DrillbitStartupException{
-    int port = initialPort;
+    int port = initialPort-1;
     while (true) {
       try {
-        b.bind(port++).sync();
+        b.bind(++port).sync();
         break;
       } catch (Exception e) {
         if (e instanceof BindException)
@@ -89,13 +122,15 @@ public abstract class BasicServer<T extends EnumLite> extends RpcBus<T>{
         throw new DrillbitStartupException("Could not bind Drillbit", e);
       }
     }
+    
     connect = !connect;
+    logger.debug("Server started on port {} of type {} ", port, this.getClass().getSimpleName());
     return port;    
   }
 
   @Override
   public void close() throws IOException {
-    if(b != null) b.shutdown();
+    eventLoopGroup.shutdownGracefully();
   }
   
   

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/CoordinationQueue.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/CoordinationQueue.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/CoordinationQueue.java
index c796e2d..70142bb 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/CoordinationQueue.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/CoordinationQueue.java
@@ -29,21 +29,21 @@ public class CoordinationQueue {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(CoordinationQueue.class);
 
   private final PositiveAtomicInteger circularInt = new PositiveAtomicInteger();
-  private final Map<Integer, DrillRpcFuture<?>> map;
+  private final Map<Integer, DrillRpcFutureImpl<?>> map;
 
   public CoordinationQueue(int segmentSize, int segmentCount) {
-    map = new ConcurrentHashMap<Integer, DrillRpcFuture<?>>(segmentSize, 0.75f, segmentCount);
+    map = new ConcurrentHashMap<Integer, DrillRpcFutureImpl<?>>(segmentSize, 0.75f, segmentCount);
   }
 
   void channelClosed(Exception ex) {
-    for (DrillRpcFuture<?> f : map.values()) {
+    for (DrillRpcFutureImpl<?> f : map.values()) {
       f.setException(ex);
     }
   }
 
-  public <V> DrillRpcFuture<V> getNewFuture(Class<V> clazz) {
+  public <V> DrillRpcFutureImpl<V> getNewFuture(Class<V> clazz) {
     int i = circularInt.getNext();
-    DrillRpcFuture<V> future = DrillRpcFuture.getNewFuture(i, clazz);
+    DrillRpcFutureImpl<V> future = DrillRpcFutureImpl.getNewFuture(i, clazz);
     // logger.debug("Writing to map coord {}, future {}", i, future);
     Object old = map.put(i, future);
     if (old != null)
@@ -52,8 +52,8 @@ public class CoordinationQueue {
     return future;
   }
 
-  private DrillRpcFuture<?> removeFromMap(int coordinationId) {
-    DrillRpcFuture<?> rpc = map.remove(coordinationId);
+  private DrillRpcFutureImpl<?> removeFromMap(int coordinationId) {
+    DrillRpcFutureImpl<?> rpc = map.remove(coordinationId);
     if (rpc == null) {
       logger.error("Rpc is null.");
       throw new IllegalStateException(
@@ -62,23 +62,25 @@ public class CoordinationQueue {
     return rpc;
   }
 
-  public <V> DrillRpcFuture<V> getFuture(int coordinationId, Class<V> clazz) {
+  public <V> DrillRpcFutureImpl<V> getFuture(int rpcType, int coordinationId, Class<V> clazz) {
     // logger.debug("Getting future for coordinationId {} and class {}", coordinationId, clazz);
-    DrillRpcFuture<?> rpc = removeFromMap(coordinationId);
+    DrillRpcFutureImpl<?> rpc = removeFromMap(coordinationId);
     // logger.debug("Got rpc from map {}", rpc);
     Class<?> outcomeClass = rpc.getOutcomeClass();
+
     if (outcomeClass != clazz) {
-      logger.error("Rpc class is not expected class. Original: {}, requested: {}", outcomeClass.getCanonicalName(), clazz.getCanonicalName());
+
       throw new IllegalStateException(
           String
               .format(
-                  "You attempted to request a future for a coordination id that has a different value class than was used when you "
-                      + "initially created the coordination id.  Requested class %s, originally expected class %s.  This shouldn't happen.  ",
-                  clazz.getCanonicalName(), outcomeClass.getCanonicalName()));
+                  "RPC Engine had a submission and response configuration mismatch.  The RPC request that you submitted was defined with an expected response type of %s.  However, "
+                      + "when the response returned, a call to getResponseDefaultInstance() with Rpc number %d provided an expected class of %s.  This means either your submission uses the wrong type definition"
+                      + "or your getResponseDefaultInstance() method responds the wrong instance type ",
+                  clazz.getCanonicalName(), rpcType, outcomeClass.getCanonicalName()));
     }
 
     @SuppressWarnings("unchecked")
-    DrillRpcFuture<V> crpc = (DrillRpcFuture<V>) rpc;
+    DrillRpcFutureImpl<V> crpc = (DrillRpcFutureImpl<V>) rpc;
 
     // logger.debug("Returning casted future");
     return crpc;
@@ -86,7 +88,7 @@ public class CoordinationQueue {
 
   public void updateFailedFuture(int coordinationId, RpcFailure failure) {
     // logger.debug("Updating failed future.");
-    DrillRpcFuture<?> rpc = removeFromMap(coordinationId);
+    DrillRpcFutureImpl<?> rpc = removeFromMap(coordinationId);
     rpc.setException(new RemoteRpcException(failure));
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/DrillRpcFuture.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/DrillRpcFuture.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/DrillRpcFuture.java
index 9a4a7f7..bae947a 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/DrillRpcFuture.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/DrillRpcFuture.java
@@ -17,80 +17,10 @@
  ******************************************************************************/
 package org.apache.drill.exec.rpc;
 
-import java.util.concurrent.ExecutionException;
+import com.google.common.util.concurrent.CheckedFuture;
 
-import com.google.common.util.concurrent.AbstractCheckedFuture;
-import com.google.common.util.concurrent.AbstractFuture;
-import com.google.common.util.concurrent.ListenableFuture;
-
-public class DrillRpcFuture<V> extends AbstractCheckedFuture<V, RpcException> {
+public interface DrillRpcFuture<T> extends CheckedFuture<T,RpcException> {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillRpcFuture.class);
 
-  final int coordinationId;
-  private final Class<V> clazz;
-
-  public DrillRpcFuture(ListenableFuture<V> delegate, int coordinationId, Class<V> clazz) {
-    super(delegate);
-    this.coordinationId = coordinationId;
-    this.clazz = clazz;
-  }
-
-  public Class<V> getOutcomeClass(){
-    return clazz;
-  }
-  
-  /**
-   * Drill doesn't currently support rpc cancellations since nearly all requests should be either instance of
-   * asynchronous. Business level cancellation is managed a separate call (e.g. canceling a query.). Calling this method
-   * will result in an UnsupportedOperationException.
-   */
-  @Override
-  public boolean cancel(boolean mayInterruptIfRunning) {
-    throw new UnsupportedOperationException(
-        "Drill doesn't currently support rpc cancellations. See javadocs for more detail.");
-  }
-
-  @Override
-  protected RpcException mapException(Exception ex) {
-    if (ex instanceof RpcException)  return (RpcException) ex;
-    
-    if (ex instanceof ExecutionException) {
-      Throwable e2 = ex.getCause();
-      
-      if (e2 instanceof RpcException) {
-        return (RpcException) e2;
-      }
-    }
-    return new RpcException(ex);
-
-  }
-
-  @SuppressWarnings("unchecked")
-  void setValue(Object value) {
-    assert clazz.isAssignableFrom(value.getClass());
-    ((InnerFuture<V>) super.delegate()).setValue((V) value);
-  }
-
-  boolean setException(Throwable t) {
-    return ((InnerFuture<V>) super.delegate()).setException(t);
-  }
-
-  public static class InnerFuture<T> extends AbstractFuture<T> {
-    // we rewrite these so that the parent can see them
-
-    void setValue(T value) {
-      super.set(value);
-    }
-
-    protected boolean setException(Throwable t) {
-      return super.setException(t);
-    }
-  }
-
-  public static <V> DrillRpcFuture<V> getNewFuture(int coordinationId, Class<V> clazz) {
-    InnerFuture<V> f = new InnerFuture<V>();
-    return new DrillRpcFuture<V>(f, coordinationId, clazz);
-  }
-
-
-}
\ No newline at end of file
+  public void addLightListener(RpcOutcomeListener<T> outcomeListener);
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/DrillRpcFutureImpl.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/DrillRpcFutureImpl.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/DrillRpcFutureImpl.java
new file mode 100644
index 0000000..ee14eeb
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/DrillRpcFutureImpl.java
@@ -0,0 +1,118 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.rpc;
+
+import java.util.concurrent.ExecutionException;
+
+import com.google.common.util.concurrent.AbstractCheckedFuture;
+import com.google.common.util.concurrent.AbstractFuture;
+import com.google.common.util.concurrent.ListenableFuture;
+import com.google.common.util.concurrent.MoreExecutors;
+
+class DrillRpcFutureImpl<V> extends AbstractCheckedFuture<V, RpcException> implements DrillRpcFuture<V>{
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillRpcFutureImpl.class);
+
+  final int coordinationId;
+  private final Class<V> clazz;
+
+  public DrillRpcFutureImpl(ListenableFuture<V> delegate, int coordinationId, Class<V> clazz) {
+    super(delegate);
+    this.coordinationId = coordinationId;
+    this.clazz = clazz;
+  }
+
+  public Class<V> getOutcomeClass(){
+    return clazz;
+  }
+  
+  /**
+   * Drill doesn't currently support rpc cancellations since nearly all requests should be either instance of
+   * asynchronous. Business level cancellation is managed a separate call (e.g. canceling a query.). Calling this method
+   * will result in an UnsupportedOperationException.
+   */
+  @Override
+  public boolean cancel(boolean mayInterruptIfRunning) {
+    throw new UnsupportedOperationException(
+        "Drill doesn't currently support rpc cancellations. See javadocs for more detail.");
+  }
+
+  @Override
+  protected RpcException mapException(Exception ex) {
+    Throwable e = ex;
+    while(e instanceof ExecutionException){
+      e = e.getCause();
+    }
+    if (e instanceof RpcException)  return (RpcException) e;
+
+    return new RpcException(ex);
+
+  }
+
+  @SuppressWarnings("unchecked")
+  void setValue(Object value) {
+    assert clazz.isAssignableFrom(value.getClass());
+    ((InnerFuture<V>) super.delegate()).setValue((V) value);
+  }
+
+  boolean setException(Throwable t) {
+    return ((InnerFuture<V>) super.delegate()).setException(t);
+  }
+
+  public static class InnerFuture<T> extends AbstractFuture<T> {
+    // we rewrite these so that the parent can see them
+
+    void setValue(T value) {
+      super.set(value);
+    }
+
+    protected boolean setException(Throwable t) {
+      return super.setException(t);
+    }
+  }
+
+  public class RpcOutcomeListenerWrapper implements Runnable{
+    final RpcOutcomeListener<V> inner;
+    
+    public RpcOutcomeListenerWrapper(RpcOutcomeListener<V> inner) {
+      super();
+      this.inner = inner;
+    }
+
+    @Override
+    public void run() {
+      try{
+        inner.success(DrillRpcFutureImpl.this.checkedGet());
+      }catch(RpcException e){
+        inner.failed(e);
+      }
+    }
+  }
+  
+  public void addLightListener(RpcOutcomeListener<V> outcomeListener){
+    this.addListener(new RpcOutcomeListenerWrapper(outcomeListener), MoreExecutors.sameThreadExecutor());
+  }
+  
+  
+  
+  public static <V> DrillRpcFutureImpl<V> getNewFuture(int coordinationId, Class<V> clazz) {
+    InnerFuture<V> f = new InnerFuture<V>();
+    return new DrillRpcFutureImpl<V>(f, coordinationId, clazz);
+  }
+
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/InboundRpcMessage.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/InboundRpcMessage.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/InboundRpcMessage.java
index ab977db..be1ff6b 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/InboundRpcMessage.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/InboundRpcMessage.java
@@ -18,15 +18,20 @@
 package org.apache.drill.exec.rpc;
 
 import io.netty.buffer.ByteBuf;
+import io.netty.buffer.ByteBufInputStream;
+
+import java.io.InputStream;
 
 import org.apache.drill.exec.proto.GeneralRPCProtos.RpcMode;
 
 public class InboundRpcMessage extends RpcMessage{
   public ByteBuf pBody;
+  public ByteBuf dBody;
   
   public InboundRpcMessage(RpcMode mode, int rpcType, int coordinationId, ByteBuf pBody, ByteBuf dBody) {
-    super(mode, rpcType, coordinationId, dBody);
+    super(mode, rpcType, coordinationId);
     this.pBody = pBody;
+    this.dBody = dBody;
   }
   
   public int getBodySize(){
@@ -37,7 +42,7 @@ public class InboundRpcMessage extends RpcMessage{
   
   void release(){
     pBody.release();
-    super.release();
+    if(dBody != null) dBody.release();
   }
 
   @Override
@@ -46,5 +51,7 @@ public class InboundRpcMessage extends RpcMessage{
         + coordinationId + ", dBody=" + dBody + "]";
   }
   
-  
+  public InputStream getProtobufBodyAsIS(){
+    return new ByteBufInputStream(pBody);
+  }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/OutboundRpcMessage.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/OutboundRpcMessage.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/OutboundRpcMessage.java
index 91c3d45..e4858c4 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/OutboundRpcMessage.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/OutboundRpcMessage.java
@@ -17,6 +17,8 @@
  ******************************************************************************/
 package org.apache.drill.exec.rpc;
 
+import java.util.Arrays;
+
 import io.netty.buffer.ByteBuf;
 
 import org.apache.drill.exec.proto.GeneralRPCProtos.RpcMode;
@@ -24,28 +26,49 @@ import org.apache.drill.exec.proto.GeneralRPCProtos.RpcMode;
 import com.google.protobuf.Internal.EnumLite;
 import com.google.protobuf.MessageLite;
 
-class OutboundRpcMessage extends RpcMessage{
+public class OutboundRpcMessage extends RpcMessage {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(OutboundRpcMessage.class);
 
   final MessageLite pBody;
-  
-  public OutboundRpcMessage(RpcMode mode, EnumLite rpcType, int coordinationId, MessageLite pBody, ByteBuf dBody) {
-    super(mode, rpcType.getNumber(), coordinationId, dBody);
+  public ByteBuf[] dBodies;
+
+  public OutboundRpcMessage(RpcMode mode, EnumLite rpcType, int coordinationId, MessageLite pBody, ByteBuf... dBodies) {
+    super(mode, rpcType.getNumber(), coordinationId);
     this.pBody = pBody;
+    this.dBodies = dBodies;
   }
-  
-  public int getBodySize(){
+
+  public int getBodySize() {
     int len = pBody.getSerializedSize();
     len += RpcEncoder.getRawVarintSize(len);
-    if(dBody != null) len += dBody.capacity();
+    len += getRawBodySize();
     return len;
   }
 
+  public int getRawBodySize(){
+    if(dBodies == null) return 0;
+    int len = 0;
+    
+    for (int i = 0; i < dBodies.length; i++) {
+      if(RpcConstants.EXTRA_DEBUGGING) logger.debug("Reader Index {}, Writer Index {}", dBodies[i].readerIndex(), dBodies[i].writerIndex());
+      len += dBodies[i].readableBytes();
+    }
+    return len;
+  }
+  
   @Override
   public String toString() {
     return "OutboundRpcMessage [pBody=" + pBody + ", mode=" + mode + ", rpcType=" + rpcType + ", coordinationId="
-        + coordinationId + ", dBody=" + dBody + "]";
+        + coordinationId + ", dBodies=" + Arrays.toString(dBodies) + "]";
   }
-
   
+  void release(){
+    if(dBodies != null){
+      for(ByteBuf b : dBodies){
+        b.release();
+      }
+    }
+  }
+
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/RemoteConnection.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/RemoteConnection.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/RemoteConnection.java
new file mode 100644
index 0000000..cedba10
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/RemoteConnection.java
@@ -0,0 +1,45 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.rpc;
+
+import io.netty.channel.Channel;
+
+public class RemoteConnection{
+  private final Channel channel;
+  
+  public RemoteConnection(Channel channel) {
+    super();
+    this.channel = channel;
+  }
+
+
+  public final Channel getChannel() {
+    return channel;
+  }
+
+
+  public ConnectionThrottle getConnectionThrottle(){
+    // can't be implemented until we switch to per query sockets.
+    return null;
+  }
+  
+  public interface ConnectionThrottle{
+    public void disableReceiving();
+    public void enableReceiving();
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/Response.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/Response.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/Response.java
index 4bd592b..0c4ab7a 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/Response.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/Response.java
@@ -17,6 +17,8 @@
  ******************************************************************************/
 package org.apache.drill.exec.rpc;
 
+import org.apache.drill.exec.proto.ExecProtos.RpcType;
+
 import io.netty.buffer.ByteBuf;
 
 import com.google.protobuf.Internal.EnumLite;
@@ -27,13 +29,13 @@ public class Response {
   
   public EnumLite rpcType;
   public MessageLite pBody;
-  public ByteBuf dBody;
+  public ByteBuf[] dBodies;
   
-  public Response(EnumLite rpcType, MessageLite pBody, ByteBuf dBody) {
+  public Response(EnumLite rpcType, MessageLite pBody, ByteBuf... dBodies) {
     super();
     this.rpcType = rpcType;
     this.pBody = pBody;
-    this.dBody = dBody;
+    this.dBodies = dBodies;
   }
   
   

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/RpcBus.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/RpcBus.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/RpcBus.java
index 76300d1..11764db 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/RpcBus.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/RpcBus.java
@@ -23,113 +23,140 @@ import io.netty.channel.Channel;
 import io.netty.channel.ChannelFuture;
 import io.netty.channel.ChannelHandlerContext;
 import io.netty.channel.ChannelInboundMessageHandlerAdapter;
-import io.netty.channel.socket.SocketChannel;
 import io.netty.util.concurrent.GenericFutureListener;
 
 import java.io.Closeable;
+import java.util.Arrays;
 import java.util.concurrent.CancellationException;
 
 import org.apache.drill.exec.proto.GeneralRPCProtos.RpcFailure;
 import org.apache.drill.exec.proto.GeneralRPCProtos.RpcMode;
+import org.slf4j.Logger;
 
 import com.google.common.base.Preconditions;
 import com.google.protobuf.Internal.EnumLite;
+import com.google.protobuf.InvalidProtocolBufferException;
 import com.google.protobuf.MessageLite;
 import com.google.protobuf.Parser;
 
 /**
- * The Rpc Bus deals with incoming and outgoing communication and is used on both the server and the client side of a system.
+ * The Rpc Bus deals with incoming and outgoing communication and is used on both the server and the client side of a
+ * system.
+ * 
  * @param <T>
  */
-public abstract class RpcBus<T extends EnumLite> implements Closeable{
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(RpcBus.class);
-  
-  private CoordinationQueue queue = new CoordinationQueue(16, 16);
-  protected Channel channel;
+public abstract class RpcBus<T extends EnumLite, C extends RemoteConnection> implements Closeable {
+  final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(this.getClass());
+
+  protected final CoordinationQueue queue = new CoordinationQueue(16, 16);
 
   protected abstract MessageLite getResponseDefaultInstance(int rpcType) throws RpcException;
-  protected abstract Response handle(SocketChannel channel, int RpcType, ByteBuf pBody, ByteBuf dBody) throws RpcException;
-  public abstract boolean isClient(); 
 
-  
-  protected <SEND extends MessageLite, RECEIVE extends MessageLite> DrillRpcFuture<RECEIVE> send(T rpcType,
-      SEND protobufBody, Class<RECEIVE> clazz, ByteBuf dataBody) throws RpcException {
+  protected abstract Response handle(C connection, int rpcType, ByteBuf pBody, ByteBuf dBody) throws RpcException;
+
+  public abstract boolean isClient();
+
+  protected final RpcConfig rpcConfig;
+
+  public RpcBus(RpcConfig rpcConfig) {
+    this.rpcConfig = rpcConfig;
+  }
+
+  public <SEND extends MessageLite, RECEIVE extends MessageLite> DrillRpcFuture<RECEIVE> send(C connection, T rpcType,
+      SEND protobufBody, Class<RECEIVE> clazz, ByteBuf... dataBodies) {
+
+    assert !Arrays.asList(dataBodies).contains(null);
+    assert rpcConfig.checkSend(rpcType, protobufBody.getClass(), clazz);
+
     ByteBuf pBuffer = null;
     boolean completed = false;
 
     try {
-//      logger.debug("Seding message");
+      // logger.debug("Seding message");
       Preconditions.checkNotNull(protobufBody);
-      DrillRpcFuture<RECEIVE> rpcFuture = queue.getNewFuture(clazz);
-      OutboundRpcMessage m = new OutboundRpcMessage(RpcMode.REQUEST, rpcType, rpcFuture.coordinationId, protobufBody, dataBody);
-      ChannelFuture channelFuture = channel.write(m);
+      DrillRpcFutureImpl<RECEIVE> rpcFuture = queue.getNewFuture(clazz);
+      OutboundRpcMessage m = new OutboundRpcMessage(RpcMode.REQUEST, rpcType, rpcFuture.coordinationId, protobufBody, dataBodies);
+      ChannelFuture channelFuture = connection.getChannel().write(m);
       channelFuture.addListener(new Listener(rpcFuture.coordinationId, clazz));
       completed = true;
       return rpcFuture;
     } finally {
       if (!completed) {
         if (pBuffer != null) pBuffer.release();
-        if (dataBody != null) dataBody.release();
+        if (dataBodies != null) {
+          for (ByteBuf b : dataBodies) {
+            b.release();
+          }
+
+        }
       }
       ;
     }
   }
 
-  
-  public class ChannelClosedHandler implements GenericFutureListener<ChannelFuture>{
+  public abstract C initRemoteConnection(Channel channel);
+
+  public class ChannelClosedHandler implements GenericFutureListener<ChannelFuture> {
     @Override
     public void operationComplete(ChannelFuture future) throws Exception {
-      logger.info("Channel closed between local {} and remote {}", future.channel().localAddress(), future.channel().remoteAddress());
-      queue.channelClosed(new ChannelClosedException());
+      logger.info("Channel closed between local {} and remote {}", future.channel().localAddress(), future.channel()
+          .remoteAddress());
+      closeQueueDueToChannelClose();
     }
   }
-  
-  protected GenericFutureListener<ChannelFuture> getCloseHandler(SocketChannel ch){
+
+  protected void closeQueueDueToChannelClose() {
+    if (this.isClient()) queue.channelClosed(new ChannelClosedException());
+  }
+
+  protected GenericFutureListener<ChannelFuture> getCloseHandler(C clientConnection) {
     return new ChannelClosedHandler();
   }
-  
+
   protected class InboundHandler extends ChannelInboundMessageHandlerAdapter<InboundRpcMessage> {
 
-    private final SocketChannel channel;
-    
-    
-    public InboundHandler(SocketChannel channel) {
+    private final C connection;
+    public InboundHandler(C connection) {
       super();
-      this.channel = channel;
+      this.connection = connection;
     }
 
-
     @Override
     public void messageReceived(ChannelHandlerContext ctx, InboundRpcMessage msg) throws Exception {
-      if(!ctx.channel().isOpen()) return;
+      if (!ctx.channel().isOpen()) return;
 
-      if(RpcConstants.EXTRA_DEBUGGING) logger.debug("Received message {}", msg);
-      switch(msg.mode){
+      if (RpcConstants.EXTRA_DEBUGGING) logger.debug("Received message {}", msg);
+      switch (msg.mode) {
       case REQUEST:
         // handle message and ack.
-        Response r = handle(channel, msg.rpcType, msg.pBody, msg.dBody);
-        OutboundRpcMessage outMessage = new OutboundRpcMessage(RpcMode.RESPONSE, r.rpcType, msg.coordinationId, r.pBody, r.dBody);
-        if(RpcConstants.EXTRA_DEBUGGING) logger.debug("Adding message to outbound buffer. {}", outMessage);
+        Response r = handle(connection, msg.rpcType, msg.pBody, msg.dBody);
+        assert rpcConfig.checkResponseSend(r.rpcType, r.pBody.getClass());
+        OutboundRpcMessage outMessage = new OutboundRpcMessage(RpcMode.RESPONSE, r.rpcType, msg.coordinationId,
+            r.pBody, r.dBodies);
+        if (RpcConstants.EXTRA_DEBUGGING) logger.debug("Adding message to outbound buffer. {}", outMessage);
         ctx.write(outMessage);
         break;
-        
+
       case RESPONSE:
         MessageLite m = getResponseDefaultInstance(msg.rpcType);
-        DrillRpcFuture<?> rpcFuture = queue.getFuture(msg.coordinationId, m.getClass());
+        assert rpcConfig.checkReceive(msg.rpcType, m.getClass());
+        DrillRpcFutureImpl<?> rpcFuture = queue.getFuture(msg.rpcType, msg.coordinationId, m.getClass());
         Parser<?> parser = m.getParserForType();
         Object value = parser.parseFrom(new ByteBufInputStream(msg.pBody, msg.pBody.readableBytes()));
         rpcFuture.setValue(value);
-        if(RpcConstants.EXTRA_DEBUGGING) logger.debug("Updated rpc future {} with value {}", rpcFuture, value);
+        if (RpcConstants.EXTRA_DEBUGGING) logger.debug("Updated rpc future {} with value {}", rpcFuture, value);
+
         break;
-        
+
       case RESPONSE_FAILURE:
         RpcFailure failure = RpcFailure.parseFrom(new ByteBufInputStream(msg.pBody, msg.pBody.readableBytes()));
         queue.updateFailedFuture(msg.coordinationId, failure);
-        if(RpcConstants.EXTRA_DEBUGGING) logger.debug("Updated rpc future with coordinationId {} with failure ", msg.coordinationId, failure);
+        if (RpcConstants.EXTRA_DEBUGGING)
+          logger.debug("Updated rpc future with coordinationId {} with failure ", msg.coordinationId, failure);
         break;
-        
+
       default:
-        throw new UnsupportedOperationException(); 
+        throw new UnsupportedOperationException();
       }
     }
 
@@ -147,18 +174,18 @@ public abstract class RpcBus<T extends EnumLite> implements Closeable{
 
     @Override
     public void operationComplete(ChannelFuture channelFuture) throws Exception {
-//      logger.debug("Completed channel write.");
-      
+      // logger.debug("Completed channel write.");
+
       if (channelFuture.isCancelled()) {
-        DrillRpcFuture<?> rpcFuture = queue.getFuture(coordinationId, clazz);
+        DrillRpcFutureImpl<?> rpcFuture = queue.getFuture(-1, coordinationId, clazz);
         rpcFuture.setException(new CancellationException("Socket operation was canceled."));
       } else if (!channelFuture.isSuccess()) {
         try {
           channelFuture.get();
-          throw new IllegalStateException(
-              "Future was described as completed and not succesful but did not throw an exception.");
+          throw new IllegalStateException("Future was described as completed and not succesful but did not throw an exception.");
         } catch (Exception e) {
-          DrillRpcFuture<?> rpcFuture = queue.getFuture(coordinationId, clazz);
+          logger.error("Error occurred during Rpc", e);
+          DrillRpcFutureImpl<?> rpcFuture = queue.getFuture(-1, coordinationId, clazz);
           rpcFuture.setException(e);
         }
       } else {
@@ -168,6 +195,13 @@ public abstract class RpcBus<T extends EnumLite> implements Closeable{
     }
 
   }
-  
-  
+
+  public static <T> T get(ByteBuf pBody, Parser<T> parser) throws RpcException{
+    try {
+      ByteBufInputStream is = new ByteBufInputStream(pBody);
+      return parser.parseFrom(is);
+    } catch (InvalidProtocolBufferException e) {
+      throw new RpcException(String.format("Failure while decoding message with parser of type. %s", parser.getClass().getCanonicalName()), e);
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/RpcConfig.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/RpcConfig.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/RpcConfig.java
new file mode 100644
index 0000000..c6b4c49
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/RpcConfig.java
@@ -0,0 +1,150 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.rpc;
+
+import java.util.Map;
+
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.Maps;
+import com.google.protobuf.Internal.EnumLite;
+import com.google.protobuf.MessageLite;
+
+public class RpcConfig {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(RpcConfig.class);
+
+  private final String name;
+  private final Map<EnumLite, RpcMessageType<?, ?, ?>> sendMap;
+  private final Map<Integer, RpcMessageType<?, ?, ?>> receiveMap;
+  
+  private RpcConfig(String name, Map<EnumLite, RpcMessageType<?, ?, ?>> sendMap, Map<Integer, RpcMessageType<?, ?, ?>> receiveMap){
+    this.name = name;
+    this.sendMap = ImmutableMap.copyOf(sendMap);
+    this.receiveMap = ImmutableMap.copyOf(receiveMap);
+  }
+  
+  public String getName() {
+    return name;
+  }
+
+  public boolean checkReceive(int rpcType, Class<?> receiveClass){
+    if(RpcConstants.EXTRA_DEBUGGING) logger.debug(String.format("Checking reception for rpcType %d and receive class %s.", rpcType, receiveClass));
+    RpcMessageType<?,?,?> type = receiveMap.get(rpcType);
+    if(type == null) throw new IllegalStateException(String.format("%s: There is no defined RpcMessage type for a Rpc receive type number of %s.", name, rpcType));
+
+    if(receiveClass != type.getRet()){
+      throw new IllegalStateException(String.format("%s: The definition for receive doesn't match implementation code.  The definition is %s however the current receive for this type was of type %s.", name, type, receiveClass.getCanonicalName()));
+    }
+    return true;
+  }
+  
+  public boolean checkSend(EnumLite send, Class<?> sendClass, Class<?> receiveClass){
+    if(RpcConstants.EXTRA_DEBUGGING) logger.debug(String.format("Checking send classes for send RpcType %s.  Send Class is %s and Receive class is %s.", send, sendClass, receiveClass));
+    RpcMessageType<?,?,?> type = sendMap.get(send);
+    if(type == null) throw new IllegalStateException(String.format("%s: There is no defined RpcMessage type for a Rpc send type of %s.", name, send));
+
+    if(type.getSend() != sendClass) throw new IllegalStateException(String.format("%s: The definition for send doesn't match implementation code.  The definition is %s however the current send is trying to send an object of type %s.", name, type, sendClass.getCanonicalName()));
+    if(type.getRet() != receiveClass) throw new IllegalStateException(String.format("%s: The definition for send doesn't match implementation code.  The definition is %s however the current send is trying to setup an expected reception of an object of type %s.", name, type, receiveClass.getCanonicalName()));
+
+    return true;
+  }
+  
+  public boolean checkResponseSend(EnumLite responseType, Class<?> responseClass){
+    if(RpcConstants.EXTRA_DEBUGGING) logger.debug(String.format("Checking responce send of type %s with response class of %s.",  responseType, responseClass));
+    RpcMessageType<?,?,?> type = receiveMap.get(responseType.getNumber());
+    if(type == null) throw new IllegalStateException(String.format("%s: There is no defined RpcMessage type for a Rpc response of type %s.", name, responseType));
+    if(type.getRet() != responseClass) throw new IllegalStateException(String.format("%s: The definition for the response doesn't match implementation code.  The definition is %s however the current response is trying to response with an object of type %s.", name, type, responseClass.getCanonicalName()));
+    
+    return true;
+  }
+  
+  public static class RpcMessageType<SEND extends MessageLite, RECEIVE extends MessageLite, T extends EnumLite>{
+    private T sendEnum;
+    private Class<SEND> send;
+    private T receiveEnum;
+    private Class<RECEIVE> ret;
+    public RpcMessageType(T sendEnum, Class<SEND> send, T receiveEnum, Class<RECEIVE> ret) {
+      super();
+      this.sendEnum = sendEnum;
+      this.send = send;
+      this.receiveEnum = receiveEnum;
+      this.ret = ret;
+    }
+    public Class<SEND> getSend() {
+      return send;
+    }
+    public void setSend(Class<SEND> send) {
+      this.send = send;
+    }
+    public T getSendEnum() {
+      return sendEnum;
+    }
+    public void setSendEnum(T sendEnum) {
+      this.sendEnum = sendEnum;
+    }
+    public Class<RECEIVE> getRet() {
+      return ret;
+    }
+    public void setRet(Class<RECEIVE> ret) {
+      this.ret = ret;
+    }
+    public T getReceiveEnum() {
+      return receiveEnum;
+    }
+    public void setReceiveEnum(T receiveEnum) {
+      this.receiveEnum = receiveEnum;
+    }
+    @Override
+    public String toString() {
+      return "RpcMessageType [sendEnum=" + sendEnum + ", send=" + send + ", receiveEnum=" + receiveEnum + ", ret="
+          + ret + "]";
+    }
+    
+    
+  }
+
+  public static RpcConfigBuilder newBuilder(String name){
+    return new RpcConfigBuilder(name);
+  }
+  
+  public static class RpcConfigBuilder {
+    private final String name;
+    private Map<EnumLite, RpcMessageType<?, ?, ?>> sendMap = Maps.newHashMap();
+    private Map<Integer, RpcMessageType<?, ?, ?>> receiveMap = Maps.newHashMap();  
+    
+    private RpcConfigBuilder(String name){
+      this.name = name;
+    }
+    
+    public <SEND extends MessageLite, RECEIVE extends MessageLite, T extends EnumLite>  RpcConfigBuilder add(T sendEnum, Class<SEND> send, T receiveEnum, Class<RECEIVE> rec){
+      RpcMessageType<SEND, RECEIVE, T> type = new RpcMessageType<SEND, RECEIVE, T>(sendEnum, send, receiveEnum, rec);
+      this.sendMap.put(sendEnum, type);
+      this.receiveMap.put(receiveEnum.getNumber(), type);
+      return this;
+    }
+    
+    public RpcConfig build(){
+      return new RpcConfig(name, sendMap, receiveMap);
+
+    }
+  }
+  
+  
+
+}
+
+

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/RpcDecoder.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/RpcDecoder.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/RpcDecoder.java
index 134e54b..4e9714b 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/RpcDecoder.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/RpcDecoder.java
@@ -19,6 +19,7 @@ package org.apache.drill.exec.rpc;
 
 import io.netty.buffer.ByteBuf;
 import io.netty.buffer.ByteBufInputStream;
+import io.netty.buffer.MessageBuf;
 import io.netty.channel.ChannelHandlerContext;
 import io.netty.handler.codec.CorruptedFrameException;
 import io.netty.handler.codec.MessageToMessageDecoder;
@@ -32,14 +33,19 @@ import org.apache.drill.exec.proto.GeneralRPCProtos.RpcHeader;
  * Converts a previously length adjusted buffer into an RpcMessage.
  */
 class RpcDecoder extends MessageToMessageDecoder<ByteBuf> {
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(RpcDecoder.class);
+  final org.slf4j.Logger logger;
   
   private final AtomicLong messageCounter = new AtomicLong();
   
+  public RpcDecoder(String name){
+    this.logger = org.slf4j.LoggerFactory.getLogger(RpcDecoder.class.getCanonicalName() + "." + name);
+  }
+
+  
   @Override
-  protected InboundRpcMessage decode(ChannelHandlerContext ctx, ByteBuf buffer) throws Exception {
+  protected void decode(ChannelHandlerContext ctx, ByteBuf buffer, MessageBuf<Object> out) throws Exception {
     if(!ctx.channel().isOpen()){
-      return null;
+      return;
     }
     
     if (RpcConstants.EXTRA_DEBUGGING) logger.debug("Inbound rpc message received.");
@@ -94,7 +100,7 @@ class RpcDecoder extends MessageToMessageDecoder<ByteBuf> {
     buffer.skipBytes(dBodyLength);
     messageCounter.incrementAndGet();
     if (RpcConstants.EXTRA_DEBUGGING) logger.trace("Inbound Rpc Message Decoded {}.", m);
-    return m;
+    out.add(m);
 
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/RpcEncoder.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/RpcEncoder.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/RpcEncoder.java
index 8d3d97c..f76d648 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/RpcEncoder.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/RpcEncoder.java
@@ -34,7 +34,7 @@ import com.google.protobuf.WireFormat;
  * Converts an RPCMessage into wire format.
  */
 class RpcEncoder extends ChannelOutboundMessageHandlerAdapter<OutboundRpcMessage>{
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(RpcEncoder.class);
+  final org.slf4j.Logger logger;
   
   static final int HEADER_TAG = makeTag(CompleteRpcMessage.HEADER_FIELD_NUMBER, WireFormat.WIRETYPE_LENGTH_DELIMITED);
   static final int PROTOBUF_BODY_TAG = makeTag(CompleteRpcMessage.PROTOBUF_BODY_FIELD_NUMBER, WireFormat.WIRETYPE_LENGTH_DELIMITED);
@@ -43,6 +43,9 @@ class RpcEncoder extends ChannelOutboundMessageHandlerAdapter<OutboundRpcMessage
   static final int PROTOBUF_BODY_TAG_LENGTH = getRawVarintSize(PROTOBUF_BODY_TAG);
   static final int RAW_BODY_TAG_LENGTH = getRawVarintSize(RAW_BODY_TAG);
   
+  public RpcEncoder(String name){
+    this.logger = org.slf4j.LoggerFactory.getLogger(RpcEncoder.class.getCanonicalName() + "." + name);
+  }
   
   @Override
   public void flush(ChannelHandlerContext ctx, OutboundRpcMessage msg) throws Exception {
@@ -61,7 +64,7 @@ class RpcEncoder extends ChannelOutboundMessageHandlerAdapter<OutboundRpcMessage
       // figure out the full length
       int headerLength = header.getSerializedSize();
       int protoBodyLength = msg.pBody.getSerializedSize();
-      int rawBodyLength = msg.dBody == null ? 0 : msg.dBody.readableBytes();
+      int rawBodyLength = msg.getRawBodySize();
       int fullLength = //
           HEADER_TAG_LENGTH + getRawVarintSize(headerLength) + headerLength +   //
           PROTOBUF_BODY_TAG_LENGTH + getRawVarintSize(protoBodyLength) + protoBodyLength; //
@@ -89,11 +92,15 @@ class RpcEncoder extends ChannelOutboundMessageHandlerAdapter<OutboundRpcMessage
       msg.pBody.writeTo(cos);
 
       // if exists, write data body and tag.
-      if(msg.dBody != null && msg.dBody.readableBytes() > 0){
+      // TODO: is it possible to avoid this copy, i think so...
+      if(msg.getRawBodySize() > 0){
+        if(RpcConstants.EXTRA_DEBUGGING) logger.debug("Writing raw body of size {}", msg.getRawBodySize());
         cos.writeRawVarint32(RAW_BODY_TAG);
         cos.writeRawVarint32(rawBodyLength);
         cos.flush(); // need to flush so that dbody goes after if cos is caching.
-        buf.writeBytes(msg.dBody);
+        for(int i =0; i < msg.dBodies.length; i++){
+          buf.writeBytes(msg.dBodies[i]);  
+        }
       }else{
         cos.flush();
       }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/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 ef1b88f..a0aed94 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
@@ -25,28 +25,23 @@ public class RpcExceptionHandler implements ChannelHandler{
   
   public RpcExceptionHandler(){
   }
-  
-  @Override
-  public void beforeAdd(ChannelHandlerContext ctx) throws Exception {
-  }
 
-  @Override
-  public void afterAdd(ChannelHandlerContext ctx) throws Exception {
-  }
 
   @Override
-  public void beforeRemove(ChannelHandlerContext ctx) throws Exception {
+  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);
+    ctx.close();
   }
 
+
   @Override
-  public void afterRemove(ChannelHandlerContext ctx) throws Exception {
+  public void handlerAdded(ChannelHandlerContext ctx) throws Exception {
   }
 
+
   @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);
-    ctx.close();
+  public void handlerRemoved(ChannelHandlerContext ctx) throws Exception {
   }
   
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/RpcMessage.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/RpcMessage.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/RpcMessage.java
index fd1938d..08ea150 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/RpcMessage.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/RpcMessage.java
@@ -27,19 +27,14 @@ public abstract class RpcMessage {
   public RpcMode mode;
   public int rpcType;
   public int coordinationId;
-  public ByteBuf dBody;
   
-  public RpcMessage(RpcMode mode, int rpcType, int coordinationId, ByteBuf dBody) {
+  public RpcMessage(RpcMode mode, int rpcType, int coordinationId) {
     this.mode = mode;
     this.rpcType = rpcType;
     this.coordinationId = coordinationId;
-    this.dBody = dBody;
   }
   
   public abstract int getBodySize();
-
-  void release(){
-    if(dBody != null) dBody.release();
-  }
+  abstract void release();
   
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/RpcOutcomeListener.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/RpcOutcomeListener.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/RpcOutcomeListener.java
new file mode 100644
index 0000000..fac908c
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/RpcOutcomeListener.java
@@ -0,0 +1,28 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.rpc;
+
+public abstract class RpcOutcomeListener<V> {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(RpcOutcomeListener.class);
+  
+  public void failed(RpcException ex){};
+  public void success(V value){};
+  
+  
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/ZeroCopyProtobufLengthDecoder.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/ZeroCopyProtobufLengthDecoder.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/ZeroCopyProtobufLengthDecoder.java
index 462bc52..20a7d7d 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/ZeroCopyProtobufLengthDecoder.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/ZeroCopyProtobufLengthDecoder.java
@@ -18,6 +18,7 @@
 package org.apache.drill.exec.rpc;
 
 import io.netty.buffer.ByteBuf;
+import io.netty.buffer.MessageBuf;
 import io.netty.channel.ChannelHandlerContext;
 import io.netty.handler.codec.ByteToMessageDecoder;
 import io.netty.handler.codec.CorruptedFrameException;
@@ -30,12 +31,14 @@ import com.google.protobuf.CodedInputStream;
 public class ZeroCopyProtobufLengthDecoder extends ByteToMessageDecoder {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ZeroCopyProtobufLengthDecoder.class);
 
+  
   @Override
-  protected ByteBuf decode(ChannelHandlerContext ctx, ByteBuf in) throws Exception {
+  protected void decode(ChannelHandlerContext ctx, ByteBuf in, MessageBuf<Object> out) throws Exception {
+
     if(!ctx.channel().isOpen()){
       logger.info("Channel is closed, discarding remaining {} byte(s) in buffer.", in.readableBytes());
       in.skipBytes(in.readableBytes());
-      return null;
+      return;
     }
     
     in.markReaderIndex();
@@ -43,7 +46,7 @@ public class ZeroCopyProtobufLengthDecoder extends ByteToMessageDecoder {
     for (int i = 0; i < buf.length; i ++) {
         if (!in.isReadable()) {
             in.resetReaderIndex();
-            return null;
+            return;
         }
 
         buf[i] = in.readByte();
@@ -60,13 +63,14 @@ public class ZeroCopyProtobufLengthDecoder extends ByteToMessageDecoder {
 
             if (in.readableBytes() < length) {
                 in.resetReaderIndex();
-                return null;
+                return;
             } else {
-                ByteBuf out = in.slice(in.readerIndex(), length);
+                ByteBuf outBuf = in.slice(in.readerIndex(), length);
                 in.retain();
                 in.skipBytes(length);
                 if(RpcConstants.EXTRA_DEBUGGING) logger.debug(String.format("ReaderIndex is %d after length header of %d bytes and frame body of length %d bytes.", in.readerIndex(), i+1, length));
-                return out;
+                out.add(outBuf);
+                return;
             }
         }
     }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/AvailabilityListener.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/AvailabilityListener.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/AvailabilityListener.java
new file mode 100644
index 0000000..ecaf8d3
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/AvailabilityListener.java
@@ -0,0 +1,22 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.rpc.bit;
+
+public interface AvailabilityListener {
+  public void isAvailable(BitConnection connection);
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/BitClient.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/BitClient.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/BitClient.java
index b16c6cb..4ba99a1 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/BitClient.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/BitClient.java
@@ -18,45 +18,88 @@
 package org.apache.drill.exec.rpc.bit;
 
 import io.netty.buffer.ByteBuf;
-import io.netty.buffer.ByteBufAllocator;
+import io.netty.channel.Channel;
 import io.netty.channel.ChannelFuture;
-import io.netty.channel.EventLoopGroup;
-import io.netty.channel.socket.SocketChannel;
 import io.netty.util.concurrent.GenericFutureListener;
 
+import java.util.concurrent.ConcurrentMap;
+
+import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
+import org.apache.drill.exec.proto.ExecProtos.BitHandshake;
 import org.apache.drill.exec.proto.ExecProtos.RpcType;
 import org.apache.drill.exec.rpc.BasicClient;
 import org.apache.drill.exec.rpc.Response;
+import org.apache.drill.exec.rpc.RpcBus;
 import org.apache.drill.exec.rpc.RpcException;
-import org.apache.drill.exec.server.DrillbitContext;
+import org.apache.drill.exec.server.BootStrapContext;
+import org.apache.drill.exec.work.batch.BitComHandler;
 
 import com.google.protobuf.MessageLite;
 
-public class BitClient  extends BasicClient<RpcType>{
-  
-  private final DrillbitContext context;
+public class BitClient  extends BasicClient<RpcType, BitConnection>{
+
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(BitClient.class);
+
   private final BitComHandler handler;
+  private final DrillbitEndpoint endpoint;
+  private BitConnection connection;
+  private final AvailabilityListener openListener;
+  private final ConcurrentMap<DrillbitEndpoint, BitConnection> registry;
+  private final ListenerPool listeners;
   
-  public BitClient(BitComHandler handler, ByteBufAllocator alloc, EventLoopGroup eventLoopGroup, DrillbitContext context) {
-    super(alloc, eventLoopGroup);
-    this.context = context;
+  public BitClient(DrillbitEndpoint endpoint, AvailabilityListener openListener, BitComHandler handler, BootStrapContext context, ConcurrentMap<DrillbitEndpoint, BitConnection> registry, ListenerPool listeners) {
+    super(BitRpcConfig.MAPPING, context.getAllocator().getUnderlyingAllocator(), context.getBitLoopGroup());
+
+    this.endpoint = endpoint;
     this.handler = handler;
+    this.openListener = openListener;
+    this.registry = registry;
+    this.listeners = listeners;
   }
   
+  public BitHandshake connect() throws RpcException, InterruptedException{
+    BitHandshake bs = connectAsClient(RpcType.HANDSHAKE, BitHandshake.newBuilder().setRpcVersion(BitRpcConfig.RPC_VERSION).build(), endpoint.getAddress(), endpoint.getBitPort(), BitHandshake.class);
+    connection.setEndpoint(endpoint);
+    return bs;
+  }
+
+  @SuppressWarnings("unchecked")
   @Override
-  protected MessageLite getResponseDefaultInstance(int rpcType) throws RpcException {
-    return handler.getResponseDefaultInstance(rpcType);
+  public BitConnection initRemoteConnection(Channel channel) {
+    this.connection = new BitConnection(openListener, channel, (RpcBus<RpcType, BitConnection>) (RpcBus<?, ?>) this, registry, listeners);
+    return connection;
   }
 
   @Override
-  protected Response handle(SocketChannel ch, int rpcType, ByteBuf pBody, ByteBuf dBody) throws RpcException {
-    return handler.handle(context, rpcType, pBody, dBody);
+  protected GenericFutureListener<ChannelFuture> getCloseHandler(BitConnection clientConnection) {
+    return clientConnection.getCloseHandler(super.getCloseHandler(clientConnection));
   }
 
   @Override
-  protected GenericFutureListener<ChannelFuture> getCloseHandler(SocketChannel ch) {
-    return super.getCloseHandler(ch);
+  public MessageLite getResponseDefaultInstance(int rpcType) throws RpcException {
+    return BitComDefaultInstanceHandler.getResponseDefaultInstance(rpcType);
   }
 
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(BitClient.class);
+  @Override
+  protected Response handle(BitConnection connection, int rpcType, ByteBuf pBody, ByteBuf dBody) throws RpcException {
+    return handler.handle(connection, rpcType, pBody, dBody);
+  }
+
+  @Override
+  protected ClientHandshakeHandler<BitHandshake> getHandshakeHandler() {
+    return new ClientHandshakeHandler<BitHandshake>(RpcType.HANDSHAKE, BitHandshake.class, BitHandshake.PARSER){
+
+      @Override
+      protected void validateHandshake(BitHandshake inbound) throws Exception {
+        logger.debug("Handling handshake from bit server to bit client. {}", inbound);
+        if(inbound.getRpcVersion() != BitRpcConfig.RPC_VERSION) throw new RpcException(String.format("Invalid rpc version.  Expected %d, actual %d.", inbound.getRpcVersion(), BitRpcConfig.RPC_VERSION));
+      }
+
+    };
+  }
+  
+  public BitConnection getConnection(){
+    return this.connection;
+  }
+  
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/BitCom.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/BitCom.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/BitCom.java
index 2349899..c60d36b 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/BitCom.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/BitCom.java
@@ -17,73 +17,34 @@
  ******************************************************************************/
 package org.apache.drill.exec.rpc.bit;
 
-import io.netty.channel.ChannelFuture;
-import io.netty.channel.socket.SocketChannel;
-import io.netty.util.concurrent.GenericFutureListener;
-
 import java.io.Closeable;
-import java.util.Collection;
 
-import org.apache.drill.common.proto.CoordinationProtos.DrillbitEndpoint;
-import org.apache.drill.exec.ops.FragmentContext;
-import org.apache.drill.exec.proto.ExecProtos.FragmentHandle;
-import org.apache.drill.exec.proto.ExecProtos.FragmentStatus;
-import org.apache.drill.exec.proto.ExecProtos.PlanFragment;
-import org.apache.drill.exec.proto.GeneralRPCProtos.Ack;
-import org.apache.drill.exec.record.RecordBatch;
-import org.apache.drill.exec.rpc.DrillRpcFuture;
-import org.apache.drill.exec.rpc.RpcBus;
+import org.apache.drill.exec.exception.DrillbitStartupException;
+import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
+import org.apache.drill.exec.work.fragment.IncomingFragmentHandler;
 
 /**
- * Service that allows one Drillbit to communicate with another. Internally manages whether each particular bit is a server
- * or a client depending on who initially made the connection. If no connection exists, BitCom is
- * responsible for making a connection.  BitCom should automatically straight route local BitCommunication rather than connecting to its self.
+ * Service that allows one Drillbit to communicate with another. Internally manages whether each particular bit is a
+ * server or a client depending on who initially made the connection. If no connection exists, BitCom is responsible for
+ * making a connection. BitCom should automatically straight route local BitCommunication rather than connecting to its
+ * self.
  */
-public interface BitCom extends Closeable{
+public interface BitCom extends Closeable {
 
   /**
-   * Routes the output of a RecordBatch to another node.  The record batch
-   * @param node The node id to send the record batch to.
-   * @param batch The record batch to send.
-   * @return A SendProgress object which can be used to monitor the sending of the batch.
-   */
-  public abstract DrillRpcFuture<Ack> sendRecordBatch(FragmentContext context, DrillbitEndpoint node, RecordBatch batch);
-
-  
-  /**
-   * Requests an iterator to access an incoming record batch.  
-   * @param fragmentId
-   * @return
-   */
-  public RecordBatch getReceivingRecordBatchHandle(int majorFragmentId, int minorFragmentId);
-  
-  /**
-   * Send a query PlanFragment to another bit.   
-   * @param context
+   * Get a Bit to Bit communication tunnel. If the BitCom doesn't have a tunnel attached to the node already, it will
+   * start creating one. This create the connection asynchronously.
+   * 
    * @param node
-   * @param fragment
    * @return
    */
-  public abstract DrillRpcFuture<FragmentHandle> sendFragment(FragmentContext context, DrillbitEndpoint node, PlanFragment fragment);
+  public BitTunnel getTunnel(DrillbitEndpoint node) ;
 
-  public abstract void startQuery(Collection<DrillbitEndpoint> firstNodes, long queryId);
-    
-  
-  public abstract DrillRpcFuture<Ack> cancelFragment(FragmentContext context, DrillbitEndpoint node, FragmentHandle handle);
-  
-  public abstract DrillRpcFuture<FragmentStatus> getFragmentStatus(FragmentContext context, DrillbitEndpoint node, FragmentHandle handle);
-  
-  
-  public interface TunnelListener extends GenericFutureListener<ChannelFuture> {
-    public void connectionEstablished(SocketChannel channel, DrillbitEndpoint endpoint, RpcBus<?> bus);
-  }
-  
-  public interface SendManager{
-    /**
-     * Sender responsible for regularly checking this value to see whether it should continue to send or yield the process
-     * @return
-     */
-    public boolean canContinue();
-  }
+  public int start() throws InterruptedException, DrillbitStartupException;
 
+  /**
+   * Register an incoming batch handler for a local foreman.  
+   * @param handler
+   */
+  public void registerIncomingBatchHandler(IncomingFragmentHandler handler);
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/BitComDefaultInstanceHandler.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/BitComDefaultInstanceHandler.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/BitComDefaultInstanceHandler.java
new file mode 100644
index 0000000..e1d4902
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/BitComDefaultInstanceHandler.java
@@ -0,0 +1,51 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.rpc.bit;
+
+import org.apache.drill.exec.proto.ExecProtos.BitHandshake;
+import org.apache.drill.exec.proto.ExecProtos.BitStatus;
+import org.apache.drill.exec.proto.ExecProtos.FragmentHandle;
+import org.apache.drill.exec.proto.ExecProtos.FragmentStatus;
+import org.apache.drill.exec.proto.ExecProtos.RpcType;
+import org.apache.drill.exec.proto.GeneralRPCProtos.Ack;
+import org.apache.drill.exec.rpc.RpcException;
+
+import com.google.protobuf.MessageLite;
+
+public class BitComDefaultInstanceHandler {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(BitComDefaultInstanceHandler.class);
+  
+
+  public static MessageLite getResponseDefaultInstance(int rpcType) throws RpcException {
+    switch (rpcType) {
+    case RpcType.ACK_VALUE:
+      return Ack.getDefaultInstance();
+    case RpcType.HANDSHAKE_VALUE:
+      return BitHandshake.getDefaultInstance();
+    case RpcType.RESP_FRAGMENT_HANDLE_VALUE:
+      return FragmentHandle.getDefaultInstance();
+    case RpcType.RESP_FRAGMENT_STATUS_VALUE:
+      return FragmentStatus.getDefaultInstance();
+    case RpcType.RESP_BIT_STATUS_VALUE:
+      return BitStatus.getDefaultInstance();
+      
+    default:
+      throw new UnsupportedOperationException();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/BitComHandler.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/BitComHandler.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/BitComHandler.java
deleted file mode 100644
index b2c5cbb..0000000
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/BitComHandler.java
+++ /dev/null
@@ -1,136 +0,0 @@
-/*******************************************************************************
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- ******************************************************************************/
-package org.apache.drill.exec.rpc.bit;
-
-import io.netty.buffer.ByteBuf;
-import io.netty.channel.ChannelFuture;
-import io.netty.channel.socket.SocketChannel;
-
-import org.apache.drill.common.proto.CoordinationProtos.DrillbitEndpoint;
-import org.apache.drill.exec.proto.ExecProtos.BitBatchChunk;
-import org.apache.drill.exec.proto.ExecProtos.BitHandshake;
-import org.apache.drill.exec.proto.ExecProtos.BitStatus;
-import org.apache.drill.exec.proto.ExecProtos.FragmentHandle;
-import org.apache.drill.exec.proto.ExecProtos.FragmentStatus;
-import org.apache.drill.exec.proto.ExecProtos.RpcType;
-import org.apache.drill.exec.proto.GeneralRPCProtos.Ack;
-import org.apache.drill.exec.rpc.Response;
-import org.apache.drill.exec.rpc.RpcBus;
-import org.apache.drill.exec.rpc.RpcException;
-import org.apache.drill.exec.rpc.bit.BitCom.TunnelListener;
-import org.apache.drill.exec.rpc.bit.BitComImpl.TunnelModifier;
-import org.apache.drill.exec.server.DrillbitContext;
-
-import com.google.protobuf.MessageLite;
-
-public class BitComHandler {
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(BitComHandler.class);
-  
-  private final TunnelModifier modifier;
-  
-  public BitComHandler(TunnelModifier modifier){
-    this.modifier = modifier;
-  }
-  
-  public TunnelListener getTunnelListener(RpcBus<?>.ChannelClosedHandler internalHandler){
-    return new Listener(internalHandler);
-  }
-  
-  public class Listener implements TunnelListener {
-    final RpcBus<?>.ChannelClosedHandler internalHandler;
-
-    public Listener(RpcBus<?>.ChannelClosedHandler internalHandler) {
-      this.internalHandler = internalHandler;
-    }
-
-    @Override
-    public void operationComplete(ChannelFuture future) throws Exception {
-      logger.debug("BitTunnel closed, removing from BitCom.");
-      internalHandler.operationComplete(future);
-      BitTunnel t = modifier.remove(future.channel());
-      if(t != null) t.shutdownIfClient();
-    }
-
-    @Override
-    public void connectionEstablished(SocketChannel channel, DrillbitEndpoint endpoint, RpcBus<?> bus) {
-      modifier.create(channel, endpoint, bus);
-    }
-
-  }
-
-  
-
-
-  public MessageLite getResponseDefaultInstance(int rpcType) throws RpcException {
-    switch (rpcType) {
-    case RpcType.ACK_VALUE:
-      return Ack.getDefaultInstance();
-    case RpcType.HANDSHAKE_VALUE:
-      return BitHandshake.getDefaultInstance();
-    case RpcType.RESP_FRAGMENT_HANDLE_VALUE:
-      return FragmentHandle.getDefaultInstance();
-    case RpcType.RESP_FRAGMENT_STATUS_VALUE:
-      return FragmentStatus.getDefaultInstance();
-    case RpcType.RESP_BIT_STATUS_VALUE:
-      return BitStatus.getDefaultInstance();
-    case RpcType.RESP_BATCH_CHUNK_VALUE:
-      return BitBatchChunk.getDefaultInstance();
-      
-    default:
-      throw new UnsupportedOperationException();
-    }
-  }
-
-  protected Response handle(DrillbitContext context, int rpcType, ByteBuf pBody, ByteBuf dBody) throws RpcException {
-    switch (rpcType) {
-    
-    case RpcType.HANDSHAKE_VALUE:
-      // parse incoming handshake.
-      // get endpoint information.
-      // record endpoint information in registry.
-      // respond with our handshake info.
-      return new Response(RpcType.HANDSHAKE, BitHandshake.getDefaultInstance(), null);
-      
-    case RpcType.REQ_BATCH_CHUNK_VALUE:
-      return new Response(RpcType.RESP_BATCH_CHUNK, BitBatchChunk.getDefaultInstance(), null);
-      
-    case RpcType.REQ_BIT_STATUS_VALUE:
-      return new Response(RpcType.RESP_BIT_STATUS, BitStatus.getDefaultInstance(), null);
-      
-    case RpcType.REQ_CANCEL_FRAGMENT_VALUE:
-      return new Response(RpcType.ACK, Ack.getDefaultInstance(), null);
-
-    case RpcType.REQ_FRAGMENT_STATUS_VALUE:
-      return new Response(RpcType.RESP_FRAGMENT_STATUS, FragmentStatus.getDefaultInstance(), null);
-      
-    case RpcType.REQ_INIATILIZE_FRAGMENT_VALUE:
-      return new Response(RpcType.ACK, Ack.getDefaultInstance(), null);
-      
-    case RpcType.REQ_RECORD_BATCH_VALUE:
-      return new Response(RpcType.RESP_BATCH_CHUNK, BitBatchChunk.getDefaultInstance(), null);
-      
-    default:
-      throw new UnsupportedOperationException();
-    }
-
-  }
-  
-
-  
-  
-}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/BitComImpl.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/BitComImpl.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/BitComImpl.java
index aada154..c98be44 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/BitComImpl.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/BitComImpl.java
@@ -17,138 +17,158 @@
  ******************************************************************************/
 package org.apache.drill.exec.rpc.bit;
 
-import io.netty.channel.Channel;
-import io.netty.channel.socket.SocketChannel;
-import io.netty.util.concurrent.Future;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Executor;
+import java.util.concurrent.Executors;
 
-import java.util.Collection;
-import java.util.Map;
-
-import org.apache.drill.common.proto.CoordinationProtos.DrillbitEndpoint;
 import org.apache.drill.exec.ExecConstants;
 import org.apache.drill.exec.exception.DrillbitStartupException;
-import org.apache.drill.exec.ops.FragmentContext;
-import org.apache.drill.exec.proto.ExecProtos.FragmentHandle;
-import org.apache.drill.exec.proto.ExecProtos.FragmentStatus;
-import org.apache.drill.exec.proto.ExecProtos.PlanFragment;
-import org.apache.drill.exec.proto.GeneralRPCProtos.Ack;
-import org.apache.drill.exec.record.RecordBatch;
-import org.apache.drill.exec.rpc.DrillRpcFuture;
-import org.apache.drill.exec.rpc.RpcBus;
-import org.apache.drill.exec.server.DrillbitContext;
+import org.apache.drill.exec.exception.FragmentSetupException;
+import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
+import org.apache.drill.exec.proto.ExecProtos.BitHandshake;
+import org.apache.drill.exec.proto.ExecProtos.RpcType;
+import org.apache.drill.exec.rpc.NamedThreadFactory;
+import org.apache.drill.exec.rpc.RpcException;
+import org.apache.drill.exec.server.BootStrapContext;
+import org.apache.drill.exec.work.batch.BitComHandler;
+import org.apache.drill.exec.work.fragment.IncomingFragmentHandler;
 
 import com.google.common.collect.Maps;
 import com.google.common.io.Closeables;
-
+import com.google.common.util.concurrent.AbstractCheckedFuture;
+import com.google.common.util.concurrent.CheckedFuture;
+import com.google.common.util.concurrent.ListenableFuture;
+import com.google.common.util.concurrent.SettableFuture;
+
+/**
+ * Manages communication tunnels between nodes.   
+ */
 public class BitComImpl implements BitCom {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(BitComImpl.class);
 
-  private Map<DrillbitEndpoint, BitTunnel> tunnels = Maps.newConcurrentMap();
-  private Map<SocketChannel, DrillbitEndpoint> endpoints = Maps.newConcurrentMap();
-  private Object lock = new Object();
-  private BitServer server;
-  private DrillbitContext context;
+  private final ConcurrentMap<DrillbitEndpoint, BitConnection> registry = Maps.newConcurrentMap();
+  private final ListenerPool listeners;
+  private volatile BitServer server;
+  private final BitComHandler handler;
+  private final BootStrapContext context;
+  
+  // TODO: this executor should be removed.
+  private final Executor exec = Executors.newCachedThreadPool(new NamedThreadFactory("BitComImpl execution pool: "));
 
-  public BitComImpl(DrillbitContext context) {
+  public BitComImpl(BootStrapContext context, BitComHandler handler) {
+    super();
+    this.handler = handler;
     this.context = context;
+    this.listeners = new ListenerPool(8);
   }
 
   public int start() throws InterruptedException, DrillbitStartupException {
-    server = new BitServer(new BitComHandler(modifier), context.getAllocator().getUnderlyingAllocator(), context.getBitLoopGroup(), context);
+    server = new BitServer(handler, context, registry, listeners);
     int port = context.getConfig().getInt(ExecConstants.INITIAL_BIT_PORT);
     return server.bind(port);
   }
 
-  private Future<BitTunnel> getNode(DrillbitEndpoint endpoint) {
-    return null;
+  private CheckedFuture<BitConnection, RpcException> getNode(final DrillbitEndpoint endpoint, boolean check) {
+    
+    
+    SettableFuture<BitConnection> future = SettableFuture.create();
+    BitComFuture<BitConnection> checkedFuture = new BitComFuture<BitConnection>(future);
+    BitConnection t = null;
+
+    if (check) {
+      t = registry.get(endpoint);
+
+      if (t != null) {
+        future.set(t);
+        return checkedFuture;
+      }
+    }
     
-//    BitTunnel t = tunnels.get(endpoint);
-//    if (t == null) {
-//      synchronized (lock) {
-//        t = tunnels.get(endpoint);
-//        if (t != null) return t;
-//        BitClient c = new BitClient(new BitComHandler(modifier), context.getAllocator().getUnderlyingAllocator(),
-//            context.getBitLoopGroup(), context);
-//
-//        // need to figure what to do here with regards to waiting for handshake before returning. Probably need to add
-//        // future registry so that new endpoint registration ping the registry.
-//        throw new UnsupportedOperationException();
-//        c.connectAsClient(endpoint.getAddress(), endpoint.getBitPort()).await();
-//        t = new BitTunnel(c);
-//        tunnels.put(endpoint, t);
-//
-//      }
-//    }
-//    return null;
+    try {
+      AvailWatcher watcher = new AvailWatcher(future);
+      BitClient c = new BitClient(endpoint, watcher, handler, context, registry, listeners);
+      c.connect();
+      return checkedFuture;
+    } catch (InterruptedException | RpcException e) {
+      future.setException(new FragmentSetupException("Unable to open connection"));
+      return checkedFuture;
+    }
+
   }
 
-  
+  private class AvailWatcher implements AvailabilityListener{
+    final SettableFuture<BitConnection> future;
+    
+    public AvailWatcher(SettableFuture<BitConnection> future) {
+      super();
+      this.future = future;
+    }
 
-  @Override
-  public DrillRpcFuture<FragmentHandle> sendFragment(FragmentContext context, DrillbitEndpoint node,
-      PlanFragment fragment) {
-    return null;
+    @Override
+    public void isAvailable(BitConnection connection) {
+      future.set(connection);
+    }
+    
+  }
+  
+  BitConnection getConnection(DrillbitEndpoint endpoint) throws RpcException {
+    BitConnection t = registry.get(endpoint);
+    if(t != null) return t;
+    return this.getNode(endpoint, false).checkedGet();
   }
 
-  @Override
-  public DrillRpcFuture<Ack> cancelFragment(FragmentContext context, DrillbitEndpoint node, FragmentHandle handle) {
-    return null;
+  
+  CheckedFuture<BitConnection, RpcException> getConnectionAsync(DrillbitEndpoint endpoint) {
+    return this.getNode(endpoint, true);
   }
 
+  
   @Override
-  public DrillRpcFuture<FragmentStatus> getFragmentStatus(FragmentContext context, DrillbitEndpoint node,
-      FragmentHandle handle) {
-    return null;
+  public BitTunnel getTunnel(DrillbitEndpoint endpoint){
+    BitConnection t = registry.get(endpoint);
+    if(t == null){
+      return new BitTunnel(exec, endpoint, this, t);
+    }else{
+      return new BitTunnel(exec, endpoint, this,  this.getNode(endpoint, false));
+    }
   }
 
-  private final TunnelModifier modifier = new TunnelModifier();
 
   /**
-   * Fully synchronized modifier. Contention should be low since endpoints shouldn't be constantly changing.
+   * A future which remaps exceptions to a BitComException.
+   * @param <T>
    */
-  class TunnelModifier {
-    public BitTunnel remove(Channel ch) {
-      synchronized (this) {
-        DrillbitEndpoint endpoint = endpoints.remove(ch);
-        if (endpoint == null) {
-          logger
-              .warn("We attempted to find a endpoint from a provided channel and found none.  This suggests a race condition or memory leak problem.");
-          return null;
-        }
-
-        BitTunnel tunnel = tunnels.remove(endpoint);
-        return tunnel;
-      }
+  private class BitComFuture<T> extends AbstractCheckedFuture<T, RpcException>{
+
+    protected BitComFuture(ListenableFuture<T> delegate) {
+      super(delegate);
     }
 
-    public void create(SocketChannel channel, DrillbitEndpoint endpoint, RpcBus<?> bus) {
-      synchronized (this) {
-        endpoints.put(channel, endpoint);
-        tunnels.put(endpoint, new BitTunnel(bus));
+    @Override
+    protected RpcException mapException(Exception e) {
+      Throwable t = e;
+      if(e instanceof ExecutionException){
+        t = e.getCause();
       }
+      
+      if(t instanceof RpcException) return (RpcException) t;
+      return new RpcException(t);
     }
   }
 
   public void close() {
     Closeables.closeQuietly(server);
-    for (BitTunnel bt : tunnels.values()) {
+    for (BitConnection bt : registry.values()) {
       bt.shutdownIfClient();
     }
   }
 
-
   @Override
-  public DrillRpcFuture<Ack> sendRecordBatch(FragmentContext context, DrillbitEndpoint node, RecordBatch batch) {
-    return null;
-  }
-
-  @Override
-  public RecordBatch getReceivingRecordBatchHandle(int majorFragmentId, int minorFragmentId) {
-    return null;
-  }
-
-  @Override
-  public void startQuery(Collection<DrillbitEndpoint> firstNodes, long queryId) {
+  public void registerIncomingBatchHandler(IncomingFragmentHandler handler) {
+    this.handler.registerIncomingFragmentHandler(handler);
   }
+  
+  
 
 }


[38/53] [abbrv] Types transition

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/sandbox/prototype/pom.xml
----------------------------------------------------------------------
diff --git a/sandbox/prototype/pom.xml b/sandbox/prototype/pom.xml
index 8dec4d1..a09d380 100644
--- a/sandbox/prototype/pom.xml
+++ b/sandbox/prototype/pom.xml
@@ -1,175 +1,212 @@
 <?xml version="1.0" encoding="UTF-8"?>
 <project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
-	xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
-	<modelVersion>4.0.0</modelVersion>
-
-	<parent>
-		<groupId>org.apache</groupId>
-		<artifactId>apache</artifactId>
-		<version>9</version>
-	</parent>
-
-	<groupId>org.apache.drill</groupId>
-	<artifactId>prototype-parent</artifactId>
-	<version>1.0-SNAPSHOT</version>
-	<packaging>pom</packaging>
-
-	<name>prototype-parent</name>
-     <repositories>
-        <repository>
-                <releases>
-                        <enabled>true</enabled>
-                        <updatePolicy>always</updatePolicy>
-                        <checksumPolicy>warn</checksumPolicy>
-                </releases>
-                <id>conjars</id>
-                <name>Conjars</name>
-                <url>http://conjars.org/repo</url>
-                <layout>default</layout>
-            </repository>
-        <repository>
-          <id>sonatype-nexus-snapshots</id>
-          <url>https://oss.sonatype.org/content/repositories/snapshots</url>
-          <releases>
-                <enabled>false</enabled>
-          </releases>
-          <snapshots>
-                <enabled>true</enabled>
-          </snapshots>
-         </repository>
-     </repositories>
-	<properties>
-		<dep.junit.version>4.11</dep.junit.version>
-		<dep.slf4j.version>1.7.2</dep.slf4j.version>
-	</properties>
-
-	<build>
-
-		<plugins>
-			<plugin>
-				<groupId>org.apache.maven.plugins</groupId>
-				<artifactId>maven-jar-plugin</artifactId>
-				<executions>
-					<execution>
-						<goals>
-							<goal>test-jar</goal>
-						</goals>
-					</execution>
-				</executions>
-			</plugin>
-			<plugin>
-				<groupId>org.apache.maven.plugins</groupId>
-				<artifactId>maven-resources-plugin</artifactId>
-				<version>2.6</version>
-				<configuration>
-					<encoding>UTF-8</encoding>
-				</configuration>
-			</plugin>
-			<plugin>
-				<groupId>org.apache.maven.plugins</groupId>
-				<artifactId>maven-compiler-plugin</artifactId>
-				<version>3.0</version>
-				<configuration>
-					<source>1.7</source>
-					<target>1.7</target>
-				</configuration>
-			</plugin>
-			<plugin>
-				<artifactId>maven-enforcer-plugin</artifactId>
-				<version>1.2</version>
-				<executions>
-					<execution>
-						<id>no_commons_logging</id>
-						<goals>
-							<goal>enforce</goal>
-						</goals>
-						<configuration>
-							<rules>
-								<bannedDependencies>
-									<excludes>
-										<exclude>commons-logging</exclude>
-									</excludes>
-								</bannedDependencies>
-							</rules>
-						</configuration>
-					</execution>
-				</executions>
-			</plugin>
-		</plugins>
-	</build>
-	<dependencies>
-
-		<dependency>
-			<groupId>com.google.guava</groupId>
-			<artifactId>guava</artifactId>
-			<version>13.0.1</version>
-		</dependency>
-
-		<dependency>
-			<groupId>org.slf4j</groupId>
-			<artifactId>slf4j-api</artifactId>
-			<version>${dep.slf4j.version}</version>
-		</dependency>
-
-		<dependency>
-			<groupId>org.slf4j</groupId>
-			<artifactId>jul-to-slf4j</artifactId>
-			<version>${dep.slf4j.version}</version>
-		</dependency>
-
-		<dependency>
-			<groupId>org.slf4j</groupId>
-			<artifactId>jcl-over-slf4j</artifactId>
-			<version>${dep.slf4j.version}</version>
-		</dependency>
-
-		<dependency>
-			<groupId>org.slf4j</groupId>
-			<artifactId>log4j-over-slf4j</artifactId>
-			<version>${dep.slf4j.version}</version>
-		</dependency>
-
-		<!-- Test Dependencies -->
-		<dependency>
-		  <groupId>com.googlecode.jmockit</groupId>
-		  <artifactId>jmockit</artifactId>
-		  <version>1.2</version>
-		  <scope>test</scope>
-		</dependency>
-		<dependency>
-			<groupId>junit</groupId>
-			<artifactId>junit</artifactId>
-			<version>${dep.junit.version}</version>
-			<scope>test</scope>
-		</dependency>
-		<dependency>
-			<groupId>ch.qos.logback</groupId>
-			<artifactId>logback-classic</artifactId>
-			<version>1.0.7</version>
-			<scope>test</scope>
-		</dependency>
-		<dependency>
-			<groupId>de.huxhorn.lilith</groupId>
-			<artifactId>de.huxhorn.lilith.logback.appender.multiplex-classic</artifactId>
-			<version>0.9.43</version>
-			<scope>test</scope>
-		</dependency>
-
-
-
-	</dependencies>
-
-	<!-- Managed Dependencies -->
-	<dependencyManagement>
-		<dependencies>
-
-		</dependencies>
-	</dependencyManagement>
-	<modules>
-		<module>common</module>
-		<module>contrib</module>
-		<module>exec</module>
-		<module>planner</module>
-		<module>sqlparser</module>
-	</modules>
+  xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+  <modelVersion>4.0.0</modelVersion>
+
+  <parent>
+    <groupId>org.apache</groupId>
+    <artifactId>apache</artifactId>
+    <version>9</version>
+  </parent>
+
+  <groupId>org.apache.drill</groupId>
+  <artifactId>prototype-parent</artifactId>
+  <version>1.0-SNAPSHOT</version>
+  <packaging>pom</packaging>
+
+  <name>prototype-parent</name>
+
+  <properties>
+    <target.gen.source.path>${project.basedir}/target/generated-sources</target.gen.source.path>
+    <proto.cas.path>${project.basedir}/src/main/protobuf/</proto.cas.path>
+    <dep.junit.version>4.11</dep.junit.version>
+    <dep.slf4j.version>1.7.2</dep.slf4j.version>
+  </properties>
+
+
+  <repositories>
+    <repository>
+      <releases>
+        <enabled>true</enabled>
+        <updatePolicy>always</updatePolicy>
+        <checksumPolicy>warn</checksumPolicy>
+      </releases>
+      <id>conjars</id>
+      <name>Conjars</name>
+      <url>http://conjars.org/repo</url>
+      <layout>default</layout>
+    </repository>
+    <repository>
+      <id>sonatype-nexus-snapshots</id>
+      <url>https://oss.sonatype.org/content/repositories/snapshots</url>
+      <releases>
+        <enabled>false</enabled>
+      </releases>
+      <snapshots>
+        <enabled>true</enabled>
+      </snapshots>
+    </repository>
+  </repositories>
+
+
+
+  <build>
+
+    <plugins>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-jar-plugin</artifactId>
+        <executions>
+          <execution>
+            <goals>
+              <goal>test-jar</goal>
+            </goals>
+          </execution>
+        </executions>
+      </plugin>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-resources-plugin</artifactId>
+        <version>2.6</version>
+        <configuration>
+          <encoding>UTF-8</encoding>
+        </configuration>
+      </plugin>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-compiler-plugin</artifactId>
+        <version>3.0</version>
+        <configuration>
+          <source>1.7</source>
+          <target>1.7</target>
+        </configuration>
+      </plugin>
+      <plugin>
+        <artifactId>maven-enforcer-plugin</artifactId>
+        <version>1.2</version>
+        <executions>
+          <execution>
+            <id>no_commons_logging</id>
+            <goals>
+              <goal>enforce</goal>
+            </goals>
+            <configuration>
+              <rules>
+                <bannedDependencies>
+                  <excludes>
+                    <exclude>commons-logging</exclude>
+                  </excludes>
+                </bannedDependencies>
+              </rules>
+            </configuration>
+          </execution>
+        </executions>
+      </plugin>
+    </plugins>
+    <pluginManagement>
+      <plugins>
+        <!--This plugin's configuration is used to store Eclipse m2e settings 
+          only. It has no influence on the Maven build itself. -->
+        <plugin>
+          <groupId>org.eclipse.m2e</groupId>
+          <artifactId>lifecycle-mapping</artifactId>
+          <version>1.0.0</version>
+          <configuration>
+            <lifecycleMappingMetadata>
+              <pluginExecutions>
+                <pluginExecution>
+                  <pluginExecutionFilter>
+                    <groupId>org.apache.maven.plugins</groupId>
+                    <artifactId>maven-antrun-plugin</artifactId>
+                    <versionRange>[1.6,)</versionRange>
+                    <goals>
+                      <goal>run</goal>
+                    </goals>
+                  </pluginExecutionFilter>
+                  <action>
+                    <ignore></ignore>
+                  </action>
+                </pluginExecution>
+              </pluginExecutions>
+            </lifecycleMappingMetadata>
+          </configuration>
+        </plugin>
+      </plugins>
+    </pluginManagement>
+  </build>
+  <dependencies>
+
+    <dependency>
+      <groupId>com.google.guava</groupId>
+      <artifactId>guava</artifactId>
+      <version>14.0.1</version>
+    </dependency>
+
+    <dependency>
+      <groupId>org.slf4j</groupId>
+      <artifactId>slf4j-api</artifactId>
+      <version>${dep.slf4j.version}</version>
+    </dependency>
+
+    <dependency>
+      <groupId>org.slf4j</groupId>
+      <artifactId>jul-to-slf4j</artifactId>
+      <version>${dep.slf4j.version}</version>
+    </dependency>
+
+    <dependency>
+      <groupId>org.slf4j</groupId>
+      <artifactId>jcl-over-slf4j</artifactId>
+      <version>${dep.slf4j.version}</version>
+    </dependency>
+
+    <dependency>
+      <groupId>org.slf4j</groupId>
+      <artifactId>log4j-over-slf4j</artifactId>
+      <version>${dep.slf4j.version}</version>
+    </dependency>
+
+    <!-- Test Dependencies -->
+    <dependency>
+      <groupId>com.googlecode.jmockit</groupId>
+      <artifactId>jmockit</artifactId>
+      <version>1.2</version>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>junit</groupId>
+      <artifactId>junit</artifactId>
+      <version>${dep.junit.version}</version>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>ch.qos.logback</groupId>
+      <artifactId>logback-classic</artifactId>
+      <version>1.0.7</version>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>de.huxhorn.lilith</groupId>
+      <artifactId>de.huxhorn.lilith.logback.appender.multiplex-classic</artifactId>
+      <version>0.9.43</version>
+      <scope>test</scope>
+    </dependency>
+
+
+
+  </dependencies>
+
+  <!-- Managed Dependencies -->
+  <dependencyManagement>
+    <dependencies>
+
+    </dependencies>
+  </dependencyManagement>
+  <modules>
+    <module>common</module>
+    <module>contrib</module>
+    <module>exec</module>
+    <module>planner</module>
+    <module>sqlparser</module>
+  </modules>
 </project>


[25/53] [abbrv] git commit: Merge branch 'master' into execwork

Posted by ja...@apache.org.
Merge branch 'master' into execwork

Conflicts:
	sandbox/prototype/common/src/main/java/org/apache/drill/common/logical/data/Order.java
	sandbox/prototype/exec/ref/src/main/java/org/apache/drill/exec/ref/rops/OrderROP.java
	sandbox/prototype/exec/ref/src/main/java/org/apache/drill/exec/ref/rse/QueueRSE.java
	sandbox/prototype/sqlparser/src/main/java/org/apache/drill/jdbc/Driver.java
	sandbox/prototype/sqlparser/src/main/java/org/apache/drill/optiq/DrillScan.java


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

Branch: refs/heads/master
Commit: 5c07ccde62323ad409562a2bfb70754bc55bb939
Parents: ea52111 3a11d29
Author: Jacques Nadeau <ja...@apache.org>
Authored: Thu Jun 6 12:00:16 2013 -0700
Committer: Jacques Nadeau <ja...@apache.org>
Committed: Thu Jun 6 12:00:16 2013 -0700

----------------------------------------------------------------------
 .../apache/drill/common/logical/data/Order.java |  61 +++++-
 .../apache/drill/common/logical/data/Union.java |  10 +-
 .../java/org/apache/drill/common/util/Hook.java |  79 ++++++++
 .../common/src/test/resources/donuts-model.json |  19 ++
 .../org/apache/drill/exec/ref/ROPConverter.java |   3 +-
 .../drill/exec/ref/ReferenceInterpreter.java    |  68 ++++++-
 .../apache/drill/exec/ref/UnbackedRecord.java   |  22 +++
 .../exec/ref/rops/CollapsingAggregateROP.java   |  10 +-
 .../org/apache/drill/exec/ref/rops/JoinROP.java |  12 +-
 .../apache/drill/exec/ref/rops/OrderROP.java    |  18 +-
 .../apache/drill/exec/ref/rops/UnionROP.java    |  79 +++++---
 .../org/apache/drill/exec/ref/rse/QueueRSE.java |  51 +++--
 .../apache/drill/exec/ref/rse/RSERegistry.java  |   4 +-
 .../apache/drill/exec/ref/values/DataValue.java |   1 +
 .../drill/exec/ref/values/ScalarValues.java     |   4 +-
 .../drill/exec/ref/values/SimpleMapValue.java   |  21 ++
 .../apache/drill/exec/ref/RSERegistryTest.java  |  40 ++++
 .../org/apache/drill/exec/ref/TestUtils.java    |  82 +++++++-
 .../exec/ref/rops/CollapsingAggregateTest.java  |  50 +++++
 .../apache/drill/exec/ref/rops/JoinROPTest.java |  39 ++++
 .../drill/exec/ref/rops/OrderROPTest.java       |  60 ++++++
 .../drill/exec/ref/rops/UnionROPTest.java       |  46 +++++
 .../ref/src/test/resources/collapse/test1.json  |  73 +++++++
 .../src/test/resources/join/departments.json    |  16 ++
 .../ref/src/test/resources/join/employees.json  |  24 +++
 .../ref/src/test/resources/join/full_join.json  |  72 +++++++
 .../ref/src/test/resources/join/left_join.json  |  71 +++++++
 .../src/test/resources/join/simple_join.json    |  71 +++++++
 .../src/test/resources/order/nulls-first.json   |  79 ++++++++
 .../src/test/resources/order/nulls-last.json    |  79 ++++++++
 .../ref/src/test/resources/union/distinct.json  |  82 ++++++++
 .../src/test/resources/union/nondistinct.json   |  82 ++++++++
 sandbox/prototype/sqlparser/pom.xml             |  10 +-
 .../java/org/apache/drill/jdbc/DrillTable.java  |  71 +++++--
 .../main/java/org/apache/drill/jdbc/Driver.java |  68 +++----
 .../org/apache/drill/optiq/DrillFilterRel.java  |  66 +++++++
 .../org/apache/drill/optiq/DrillFilterRule.java |  52 +++++
 .../apache/drill/optiq/DrillImplementor.java    |   5 +
 .../java/org/apache/drill/optiq/DrillOptiq.java | 103 ++++++++--
 .../apache/drill/optiq/DrillPrepareImpl.java    |  47 +++++
 .../org/apache/drill/optiq/DrillProjectRel.java |  88 +++++++++
 .../apache/drill/optiq/DrillProjectRule.java    |  51 +++++
 .../java/org/apache/drill/optiq/DrillRel.java   |   5 +
 .../java/org/apache/drill/optiq/DrillScan.java  |   7 +-
 .../org/apache/drill/optiq/DrillValuesRel.java  |  58 ++++++
 .../org/apache/drill/optiq/DrillValuesRule.java |  48 +++++
 .../org/apache/drill/optiq/EnumerableDrill.java | 145 ++++++++------
 .../apache/drill/optiq/EnumerableDrillRel.java  |  53 +++--
 .../apache/drill/optiq/EnumerableDrillRule.java |  12 +-
 .../org/apache/drill/jdbc/test/JdbcAssert.java  | 148 ++++++++++++++
 .../org/apache/drill/jdbc/test/JdbcTest.java    | 197 +++++++++++++++----
 51 files changed, 2385 insertions(+), 277 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/5c07ccde/sandbox/prototype/exec/ref/src/main/java/org/apache/drill/exec/ref/rse/QueueRSE.java
----------------------------------------------------------------------
diff --cc sandbox/prototype/exec/ref/src/main/java/org/apache/drill/exec/ref/rse/QueueRSE.java
index 9a0a132,8f7a6a5..3bb1648
--- a/sandbox/prototype/exec/ref/src/main/java/org/apache/drill/exec/ref/rse/QueueRSE.java
+++ b/sandbox/prototype/exec/ref/src/main/java/org/apache/drill/exec/ref/rse/QueueRSE.java
@@@ -48,7 -52,25 +52,24 @@@ public class QueueRSE extends RSEBase 
      return sinkQueues.get(number);
    }
    
-   @JsonTypeName("queue") public static class QueueRSEConfig extends StorageEngineConfigBase {}
+   @JsonTypeName("queue")
+   public static class QueueRSEConfig extends StorageEngineConfigBase {
+     
+     public static enum Encoding {JSON, RECORD};
+     
+     private final Encoding encoding;
+     
+     @JsonCreator
 -    public QueueRSEConfig(@JsonProperty("name") String name, @JsonProperty("encoding") Encoding encoding) {
 -      super(name);
++    public QueueRSEConfig(@JsonProperty("encoding") Encoding encoding) {
+       this.encoding = encoding == null ? Encoding.JSON : encoding;
+     }
+ 
+     public Encoding getEncoding() {
+       return encoding;
+     }
+     
+     
+   }
    
    public static class QueueOutputInfo{
      public int number;

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/5c07ccde/sandbox/prototype/exec/ref/src/test/java/org/apache/drill/exec/ref/RSERegistryTest.java
----------------------------------------------------------------------
diff --cc sandbox/prototype/exec/ref/src/test/java/org/apache/drill/exec/ref/RSERegistryTest.java
index 0000000,efb3855..fe9d497
mode 000000,100644..100644
--- a/sandbox/prototype/exec/ref/src/test/java/org/apache/drill/exec/ref/RSERegistryTest.java
+++ b/sandbox/prototype/exec/ref/src/test/java/org/apache/drill/exec/ref/RSERegistryTest.java
@@@ -1,0 -1,40 +1,40 @@@
+ /*******************************************************************************
+  * Licensed to the Apache Software Foundation (ASF) under one
+  * or more contributor license agreements.  See the NOTICE file
+  * distributed with this work for additional information
+  * regarding copyright ownership.  The ASF licenses this file
+  * to you under the Apache License, Version 2.0 (the
+  * "License"); you may not use this file except in compliance
+  * with the License.  You may obtain a copy of the License at
+  *
+  * http://www.apache.org/licenses/LICENSE-2.0
+  *
+  * Unless required by applicable law or agreed to in writing, software
+  * distributed under the License is distributed on an "AS IS" BASIS,
+  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  * See the License for the specific language governing permissions and
+  * limitations under the License.
+  ******************************************************************************/
+ package org.apache.drill.exec.ref;
+ 
+ import org.apache.drill.common.config.DrillConfig;
+ import org.apache.drill.common.logical.StorageEngineConfig;
+ import org.apache.drill.exec.ref.rse.ConsoleRSE;
+ import org.apache.drill.exec.ref.rse.RSERegistry;
+ import org.junit.Test;
+ 
+ import static junit.framework.Assert.assertSame;
+ 
+ 
+ public class RSERegistryTest {
+ 
+   @Test
+   public void testEnginesWithTheSameNameAreEqual() {
+     DrillConfig config = DrillConfig.create();
+     RSERegistry rses = new RSERegistry(config);
 -    StorageEngineConfig hconfig = new ConsoleRSE.ConsoleRSEConfig("console");
++    StorageEngineConfig hconfig = new ConsoleRSE.ConsoleRSEConfig();
+     ConsoleRSE engine = (ConsoleRSE) rses.getEngine(hconfig);
+     ConsoleRSE engine2 = (ConsoleRSE) rses.getEngine(hconfig);
+     assertSame(engine, engine2);
+   }
+ }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/5c07ccde/sandbox/prototype/exec/ref/src/test/resources/collapse/test1.json
----------------------------------------------------------------------
diff --cc sandbox/prototype/exec/ref/src/test/resources/collapse/test1.json
index 0000000,43ef233..d4b1e24
mode 000000,100644..100644
--- a/sandbox/prototype/exec/ref/src/test/resources/collapse/test1.json
+++ b/sandbox/prototype/exec/ref/src/test/resources/collapse/test1.json
@@@ -1,0 -1,71 +1,73 @@@
+ {
+   "head" : {
 -    "type" : "apache_drill_logical_plan",
++    "type" : "APACHE_DRILL_LOGICAL",
+     "version" : 1,
+     "generator" : {
+       "type" : "manual",
+       "info" : "na"
+     }
+   },
 -  "storage" : [ {
 -    "type" : "queue",
 -    "name" : "queue",
 -    "encoding" : "RECORD"
 -  }, {
 -    "type" : "classpath",
 -    "name" : "donuts-json"
 -  } ],
++  "storage" : {
++    "queue" : {
++        "type" : "queue",
++        "encoding" : "RECORD"
++    },
++    "donuts-json" : {
++          "type": "classpath"
++    }
++  },
++
+   "query" : [ {
+     "op" : "scan",
+     "@id" : 1,
+     "memo" : "initial_scan",
+     "storageengine" : "donuts-json",
+     "selection" : {
+       "path" : "/employees.json",
+       "type" : "JSON"
+     },
+     "ref" : "_MAP"
+   }, {
+     "op" : "project",
+     "input" : 1,
+     "@id" : 2,
+     "projections" : [ {
+       "ref" : "output.deptId",
+       "expr" : "_MAP.deptId"
+     } ]
+   },  {
+     op: "segment",
+     "input" : 2,
+     "@id" : 3,
+     ref: "segment",
+     exprs: ["deptId"]
+   }, {
+     "input" : 3,
+     "@id" : 4,
+     op: "collapsingaggregate",
+     within: "segment",
+     carryovers: [ "deptId" ],
+     aggregations: [
+                 { ref: "typeCount",  expr: "count(1)" }
+               ]
+   }, 
+   {
+     op: "order",
+     "input" : 4,
+     "@id" : 5,
+     orderings: [
+     	{order: "asc", expr: "deptId"}
+     ]
+   },
+   {
+     "op" : "store",
+     "input" : 5,
+     "@id" : 6,
+     "memo" : "output sink",
+     "target" : {
+       "number" : 0
+     },
+     "partition" : null,
+     "storageEngine" : "queue"
+   } ]
+ }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/5c07ccde/sandbox/prototype/exec/ref/src/test/resources/join/full_join.json
----------------------------------------------------------------------
diff --cc sandbox/prototype/exec/ref/src/test/resources/join/full_join.json
index 0000000,d565a08..59a53f3
mode 000000,100644..100644
--- a/sandbox/prototype/exec/ref/src/test/resources/join/full_join.json
+++ b/sandbox/prototype/exec/ref/src/test/resources/join/full_join.json
@@@ -1,0 -1,74 +1,72 @@@
+ {
+    head: {
 -      type: "apache_drill_logical_plan",
++      type: "APACHE_DRILL_LOGICAL",
+       version: "1",
+       generator: {
+          type: "manual",
+          info: "na"
+       }
+    },
 -   storage:[
 -       {
 -         type:"console",
 -         name:"console"
 -       },
 -       {
 -         type:"fs",
 -         name:"fs1",
 -         root:"file:///"
 -       },
 -       {
 -         type:"classpath",
 -         name:"cp"
 -       },
 -       {
 -         type: "queue",
 -         name: "queue"
 -       }
 -   ],
 -   query: [
++    "storage" : {
++        "queue" : {
++            "type" : "queue",
++            "encoding" : "RECORD"
++        },
++        "cp" : {
++            "type": "classpath"
++        },
++        "fs1" : {
++            "type" : "fs",
++            "root" : "file:///"
++        },
++        "console" : {
++            "type" : "console"
++        }
++    },
++
++   "query": [
+       {
+          @id: 1,
+          op: "scan",
+          memo: "initial_scan",
+          ref: "employees",
+          storageengine: "cp",
+          selection: {
+          	 path: "/join/employees.json",
+          	 type: "JSON"
+          }
+       },
+       {
+          @id: 2,
+          op: "scan",
+          memo: "second_scan",
+          ref: "departments",
+          storageengine: "cp",
+          selection: {
+              path: "/join/departments.json",
+              type: "JSON"
+          }
+       },
+       {
+          @id: 3,
+          op: "join",
+          left: 1,
+          right: 2,
+          type: "outer",
+          conditions: [
+             {
+                relationship: "==",
+                left: "employees.deptId",
+                right: "departments.deptId"
+             }
+          ]
+       },
+       {
+          input: 3,
+          op: "store",
+          memo: "output sink",
+          storageengine: "queue",
+          target: {number: 0}
+       }
+    ]
+ }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/5c07ccde/sandbox/prototype/exec/ref/src/test/resources/join/left_join.json
----------------------------------------------------------------------
diff --cc sandbox/prototype/exec/ref/src/test/resources/join/left_join.json
index 0000000,ac43be4..96e6a79
mode 000000,100644..100644
--- a/sandbox/prototype/exec/ref/src/test/resources/join/left_join.json
+++ b/sandbox/prototype/exec/ref/src/test/resources/join/left_join.json
@@@ -1,0 -1,74 +1,71 @@@
+ {
+    head: {
 -      type: "apache_drill_logical_plan",
++      type: "APACHE_DRILL_LOGICAL",
+       version: "1",
+       generator: {
+          type: "manual",
+          info: "na"
+       }
+    },
 -   storage:[
 -       {
 -         type:"console",
 -         name:"console"
++   storage:{
++       "queue" : {
++           "type" : "queue",
++           "encoding" : "RECORD"
+        },
 -       {
 -         type:"fs",
 -         name:"fs1",
 -         root:"file:///"
++       "cp" : {
++           "type": "classpath"
+        },
 -       {
 -         type:"classpath",
 -         name:"cp"
++       "fs1" : {
++           "type" : "fs",
++           "root" : "file:///"
+        },
 -       {
 -         type: "queue",
 -         name: "queue"
++       "console" : {
++           "type" : "console"
+        }
 -   ],
++   },
+    query: [
+       {
+          @id: 1,
+          op: "scan",
+          memo: "initial_scan",
+          ref: "employees",
+          storageengine: "cp",
+          selection: {
+          	 path: "/join/employees.json",
+          	 type: "JSON"
+          }
+       },
+       {
+          @id: 2,
+          op: "scan",
+          memo: "second_scan",
+          ref: "departments",
+          storageengine: "cp",
+          selection: {
+              path: "/join/departments.json",
+              type: "JSON"
+          }
+       },
+       {
+          @id: 3,
+          op: "join",
+          left: 1,
+          right: 2,
+          type: "left",
+          conditions: [
+             {
+                relationship: "==",
+                left: "employees.deptId",
+                right: "departments.deptId"
+             }
+          ]
+       },
+       {
+          input: 3,
+          op: "store",
+          memo: "output sink",
+          storageengine: "queue",
+          target: {number: 0}
+       }
+    ]
+ }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/5c07ccde/sandbox/prototype/exec/ref/src/test/resources/join/simple_join.json
----------------------------------------------------------------------
diff --cc sandbox/prototype/exec/ref/src/test/resources/join/simple_join.json
index 0000000,65c82cf..1587a78
mode 000000,100644..100644
--- a/sandbox/prototype/exec/ref/src/test/resources/join/simple_join.json
+++ b/sandbox/prototype/exec/ref/src/test/resources/join/simple_join.json
@@@ -1,0 -1,74 +1,71 @@@
+ {
+    head: {
 -      type: "apache_drill_logical_plan",
++      type: "APACHE_DRILL_LOGICAL",
+       version: "1",
+       generator: {
+          type: "manual",
+          info: "na"
+       }
+    },
 -   storage:[
 -       {
 -         type:"console",
 -         name:"console"
++   storage:{
++       "queue" : {
++           "type" : "queue",
++           "encoding" : "RECORD"
+        },
 -       {
 -         type:"fs",
 -         name:"fs1",
 -         root:"file:///"
++       "cp" : {
++           "type": "classpath"
+        },
 -       {
 -         type:"classpath",
 -         name:"cp"
++       "fs1" : {
++           "type" : "fs",
++           "root" : "file:///"
+        },
 -       {
 -         type: "queue",
 -         name: "queue"
++       "console" : {
++           "type" : "console"
+        }
 -   ],
++   },
+    query: [
+       {
+          @id: 1,
+          op: "scan",
+          memo: "initial_scan",
+          ref: "employees",
+          storageengine: "cp",
+          selection: {
+          	 path: "/join/employees.json",
+          	 type: "JSON"
+          }
+       },
+       {
+          @id: 2,
+          op: "scan",
+          memo: "second_scan",
+          ref: "departments",
+          storageengine: "cp",
+          selection: {
+              path: "/join/departments.json",
+              type: "JSON"
+          }
+       },
+       {
+          @id: 3,
+          op: "join",
+          left: 1,
+          right: 2,
+          type: "inner",
+          conditions: [
+             {
+                relationship: "==",
+                left: "employees.deptId",
+                right: "departments.deptId"
+             }
+          ]
+       },
+       {
+          input: 3,
+          op: "store",
+          memo: "output sink",
+          storageengine: "queue",
+          target: {number: 0}
+       }
+    ]
+ }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/5c07ccde/sandbox/prototype/exec/ref/src/test/resources/order/nulls-first.json
----------------------------------------------------------------------
diff --cc sandbox/prototype/exec/ref/src/test/resources/order/nulls-first.json
index 0000000,a0810e5..6e541e9
mode 000000,100644..100644
--- a/sandbox/prototype/exec/ref/src/test/resources/order/nulls-first.json
+++ b/sandbox/prototype/exec/ref/src/test/resources/order/nulls-first.json
@@@ -1,0 -1,71 +1,79 @@@
+ {
+   "head" : {
 -    "type" : "apache_drill_logical_plan",
++    "type" : "APACHE_DRILL_LOGICAL",
+     "version" : 1,
+     "generator" : {
+       "type" : "manual",
+       "info" : "na"
+     }
+   },
 -  "storage" : [ {
 -    "type" : "queue",
 -    "name" : "queue",
 -    "encoding" : "RECORD"
 -  }, {
 -    "type" : "classpath",
 -    "name" : "donuts-json"
 -  } ],
++  "storage" : {
++      "queue" : {
++          "type" : "queue",
++          "encoding" : "RECORD"
++      },
++      "cp" : {
++          "type": "classpath"
++      },
++      "fs1" : {
++          "type" : "fs",
++          "root" : "file:///"
++      },
++      "console" : {
++          "type" : "console"
++      }
++  },
+   "query" : [ {
+     "op" : "scan",
+     "@id" : 1,
+     "memo" : "initial_scan",
 -    "storageengine" : "donuts-json",
++    "storageengine" : "cp",
+     "selection" : {
+       "path" : "/employees.json",
+       "type" : "JSON"
+     },
+     "ref" : "_MAP"
+   }, {
+     "op" : "project",
+     "input" : 1,
+     "@id" : 2,
+     "projections" : [ {
+       "ref" : "output.deptId",
+       "expr" : "_MAP.deptId"
+     } ]
+   },  {
+     op: "segment",
+     "input" : 2,
+     "@id" : 3,
+     ref: "segment",
+     exprs: ["deptId"]
+   }, {
+     "input" : 3,
+     "@id" : 4,
+     op: "collapsingaggregate",
+     within: "segment",
+     carryovers: [ "deptId" ],
+     aggregations: [
+                 { ref: "typeCount",  expr: "count(1)" }
+               ]
+   }, 
+   {
+     op: "order",
+     "input" : 4,
+     "@id" : 5,
+     orderings: [
+     	{order: "asc", expr: "deptId", nullCollation: "first"}
+     ]
+   },
+   {
+     "op" : "store",
+     "input" : 5,
+     "@id" : 6,
+     "memo" : "output sink",
+     "target" : {
+       "number" : 0
+     },
+     "partition" : null,
+     "storageEngine" : "queue"
+   } ]
+ }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/5c07ccde/sandbox/prototype/exec/ref/src/test/resources/order/nulls-last.json
----------------------------------------------------------------------
diff --cc sandbox/prototype/exec/ref/src/test/resources/order/nulls-last.json
index 0000000,117a562..83338b7
mode 000000,100644..100644
--- a/sandbox/prototype/exec/ref/src/test/resources/order/nulls-last.json
+++ b/sandbox/prototype/exec/ref/src/test/resources/order/nulls-last.json
@@@ -1,0 -1,71 +1,79 @@@
+ {
+   "head" : {
 -    "type" : "apache_drill_logical_plan",
++    "type" : "APACHE_DRILL_LOGICAL",
+     "version" : 1,
+     "generator" : {
+       "type" : "manual",
+       "info" : "na"
+     }
+   },
 -  "storage" : [ {
 -    "type" : "queue",
 -    "name" : "queue",
 -    "encoding" : "RECORD"
 -  }, {
 -    "type" : "classpath",
 -    "name" : "donuts-json"
 -  } ],
++  "storage" : {
++      "queue" : {
++          "type" : "queue",
++          "encoding" : "RECORD"
++      },
++      "cp" : {
++          "type": "classpath"
++      },
++      "fs1" : {
++          "type" : "fs",
++          "root" : "file:///"
++      },
++      "console" : {
++          "type" : "console"
++      }
++  },
+   "query" : [ {
+     "op" : "scan",
+     "@id" : 1,
+     "memo" : "initial_scan",
 -    "storageengine" : "donuts-json",
++    "storageengine" : "cp",
+     "selection" : {
+       "path" : "/employees.json",
+       "type" : "JSON"
+     },
+     "ref" : "_MAP"
+   }, {
+     "op" : "project",
+     "input" : 1,
+     "@id" : 2,
+     "projections" : [ {
+       "ref" : "output.deptId",
+       "expr" : "_MAP.deptId"
+     } ]
+   },  {
+     op: "segment",
+     "input" : 2,
+     "@id" : 3,
+     ref: "segment",
+     exprs: ["deptId"]
+   }, {
+     "input" : 3,
+     "@id" : 4,
+     op: "collapsingaggregate",
+     within: "segment",
+     carryovers: [ "deptId" ],
+     aggregations: [
+                 { ref: "typeCount",  expr: "count(1)" }
+               ]
+   }, 
+   {
+     op: "order",
+     "input" : 4,
+     "@id" : 5,
+     orderings: [
+     	{order: "asc", expr: "deptId", nullCollation: "last"}
+     ]
+   },
+   {
+     "op" : "store",
+     "input" : 5,
+     "@id" : 6,
+     "memo" : "output sink",
+     "target" : {
+       "number" : 0
+     },
+     "partition" : null,
+     "storageEngine" : "queue"
+   } ]
+ }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/5c07ccde/sandbox/prototype/exec/ref/src/test/resources/union/distinct.json
----------------------------------------------------------------------
diff --cc sandbox/prototype/exec/ref/src/test/resources/union/distinct.json
index 0000000,b975a77..9361009
mode 000000,100644..100644
--- a/sandbox/prototype/exec/ref/src/test/resources/union/distinct.json
+++ b/sandbox/prototype/exec/ref/src/test/resources/union/distinct.json
@@@ -1,0 -1,71 +1,82 @@@
+ {
 -  "head" : {
 -    "type" : "apache_drill_logical_plan",
 -    "version" : 1,
 -    "generator" : {
 -      "type" : "manual",
 -      "info" : "na"
 -    }
 -  },
 -  "storage" : [ {
 -    "type" : "queue",
 -    "name" : "queue",
 -    "encoding" : "RECORD"
 -    
 -  }, {
 -    "type" : "classpath",
 -    "name" : "donuts-json"
 -  } ],
 -  "query" : [ {
 -    "op" : "scan",
 -    "@id" : 1,
 -    "memo" : "initial_scan",
 -    "storageengine" : "donuts-json",
 -    "selection" : {
 -      "path" : "/employees.json",
 -      "type" : "JSON"
++    "head": {
++        "type": "APACHE_DRILL_LOGICAL",
++        "version": 1,
++        "generator": {
++            "type": "manual",
++            "info": "na"
++        }
+     },
 -    "ref" : "_MAP"
 -  }, {
 -    "op" : "project",
 -    "input" : 1,
 -    "@id" : 2,
 -    "projections" : [ {
 -      "ref" : "output.deptId",
 -      "expr" : "_MAP.deptId"
 -    } ]
 -  }, {
 -    "op" : "scan",
 -    "@id" : 3,
 -    "memo" : "initial_scan",
 -    "storageengine" : "donuts-json",
 -    "selection" : {
 -      "path" : "/departments.json",
 -      "type" : "JSON"
++    "storage": {
++        "queue": {
++            "type": "queue",
++            "encoding": "RECORD"
++        },
++        "donuts-json": {
++            "type": "classpath"
++        }
+     },
 -    "ref" : "_MAP"
 -  }, {
 -    "op" : "project",
 -    "input" : 3,
 -    "@id" : 4,
 -    "projections" : [ {
 -      "ref" : "output.deptId",
 -      "expr" : "_MAP.deptId"
 -    } ]
 -  },  {
 -    "op": "union",
 -    "@id" : 5,
 -    "distinct": true,
 -    "inputs": [2, 4]
 -  }, {
 -    "op" : "store",
 -    "input" : 5,
 -    "@id" : 6,
 -    "memo" : "output sink",
 -    "target" : {
 -      "number" : 0
 -    },
 -    "partition" : null,
 -    "storageEngine" : "queue"
 -  } ]
++    "query": [
++        {
++            "op": "scan",
++            "@id": 1,
++            "memo": "initial_scan",
++            "storageengine": "donuts-json",
++            "selection": {
++                "path": "/employees.json",
++                "type": "JSON"
++            },
++            "ref": "_MAP"
++        },
++        {
++            "op": "project",
++            "input": 1,
++            "@id": 2,
++            "projections": [
++                {
++                    "ref": "output.deptId",
++                    "expr": "_MAP.deptId"
++                }
++            ]
++        },
++        {
++            "op": "scan",
++            "@id": 3,
++            "memo": "initial_scan",
++            "storageengine": "donuts-json",
++            "selection": {
++                "path": "/departments.json",
++                "type": "JSON"
++            },
++            "ref": "_MAP"
++        },
++        {
++            "op": "project",
++            "input": 3,
++            "@id": 4,
++            "projections": [
++                {
++                    "ref": "output.deptId",
++                    "expr": "_MAP.deptId"
++                }
++            ]
++        },
++        {
++            "op": "union",
++            "@id": 5,
++            "distinct": true,
++            "inputs": [2, 4]
++        },
++        {
++            "op": "store",
++            "input": 5,
++            "@id": 6,
++            "memo": "output sink",
++            "target": {
++                "number": 0
++            },
++            "partition": null,
++            "storageEngine": "queue"
++        }
++    ]
+ }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/5c07ccde/sandbox/prototype/exec/ref/src/test/resources/union/nondistinct.json
----------------------------------------------------------------------
diff --cc sandbox/prototype/exec/ref/src/test/resources/union/nondistinct.json
index 0000000,817ed48..35ad70e
mode 000000,100644..100644
--- a/sandbox/prototype/exec/ref/src/test/resources/union/nondistinct.json
+++ b/sandbox/prototype/exec/ref/src/test/resources/union/nondistinct.json
@@@ -1,0 -1,71 +1,82 @@@
+ {
 -  "head" : {
 -    "type" : "apache_drill_logical_plan",
 -    "version" : 1,
 -    "generator" : {
 -      "type" : "manual",
 -      "info" : "na"
 -    }
 -  },
 -  "storage" : [ {
 -    "type" : "queue",
 -    "name" : "queue",
 -    "encoding" : "RECORD"
 -    
 -  }, {
 -    "type" : "classpath",
 -    "name" : "donuts-json"
 -  } ],
 -  "query" : [ {
 -    "op" : "scan",
 -    "@id" : 1,
 -    "memo" : "initial_scan",
 -    "storageengine" : "donuts-json",
 -    "selection" : {
 -      "path" : "/employees.json",
 -      "type" : "JSON"
++    "head": {
++        "type": "APACHE_DRILL_LOGICAL",
++        "version": 1,
++        "generator": {
++            "type": "manual",
++            "info": "na"
++        }
+     },
 -    "ref" : "_MAP"
 -  }, {
 -    "op" : "project",
 -    "input" : 1,
 -    "@id" : 2,
 -    "projections" : [ {
 -      "ref" : "output.deptId",
 -      "expr" : "_MAP.deptId"
 -    } ]
 -  }, {
 -    "op" : "scan",
 -    "@id" : 3,
 -    "memo" : "initial_scan",
 -    "storageengine" : "donuts-json",
 -    "selection" : {
 -      "path" : "/departments.json",
 -      "type" : "JSON"
++    "storage": {
++        "queue": {
++            "type": "queue",
++            "encoding": "RECORD"
++        },
++        "donuts-json": {
++            "type": "classpath"
++        }
+     },
 -    "ref" : "_MAP"
 -  }, {
 -    "op" : "project",
 -    "input" : 3,
 -    "@id" : 4,
 -    "projections" : [ {
 -      "ref" : "output.deptId",
 -      "expr" : "_MAP.deptId"
 -    } ]
 -  },  {
 -    "op": "union",
 -    "@id" : 5,
 -    "distinct": false,
 -    "inputs": [2, 4]
 -  }, {
 -    "op" : "store",
 -    "input" : 5,
 -    "@id" : 6,
 -    "memo" : "output sink",
 -    "target" : {
 -      "number" : 0
 -    },
 -    "partition" : null,
 -    "storageEngine" : "queue"
 -  } ]
++    "query": [
++        {
++            "op": "scan",
++            "@id": 1,
++            "memo": "initial_scan",
++            "storageengine": "donuts-json",
++            "selection": {
++                "path": "/employees.json",
++                "type": "JSON"
++            },
++            "ref": "_MAP"
++        },
++        {
++            "op": "project",
++            "input": 1,
++            "@id": 2,
++            "projections": [
++                {
++                    "ref": "output.deptId",
++                    "expr": "_MAP.deptId"
++                }
++            ]
++        },
++        {
++            "op": "scan",
++            "@id": 3,
++            "memo": "initial_scan",
++            "storageengine": "donuts-json",
++            "selection": {
++                "path": "/departments.json",
++                "type": "JSON"
++            },
++            "ref": "_MAP"
++        },
++        {
++            "op": "project",
++            "input": 3,
++            "@id": 4,
++            "projections": [
++                {
++                    "ref": "output.deptId",
++                    "expr": "_MAP.deptId"
++                }
++            ]
++        },
++        {
++            "op": "union",
++            "@id": 5,
++            "distinct": false,
++            "inputs": [2, 4]
++        },
++        {
++            "op": "store",
++            "input": 5,
++            "@id": 6,
++            "memo": "output sink",
++            "target": {
++                "number": 0
++            },
++            "partition": null,
++            "storageEngine": "queue"
++        }
++    ]
+ }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/5c07ccde/sandbox/prototype/sqlparser/pom.xml
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/5c07ccde/sandbox/prototype/sqlparser/src/main/java/org/apache/drill/jdbc/DrillTable.java
----------------------------------------------------------------------
diff --cc sandbox/prototype/sqlparser/src/main/java/org/apache/drill/jdbc/DrillTable.java
index f530ea8,022e05d..23d7237
--- a/sandbox/prototype/sqlparser/src/main/java/org/apache/drill/jdbc/DrillTable.java
+++ b/sandbox/prototype/sqlparser/src/main/java/org/apache/drill/jdbc/DrillTable.java
@@@ -48,6 -48,6 +48,7 @@@ public class DrillTable extends BaseQue
  {
    private final Schema schema;
    private final String name;
++  private final String storageEngineName;
    private final RelDataType rowType;
    public final StorageEngineConfig storageEngineConfig;
    public final Object selection;
@@@ -59,18 -59,21 +60,26 @@@
        RelDataType rowType,
        String name,
        StorageEngineConfig storageEngineConfig,
-       Object selection
 -      Object selection) {
++      Object selection,
++      String storageEngineName
 +      ) {
      super(schema.getQueryProvider(), elementType, expression);
      this.schema = schema;
      this.name = name;
      this.rowType = rowType;
      this.storageEngineConfig = storageEngineConfig;
      this.selection = selection;
++    this.storageEngineName = storageEngineName;
    }
  
-   static void addTable(RelDataTypeFactory typeFactory, MutableSchema schema,
-       String name, StorageEngineConfig storageEngineConfig, Object selection) {
+   private static DrillTable createTable(
+       RelDataTypeFactory typeFactory,
+       MutableSchema schema,
+       String name,
+       StorageEngineConfig storageEngineConfig,
 -      Object selection) {
++      Object selection,
++      String storageEngineName
++      ) {
      final MethodCallExpression call = Expressions.call(schema.getExpression(),
          BuiltinMethod.DATA_CONTEXT_GET_TABLE.method,
          Expressions.constant(name),
@@@ -78,11 -81,12 +87,12 @@@
      final RelDataType rowType =
          typeFactory.createStructType(
              Collections.singletonList(
-                 typeFactory.createSqlType(SqlTypeName.ANY)),
-             Collections.singletonList("_extra"));
-     final DrillTable table =
-         new DrillTable(schema, Object.class, call, rowType, name, storageEngineConfig, selection);
-     schema.addTable(name, table);
+                 typeFactory.createMapType(
+                     typeFactory.createSqlType(SqlTypeName.VARCHAR),
+                     typeFactory.createSqlType(SqlTypeName.ANY))),
+             Collections.singletonList("_MAP"));
+       return new DrillTable(schema, Object.class, call, rowType, name,
 -          storageEngineConfig, selection);
++          storageEngineConfig, selection, storageEngineName);
    }
  
    @Override
@@@ -96,15 -100,40 +106,44 @@@
    }
  
    @Override
+   public Statistic getStatistic() {
+     return Statistics.UNKNOWN;
+   }
+ 
++  public String getStorageEngineName() {
++    return storageEngineName;
++  }
++
+   @Override
    public Enumerator<Object> enumerator() {
      return Linq4j.emptyEnumerator();
    }
  
    public RelNode toRel(RelOptTable.ToRelContext context, RelOptTable table) {
      return new DrillScan(context.getCluster(),
-         context.getCluster().traitSetOf(DrillOptiq.CONVENTION),
+         context.getCluster().traitSetOf(DrillRel.CONVENTION),
          table);
    }
+ 
+   private static <T> T last(T t0, T t1) {
+     return t0 != null ? t0 : t1;
+   }
+ 
+   /** Factory for custom tables in Optiq schema. */
+   @SuppressWarnings("UnusedDeclaration")
+   public static class Factory implements TableFactory<DrillTable> {
+     @Override
+     public DrillTable create(Schema schema, String name,
+         Map<String, Object> operand, RelDataType rowType) {
+       final ClasspathRSE.ClasspathRSEConfig rseConfig =
 -          new ClasspathRSE.ClasspathRSEConfig("donuts-json");
++          new ClasspathRSE.ClasspathRSEConfig();
+       final ClasspathInputConfig inputConfig = new ClasspathInputConfig();
+       inputConfig.path = last((String) operand.get("path"), "/donuts.json");
+       inputConfig.type = DataWriter.ConverterType.JSON;
+       return createTable(schema.getTypeFactory(), (MutableSchema) schema, name,
 -          rseConfig, inputConfig);
++          rseConfig, inputConfig, "donuts-json");
+     }
+   }
  }
  
  // End DrillTable.java

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/5c07ccde/sandbox/prototype/sqlparser/src/main/java/org/apache/drill/optiq/DrillImplementor.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/5c07ccde/sandbox/prototype/sqlparser/src/main/java/org/apache/drill/optiq/DrillScan.java
----------------------------------------------------------------------
diff --cc sandbox/prototype/sqlparser/src/main/java/org/apache/drill/optiq/DrillScan.java
index 2e2849a,89713ae..11897d5
--- a/sandbox/prototype/sqlparser/src/main/java/org/apache/drill/optiq/DrillScan.java
+++ b/sandbox/prototype/sqlparser/src/main/java/org/apache/drill/optiq/DrillScan.java
@@@ -37,9 -37,8 +37,8 @@@ public class DrillScan extends TableAcc
      final ObjectNode node = implementor.mapper.createObjectNode();
      node.put("op", "scan");
      node.put("memo", "initial_scan");
-     node.put("ref", "donuts");
-     final ObjectNode engines = implementor.mapper.createObjectNode();
-     node.put("storageengine", "donuts-json");
+     node.put("ref", "_MAP"); // output is a record with a single field, '_MAP'
 -    node.put("storageengine", drillTable.storageEngineConfig.getName());
++    node.put("storageengine", drillTable.getStorageEngineName());
      node.put("selection", implementor.mapper.convertValue(drillTable.selection, JsonNode.class));
      implementor.add(node);
    }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/5c07ccde/sandbox/prototype/sqlparser/src/test/java/org/apache/drill/jdbc/test/JdbcTest.java
----------------------------------------------------------------------
diff --cc sandbox/prototype/sqlparser/src/test/java/org/apache/drill/jdbc/test/JdbcTest.java
index eb0b51a,71ed10d..2d57732
--- a/sandbox/prototype/sqlparser/src/test/java/org/apache/drill/jdbc/test/JdbcTest.java
+++ b/sandbox/prototype/sqlparser/src/test/java/org/apache/drill/jdbc/test/JdbcTest.java
@@@ -49,39 -87,118 +87,118 @@@ public class JdbcTest extends TestCase 
  
    /** Simple query against JSON. */
    public void testSelectJson() throws Exception {
-     Class.forName("org.apache.drill.jdbc.Driver");
-     final Connection connection = DriverManager.getConnection(
-         "jdbc:drill:schema=DONUTS;tables=DONUTS");
-     final Statement statement = connection.createStatement();
-     final ResultSet resultSet = statement.executeQuery(
-         "select * from donuts");
-     assertEquals(
-         "_extra={donuts={batters={batter=[{id=1001, type=Regular}, {id=1002, type=Chocolate}, {id=1003, type=Blueberry}, {id=1004, type=Devil's Food}]}, id=0001, name=Cake, ppu=0.55, sales=35, topping=[{id=5001, type=None}, {id=5002, type=Glazed}, {id=5005, type=Sugar}, {id=5007, type=Powdered Sugar}, {id=5006, type=Chocolate with Sprinkles}, {id=5003, type=Chocolate}, {id=5004, type=Maple}], type=donut}}\n"
-         + "_extra={donuts={batters={batter=[{id=1001, type=Regular}]}, id=0002, name=Raised, ppu=0.69, sales=145, topping=[{id=5001, type=None}, {id=5002, type=Glazed}, {id=5005, type=Sugar}, {id=5003, type=Chocolate}, {id=5004, type=Maple}], type=donut}}\n"
-         + "_extra={donuts={batters={batter=[{id=1001, type=Regular}, {id=1002, type=Chocolate}]}, id=0003, name=Old Fashioned, ppu=0.55, sales=300, topping=[{id=5001, type=None}, {id=5002, type=Glazed}, {id=5003, type=Chocolate}, {id=5004, type=Maple}], type=donut}}\n"
-         + "_extra={donuts={batters={batter=[{id=1001, type=Regular}, {id=1002, type=Chocolate}, {id=1003, type=Blueberry}, {id=1004, type=Devil's Food}]}, filling=[{id=6001, type=None}, {id=6002, type=Raspberry}, {id=6003, type=Lemon}, {id=6004, type=Chocolate}, {id=6005, type=Kreme}], id=0004, name=Filled, ppu=0.69, sales=14, topping=[{id=5001, type=None}, {id=5002, type=Glazed}, {id=5005, type=Sugar}, {id=5007, type=Powdered Sugar}, {id=5006, type=Chocolate with Sprinkles}, {id=5003, type=Chocolate}, {id=5004, type=Maple}], type=donut}}\n"
-         + "_extra={donuts={batters={batter=[{id=1001, type=Regular}]}, id=0005, name=Apple Fritter, ppu=1.0, sales=700, topping=[{id=5002, type=Glazed}], type=donut}}\n",
-         toString(resultSet));
-     resultSet.close();
-     statement.close();
-     connection.close();
+     JdbcAssert.withModel(MODEL, "DONUTS")
+         .sql("select * from donuts")
+         .returns(EXPECTED);
+   }
+ 
+   /** Query with project list. No field references yet. */
+   public void testProjectConstant() throws Exception {
+     JdbcAssert.withModel(MODEL, "DONUTS")
+         .sql("select 1 + 3 as c from donuts")
+         .returns("C=4\n"
+             + "C=4\n"
+             + "C=4\n"
+             + "C=4\n"
+             + "C=4\n");
    }
  
-   static String toString(ResultSet resultSet) throws SQLException {
-     StringBuilder buf = new StringBuilder();
-     while (resultSet.next()) {
-       int n = resultSet.getMetaData().getColumnCount();
-       String sep = "";
-       for (int i = 1; i <= n; i++) {
-         buf.append(sep)
-             .append(resultSet.getMetaData().getColumnLabel(i))
-             .append("=")
-             .append(resultSet.getObject(i));
-         sep = "; ";
-       }
-       buf.append("\n");
-     }
-     return buf.toString();
+   /** Query that projects an element from the map. */
+   public void testProject() throws Exception {
+     JdbcAssert.withModel(MODEL, "DONUTS")
+         .sql("select _MAP['ppu'] as ppu from donuts")
+         .returns("PPU=0.55\n"
+             + "PPU=0.69\n"
+             + "PPU=0.55\n"
+             + "PPU=0.69\n"
+             + "PPU=1.0\n");
+   }
+ 
+   /** Same logic as {@link #testProject()}, but using a subquery. */
+   public void testProjectOnSubquery() throws Exception {
+     JdbcAssert.withModel(MODEL, "DONUTS")
+         .sql("select d['ppu'] as ppu from (\n"
+              + " select _MAP as d from donuts)")
+         .returns("PPU=0.55\n"
+             + "PPU=0.69\n"
+             + "PPU=0.55\n"
+             + "PPU=0.69\n"
+             + "PPU=1.0\n");
+   }
+ 
+   /** Checks the logical plan. */
+   public void testProjectPlan() throws Exception {
+     JdbcAssert.withModel(MODEL, "DONUTS")
+         .sql("select _MAP['ppu'] as ppu from donuts")
+         .planContains(
 -            "{'head':{'type':'apache_drill_logical_plan','version':'1','generator':{'type':'manual','info':'na'}},"
 -            + "'storage':[{'name':'donuts-json','type':'classpath'},{'name':'queue','type':'queue'}],"
++            "{'head':{'type':'APACHE_DRILL_LOGICAL','version':'1','generator':{'type':'manual','info':'na'}},"
++            + "'storage':{'donuts-json':{'type':'classpath'},'queue':{'type':'queue'}},"
+             + "'query':["
+             + "{'op':'sequence','do':["
+             + "{'op':'scan','memo':'initial_scan','ref':'_MAP','storageengine':'donuts-json','selection':{'path':'/donuts.json','type':'JSON'}},"
+             + "{'op':'project','projections':[{'expr':'_MAP.ppu','ref':'output.PPU'}]},"
+             + "{'op':'store','storageengine':'queue','memo':'output sink','target':{'number':0}}]}]}");
+   }
+ 
+   /** Query with subquery, filter, and projection of one real and one
+    * nonexistent field from a map field. */
+   public void testProjectFilterSubquery() throws Exception {
+     JdbcAssert.withModel(MODEL, "DONUTS")
+         .sql("select d['name'] as name, d['xx'] as xx from (\n"
+             + " select _MAP as d from donuts)\n"
+             + "where cast(d['ppu'] as double) > 0.6")
+         .returns("NAME=Raised; XX=null\n"
+             + "NAME=Filled; XX=null\n"
+             + "NAME=Apple Fritter; XX=null\n");
+   }
+ 
+   public void testProjectFilterSubqueryPlan() throws Exception {
+     JdbcAssert.withModel(MODEL, "DONUTS")
+         .sql("select d['name'] as name, d['xx'] as xx from (\n"
+             + " select _MAP['donuts'] as d from donuts)\n"
+             + "where cast(d['ppu'] as double) > 0.6")
+         .planContains(
 -            "{'head':{'type':'apache_drill_logical_plan','version':'1','generator':{'type':'manual','info':'na'}},'storage':[{'name':'donuts-json','type':'classpath'},{'name':'queue','type':'queue'}],"
++            "{'head':{'type':'APACHE_DRILL_LOGICAL','version':'1','generator':{'type':'manual','info':'na'}},'storage':{'donuts-json':{'type':'classpath'},'queue':{'type':'queue'}},"
+             + "'query':["
+             + "{'op':'sequence','do':["
+             + "{'op':'scan','memo':'initial_scan','ref':'_MAP','storageengine':'donuts-json','selection':{'path':'/donuts.json','type':'JSON'}},"
+             + "{'op':'filter','expr':'(_MAP.donuts.ppu > 0.6)'},"
+             + "{'op':'project','projections':[{'expr':'_MAP.donuts','ref':'output.D'}]},"
+             + "{'op':'project','projections':[{'expr':'D.name','ref':'output.NAME'},{'expr':'D.xx','ref':'output.XX'}]},"
+             + "{'op':'store','storageengine':'queue','memo':'output sink','target':{'number':0}}]}]}");
+   }
+ 
+   /** Query that projects one field. (Disabled; uses sugared syntax.) */
+   public void _testProjectNestedFieldSugared() throws Exception {
+     JdbcAssert.withModel(MODEL, "DONUTS")
+         .sql("select donuts.ppu from donuts")
+         .returns("C=4\n"
+             + "C=4\n"
+             + "C=4\n"
+             + "C=4\n"
+             + "C=4\n");
+   }
+ 
+   /** Query with filter. No field references yet. */
+   public void testFilterConstantFalse() throws Exception {
+     JdbcAssert.withModel(MODEL, "DONUTS")
+         .sql("select * from donuts where 3 > 4")
+         .returns("");
+   }
+ 
+   public void testFilterConstant() throws Exception {
+     JdbcAssert.withModel(MODEL, "DONUTS")
+         .sql("select * from donuts where 3 < 4")
+         .returns(EXPECTED);
+   }
+ 
+   public void testValues() throws Exception {
+     JdbcAssert.withModel(MODEL, "DONUTS")
+         .sql("values (1)")
+         .returns("EXPR$0=1\n");
+ 
+     // Enable when https://issues.apache.org/jira/browse/DRILL-57 fixed
+     // .planContains("store");
    }
  }
  


[09/53] [abbrv] Update typing system. Update RPC system. Add Fragmenting Implementation. Working single node. Distributed failing due to threading issues.

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/FragmentRunner.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/FragmentRunner.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/FragmentRunner.java
new file mode 100644
index 0000000..2900d99
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/FragmentRunner.java
@@ -0,0 +1,124 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.work;
+
+import java.util.concurrent.atomic.AtomicInteger;
+
+import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.physical.impl.RootExec;
+import org.apache.drill.exec.proto.ExecProtos.FragmentStatus;
+import org.apache.drill.exec.proto.ExecProtos.FragmentStatus.FragmentState;
+import org.apache.drill.exec.rpc.user.UserServer.UserClientConnection;
+
+import com.yammer.metrics.Timer;
+
+/**
+ * Responsible for running a single fragment on a single Drillbit. Listens/responds to status request and cancellation
+ * messages. Two child implementation, root (driving) and child (driven) exist. 
+ */
+public class FragmentRunner implements Runnable, CancelableQuery, StatusProvider, Comparable<Object>{
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(FragmentRunner.class);
+
+  private final AtomicInteger state = new AtomicInteger(FragmentState.AWAITING_ALLOCATION_VALUE);
+  private final RootExec root;
+  private final FragmentContext context;
+  private final FragmentRunnerListener listener;
+  
+  public FragmentRunner(FragmentContext context, RootExec root, FragmentRunnerListener listener){
+    this.context = context;
+    this.root = root;
+    this.listener = listener;
+  }
+
+  @Override
+  public FragmentStatus getStatus() {
+    return FragmentStatus.newBuilder() //
+        .setBatchesCompleted(context.batchesCompleted.get()) //
+        .setDataProcessed(context.dataProcessed.get()) //
+        .setMemoryUse(context.getAllocator().getAllocatedMemory()) //
+        .build();
+  }
+
+  @Override
+  public void cancel() {
+    updateState(FragmentState.CANCELLED);
+  }
+
+  public UserClientConnection getClient(){
+    return context.getConnection();
+  }
+  
+  @Override
+  public void run() {
+    if(!updateState(FragmentState.AWAITING_ALLOCATION, FragmentState.RUNNING, false)){
+      internalFail(new RuntimeException(String.format("Run was called when fragment was in %s state.  FragmentRunnables should only be started when they are currently in awaiting allocation state.", FragmentState.valueOf(state.get()))));
+      return;
+    }
+    
+    Timer.Context t = context.fragmentTime.time();
+    
+    // run the query until root.next returns false.
+    try{
+      while(state.get() == FragmentState.RUNNING_VALUE){
+        if(!root.next()){
+          updateState(FragmentState.RUNNING, FragmentState.FINISHED, false);
+        }
+      }
+      
+      // If this isn't a finished stop, we'll inform other batches to finish up.
+      if(state.get() != FragmentState.FINISHED_VALUE){
+        root.stop();
+      }
+      
+    }catch(Exception ex){
+      internalFail(ex);
+    }finally{
+      t.stop();
+    }
+    
+  }
+  
+  private void internalFail(Throwable excep){
+    state.set(FragmentState.FAILED_VALUE);
+    listener.fail(context.getHandle(), "Failure while running fragment.", excep);
+  }
+  
+  private void updateState(FragmentState update){
+    state.set(update.getNumber());
+    listener.stateChanged(context.getHandle(), update);
+  }
+  
+  private boolean updateState(FragmentState current, FragmentState update, boolean exceptionOnFailure) {
+    boolean success = state.compareAndSet(current.getNumber(), update.getNumber());
+    if (!success && exceptionOnFailure) {
+      internalFail(new RuntimeException(String.format(
+          "State was different than expected.  Attempting to update state from %s to %s however current state was %s.",
+          current.name(), update.name(), FragmentState.valueOf(state.get()))));
+      return false;
+    }
+    listener.stateChanged(context.getHandle(), update);
+    return true;
+  }
+
+  @Override
+  public int compareTo(Object o) {
+    return o.hashCode() - this.hashCode();
+  }
+
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/FragmentRunnerListener.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/FragmentRunnerListener.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/FragmentRunnerListener.java
new file mode 100644
index 0000000..d978470
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/FragmentRunnerListener.java
@@ -0,0 +1,28 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.work;
+
+import org.apache.drill.exec.proto.ExecProtos.FragmentHandle;
+import org.apache.drill.exec.proto.ExecProtos.FragmentStatus.FragmentState;
+
+public interface FragmentRunnerListener {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(FragmentRunnerListener.class);
+  
+  void fail(FragmentHandle handle, String message, Throwable excep);
+  void stateChanged(FragmentHandle handle, FragmentState newState);
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/QueryWorkUnit.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/QueryWorkUnit.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/QueryWorkUnit.java
new file mode 100644
index 0000000..243d677
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/QueryWorkUnit.java
@@ -0,0 +1,64 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.work;
+
+import java.util.List;
+
+import org.apache.drill.exec.physical.base.FragmentRoot;
+import org.apache.drill.exec.physical.base.PhysicalOperator;
+import org.apache.drill.exec.proto.ExecProtos.PlanFragment;
+
+import com.google.common.base.Preconditions;
+
+public class QueryWorkUnit {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(QueryWorkUnit.class);
+  
+  private final PlanFragment rootFragment; // for local
+  private final FragmentRoot rootOperator; // for local
+  private final List<PlanFragment> fragments;
+  
+  public QueryWorkUnit(FragmentRoot rootOperator, PlanFragment rootFragment, List<PlanFragment> fragments) {
+    super();
+    Preconditions.checkNotNull(rootFragment);
+    Preconditions.checkNotNull(fragments);
+    Preconditions.checkNotNull(rootOperator);
+    
+    this.rootFragment = rootFragment;
+    this.fragments = fragments;
+    this.rootOperator = rootOperator;
+  }
+
+  public PlanFragment getRootFragment() {
+    return rootFragment;
+  }
+
+  public List<PlanFragment> getFragments() {
+    return fragments;
+  }
+
+  public FragmentRoot getRootOperator() {
+    return rootOperator;
+  }
+  
+  
+  
+  
+  
+  
+  
+}

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

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/RemotingFragmentRunnerListener.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/RemotingFragmentRunnerListener.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/RemotingFragmentRunnerListener.java
new file mode 100644
index 0000000..74fcd2b
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/RemotingFragmentRunnerListener.java
@@ -0,0 +1,46 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.work;
+
+import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.proto.ExecProtos.FragmentHandle;
+import org.apache.drill.exec.proto.ExecProtos.FragmentStatus;
+import org.apache.drill.exec.proto.ExecProtos.FragmentStatus.Builder;
+import org.apache.drill.exec.proto.ExecProtos.FragmentStatus.FragmentState;
+import org.apache.drill.exec.rpc.bit.BitTunnel;
+import org.apache.drill.exec.work.foreman.ErrorHelper;
+
+/**
+ * Informs remote node as fragment changes state.
+ */
+public class RemotingFragmentRunnerListener extends AbstractFragmentRunnerListener{
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(RemotingFragmentRunnerListener.class);
+  
+  private final BitTunnel tunnel;
+
+  public RemotingFragmentRunnerListener(FragmentContext context, BitTunnel tunnel) {
+    super(context);
+    this.tunnel = tunnel;
+  }
+  
+  @Override
+  protected void statusChange(FragmentHandle handle, FragmentStatus status) {
+    tunnel.sendFragmentStatus(status);
+  }
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/ResourceRequest.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/ResourceRequest.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/ResourceRequest.java
new file mode 100644
index 0000000..2e1296e
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/ResourceRequest.java
@@ -0,0 +1,30 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.work;
+
+public class ResourceRequest {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ResourceRequest.class);
+  
+  public long memoryMin;
+  public long memoryDesired;
+  
+
+  public static class ResourceAllocation {
+    public long memory;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/RootNodeDriver.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/RootNodeDriver.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/RootNodeDriver.java
new file mode 100644
index 0000000..12da7ba
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/RootNodeDriver.java
@@ -0,0 +1,25 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.work;
+
+public interface RootNodeDriver {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(RootNodeDriver.class);
+  
+  public boolean doNext();
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/StatusProvider.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/StatusProvider.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/StatusProvider.java
new file mode 100644
index 0000000..bbd9df7
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/StatusProvider.java
@@ -0,0 +1,24 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.work;
+
+import org.apache.drill.exec.proto.ExecProtos.FragmentStatus;
+
+public interface StatusProvider {
+  public FragmentStatus getStatus();
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/WorkManager.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/WorkManager.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/WorkManager.java
new file mode 100644
index 0000000..d3664a0
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/WorkManager.java
@@ -0,0 +1,168 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.work;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.Collections;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.Executor;
+import java.util.concurrent.Executors;
+import java.util.concurrent.PriorityBlockingQueue;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.drill.exec.cache.DistributedCache;
+import org.apache.drill.exec.coord.ClusterCoordinator;
+import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
+import org.apache.drill.exec.proto.ExecProtos.FragmentHandle;
+import org.apache.drill.exec.proto.UserBitShared.QueryId;
+import org.apache.drill.exec.rpc.bit.BitCom;
+import org.apache.drill.exec.server.BootStrapContext;
+import org.apache.drill.exec.server.DrillbitContext;
+import org.apache.drill.exec.work.batch.BitComHandler;
+import org.apache.drill.exec.work.batch.BitComHandlerImpl;
+import org.apache.drill.exec.work.foreman.Foreman;
+import org.apache.drill.exec.work.fragment.IncomingFragmentHandler;
+import org.apache.drill.exec.work.fragment.RemoteFragmentHandler;
+import org.apache.drill.exec.work.user.UserWorker;
+
+import com.google.common.collect.Maps;
+import com.google.common.collect.Queues;
+
+public class WorkManager implements Closeable{
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(WorkManager.class);
+  
+  private Set<IncomingFragmentHandler> incomingFragments = Collections.newSetFromMap(Maps.<IncomingFragmentHandler, Boolean> newConcurrentMap());
+
+  private PriorityBlockingQueue<Runnable> pendingTasks = Queues.newPriorityBlockingQueue();
+  
+  private Map<FragmentHandle, FragmentRunner> runningFragments = Maps.newConcurrentMap();
+  
+  private ConcurrentMap<QueryId, Foreman> queries = Maps.newConcurrentMap();
+
+  private BootStrapContext bContext;
+  private DrillbitContext dContext;
+
+  private final BitComHandler bitComWorker;
+  private final UserWorker userWorker;
+  private final WorkerBee bee;
+  private Executor executor = Executors.newFixedThreadPool(4);
+  private final EventThread eventThread;
+  
+  public WorkManager(BootStrapContext context){
+    this.bee = new WorkerBee();
+    this.bContext = context;
+    this.bitComWorker = new BitComHandlerImpl(bee);
+    this.userWorker = new UserWorker(bee);
+    this.eventThread = new EventThread();
+    
+  }
+  
+  public void start(DrillbitEndpoint endpoint, DistributedCache cache, BitCom com, ClusterCoordinator coord){
+    this.dContext = new DrillbitContext(endpoint, bContext, coord, com, cache);
+    eventThread.start();
+  }
+  
+  public BitComHandler getBitComWorker(){
+    return bitComWorker;
+  }
+
+  public UserWorker getUserWorker(){
+    return userWorker;
+  }
+  
+  @Override
+  public void close() throws IOException {
+  }
+  
+
+  public DrillbitContext getContext() {
+    return dContext;
+  }
+
+  // create this so items can see the data here whether or not they are in this package.
+  public class WorkerBee{
+
+    public void addFragmentRunner(FragmentRunner runner){
+      pendingTasks.add(runner);
+    }
+    
+    public void addNewForeman(Foreman foreman){
+      pendingTasks.add(foreman);
+    }
+
+
+    public void addFragmentPendingRemote(IncomingFragmentHandler handler){
+      incomingFragments.add(handler);
+    }
+    
+    public void startFragmentPendingRemote(IncomingFragmentHandler handler){
+      incomingFragments.remove(handler);
+      pendingTasks.add(handler.getRunnable());
+    }
+    
+    public FragmentRunner getFragmentRunner(FragmentHandle handle){
+      return runningFragments.get(handle);
+    }
+    
+    public Foreman getForemanForQueryId(QueryId queryId){
+      return queries.get(queryId);
+    }
+    
+    public void retireForeman(Foreman foreman){
+      queries.remove(foreman.getQueryId(), foreman);
+    }
+
+    public DrillbitContext getContext() {
+      return dContext;
+    }
+
+  }
+
+
+
+ private class EventThread extends Thread{
+   public EventThread(){
+     this.setDaemon(true);
+     this.setName("WorkManager Event Thread");
+   }
+
+  @Override
+  public void run() {
+    try {
+    while(true){
+      logger.debug("Checking for pending work tasks.");
+      Runnable r = pendingTasks.poll(10, TimeUnit.SECONDS);
+      if(r != null){
+        executor.execute(r);  
+      }
+      
+    }
+    } catch (InterruptedException e) {
+      logger.info("Work Manager stopping as it was interrupted.");
+    }
+  }
+   
+   
+ }
+
+  
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/AbstractFragmentCollector.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/AbstractFragmentCollector.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/AbstractFragmentCollector.java
new file mode 100644
index 0000000..5dacb71
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/AbstractFragmentCollector.java
@@ -0,0 +1,84 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.work.batch;
+
+import java.util.List;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicIntegerArray;
+
+import org.apache.drill.exec.physical.base.Receiver;
+import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
+import org.apache.drill.exec.record.RawFragmentBatch;
+import org.apache.drill.exec.rpc.RemoteConnection.ConnectionThrottle;
+
+import com.google.common.base.Preconditions;
+
+public abstract class AbstractFragmentCollector implements BatchCollector{
+  private final List<DrillbitEndpoint> incoming;
+  private final int oppositeMajorFragmentId;
+  private final AtomicIntegerArray remainders;
+  private final AtomicInteger remainingRequired;
+  protected final RawBatchBuffer[] buffers;
+  private final AtomicInteger parentAccounter;
+  private final AtomicInteger finishedStreams = new AtomicInteger();
+  
+  public AbstractFragmentCollector(AtomicInteger parentAccounter, Receiver receiver, int minInputsRequired) {
+    Preconditions.checkArgument(minInputsRequired > 0);
+    Preconditions.checkNotNull(receiver);
+    Preconditions.checkNotNull(parentAccounter);
+
+    this.parentAccounter = parentAccounter;
+    this.incoming = receiver.getProvidingEndpoints();
+    this.remainders = new AtomicIntegerArray(incoming.size());
+    this.oppositeMajorFragmentId = receiver.getOppositeMajorFragmentId();
+    this.buffers = new RawBatchBuffer[minInputsRequired];
+    for(int i = 0; i < buffers.length; i++){
+      buffers[i] = new UnlmitedRawBatchBuffer();
+    }
+    if (receiver.supportsOutOfOrderExchange()) {
+      this.remainingRequired = new AtomicInteger(1);
+    } else {
+      this.remainingRequired = new AtomicInteger(minInputsRequired);
+    }
+  }
+
+  public int getOppositeMajorFragmentId() {
+    return oppositeMajorFragmentId;
+  }
+
+  public RawBatchBuffer[] getBuffers(){
+    return buffers;
+  }
+  
+  public abstract void streamFinished(int minorFragmentId);
+  
+  public void batchArrived(ConnectionThrottle throttle, int minorFragmentId, RawFragmentBatch batch) {
+    if (remainders.compareAndSet(minorFragmentId, 0, 1)) {
+      int rem = remainingRequired.decrementAndGet();
+      if (rem == 0) {
+        parentAccounter.decrementAndGet();
+      }
+    }
+    if(batch.getHeader().getIsLastBatch()){
+      streamFinished(minorFragmentId);
+    }
+    getBuffer(minorFragmentId).enqueue(throttle, batch);
+  }
+
+  protected abstract RawBatchBuffer getBuffer(int minorFragmentId);
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/BatchCollector.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/BatchCollector.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/BatchCollector.java
new file mode 100644
index 0000000..ff091d7
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/BatchCollector.java
@@ -0,0 +1,32 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.work.batch;
+
+
+import org.apache.drill.exec.record.RawFragmentBatch;
+import org.apache.drill.exec.rpc.RemoteConnection.ConnectionThrottle;
+
+
+interface BatchCollector {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(BatchCollector.class);
+
+  public void batchArrived(ConnectionThrottle throttle, int minorFragmentId, RawFragmentBatch batch);
+  public int getOppositeMajorFragmentId();
+  public RawBatchBuffer[] getBuffers();
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/BitComHandler.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/BitComHandler.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/BitComHandler.java
new file mode 100644
index 0000000..97064e3
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/BitComHandler.java
@@ -0,0 +1,41 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.work.batch;
+
+import io.netty.buffer.ByteBuf;
+
+import org.apache.drill.exec.proto.ExecProtos.FragmentHandle;
+import org.apache.drill.exec.proto.ExecProtos.PlanFragment;
+import org.apache.drill.exec.proto.GeneralRPCProtos.Ack;
+import org.apache.drill.exec.rpc.Response;
+import org.apache.drill.exec.rpc.RpcException;
+import org.apache.drill.exec.rpc.bit.BitConnection;
+import org.apache.drill.exec.work.fragment.IncomingFragmentHandler;
+
+public interface BitComHandler {
+
+  public abstract Response handle(BitConnection connection, int rpcType, ByteBuf pBody, ByteBuf dBody)
+      throws RpcException;
+
+  public abstract void startNewRemoteFragment(PlanFragment fragment);
+
+  public abstract Ack cancelFragment(FragmentHandle handle);
+
+  public abstract void registerIncomingFragmentHandler(IncomingFragmentHandler handler);
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/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
new file mode 100644
index 0000000..9b227da
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/BitComHandlerImpl.java
@@ -0,0 +1,205 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.work.batch;
+
+import static org.apache.drill.exec.rpc.RpcBus.get;
+import io.netty.buffer.ByteBuf;
+
+import java.io.IOException;
+import java.util.concurrent.ConcurrentMap;
+
+import org.apache.drill.common.exceptions.ExecutionSetupException;
+import org.apache.drill.exec.exception.FragmentSetupException;
+import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.physical.base.FragmentRoot;
+import org.apache.drill.exec.physical.impl.ImplCreator;
+import org.apache.drill.exec.physical.impl.RootExec;
+import org.apache.drill.exec.proto.ExecProtos.BitHandshake;
+import org.apache.drill.exec.proto.ExecProtos.BitStatus;
+import org.apache.drill.exec.proto.ExecProtos.FragmentHandle;
+import org.apache.drill.exec.proto.ExecProtos.FragmentRecordBatch;
+import org.apache.drill.exec.proto.ExecProtos.FragmentStatus;
+import org.apache.drill.exec.proto.ExecProtos.PlanFragment;
+import org.apache.drill.exec.proto.ExecProtos.RpcType;
+import org.apache.drill.exec.proto.GeneralRPCProtos.Ack;
+import org.apache.drill.exec.record.RawFragmentBatch;
+import org.apache.drill.exec.rpc.Acks;
+import org.apache.drill.exec.rpc.RemoteConnection;
+import org.apache.drill.exec.rpc.Response;
+import org.apache.drill.exec.rpc.RpcConstants;
+import org.apache.drill.exec.rpc.RpcException;
+import org.apache.drill.exec.rpc.bit.BitConnection;
+import org.apache.drill.exec.rpc.bit.BitRpcConfig;
+import org.apache.drill.exec.rpc.bit.BitTunnel;
+import org.apache.drill.exec.work.FragmentRunner;
+import org.apache.drill.exec.work.RemotingFragmentRunnerListener;
+import org.apache.drill.exec.work.WorkManager.WorkerBee;
+import org.apache.drill.exec.work.fragment.IncomingFragmentHandler;
+import org.apache.drill.exec.work.fragment.RemoteFragmentHandler;
+
+import com.google.common.collect.Maps;
+import com.google.protobuf.MessageLite;
+
+public class BitComHandlerImpl implements BitComHandler {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(BitComHandlerImpl.class);
+  
+  private ConcurrentMap<FragmentHandle, IncomingFragmentHandler> handlers = Maps.newConcurrentMap();
+  private final WorkerBee bee;
+  
+  public BitComHandlerImpl(WorkerBee bee) {
+    super();
+    this.bee = bee;
+  }
+
+  /* (non-Javadoc)
+   * @see org.apache.drill.exec.work.batch.BitComHandler#handle(org.apache.drill.exec.rpc.bit.BitConnection, int, io.netty.buffer.ByteBuf, io.netty.buffer.ByteBuf)
+   */
+  @Override
+  public Response handle(BitConnection connection, int rpcType, ByteBuf pBody, ByteBuf dBody) throws RpcException {
+    if(RpcConstants.EXTRA_DEBUGGING) logger.debug("Received bit com message of type {}", rpcType);
+
+    switch (rpcType) {
+    
+    case RpcType.REQ_CANCEL_FRAGMENT_VALUE:
+      FragmentHandle handle = get(pBody, FragmentHandle.PARSER);
+      cancelFragment(handle);
+      return BitRpcConfig.OK;
+
+    case RpcType.REQ_FRAGMENT_STATUS_VALUE:
+      connection.getListenerPool().status( get(pBody, FragmentStatus.PARSER));
+      // TODO: Support a type of message that has no response.
+      return BitRpcConfig.OK;
+
+    case RpcType.REQ_INIATILIZE_FRAGMENT_VALUE:
+      PlanFragment fragment = get(pBody, PlanFragment.PARSER);
+      startNewRemoteFragment(fragment);
+      return BitRpcConfig.OK;
+      
+    case RpcType.REQ_RECORD_BATCH_VALUE:
+      try {
+        FragmentRecordBatch header = get(pBody, FragmentRecordBatch.PARSER);
+        incomingRecordBatch(connection, header, dBody);
+        return BitRpcConfig.OK;
+      } catch (FragmentSetupException e) {
+        throw new RpcException("Failure receiving record batch.", e);
+      }
+
+    default:
+      throw new RpcException("Not yet supported.");
+    }
+
+  }
+  
+  
+  
+  /* (non-Javadoc)
+   * @see org.apache.drill.exec.work.batch.BitComHandler#startNewRemoteFragment(org.apache.drill.exec.proto.ExecProtos.PlanFragment)
+   */
+  @Override
+  public void startNewRemoteFragment(PlanFragment fragment){
+    logger.debug("Received remote fragment start instruction", fragment);
+    FragmentContext context = new FragmentContext(bee.getContext(), fragment.getHandle(), null, null);
+    BitTunnel tunnel = bee.getContext().getBitCom().getTunnel(fragment.getForeman());
+    RemotingFragmentRunnerListener listener = new RemotingFragmentRunnerListener(context, tunnel);
+    try{
+      FragmentRoot rootOperator = bee.getContext().getPlanReader().readFragmentOperator(fragment.getFragmentJson());
+      RootExec exec = ImplCreator.getExec(context, rootOperator);
+      FragmentRunner fr = new FragmentRunner(context, exec, listener);
+      bee.addFragmentRunner(fr);
+
+    }catch(IOException e){
+      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);
+    }
+    
+  }
+  
+  /* (non-Javadoc)
+   * @see org.apache.drill.exec.work.batch.BitComHandler#cancelFragment(org.apache.drill.exec.proto.ExecProtos.FragmentHandle)
+   */
+  @Override
+  public Ack cancelFragment(FragmentHandle handle){
+    IncomingFragmentHandler handler = handlers.get(handle);
+    if(handler != null){
+      // try remote fragment cancel.
+      handler.cancel();
+    }else{
+      // then try local cancel.
+      FragmentRunner runner = bee.getFragmentRunner(handle);
+      if(runner != null) runner.cancel();
+    }
+    
+    return Acks.OK;
+  }
+  
+  
+  /**
+   * Returns a positive Ack if this fragment is accepted.  
+   */
+  private Ack incomingRecordBatch(RemoteConnection connection, FragmentRecordBatch fragmentBatch, ByteBuf body) throws FragmentSetupException{
+    FragmentHandle handle = fragmentBatch.getHandle();
+    IncomingFragmentHandler handler = handlers.get(handle);
+
+    // Create a handler if there isn't already one.
+    if(handler == null){
+      
+      
+      
+      PlanFragment fragment = bee.getContext().getCache().getFragment(handle);
+      if(fragment == null){
+        logger.error("Received batch where fragment was not in cache.");
+        return Acks.FAIL;
+      }
+      
+
+      IncomingFragmentHandler newHandler = new RemoteFragmentHandler(fragment, bee.getContext(), bee.getContext().getBitCom().getTunnel(fragment.getForeman()));
+      
+      // since their could be a race condition on the check, we'll use putIfAbsent so we don't have two competing handlers.
+      handler = handlers.putIfAbsent(fragment.getHandle(), newHandler);
+          
+      if(handler == null){
+        // we added a handler, inform foreman that we did so.  This way, the foreman can track status.  We also tell foreman that we don't need inform ourself.
+        bee.addFragmentPendingRemote(newHandler);
+        handler = newHandler;
+      }
+    }
+    
+    boolean canRun = handler.handle(connection.getConnectionThrottle(), new RawFragmentBatch(fragmentBatch, body));
+    if(canRun){
+      // if we've reached the canRun threshold, we'll proceed.  This expects handler.handle() to only return a single true.
+      bee.startFragmentPendingRemote(handler);
+    }
+    if(handler.isDone()){
+      handlers.remove(handler.getHandle());
+    }
+    
+    return Acks.OK;
+  }
+  
+  /* (non-Javadoc)
+   * @see org.apache.drill.exec.work.batch.BitComHandler#registerIncomingFragmentHandler(org.apache.drill.exec.work.fragment.IncomingFragmentHandler)
+   */
+  @Override
+  public void registerIncomingFragmentHandler(IncomingFragmentHandler handler){
+    IncomingFragmentHandler old = handlers.putIfAbsent(handler.getHandle(), handler);
+    assert old == null : "You can only register a fragment handler if one hasn't been registered already.";
+  }
+  
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/IncomingBuffers.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/IncomingBuffers.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/IncomingBuffers.java
new file mode 100644
index 0000000..20775c5
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/IncomingBuffers.java
@@ -0,0 +1,108 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.work.batch;
+
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import org.apache.drill.exec.exception.FragmentSetupException;
+import org.apache.drill.exec.physical.base.AbstractPhysicalVisitor;
+import org.apache.drill.exec.physical.base.PhysicalOperator;
+import org.apache.drill.exec.physical.base.Receiver;
+import org.apache.drill.exec.record.RawFragmentBatch;
+import org.apache.drill.exec.rpc.RemoteConnection.ConnectionThrottle;
+
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.Maps;
+
+/**
+ * Determines when a particular fragment has enough data for each of its receiving exchanges to commence execution.
+ */
+public class IncomingBuffers {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(IncomingBuffers.class);
+
+  private final AtomicInteger streamsRemaining = new AtomicInteger(0);
+  private final AtomicInteger remainingRequired = new AtomicInteger(0);
+  private final Map<Integer, BatchCollector> fragCounts;
+
+  public IncomingBuffers(PhysicalOperator root) {
+    Map<Integer, BatchCollector> counts = Maps.newHashMap();
+    root.accept(new CountRequiredFragments(), counts);
+    streamsRemaining.set(remainingRequired.get());
+    fragCounts = ImmutableMap.copyOf(counts);
+  }
+
+  public boolean batchArrived(ConnectionThrottle throttle, RawFragmentBatch batch) throws FragmentSetupException {
+    // no need to do anything if we've already enabled running.
+    logger.debug("New Batch Arrived {}", batch);
+    if(batch.getHeader().getIsLastBatch()){
+      streamsRemaining.decrementAndGet();
+    }
+    
+    BatchCollector fSet = fragCounts.get(batch.getHeader().getSendingMajorFragmentId());
+    if (fSet == null) throw new FragmentSetupException("We received a major fragment id that we were not expecting.");
+    fSet.batchArrived(throttle, batch.getHeader().getSendingMinorFragmentId(), batch);
+    return remainingRequired.get() == 0;
+  }
+
+  public int getRemainingRequired() {
+    int rem = remainingRequired.get();
+    if (rem < 0) return 0;
+    return rem;
+  }
+
+  public RawBatchBuffer[] getBuffers(int senderMajorFragmentId){
+    return fragCounts.get(senderMajorFragmentId).getBuffers();
+  }
+  
+  
+  /**
+   * Designed to setup initial values for arriving fragment accounting.
+   */
+  public class CountRequiredFragments extends AbstractPhysicalVisitor<Void, Map<Integer, BatchCollector>, RuntimeException> {
+
+    @Override
+    public Void visitReceiver(Receiver receiver, Map<Integer, BatchCollector> counts) throws RuntimeException {
+      BatchCollector set;
+      if (receiver.supportsOutOfOrderExchange()) {
+        set = new MergingCollector(remainingRequired, receiver);
+      } else {
+        set = new PartitionedCollector(remainingRequired, receiver);
+      }
+
+      counts.put(set.getOppositeMajorFragmentId(), set);
+      remainingRequired.incrementAndGet();
+      return null;
+    }
+
+    
+    @Override
+    public Void visitOp(PhysicalOperator op, Map<Integer, BatchCollector> value) throws RuntimeException {
+      for(PhysicalOperator o : op){
+        o.accept(this, value);
+      }
+      return null;
+    }
+
+
+  }
+
+  public boolean isDone(){
+    return streamsRemaining.get() < 1;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/MergingCollector.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/MergingCollector.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/MergingCollector.java
new file mode 100644
index 0000000..e21d69a
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/MergingCollector.java
@@ -0,0 +1,45 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.work.batch;
+
+import java.util.concurrent.atomic.AtomicInteger;
+
+import org.apache.drill.exec.physical.base.Receiver;
+
+public class MergingCollector extends AbstractFragmentCollector{
+
+  private AtomicInteger streamsRunning;
+  
+  public MergingCollector(AtomicInteger parentAccounter, Receiver receiver) {
+    super(parentAccounter, receiver, 1);
+    streamsRunning = new AtomicInteger(parentAccounter.get());
+  }
+
+  @Override
+  protected RawBatchBuffer getBuffer(int minorFragmentId) {
+    return buffers[0];
+  }
+
+  @Override
+  public void streamFinished(int minorFragmentId) {
+    if(streamsRunning.decrementAndGet() == 0) buffers[0].finished();
+  }
+  
+  
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/PartitionedCollector.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/PartitionedCollector.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/PartitionedCollector.java
new file mode 100644
index 0000000..116ca26
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/PartitionedCollector.java
@@ -0,0 +1,42 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.work.batch;
+
+import java.util.concurrent.atomic.AtomicInteger;
+
+import org.apache.drill.exec.physical.base.Receiver;
+
+public class PartitionedCollector extends AbstractFragmentCollector{
+
+  public PartitionedCollector(AtomicInteger parentAccounter, Receiver receiver) {
+    super(parentAccounter, receiver, receiver.getProvidingEndpoints().size());
+  }
+
+  @Override
+  protected RawBatchBuffer getBuffer(int minorFragmentId) {
+    return buffers[minorFragmentId];
+  }
+
+  @Override
+  public void streamFinished(int minorFragmentId) {
+    buffers[minorFragmentId].finished();
+  }
+  
+  
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/RawBatchBuffer.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/RawBatchBuffer.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/RawBatchBuffer.java
new file mode 100644
index 0000000..0f10e26
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/RawBatchBuffer.java
@@ -0,0 +1,33 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.work.batch;
+
+import org.apache.drill.exec.record.RawFragmentBatch;
+import org.apache.drill.exec.record.RawFragmentBatchProvider;
+import org.apache.drill.exec.rpc.RemoteConnection.ConnectionThrottle;
+
+public interface RawBatchBuffer extends RawFragmentBatchProvider{
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(RawBatchBuffer.class);
+  
+  public void enqueue(ConnectionThrottle throttle, RawFragmentBatch batch);
+  
+  /**
+   * Inform the buffer that no more records are expected.
+   */
+  public void finished();
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/UnlmitedRawBatchBuffer.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/UnlmitedRawBatchBuffer.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/UnlmitedRawBatchBuffer.java
new file mode 100644
index 0000000..f97d878
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/UnlmitedRawBatchBuffer.java
@@ -0,0 +1,73 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.work.batch;
+
+import java.util.Iterator;
+import java.util.concurrent.LinkedBlockingDeque;
+
+import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.record.RawFragmentBatch;
+import org.apache.drill.exec.rpc.RemoteConnection.ConnectionThrottle;
+
+import com.google.common.collect.Queues;
+
+public class UnlmitedRawBatchBuffer implements RawBatchBuffer{
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(UnlmitedRawBatchBuffer.class);
+
+  private final LinkedBlockingDeque<RawFragmentBatch> buffer = Queues.newLinkedBlockingDeque();
+  private volatile boolean finished = false;
+  
+  @Override
+  public void enqueue(ConnectionThrottle throttle, RawFragmentBatch batch) {
+    buffer.add(batch);
+  }
+
+//  @Override
+//  public RawFragmentBatch dequeue() {
+//    return buffer.poll();
+//  }
+
+  @Override
+  public void kill(FragmentContext context) {
+    // TODO: Pass back or kill handler?
+  }
+
+  
+  @Override
+  public void finished() {
+    finished = true;
+  }
+
+  @Override
+  public RawFragmentBatch getNext(){
+    
+    RawFragmentBatch b = buffer.poll();
+    if(b == null && !finished){
+      try {
+        return buffer.take();
+      } catch (InterruptedException e) {
+        return null;
+      }
+    }
+    
+    return null;
+    
+  }
+
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/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
new file mode 100644
index 0000000..d4c4014
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/ErrorHelper.java
@@ -0,0 +1,47 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.work.foreman;
+
+import java.util.UUID;
+
+import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
+import org.apache.drill.exec.proto.UserBitShared.DrillPBError;
+import org.slf4j.Logger;
+
+
+public class ErrorHelper {
+  
+  public static DrillPBError logAndConvertError(DrillbitEndpoint endpoint, String message, Throwable t, Logger logger){
+    String id = UUID.randomUUID().toString();
+    DrillPBError.Builder builder = DrillPBError.newBuilder();
+    builder.setEndpoint(endpoint);
+    builder.setErrorId(id);
+    if(message != null){
+      builder.setMessage(message);  
+    }else{
+      builder.setMessage(t.getMessage());
+    }
+    builder.setErrorType(0);
+    
+    // record the error to the log for later reference.
+    logger.error("Error {}: {}", id, message, t);
+    
+    
+    return builder.build();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/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
new file mode 100644
index 0000000..dea8282
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/Foreman.java
@@ -0,0 +1,272 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.work.foreman;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.List;
+
+import org.apache.drill.common.exceptions.ExecutionSetupException;
+import org.apache.drill.common.logical.LogicalPlan;
+import org.apache.drill.exec.exception.FragmentSetupException;
+import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.ops.QueryContext;
+import org.apache.drill.exec.physical.PhysicalPlan;
+import org.apache.drill.exec.physical.base.PhysicalOperator;
+import org.apache.drill.exec.physical.impl.ImplCreator;
+import org.apache.drill.exec.physical.impl.RootExec;
+import org.apache.drill.exec.physical.impl.materialize.QueryWritableBatch;
+import org.apache.drill.exec.planner.fragment.Fragment;
+import org.apache.drill.exec.planner.fragment.MakeFragmentsVisitor;
+import org.apache.drill.exec.planner.fragment.PlanningSet;
+import org.apache.drill.exec.planner.fragment.SimpleParallelizer;
+import org.apache.drill.exec.planner.fragment.StatsCollector;
+import org.apache.drill.exec.proto.ExecProtos.FragmentHandle;
+import org.apache.drill.exec.proto.ExecProtos.PlanFragment;
+import org.apache.drill.exec.proto.GeneralRPCProtos.Ack;
+import org.apache.drill.exec.proto.UserBitShared.DrillPBError;
+import org.apache.drill.exec.proto.UserBitShared.QueryId;
+import org.apache.drill.exec.proto.UserProtos.QueryResult;
+import org.apache.drill.exec.proto.UserProtos.QueryResult.QueryState;
+import org.apache.drill.exec.proto.UserProtos.RequestResults;
+import org.apache.drill.exec.proto.UserProtos.RunQuery;
+import org.apache.drill.exec.rpc.RpcException;
+import org.apache.drill.exec.rpc.RpcOutcomeListener;
+import org.apache.drill.exec.rpc.user.UserServer.UserClientConnection;
+import org.apache.drill.exec.server.DrillbitContext;
+import org.apache.drill.exec.util.AtomicState;
+import org.apache.drill.exec.work.QueryWorkUnit;
+import org.apache.drill.exec.work.WorkManager.WorkerBee;
+
+import com.google.common.collect.Lists;
+
+/**
+ * Foreman manages all queries where this is the driving/root node.
+ */
+public class Foreman implements Runnable, Closeable, Comparable<Object>{
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(Foreman.class);
+
+  private QueryId queryId;
+  private RunQuery queryRequest;
+  private QueryContext context;
+  private RunningFragmentManager fragmentManager;
+  private WorkerBee bee;
+  private UserClientConnection initiatingClient;
+  private final AtomicState<QueryState> state;
+  
+  
+  public Foreman(WorkerBee bee, DrillbitContext dContext, UserClientConnection connection, QueryId queryId,
+      RunQuery queryRequest) {
+    this.queryId = queryId;
+    this.queryRequest = queryRequest;
+    this.context = new QueryContext(queryId, dContext);
+    this.initiatingClient = connection;
+    this.fragmentManager = new RunningFragmentManager(new ForemanManagerListener(), new TunnelManager(dContext.getBitCom()));
+    this.bee = bee;
+    
+    this.state = new AtomicState<QueryState>(QueryState.PENDING) {
+      protected QueryState getStateFromNumber(int i) {
+        return QueryState.valueOf(i);
+      }
+    };
+  }
+  
+  private boolean isFinished(){
+    switch(state.getState()){
+    case PENDING:
+    case RUNNING:
+      return false;
+    default:
+      return true;
+    }
+    
+  }
+
+  private void fail(String message, Throwable t) {
+    if(isFinished()){
+      logger.error("Received a failure message query finished of: {}", message, t);
+    }
+    DrillPBError error = ErrorHelper.logAndConvertError(context.getCurrentEndpoint(), message, t, logger);
+    QueryResult result = QueryResult //
+        .newBuilder() //
+        .addError(error) //
+        .setIsLastChunk(true) //
+        .setQueryState(QueryState.FAILED) //
+        .build();
+    cleanupAndSendResult(result);
+  }
+
+  
+  public void cancel() {
+    if(isFinished()){
+      return;
+    }
+    
+    // cancel remote fragments.
+    fragmentManager.cancel();
+    
+    QueryResult result = QueryResult.newBuilder().setQueryState(QueryState.CANCELED).setIsLastChunk(true).setQueryId(queryId).build();
+    cleanupAndSendResult(result);
+  }
+  
+  void cleanupAndSendResult(QueryResult result){
+    bee.retireForeman(this);
+    initiatingClient.sendResult(new QueryWritableBatch(result)).addLightListener(new ResponseSendListener());
+  }
+
+  private class ResponseSendListener extends RpcOutcomeListener<Ack> {
+    @Override
+    public void failed(RpcException ex) {
+      logger
+          .info(
+              "Failure while trying communicate query result to initating client.  This would happen if a client is disconnected before response notice can be sent.",
+              ex);
+    }
+  }
+  
+
+
+  /**
+   * Called by execution pool to do foreman setup. Actual query execution is a separate phase (and can be scheduled).
+   */
+  public void run() {
+    // convert a run query request into action
+
+    switch (queryRequest.getType()) {
+
+    case LOGICAL:
+      parseAndRunLogicalPlan(queryRequest.getPlan());
+      break;
+    case PHYSICAL:
+      parseAndRunPhysicalPlan(queryRequest.getPlan());
+      break;
+    case SQL:
+      runSQL(queryRequest.getPlan());
+      break;
+    default:
+      throw new UnsupportedOperationException();
+    }
+  }
+
+  private void parseAndRunLogicalPlan(String json) {
+    try {
+      LogicalPlan logicalPlan = context.getPlanReader().readLogicalPlan(json);
+      PhysicalPlan physicalPlan = convert(logicalPlan);
+      runPhysicalPlan(physicalPlan);
+    } catch (IOException e) {
+      fail("Failure while parsing logical plan.", e);
+    }
+  }
+
+  private void parseAndRunPhysicalPlan(String json) {
+    try {
+      PhysicalPlan plan = context.getPlanReader().readPhysicalPlan(json);
+      runPhysicalPlan(plan);
+    } catch (IOException e) {
+      fail("Failure while parsing physical plan.", e);
+    }
+  }
+
+  private void runPhysicalPlan(PhysicalPlan plan) {
+
+    PhysicalOperator rootOperator = plan.getSortedOperators(false).iterator().next();
+    MakeFragmentsVisitor makeFragmentsVisitor = new MakeFragmentsVisitor();
+    Fragment rootFragment;
+    try {
+      rootFragment = rootOperator.accept(makeFragmentsVisitor, null);
+    } catch (FragmentSetupException e) {
+      fail("Failure while fragmenting query.", e);
+      return;
+    }
+    PlanningSet planningSet = StatsCollector.collectStats(rootFragment);
+    SimpleParallelizer parallelizer = new SimpleParallelizer();
+
+    try {
+      QueryWorkUnit work = parallelizer.getFragments(context.getCurrentEndpoint(), queryId, context.getActiveEndpoints(), context.getPlanReader(), rootFragment, planningSet, 10);
+
+      List<PlanFragment> leafFragments = Lists.newArrayList();
+
+      // store fragments in distributed grid.
+      for (PlanFragment f : work.getFragments()) {
+        if (f.getLeafFragment()) {
+          leafFragments.add(f);
+        } else {
+          context.getCache().storeFragment(f);
+        }
+      }
+
+      fragmentManager.runFragments(bee, work.getRootFragment(), work.getRootOperator(), initiatingClient, leafFragments);
+
+    
+    } catch (ExecutionSetupException e) {
+      fail("Failure while setting up query.", e);
+    }
+
+  }
+
+  private void runSQL(String json) {
+    throw new UnsupportedOperationException();
+  }
+
+  private PhysicalPlan convert(LogicalPlan plan) {
+    throw new UnsupportedOperationException();
+  }
+
+  public QueryResult getResult(UserClientConnection connection, RequestResults req) {
+
+    return null;
+  }
+
+
+  public QueryId getQueryId() {
+    return queryId;
+  }
+
+  @Override
+  public void close() throws IOException {
+  }
+  
+  QueryState getQueryState(){
+    return this.state.getState();
+  }
+
+  public boolean rootCoorespondsTo(FragmentHandle handle){
+    throw new UnsupportedOperationException();
+  }
+  
+  class ForemanManagerListener{
+    void fail(String message, Throwable t) {
+      ForemanManagerListener.this.fail(message, t);
+    }
+    
+    void cleanupAndSendResult(QueryResult result){
+      ForemanManagerListener.this.cleanupAndSendResult(result);
+    }
+    
+  }
+
+
+
+  @Override
+  public int compareTo(Object o) {
+    return o.hashCode() - o.hashCode();
+  }
+  
+  
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/FragmentStatusListener.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/FragmentStatusListener.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/FragmentStatusListener.java
new file mode 100644
index 0000000..d906ba2
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/FragmentStatusListener.java
@@ -0,0 +1,26 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.work.foreman;
+
+import org.apache.drill.exec.proto.ExecProtos.FragmentStatus;
+
+public interface FragmentStatusListener {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(FragmentStatusListener.class);
+  
+  public void statusUpdate(FragmentStatus status);
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/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
new file mode 100644
index 0000000..20797b8
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/RunningFragmentManager.java
@@ -0,0 +1,266 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.work.foreman;
+
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import org.apache.drill.common.exceptions.ExecutionSetupException;
+import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.physical.base.FragmentRoot;
+import org.apache.drill.exec.physical.impl.ImplCreator;
+import org.apache.drill.exec.physical.impl.RootExec;
+import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
+import org.apache.drill.exec.proto.ExecProtos.FragmentHandle;
+import org.apache.drill.exec.proto.ExecProtos.FragmentStatus;
+import org.apache.drill.exec.proto.ExecProtos.FragmentStatus.FragmentState;
+import org.apache.drill.exec.proto.ExecProtos.PlanFragment;
+import org.apache.drill.exec.proto.GeneralRPCProtos.Ack;
+import org.apache.drill.exec.proto.UserProtos.QueryResult;
+import org.apache.drill.exec.proto.UserProtos.QueryResult.QueryState;
+import org.apache.drill.exec.rpc.RpcException;
+import org.apache.drill.exec.rpc.RpcOutcomeListener;
+import org.apache.drill.exec.rpc.user.UserServer.UserClientConnection;
+import org.apache.drill.exec.work.AbstractFragmentRunnerListener;
+import org.apache.drill.exec.work.EndpointListener;
+import org.apache.drill.exec.work.FragmentRunner;
+import org.apache.drill.exec.work.WorkManager.WorkerBee;
+import org.apache.drill.exec.work.batch.IncomingBuffers;
+import org.apache.drill.exec.work.foreman.Foreman.ForemanManagerListener;
+import org.apache.drill.exec.work.fragment.LocalFragmentHandler;
+
+import com.google.common.collect.Maps;
+
+/**
+ * Each Foreman holds its own fragment manager.  This manages the events associated with execution of a particular query across all fragments.  
+ */
+class RunningFragmentManager implements FragmentStatusListener{
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(RunningFragmentManager.class);
+  
+  public Map<FragmentHandle, FragmentData> map = Maps.newHashMap(); // doesn't need to be
+  private final TunnelManager tun;
+  private ForemanManagerListener foreman;
+  private AtomicInteger remainingFragmentCount;
+  private FragmentRunner rootRunner;
+  
+  public RunningFragmentManager(ForemanManagerListener foreman, TunnelManager tun) {
+    super();
+    this.foreman = foreman;
+    this.tun = tun;
+    this.remainingFragmentCount = new AtomicInteger(0);
+  }
+
+  public void runFragments(WorkerBee bee, PlanFragment rootFragment, FragmentRoot rootOperator, UserClientConnection rootClient, List<PlanFragment> leafFragments) throws ExecutionSetupException{
+    remainingFragmentCount.set(leafFragments.size()+1);
+
+    // set up the root framgnet first so we'll have incoming buffers available.
+    {
+      IncomingBuffers buffers = new IncomingBuffers(rootOperator);
+      
+      FragmentContext rootContext = new FragmentContext(bee.getContext(), rootFragment.getHandle(), rootClient, buffers);
+      RootExec rootExec = ImplCreator.getExec(rootContext, rootOperator);
+      // add fragment to local node.
+      map.put(rootFragment.getHandle(), new FragmentData(rootFragment.getHandle(), null, true));
+      rootRunner = new FragmentRunner(rootContext, rootExec, new RootFragmentManager(rootContext, rootFragment));
+      LocalFragmentHandler handler = new LocalFragmentHandler(rootFragment.getHandle(), buffers, rootRunner);
+      if(buffers.isDone()){
+        bee.addFragmentRunner(handler.getRunnable());
+      }else{
+        bee.getContext().getBitCom().registerIncomingBatchHandler(handler);
+      }
+      
+    }
+    
+    // send remote fragments.
+    for (PlanFragment f : leafFragments) {
+      sendRemoteFragment(f);
+    }
+    
+  }
+    
+  private void sendRemoteFragment(PlanFragment fragment){
+    map.put(fragment.getHandle(), new FragmentData(fragment.getHandle(), fragment.getAssignment(), false));
+    FragmentSubmitListener listener = new FragmentSubmitListener(fragment.getAssignment(), fragment);
+    tun.get(fragment.getAssignment()).sendFragment(fragment).addLightListener(listener);
+  }
+  
+  
+  @Override
+  public void statusUpdate(FragmentStatus status) {
+    
+    switch(status.getState()){
+    case AWAITING_ALLOCATION:
+      updateStatus(status);
+      break;
+    case CANCELLED:
+      // we don't care about cancellation messages since we're the only entity that should drive cancellations.
+      break;
+    case FAILED:
+      fail(status);
+      break;
+    case FINISHED:
+      finished(status);
+      break;
+    case RUNNING:
+      updateStatus(status);
+      break;
+    default:
+      throw new UnsupportedOperationException();
+    }
+  }
+  
+  private void updateStatus(FragmentStatus status){
+    map.get(status.getHandle()).setStatus(status);
+  }
+  
+  private void finished(FragmentStatus status){
+    updateStatus(status);
+    int remaining = remainingFragmentCount.decrementAndGet();
+    if(remaining == 0){
+      QueryResult result = QueryResult.newBuilder().setQueryState(QueryState.COMPLETED).build();
+      foreman.cleanupAndSendResult(result);
+    }
+  }
+  
+  private void fail(FragmentStatus status){
+    updateStatus(status);
+    stopQuery();
+    QueryResult result = QueryResult.newBuilder().setQueryState(QueryState.FAILED).build();
+    foreman.cleanupAndSendResult(result);
+  }
+ 
+  
+  private void stopQuery(){
+    // Stop all queries with a currently active status.
+//    for(FragmentData data: map.values()){
+//      FragmentHandle handle = data.getStatus().getHandle();
+//      switch(data.getStatus().getState()){
+//      case SENDING:
+//      case AWAITING_ALLOCATION:
+//      case RUNNING:
+//        if(data.isLocal()){
+//          rootRunner.cancel();
+//        }else{
+//          tun.get(data.getEndpoint()).cancelFragment(handle).addLightListener(new CancelListener(data.endpoint, handle));
+//        }
+//        break;
+//      default:
+//        break;
+//      }
+//    }
+  }
+  
+  public void cancel(){
+    stopQuery();
+  }
+
+  private class CancelListener extends EndpointListener<Ack, FragmentHandle>{
+    
+    public CancelListener(DrillbitEndpoint endpoint, FragmentHandle handle) {
+      super(endpoint, handle);
+    }
+
+    @Override
+    public void failed(RpcException ex) {
+      logger.error("Failure while attempting to cancel fragment {} on endpoint {}.", value, endpoint, ex);
+    }
+
+    @Override
+    public void success(Ack value) {
+      if(!value.getOk()){
+        logger.warn("Remote node {} responded negative on cancellation request for fragment {}.", endpoint, value);
+      }
+      // do nothing.
+    }
+
+  };
+  
+  public RpcOutcomeListener<Ack> getSubmitListener(DrillbitEndpoint endpoint, PlanFragment value){
+    return new FragmentSubmitListener(endpoint, value);
+  }
+  
+  
+  
+  private class FragmentSubmitListener extends EndpointListener<Ack, PlanFragment>{
+    
+    public FragmentSubmitListener(DrillbitEndpoint endpoint, PlanFragment value) {
+      super(endpoint, value);
+    }
+
+    @Override
+    public void failed(RpcException ex) {
+      stopQuery();
+    }
+
+  }
+  
+  
+  private class FragmentData{
+    private final boolean isLocal;
+    private volatile FragmentStatus status;
+    private volatile long lastStatusUpdate = 0;
+    private final DrillbitEndpoint endpoint;
+    
+    public FragmentData(FragmentHandle handle, DrillbitEndpoint endpoint, boolean isLocal) {
+      super();
+      this.status = FragmentStatus.newBuilder().setHandle(handle).setState(FragmentState.SENDING).build();
+      this.endpoint = endpoint;
+      this.isLocal = isLocal;
+    }
+    
+    public void setStatus(FragmentStatus status){
+      this.status = status;
+      lastStatusUpdate = System.currentTimeMillis();
+    }
+
+    public FragmentStatus getStatus() {
+      return status;
+    }
+
+    public boolean isLocal() {
+      return isLocal;
+    }
+
+    public long getLastStatusUpdate() {
+      return lastStatusUpdate;
+    }
+
+    public DrillbitEndpoint getEndpoint() {
+      return endpoint;
+    }
+    
+    
+  }
+
+  private class RootFragmentManager extends AbstractFragmentRunnerListener{
+
+    private RootFragmentManager(FragmentContext context, PlanFragment fragment){
+      super(context);
+    }
+
+    @Override
+    protected void statusChange(FragmentHandle handle, FragmentStatus status) {
+      RunningFragmentManager.this.updateStatus(status);
+    }
+
+
+    
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/TunnelManager.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/TunnelManager.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/TunnelManager.java
new file mode 100644
index 0000000..ad3534c
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/TunnelManager.java
@@ -0,0 +1,53 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.work.foreman;
+
+import java.util.Map;
+
+import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
+import org.apache.drill.exec.rpc.RpcException;
+import org.apache.drill.exec.rpc.bit.BitCom;
+import org.apache.drill.exec.rpc.bit.BitTunnel;
+
+import com.google.common.collect.Maps;
+
+/**
+ * Keeps a local list of tunnels associated with a particular Foreman.
+ */
+public class TunnelManager {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TunnelManager.class);
+  
+  private final BitCom com;
+  private Map<DrillbitEndpoint, BitTunnel> tunnels = Maps.newHashMap();
+  
+  public TunnelManager(BitCom com){
+    this.com = com;
+  }
+  
+  public BitTunnel get(DrillbitEndpoint ep){
+    BitTunnel bt = tunnels.get(ep);
+    if(bt == null){
+      bt = com.getTunnel(ep);
+      tunnels.put(ep, bt);
+    }
+    
+    return bt;
+  }
+  
+   
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/IncomingFragmentHandler.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/IncomingFragmentHandler.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/IncomingFragmentHandler.java
new file mode 100644
index 0000000..b4e9308
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/IncomingFragmentHandler.java
@@ -0,0 +1,49 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.work.fragment;
+
+import org.apache.drill.exec.exception.FragmentSetupException;
+import org.apache.drill.exec.proto.ExecProtos.FragmentHandle;
+import org.apache.drill.exec.record.RawFragmentBatch;
+import org.apache.drill.exec.rpc.RemoteConnection.ConnectionThrottle;
+import org.apache.drill.exec.work.FragmentRunner;
+
+/**
+ * Handles incoming fragments as they arrive, routing them as apporpriate. 
+ */
+public interface IncomingFragmentHandler {
+
+  /**
+   * Handle the next incoming fragment.
+   * @param throttle
+   * @param batch
+   * @return True if the fragment has enough incoming data to be able to be run.
+   * @throws FragmentSetupException
+   */
+  public abstract boolean handle(ConnectionThrottle throttle, RawFragmentBatch batch) throws FragmentSetupException;
+
+  /**
+   * Get the fragment runner for this incoming fragment.  Note, this can only be requested once.
+   * @return
+   */
+  public abstract FragmentRunner getRunnable();
+
+  public abstract void cancel();
+  public boolean isDone();
+  public abstract FragmentHandle getHandle();
+}
\ No newline at end of file


[45/53] [abbrv] Working project operator. Update to ASM 4.1

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/80b1d24f/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/SelectionVector2.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/SelectionVector2.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/SelectionVector2.java
index c314cd4..b808dc6 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/SelectionVector2.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/SelectionVector2.java
@@ -43,4 +43,7 @@ public class SelectionVector2{
     return buffer.getChar(directIndex);
   }
 
+  public void setIndex(int directIndex, char value){
+    buffer.setChar(directIndex, value);
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/80b1d24f/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 47aee7d..cb1e1d6 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
@@ -37,15 +37,16 @@ public class TestClassTransformation {
   
   private void testBasicClassCompilation(boolean useJanino) throws ClassTransformationException{
     final String output = "hello world, the time is now " + System.currentTimeMillis();
-    @SuppressWarnings("unchecked")
-    TemplateClassDefinition<ExampleExternalInterface, Object> def = new TemplateClassDefinition(
+
+    TemplateClassDefinition<ExampleExternalInterface> def = new TemplateClassDefinition<ExampleExternalInterface>(
         ExampleExternalInterface.class, "org.apache.drill.exec.compile.ExampleTemplate",
-        ExampleInternalInterface.class, Object.class);
+        ExampleInternalInterface.class, "a", "b");
+    
+    
     ClassTransformer ct = new ClassTransformer();
     QueryClassLoader loader = new QueryClassLoader(useJanino);
-    ExampleExternalInterface instance = ct.getImplementationClass(loader, def,
-        "public String getInternalData(){return \"" + output + "\";}",
-        new Object());
+    ExampleExternalInterface instance = ct.getImplementationClassByBody(loader, def,
+        "public String getInternalData(){return \"" + output + "\";}");
     System.out.println(String.format("Generated a new class %s that provides the following getData response '%s'.",
         instance.getClass().getCanonicalName(), instance.getData()));
     assertEquals(instance.getData(), output);

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/80b1d24f/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/expr/ExpressionTest.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/expr/ExpressionTest.java b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/expr/ExpressionTest.java
index 16f7802..007abb3 100644
--- a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/expr/ExpressionTest.java
+++ b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/expr/ExpressionTest.java
@@ -23,6 +23,7 @@ import org.apache.drill.common.types.TypeProtos.MajorType;
 import org.apache.drill.common.types.TypeProtos.MinorType;
 import org.apache.drill.exec.expr.CodeGenerator.HoldingContainer;
 import org.apache.drill.exec.expr.fn.FunctionImplementationRegistry;
+import org.apache.drill.exec.physical.impl.project.Projector;
 import org.apache.drill.exec.record.RecordBatch;
 import org.apache.drill.exec.record.RecordBatch.TypedFieldId;
 import org.apache.drill.exec.record.vector.Fixed4;
@@ -48,7 +49,7 @@ public class ExpressionTest {
 
     new NonStrictExpectations() {
       {
-        batch.getValueVector(new SchemaPath("alpha", ExpressionPosition.UNKNOWN));
+        batch.getValueVectorId(new SchemaPath("alpha", ExpressionPosition.UNKNOWN));
         result = tfid;
         batch.getValueVectorById(tfid.getFieldId(), Fixed4.class);
         result = new Fixed4(null, null);
@@ -65,7 +66,7 @@ public class ExpressionTest {
 
     new Expectations() {
       {
-        batch.getValueVector(new SchemaPath("alpha", ExpressionPosition.UNKNOWN));
+        batch.getValueVectorId(new SchemaPath("alpha", ExpressionPosition.UNKNOWN));
         result = tfid;
         // batch.getValueVectorById(tfid); result = new Fixed4(null, null);
       }
@@ -95,7 +96,7 @@ public class ExpressionTest {
       assertEquals(0, error.getErrorCount());
     }
 
-    CodeGenerator cg = new CodeGenerator("setup", "eval", new FunctionImplementationRegistry(DrillConfig.create()));
+    CodeGenerator<Projector> cg = new CodeGenerator<Projector>(Projector.TEMPLATE_DEFINITION, new FunctionImplementationRegistry(DrillConfig.create()));
     cg.addNextWrite(new ValueVectorWriteExpression(-1, materializedExpr));
     return cg.generate();
   }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/80b1d24f/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/SimpleRootExec.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/SimpleRootExec.java b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/SimpleRootExec.java
new file mode 100644
index 0000000..093c58f
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/SimpleRootExec.java
@@ -0,0 +1,40 @@
+package org.apache.drill.exec.physical.impl;
+
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.exec.physical.impl.ScreenCreator.ScreenRoot;
+import org.apache.drill.exec.record.RecordBatch;
+import org.apache.drill.exec.record.RecordBatch.IterOutcome;
+import org.apache.drill.exec.record.RecordBatch.TypedFieldId;
+import org.apache.drill.exec.record.vector.ValueVector;
+
+public class SimpleRootExec implements RootExec{
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(SimpleRootExec.class);
+
+  private RecordBatch incoming;
+  
+  public SimpleRootExec(RootExec e){
+    if(e instanceof ScreenRoot){
+      incoming = ((ScreenRoot)e).getIncoming();  
+    }else{
+      throw new UnsupportedOperationException();
+    }
+    
+  }
+
+
+  public <T extends ValueVector<T>> T getValueVectorById(SchemaPath path, Class<?> vvClass){
+    TypedFieldId tfid = incoming.getValueVectorId(path);
+    return incoming.getValueVectorById(tfid.getFieldId(), vvClass);
+  }
+  
+  @Override
+  public boolean next() {
+    return incoming.next() != IterOutcome.NONE;
+  }
+
+  @Override
+  public void stop() {
+  }
+
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/80b1d24f/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/project/TestSimpleProjection.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/project/TestSimpleProjection.java b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/project/TestSimpleProjection.java
new file mode 100644
index 0000000..4144a54
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/project/TestSimpleProjection.java
@@ -0,0 +1,69 @@
+package org.apache.drill.exec.physical.impl.project;
+
+import static org.junit.Assert.assertEquals;
+import mockit.Injectable;
+import mockit.NonStrictExpectations;
+
+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.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.FragmentHandle;
+import org.apache.drill.exec.record.vector.Fixed8;
+import org.apache.drill.exec.rpc.user.UserServer.UserClientConnection;
+import org.apache.drill.exec.server.DrillbitContext;
+import org.junit.After;
+import org.junit.Test;
+
+import com.google.common.base.Charsets;
+import com.google.common.io.Files;
+import com.yammer.metrics.MetricRegistry;
+
+public class TestSimpleProjection {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestSimpleProjection.class);
+  DrillConfig c = DrillConfig.create();
+  
+  
+  @Test
+  public void project(@Injectable final DrillbitContext bitContext, @Injectable UserClientConnection connection) throws Exception{
+    System.out.println(System.getProperty("java.class.path"));
+
+
+    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("/project/test1.json"), Charsets.UTF_8));
+    FunctionImplementationRegistry registry = new FunctionImplementationRegistry(c);
+    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()){
+      Fixed8 c1 = exec.getValueVectorById(new SchemaPath("col1", ExpressionPosition.UNKNOWN), Fixed8.class);
+      Fixed8 c2 = exec.getValueVectorById(new SchemaPath("col2", ExpressionPosition.UNKNOWN), Fixed8.class);
+      int x = 0;
+      for(int i =0; i < c1.getRecordCount(); i++){
+        assertEquals(c1.get(i)+1, c2.get(i));
+        x += c1.get(i);
+      }
+      System.out.println(x);
+    }
+  }
+  
+  @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/80b1d24f/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 99bf820..925faf7 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
@@ -66,7 +66,7 @@ public class ExpressionTreeMaterializerTest {
     
     new NonStrictExpectations() {
       {
-        batch.getValueVector(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);
       }
     };
     
@@ -81,8 +81,8 @@ public class ExpressionTreeMaterializerTest {
   public void testMaterializingLateboundTree(final @Injectable RecordBatch batch) throws SchemaChangeException {
     new NonStrictExpectations() {
       {
-        batch.getValueVector(new FieldReference("test", ExpressionPosition.UNKNOWN)); result = new TypedFieldId(Types.required(MinorType.BOOLEAN), -4);
-        batch.getValueVector(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.BOOLEAN), -4);
+        batch.getValueVectorId(new FieldReference("test1", ExpressionPosition.UNKNOWN)); result = new TypedFieldId(Types.required(MinorType.BIGINT), -5);
       }
     };
     
@@ -175,7 +175,7 @@ public class ExpressionTreeMaterializerTest {
 
     new NonStrictExpectations() {
       {
-        batch.getValueVector(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);
       }
     };
     

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/80b1d24f/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/test/generated/TestGeneratedClass.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/test/generated/TestGeneratedClass.java b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/test/generated/TestGeneratedClass.java
new file mode 100644
index 0000000..4e5e4c7
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/test/generated/TestGeneratedClass.java
@@ -0,0 +1,15 @@
+package org.apache.drill.exec.test.generated;
+
+import org.apache.drill.exec.compile.JaninoClassCompiler;
+
+
+public class TestGeneratedClass {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestGeneratedClass.class);
+  public static void main(String[] args) throws Exception{
+    String s = "package org.apache.drill.exec.test.generated;\n\nimport org.apache.drill.exec.expr.holders.LongHolder;\nimport org.apache.drill.exec.ops.FragmentContext;\nimport org.apache.drill.exec.record.RecordBatch;\nimport org.apache.drill.exec.record.vector.NullableFixed8;\n\npublic class Test1 {\n\n    NullableFixed8 vv0;\n    NullableFixed8 vv5;\n    NullableFixed8 vv7;\n    NullableFixed8 vv12;\n\n    public void setupEvaluators(FragmentContext context, RecordBatch incoming, RecordBatch outgoing) {\n        {\n            {\n                Object obj1 = (outgoing).getValueVectorById(1, NullableFixed8 .class);\n                vv0 = ((NullableFixed8) obj1);\n                Object obj6 = (outgoing).getValueVectorById(0, NullableFixed8 .class);\n                vv5 = ((NullableFixed8) obj6);\n            }\n            {\n                Object obj8 = (outgoing).getValueVectorById(1, NullableFixed8 .class);\n                vv7 = ((NullableFixed8) obj8);\n                Obj
 ect obj13 = (outgoing).getValueVectorById(1, NullableFixed8 .class);\n                vv12 = ((NullableFixed8) obj13);\n            }\n        }\n    }\n\n    public void doPerRecordWork(int inIndex, int outIndex) {\n        {\n            {\n                LongHolder out2 = new LongHolder();\n                out2 .value = vv0 .get((inIndex));\n                LongHolder out3 = new LongHolder();\n                out3 .value = 1L;\n                LongHolder out4 = new LongHolder();\n                {\n                    final LongHolder out = new LongHolder();\n                    final LongHolder left = out2;\n                    final LongHolder right = out3;\n                        out.value = left.value + right.value;\n\n                    out4 = out;\n                }\n                vv5 .set((outIndex), out4 .value);\n            }\n            {\n                LongHolder out9 = new LongHolder();\n                out9 .value = vv7 .get((inIndex));\n                Long
 Holder out10 = new LongHolder();\n                out10 .value = 2L;\n                LongHolder out11 = new LongHolder();\n                {\n                    final LongHolder out = new LongHolder();\n                    final LongHolder left = out9;\n                    final LongHolder right = out10;\n                        out.value = left.value + right.value;\n\n                    out11 = out;\n                }\n                vv12 .set((outIndex), out11 .value);\n            }\n        }\n    }\n\n}";
+    JaninoClassCompiler cc = new JaninoClassCompiler(ClassLoader.getSystemClassLoader());
+    byte[] b = cc.getClassByteCode("Test12", s);
+  }
+  
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/80b1d24f/sandbox/prototype/exec/java-exec/src/test/resources/project/test1.json
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/resources/project/test1.json b/sandbox/prototype/exec/java-exec/src/test/resources/project/test1.json
new file mode 100644
index 0000000..70a5115
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/test/resources/project/test1.json
@@ -0,0 +1,37 @@
+{
+    head:{
+        type:"APACHE_DRILL_PHYSICAL",
+        version:"1",
+        generator:{
+            type:"manual"
+        }
+    },
+	graph:[
+        {
+            @id:1,
+            pop:"mock-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,
+            child: 1,
+            pop:"project",
+            exprs: [
+              { ref: "col1", expr:"red + 1" },
+              { ref: "col2", expr:"red + 2" }
+            ]
+        },
+        {
+            @id: 3,
+            child: 2,
+            pop: "screen"
+        }
+    ]
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/80b1d24f/sandbox/prototype/pom.xml
----------------------------------------------------------------------
diff --git a/sandbox/prototype/pom.xml b/sandbox/prototype/pom.xml
index a09d380..b7b9a40 100644
--- a/sandbox/prototype/pom.xml
+++ b/sandbox/prototype/pom.xml
@@ -105,6 +105,10 @@
     </plugins>
     <pluginManagement>
       <plugins>
+        <plugin>
+          <artifactId>maven-surefire-plugin</artifactId>
+          <version>2.15</version>
+        </plugin>
         <!--This plugin's configuration is used to store Eclipse m2e settings 
           only. It has no influence on the Maven build itself. -->
         <plugin>
@@ -127,6 +131,19 @@
                     <ignore></ignore>
                   </action>
                 </pluginExecution>
+                <pluginExecution>
+                  <pluginExecutionFilter>
+                    <groupId>com.github.igor-petruk.protobuf</groupId>
+                    <artifactId>protobuf-maven-plugin</artifactId>
+                    <versionRange>[0.6.3,)</versionRange>
+                    <goals>
+                      <goal>run</goal>
+                    </goals>
+                  </pluginExecutionFilter>
+                  <action>
+                    <ignore></ignore>
+                  </action>
+                </pluginExecution>
               </pluginExecutions>
             </lifecycleMappingMetadata>
           </configuration>


Re: [49/53] [abbrv] git commit: Add workspace field and function setup support. Add alternate function Add slightly better filter test.

Posted by Ted Dunning <te...@gmail.com>.
On Fri, Jul 19, 2013 at 6:58 PM, <ja...@apache.org> wrote:

> Add workspace field and function setup support.
> Add alternate function
> Add slightly better filter test.
>
>
> Project: http://git-wip-us.apache.org/repos/asf/incubator-drill/repo
> Commit:
> http://git-wip-us.apache.org/repos/asf/incubator-drill/commit/7779a723
> Tree: http://git-wip-us.apache.org/repos/asf/incubator-drill/tree/7779a723
> Diff: http://git-wip-us.apache.org/repos/asf/incubator-drill/diff/7779a723
>
> Branch: refs/heads/master
> Commit: 7779a7232d292f5a7f4f2838c04687360d19d90b
> Parents: 65e2cfc
> Author: Jacques Nadeau <ja...@apache.org>
> Authored: Wed Jul 17 20:06:00 2013 -0700
> Committer: Jacques Nadeau <ja...@apache.org>
> Committed: Fri Jul 19 14:53:31 2013 -0700
>
> ----------------------------------------------------------------------
>  .../common/expression/ArgumentValidators.java   |   8 +-
>  .../drill/common/expression/FunctionCall.java   |   4 +
>  .../common/expression/FunctionRegistry.java     |   5 +-
>  .../drill/common/expression/NoArgValidator.java |  21 ++++
>  .../apache/drill/exec/expr/CodeGenerator.java   |   2 +
>  .../drill/exec/expr/EvaluationVisitor.java      |   5 +-
>  .../drill/exec/expr/annotations/WorkSpace.java  |   2 +-
>  .../drill/exec/expr/fn/FunctionConverter.java   |  28 ++++--
>  .../drill/exec/expr/fn/FunctionHolder.java      | 100 +++++++++++++------
>  .../drill/exec/expr/fn/ImportGrabber.java       |  55 ++++++++++
>  .../exec/expr/fn/MethodGrabbingVisitor.java     |   4 +
>  .../drill/exec/expr/fn/impl/Alternator.java     |  49 +++++++++
>  .../apache/drill/exec/vector/ValueVector.java   |   1 -
>  .../src/main/resources/drill-module.conf        |   3 +
>  .../physical/impl/filter/TestSimpleFilter.java  |   3 +-
>  .../src/test/resources/filter/test1.json        |   2 +-
>  16 files changed, 239 insertions(+), 53 deletions(-)
> ----------------------------------------------------------------------
>
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/7779a723/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/ArgumentValidators.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/ArgumentValidators.java
> b/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/ArgumentValidators.java
> index f30733d..b745277 100644
> ---
> a/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/ArgumentValidators.java
> +++
> b/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/ArgumentValidators.java
> @@ -37,12 +37,12 @@ public class ArgumentValidators {
>
>      public AnyTypeAllowed(int argCount) {
>        super();
> -      argumentCount = Ranges.singleton(argCount);
> +      argumentCount = Range.singleton(argCount);
>      }
>
>      public AnyTypeAllowed(int minArguments, int maxArguments) {
>        super();
> -      argumentCount = Ranges.closedOpen(minArguments, maxArguments);
> +      argumentCount = Range.closedOpen(minArguments, maxArguments);
>      }
>
>      @Override
> @@ -66,14 +66,14 @@ public class ArgumentValidators {
>
>      public PredicateValidator(int argCount, Predicate<MajorType>
> predicate, boolean allSame) {
>        super();
> -      this.argumentCount = Ranges.singleton(argCount);
> +      this.argumentCount = Range.singleton(argCount);
>        this.predicate = predicate;
>        this.allSame = allSame;
>      }
>
>      public PredicateValidator(int minArguments, int maxArguments,
> Predicate<MajorType> predicate, boolean allSame) {
>        super();
> -      this.argumentCount = Ranges.closedOpen(minArguments, maxArguments);
> +      this.argumentCount = Range.closedOpen(minArguments, maxArguments);
>        this.predicate = predicate;
>        this.allSame = allSame;
>      }
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/7779a723/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/FunctionCall.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/FunctionCall.java
> b/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/FunctionCall.java
> index e13e87e..d27b584 100644
> ---
> a/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/FunctionCall.java
> +++
> b/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/FunctionCall.java
> @@ -24,6 +24,7 @@ import
> org.apache.drill.common.expression.visitors.ExprVisitor;
>  import org.apache.drill.common.types.TypeProtos.MajorType;
>
>  import com.google.common.collect.ImmutableList;
> +import com.google.common.collect.Lists;
>
>  public class FunctionCall extends LogicalExpressionBase implements
> Iterable<LogicalExpression> {
>    private final FunctionDefinition func;
> @@ -33,6 +34,9 @@ public class FunctionCall extends LogicalExpressionBase
> implements Iterable<Logi
>    public FunctionCall(FunctionDefinition func, List<LogicalExpression>
> args, ExpressionPosition pos) {
>      super(pos);
>      this.func = func;
> +
> +    if(args == null) args = Lists.newArrayList();
> +
>      if (!(args instanceof ImmutableList)) {
>        args = ImmutableList.copyOf(args);
>      }
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/7779a723/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/FunctionRegistry.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/FunctionRegistry.java
> b/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/FunctionRegistry.java
> index ed2c63e..28fa2db 100644
> ---
> a/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/FunctionRegistry.java
> +++
> b/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/FunctionRegistry.java
> @@ -32,7 +32,8 @@ import org.apache.drill.common.util.PathScanner;
>  import com.google.common.collect.Lists;
>
>  public class FunctionRegistry {
> -
> +  static final org.slf4j.Logger logger =
> org.slf4j.LoggerFactory.getLogger(FunctionRegistry.class);
> +
>    private final Map<String, FunctionDefinition> funcMap;
>
>    public FunctionRegistry(DrillConfig config){
> @@ -44,7 +45,9 @@ public class FunctionRegistry {
>          FunctionDefinition[] defs = p.getFunctionDefintions();
>          for(FunctionDefinition d : defs){
>            for(String rn : d.getRegisteredNames()){
> +
>              FunctionDefinition d2 = funcs.put(rn, d);
> +            logger.debug("Registering function {}", d);
>              if(d2 != null){
>                throw new
> ExceptionInInitializerError(String.format("Failure while registering
> functions.  The function %s tried to register with the name %s but the
> function %s already registered with that name.", d.getName(), rn,
> d2.getName()) );
>              }
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/7779a723/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/NoArgValidator.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/NoArgValidator.java
> b/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/NoArgValidator.java
> new file mode 100644
> index 0000000..2b2c46d
> --- /dev/null
> +++
> b/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/NoArgValidator.java
> @@ -0,0 +1,21 @@
> +package org.apache.drill.common.expression;
> +
> +import java.util.List;
> +
> +public class NoArgValidator implements ArgumentValidator{
> +  static final org.slf4j.Logger logger =
> org.slf4j.LoggerFactory.getLogger(NoArgValidator.class);
> +
> +  @Override
> +  public void validateArguments(ExpressionPosition expr,
> List<LogicalExpression> expressions, ErrorCollector errors) {
> +    if(!expressions.isEmpty()){
> +      errors.addGeneralError(expr, "Expected zero arguments, received
> more than that.");
> +    }
> +  }
> +
> +  @Override
> +  public String[] getArgumentNamesByPosition() {
> +    return new String[0];
> +  }
> +
> +  public static final NoArgValidator VALIDATOR = new NoArgValidator();
> +}
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/7779a723/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 241c1cc..6e31dec 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
> @@ -57,6 +57,8 @@ public class CodeGenerator<T> {
>        throw new IllegalStateException(e);
>      }
>    }
> +
> +
>
>    public void addExpr(LogicalExpression ex){
>      logger.debug("Adding next write {}", ex);
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/7779a723/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 c219d9c..c9e3c22 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
> @@ -42,7 +42,8 @@ public class EvaluationVisitor extends
> AbstractExprVisitor<HoldingContainer, Cod
>        args[i] = call.args.get(i).accept(this, generator);
>      }
>      FunctionHolder holder = registry.getFunction(call);
> -    return holder.generateEvalBody(generator, args);
> +
> +    return holder.renderFunction(generator, args);
>    }
>
>    @Override
> @@ -198,7 +199,7 @@ public class EvaluationVisitor extends
> AbstractExprVisitor<HoldingContainer, Cod
>
>    private HoldingContainer
> visitReturnValueExpression(ReturnValueExpression e, CodeGenerator<?>
> generator){
>      LogicalExpression child = e.getChild();
> -
>  Preconditions.checkArgument(child.getMajorType().equals(Types.REQUIRED_BOOLEAN));
> +//
>  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;
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/7779a723/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/annotations/WorkSpace.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/annotations/WorkSpace.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/annotations/WorkSpace.java
> index cc106a7..75c97b5 100644
> ---
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/annotations/WorkSpace.java
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/annotations/WorkSpace.java
> @@ -10,5 +10,5 @@ import java.lang.annotation.Target;
>   */
>  @Retention(RetentionPolicy.RUNTIME)
>  @Target({ElementType.FIELD})
> -public @interface WorkSpace {
> +public @interface Workspace {
>  }
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/7779a723/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 8e0f1be..3525cbb 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
> @@ -14,7 +14,7 @@ import org.apache.drill.exec.expr.DrillFunc;
>  import org.apache.drill.exec.expr.annotations.FunctionTemplate;
>  import org.apache.drill.exec.expr.annotations.Output;
>  import org.apache.drill.exec.expr.annotations.Param;
> -import org.apache.drill.exec.expr.annotations.WorkSpace;
> +import org.apache.drill.exec.expr.annotations.Workspace;
>  import org.apache.drill.exec.expr.fn.FunctionHolder.ValueReference;
>  import org.apache.drill.exec.expr.fn.FunctionHolder.WorkspaceReference;
>  import org.apache.drill.exec.expr.holders.ValueHolder;
> @@ -53,16 +53,16 @@ public class FunctionConverter {
>
>        Param param = field.getAnnotation(Param.class);
>        Output output = field.getAnnotation(Output.class);
> -      WorkSpace workspace = field.getAnnotation(WorkSpace.class);
> +      Workspace workspace = field.getAnnotation(Workspace.class);
>
>        int i =0;
>        if(param != null) i++;
>        if(output != null) i++;
>        if(workspace != null) i++;
>        if(i == 0){
> -        return failure("The field must be either a @Param, @Output or
> @WorkSpace field.", clazz, field);
> +        return failure("The field must be either a @Param, @Output or
> @Workspace field.", clazz, field);
>        }else if(i > 1){
> -        return failure("The field must be only one of @Param, @Output or
> @WorkSpace.  It currently has more than one of these annotations.", clazz,
> field);
> +        return failure("The field must be only one of @Param, @Output or
> @Workspace.  It currently has more than one of these annotations.", clazz,
> field);
>        }
>
>
> @@ -98,13 +98,14 @@ public class FunctionConverter {
>
>        }else{
>          // workspace work.
> +        logger.debug("Found workspace field {}:{}", field.getType(),
> field.getName());
>          workspaceFields.add(new WorkspaceReference(field.getType(),
> field.getName()));
>        }
>
>      }
>
>
> -    if(!workspaceFields.isEmpty()) return failure("This function declares
> one or more workspace fields.  However, those have not yet been
> implemented.", clazz);
> +   // if(!workspaceFields.isEmpty()) return failure("This function
> declares one or more workspace fields.  However, those have not yet been
> implemented.", clazz);
>      if(outputField == null)  return failure("This function declares zero
> output fields.  A function must declare one output field.", clazz);
>
>      // get function body.
> @@ -117,12 +118,21 @@ public class FunctionConverter {
>      }
>
>      Map<String, String> methods = MethodGrabbingVisitor.getMethods(cu,
> clazz);
> -
> +    List<String> imports = ImportGrabber.getMethods(cu);
>      // return holder
>      ValueReference[] ps = params.toArray(new
> ValueReference[params.size()]);
>      WorkspaceReference[] works = workspaceFields.toArray(new
> WorkspaceReference[workspaceFields.size()]);
> -    FunctionHolder fh = new FunctionHolder(template.scope(),
> template.nulls(), template.isBinaryCommutative(), template.name(), ps,
> outputField, works, methods);
> -    return fh;
> +    if(!methods.containsKey("eval")){
> +      return failure("Failure finding eval method for function.", clazz);
> +    }
> +
> +    try{
> +      FunctionHolder fh = new FunctionHolder(template.scope(),
> template.nulls(), template.isBinaryCommutative(), template.name(), ps,
> outputField, works, methods, imports);
> +      return fh;
> +    }catch(Exception ex){
> +      return failure("Failure while creating function holder.", ex,
> clazz);
> +    }
> +
>    }
>
>
> @@ -141,7 +151,7 @@ public class FunctionConverter {
>        String body = IO.toString(is);
>
>        //TODO: Hack to remove annotations so Janino doesn't choke.  Need
> to reconsider this problem...
> -      body =
> body.replaceAll("@(?:Output|Param|FunctionTemplate\\([^\\\\]*?\\))", "");
> +      body =
> body.replaceAll("@(?:Output|Param|Workspace|Override|FunctionTemplate\\([^\\\\]*?\\))",
> "");
>        return new Parser(new Scanner(null, new
> StringReader(body))).parseCompilationUnit();
>      }
>
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/7779a723/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 9d7fc35..d249f4d 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,6 +1,7 @@
>  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;
> @@ -14,6 +15,8 @@ import
> org.apache.drill.exec.expr.annotations.FunctionTemplate;
>  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;
> @@ -32,11 +35,12 @@ public class FunctionHolder {
>    private String evalBody;
>    private String addBody;
>    private String setupBody;
> +  private List<String> imports;
>    private WorkspaceReference[] workspaceVars;
>    private ValueReference[] parameters;
>    private ValueReference returnValue;
>
> -  public FunctionHolder(FunctionScope scope, NullHandling nullHandling,
> boolean isBinaryCommutative, String functionName, ValueReference[]
> parameters, ValueReference returnValue, WorkspaceReference[] workspaceVars,
> Map<String, String> methods) {
> +  public FunctionHolder(FunctionScope scope, NullHandling nullHandling,
> boolean isBinaryCommutative, String functionName, ValueReference[]
> parameters, ValueReference returnValue, WorkspaceReference[] workspaceVars,
> Map<String, String> methods, List<String> imports) {
>      super();
>      this.scope = scope;
>      this.nullHandling = nullHandling;
> @@ -46,17 +50,66 @@ public class FunctionHolder {
>      this.setupBody = methods.get("setup");
>      this.addBody = methods.get("add");
>      this.evalBody = methods.get("eval");
> +    Preconditions.checkNotNull(evalBody);
>      this.parameters = parameters;
>      this.returnValue = returnValue;
> +    this.imports = imports;
> +  }
> +
> +  public List<String> getImports() {
> +    return imports;
>    }
>
> -  public HoldingContainer generateEvalBody(CodeGenerator<?> g,
> HoldingContainer[] inputVariables){
> +  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 void addProtectedBlock(CodeGenerator<?> g, JBlock sub, String
> body, HoldingContainer[] inputVariables){
>
> -    //g.getBlock().directStatement(String.format("//---- start of eval
> portion of %s function. ----//", functionName));
> +    // create sub block with defined workspace variables.
> +    JVar[] workspaceJVars = new JVar[workspaceVars.length];
> +    for(int i =0 ; i < workspaceVars.length; i++){
> +      workspaceJVars[i] = g.declareClassField("work",
> g.getModel()._ref(workspaceVars[i].type));
> +    }
> +
> +    if(inputVariables != null){
> +      for(int i =0; i < inputVariables.length; i++){
> +        ValueReference parameter = parameters[i];
> +        HoldingContainer inputVariable = inputVariables[i];
> +        sub.decl(JMod.FINAL, inputVariable.getHolder().type(),
> parameter.name, inputVariable.getHolder());
> +      }
> +    }
> +
> +    JVar[] internalVars = new JVar[workspaceJVars.length];
> +    for(int i =0; i < workspaceJVars.length; i++){
> +      internalVars[i] = sub.decl(JMod.FINAL,
> g.getModel()._ref(workspaceVars[i].type),  workspaceVars[i].name,
> workspaceJVars[i]);
> +    }
>
> -    JBlock sub = new JBlock(true, true);
> +    Preconditions.checkNotNull(body);
> +    sub.directStatement(body);
> +
> +    // reassign workspace variables back to global space.
> +    for(int i =0; i < workspaceJVars.length; i++){
> +      sub.assign(workspaceJVars[i], internalVars[i]);
> +    }
> +  }
> +
> +  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);
>
>      HoldingContainer out = null;
>
> @@ -89,40 +142,16 @@ public class FunctionHolder {
>      // add the subblock after the out declaration.
>      g.getBlock().add(sub);
>
> -    JVar[] workspaceJVars = new JVar[workspaceVars.length];
> -    for(int i =0 ; i < workspaceVars.length; i++){
> -      workspaceJVars[i] = g.declareClassField("work",
> g.getModel()._ref(workspaceVars[i].type));
> -    }
>
> -//    for(WorkspaceReference r : workspaceVars){
> -//      g.declareClassField(, t)
> -//    }
> -//
> -//    g.declareClassField(prefix, t)
> -
> -
> -    // locally name external blocks.
> -
> -    // internal out value.
>      JVar internalOutput = sub.decl(JMod.FINAL,
> g.getHolderType(returnValue.type), returnValue.name,
> JExpr._new(g.getHolderType(returnValue.type)));
> -
> -    for(int i =0; i < inputVariables.length; i++){
> -
> -      ValueReference parameter = parameters[i];
> -      HoldingContainer inputVariable = inputVariables[i];
> -      sub.decl(JMod.FINAL, inputVariable.getHolder().type(),
> parameter.name, inputVariable.getHolder());
> -    }
> -
> -
> -    // add function body.
> -    sub.directStatement(evalBody);
> -
> +    addProtectedBlock(g, sub, evalBody, inputVariables);
>      sub.assign(out.getHolder(), internalOutput);
>
> -    //g.getBlock().directStatement(String.format("//---- end of eval
> portion of %s function. ----//\n", functionName));
>      return out;
>    }
>
> +
> +
>    public boolean matches(FunctionCall call){
>      if(!softCompare(call.getMajorType(), returnValue.type)) return false;
>      if(call.args.size() != parameters.length) return false;
> @@ -148,6 +177,8 @@ public class FunctionHolder {
>      String name;
>      public ValueReference(MajorType type, String name) {
>        super();
> +      Preconditions.checkNotNull(type);
> +      Preconditions.checkNotNull(name);
>        this.type = type;
>        this.name = name;
>      }
> @@ -155,15 +186,18 @@ public class FunctionHolder {
>      public String toString() {
>        return "ValueReference [type=" + type + ", name=" + name + "]";
>      }
> -
> -
>    }
>
> +
>    public static class WorkspaceReference{
>      Class<?> type;
>      String name;
> +
> +
>      public WorkspaceReference(Class<?> type, String name) {
>        super();
> +      Preconditions.checkNotNull(type);
> +      Preconditions.checkNotNull(name);
>        this.type = type;
>        this.name = name;
>      }
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/7779a723/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/ImportGrabber.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/ImportGrabber.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/ImportGrabber.java
> new file mode 100644
> index 0000000..8fc73c2
> --- /dev/null
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/ImportGrabber.java
> @@ -0,0 +1,55 @@
> +package org.apache.drill.exec.expr.fn;
> +
> +import java.util.List;
> +
> +import org.codehaus.janino.Java;
> +import
> org.codehaus.janino.Java.CompilationUnit.SingleStaticImportDeclaration;
> +import
> org.codehaus.janino.Java.CompilationUnit.SingleTypeImportDeclaration;
> +import
> org.codehaus.janino.Java.CompilationUnit.StaticImportOnDemandDeclaration;
> +import
> org.codehaus.janino.Java.CompilationUnit.TypeImportOnDemandDeclaration;
> +import org.codehaus.janino.util.Traverser;
> +
> +import com.google.common.collect.Lists;
> +
> +
> +public class ImportGrabber{
> +  static final org.slf4j.Logger logger =
> org.slf4j.LoggerFactory.getLogger(ImportGrabber.class);
> +
> +  private List<String> imports = Lists.newArrayList();
> +  private final ImportFinder finder = new ImportFinder();
> +
> +  private ImportGrabber() {
> +  }
> +
> +  public class ImportFinder extends Traverser{
> +
> +    @Override
> +    public void
> traverseSingleTypeImportDeclaration(SingleTypeImportDeclaration stid) {
> +      imports.add(stid.toString());
> +    }
> +
> +    @Override
> +    public void
> traverseSingleStaticImportDeclaration(SingleStaticImportDeclaration stid) {
> +      imports.add(stid.toString());
> +    }
> +
> +    @Override
> +    public void
> traverseTypeImportOnDemandDeclaration(TypeImportOnDemandDeclaration tiodd) {
> +      imports.add(tiodd.toString());
> +    }
> +
> +    @Override
> +    public void
> traverseStaticImportOnDemandDeclaration(StaticImportOnDemandDeclaration
> siodd) {
> +      imports.add(siodd.toString());
> +    }
> +
> +
> +  }
> +
> +  public static List<String> getMethods(Java.CompilationUnit cu){
> +    ImportGrabber visitor = new ImportGrabber();
> +
>  cu.getPackageMemberTypeDeclarations()[0].accept(visitor.finder.comprehensiveVisitor());
> +    return visitor.imports;
> +  }
> +
> +}
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/7779a723/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/MethodGrabbingVisitor.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/MethodGrabbingVisitor.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/MethodGrabbingVisitor.java
> index 57268ee..d46d008 100644
> ---
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/MethodGrabbingVisitor.java
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/MethodGrabbingVisitor.java
> @@ -28,12 +28,16 @@ public class MethodGrabbingVisitor{
>
>      @Override
>      public void traverseClassDeclaration(ClassDeclaration cd) {
> +      logger.debug("Traversing: {}", cd.getClassName());
> +      boolean prevCapture = captureMethods;
>        captureMethods = c.getName().equals(cd.getClassName());
>        super.traverseClassDeclaration(cd);
> +      captureMethods = prevCapture;
>      }
>
>      @Override
>      public void traverseMethodDeclarator(MethodDeclarator md) {
> +      logger.debug(c.getName() + ": Found {}, include {}", md.name,
> captureMethods);
>        if(captureMethods){
>          StringWriter writer = new StringWriter();
>          ModifiedUnparseVisitor v = new ModifiedUnparseVisitor(writer);
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/7779a723/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/Alternator.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/Alternator.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/Alternator.java
> new file mode 100644
> index 0000000..83f8368
> --- /dev/null
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/Alternator.java
> @@ -0,0 +1,49 @@
> +package org.apache.drill.exec.expr.fn.impl;
> +
> +import org.apache.drill.common.expression.CallProvider;
> +import org.apache.drill.common.expression.FunctionDefinition;
> +import org.apache.drill.common.expression.NoArgValidator;
> +import org.apache.drill.common.expression.OutputTypeDeterminer;
> +import org.apache.drill.common.types.TypeProtos.MinorType;
> +import org.apache.drill.common.types.Types;
> +import org.apache.drill.exec.expr.DrillFunc;
> +import org.apache.drill.exec.expr.annotations.FunctionTemplate;
> +import
> org.apache.drill.exec.expr.annotations.FunctionTemplate.FunctionScope;
> +import org.apache.drill.exec.expr.annotations.Output;
> +import org.apache.drill.exec.expr.annotations.Workspace;
> +import org.apache.drill.exec.expr.holders.LongHolder;
> +import org.apache.drill.exec.record.RecordBatch;
> +
> +
> +
> +@FunctionTemplate(name = "alternate", scope = FunctionScope.SIMPLE)
> +public class Alternator implements DrillFunc{
> +
> +  @Workspace int val;
> +  @Output LongHolder out;
> +
> +  public void setup(RecordBatch incoming) {
> +    val = 0;
> +  }
> +
> +
> +  public void eval() {
> +    out.value = val;
> +    if(val == 0){
> +      val = 1;
> +    }else{
> +      val = 0;
> +    }
> +  }
> +
> +  public static class Provider implements CallProvider{
> +
> +    @Override
> +    public FunctionDefinition[] getFunctionDefintions() {
> +      return new FunctionDefinition[]{
> +          FunctionDefinition.simple("alternate",
> NoArgValidator.VALIDATOR, new
> OutputTypeDeterminer.FixedType(Types.required(MinorType.BIGINT)),
> "alternate")
> +      };
> +    }
> +
> +  }
> +}
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/7779a723/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/ValueVector.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/ValueVector.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/ValueVector.java
> index 27089ac..b188d5b 100644
> ---
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/ValueVector.java
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/ValueVector.java
> @@ -116,7 +116,6 @@ public interface ValueVector extends Closeable {
>       * @param index   Index of the value to get
>       */
>      public abstract Object getObject(int index);
> -
>      public void reset();
>    }
>
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/7779a723/sandbox/prototype/exec/java-exec/src/main/resources/drill-module.conf
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/resources/drill-module.conf
> b/sandbox/prototype/exec/java-exec/src/main/resources/drill-module.conf
> index 3ce903d..a543197 100644
> --- a/sandbox/prototype/exec/java-exec/src/main/resources/drill-module.conf
> +++ b/sandbox/prototype/exec/java-exec/src/main/resources/drill-module.conf
> @@ -1,6 +1,9 @@
>  //  This file tells Drill to consider this module when class path
> scanning.
>  //  This file can also include any supplementary configuration
> information.
>  //  This file is in HOCON format, see
> https://github.com/typesafehub/config/blob/master/HOCON.md for more
> information.
> +
> +drill.logical.function.packages += "org.apache.drill.exec.expr.fn.impl"
> +
>  drill.exec: {
>    cluster-id: "drillbits1"
>    rpc: {
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/7779a723/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 df11aa7..e2bf25b 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
> @@ -1,5 +1,6 @@
>  package org.apache.drill.exec.physical.impl.filter;
>
> +import static org.junit.Assert.*;
>  import mockit.Injectable;
>  import mockit.NonStrictExpectations;
>
> @@ -46,7 +47,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()){
> -      System.out.println(exec.getSelectionVector2().getCount());
> +      assertEquals(50, exec.getSelectionVector2().getCount());
>      }
>    }
>
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/7779a723/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 a892c70..c9b367f 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
> @@ -23,7 +23,7 @@
>              @id:2,
>              child: 1,
>              pop:"filter",
> -            expr: "true"
> +            expr: "alternate()"
>          },
>          {
>              @id: 3,
>
>

[49/53] [abbrv] git commit: Add workspace field and function setup support. Add alternate function Add slightly better filter test.

Posted by ja...@apache.org.
Add workspace field and function setup support.
Add alternate function
Add slightly better filter test.


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

Branch: refs/heads/master
Commit: 7779a7232d292f5a7f4f2838c04687360d19d90b
Parents: 65e2cfc
Author: Jacques Nadeau <ja...@apache.org>
Authored: Wed Jul 17 20:06:00 2013 -0700
Committer: Jacques Nadeau <ja...@apache.org>
Committed: Fri Jul 19 14:53:31 2013 -0700

----------------------------------------------------------------------
 .../common/expression/ArgumentValidators.java   |   8 +-
 .../drill/common/expression/FunctionCall.java   |   4 +
 .../common/expression/FunctionRegistry.java     |   5 +-
 .../drill/common/expression/NoArgValidator.java |  21 ++++
 .../apache/drill/exec/expr/CodeGenerator.java   |   2 +
 .../drill/exec/expr/EvaluationVisitor.java      |   5 +-
 .../drill/exec/expr/annotations/WorkSpace.java  |   2 +-
 .../drill/exec/expr/fn/FunctionConverter.java   |  28 ++++--
 .../drill/exec/expr/fn/FunctionHolder.java      | 100 +++++++++++++------
 .../drill/exec/expr/fn/ImportGrabber.java       |  55 ++++++++++
 .../exec/expr/fn/MethodGrabbingVisitor.java     |   4 +
 .../drill/exec/expr/fn/impl/Alternator.java     |  49 +++++++++
 .../apache/drill/exec/vector/ValueVector.java   |   1 -
 .../src/main/resources/drill-module.conf        |   3 +
 .../physical/impl/filter/TestSimpleFilter.java  |   3 +-
 .../src/test/resources/filter/test1.json        |   2 +-
 16 files changed, 239 insertions(+), 53 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/7779a723/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/ArgumentValidators.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/ArgumentValidators.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/ArgumentValidators.java
index f30733d..b745277 100644
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/ArgumentValidators.java
+++ b/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/ArgumentValidators.java
@@ -37,12 +37,12 @@ public class ArgumentValidators {
 
     public AnyTypeAllowed(int argCount) {
       super();
-      argumentCount = Ranges.singleton(argCount);
+      argumentCount = Range.singleton(argCount);
     }
 
     public AnyTypeAllowed(int minArguments, int maxArguments) {
       super();
-      argumentCount = Ranges.closedOpen(minArguments, maxArguments);
+      argumentCount = Range.closedOpen(minArguments, maxArguments);
     }
 
     @Override
@@ -66,14 +66,14 @@ public class ArgumentValidators {
 
     public PredicateValidator(int argCount, Predicate<MajorType> predicate, boolean allSame) {
       super();
-      this.argumentCount = Ranges.singleton(argCount);
+      this.argumentCount = Range.singleton(argCount);
       this.predicate = predicate;
       this.allSame = allSame;
     }
 
     public PredicateValidator(int minArguments, int maxArguments, Predicate<MajorType> predicate, boolean allSame) {
       super();
-      this.argumentCount = Ranges.closedOpen(minArguments, maxArguments);
+      this.argumentCount = Range.closedOpen(minArguments, maxArguments);
       this.predicate = predicate;
       this.allSame = allSame;
     }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/7779a723/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/FunctionCall.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/FunctionCall.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/FunctionCall.java
index e13e87e..d27b584 100644
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/FunctionCall.java
+++ b/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/FunctionCall.java
@@ -24,6 +24,7 @@ import org.apache.drill.common.expression.visitors.ExprVisitor;
 import org.apache.drill.common.types.TypeProtos.MajorType;
 
 import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Lists;
 
 public class FunctionCall extends LogicalExpressionBase implements Iterable<LogicalExpression> {
   private final FunctionDefinition func;
@@ -33,6 +34,9 @@ public class FunctionCall extends LogicalExpressionBase implements Iterable<Logi
   public FunctionCall(FunctionDefinition func, List<LogicalExpression> args, ExpressionPosition pos) {
     super(pos);
     this.func = func;
+    
+    if(args == null) args = Lists.newArrayList();
+    
     if (!(args instanceof ImmutableList)) {
       args = ImmutableList.copyOf(args);
     }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/7779a723/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/FunctionRegistry.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/FunctionRegistry.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/FunctionRegistry.java
index ed2c63e..28fa2db 100644
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/FunctionRegistry.java
+++ b/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/FunctionRegistry.java
@@ -32,7 +32,8 @@ import org.apache.drill.common.util.PathScanner;
 import com.google.common.collect.Lists;
 
 public class FunctionRegistry {
-
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(FunctionRegistry.class);
+  
   private final Map<String, FunctionDefinition> funcMap;
 
   public FunctionRegistry(DrillConfig config){
@@ -44,7 +45,9 @@ public class FunctionRegistry {
         FunctionDefinition[] defs = p.getFunctionDefintions();
         for(FunctionDefinition d : defs){
           for(String rn : d.getRegisteredNames()){
+            
             FunctionDefinition d2 = funcs.put(rn, d);
+            logger.debug("Registering function {}", d);
             if(d2 != null){
               throw new ExceptionInInitializerError(String.format("Failure while registering functions.  The function %s tried to register with the name %s but the function %s already registered with that name.", d.getName(), rn, d2.getName()) );
             }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/7779a723/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/NoArgValidator.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/NoArgValidator.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/NoArgValidator.java
new file mode 100644
index 0000000..2b2c46d
--- /dev/null
+++ b/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/NoArgValidator.java
@@ -0,0 +1,21 @@
+package org.apache.drill.common.expression;
+
+import java.util.List;
+
+public class NoArgValidator implements ArgumentValidator{
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(NoArgValidator.class);
+
+  @Override
+  public void validateArguments(ExpressionPosition expr, List<LogicalExpression> expressions, ErrorCollector errors) {
+    if(!expressions.isEmpty()){
+      errors.addGeneralError(expr, "Expected zero arguments, received more than that.");
+    }
+  }
+
+  @Override
+  public String[] getArgumentNamesByPosition() {
+    return new String[0];
+  }
+  
+  public static final NoArgValidator VALIDATOR = new NoArgValidator();
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/7779a723/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 241c1cc..6e31dec 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
@@ -57,6 +57,8 @@ public class CodeGenerator<T> {
       throw new IllegalStateException(e);
     }
   }
+  
+
 
   public void addExpr(LogicalExpression ex){
     logger.debug("Adding next write {}", ex);

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/7779a723/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 c219d9c..c9e3c22 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
@@ -42,7 +42,8 @@ public class EvaluationVisitor extends AbstractExprVisitor<HoldingContainer, Cod
       args[i] = call.args.get(i).accept(this, generator);
     }
     FunctionHolder holder = registry.getFunction(call);
-    return holder.generateEvalBody(generator, args);
+    
+    return holder.renderFunction(generator, args);
   }
   
   @Override
@@ -198,7 +199,7 @@ public class EvaluationVisitor extends AbstractExprVisitor<HoldingContainer, Cod
   
   private HoldingContainer visitReturnValueExpression(ReturnValueExpression e, CodeGenerator<?> generator){
     LogicalExpression child = e.getChild();
-    Preconditions.checkArgument(child.getMajorType().equals(Types.REQUIRED_BOOLEAN));
+//    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;

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/7779a723/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/annotations/WorkSpace.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/annotations/WorkSpace.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/annotations/WorkSpace.java
index cc106a7..75c97b5 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/annotations/WorkSpace.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/annotations/WorkSpace.java
@@ -10,5 +10,5 @@ import java.lang.annotation.Target;
  */
 @Retention(RetentionPolicy.RUNTIME)
 @Target({ElementType.FIELD})
-public @interface WorkSpace {
+public @interface Workspace {
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/7779a723/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 8e0f1be..3525cbb 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
@@ -14,7 +14,7 @@ import org.apache.drill.exec.expr.DrillFunc;
 import org.apache.drill.exec.expr.annotations.FunctionTemplate;
 import org.apache.drill.exec.expr.annotations.Output;
 import org.apache.drill.exec.expr.annotations.Param;
-import org.apache.drill.exec.expr.annotations.WorkSpace;
+import org.apache.drill.exec.expr.annotations.Workspace;
 import org.apache.drill.exec.expr.fn.FunctionHolder.ValueReference;
 import org.apache.drill.exec.expr.fn.FunctionHolder.WorkspaceReference;
 import org.apache.drill.exec.expr.holders.ValueHolder;
@@ -53,16 +53,16 @@ public class FunctionConverter {
 
       Param param = field.getAnnotation(Param.class);
       Output output = field.getAnnotation(Output.class);
-      WorkSpace workspace = field.getAnnotation(WorkSpace.class);
+      Workspace workspace = field.getAnnotation(Workspace.class);
       
       int i =0;
       if(param != null) i++;
       if(output != null) i++;
       if(workspace != null) i++;
       if(i == 0){
-        return failure("The field must be either a @Param, @Output or @WorkSpace field.", clazz, field);
+        return failure("The field must be either a @Param, @Output or @Workspace field.", clazz, field);
       }else if(i > 1){
-        return failure("The field must be only one of @Param, @Output or @WorkSpace.  It currently has more than one of these annotations.", clazz, field);
+        return failure("The field must be only one of @Param, @Output or @Workspace.  It currently has more than one of these annotations.", clazz, field);
       }
 
       
@@ -98,13 +98,14 @@ public class FunctionConverter {
         
       }else{
         // workspace work.
+        logger.debug("Found workspace field {}:{}", field.getType(), field.getName());
         workspaceFields.add(new WorkspaceReference(field.getType(), field.getName()));
       }
       
     }
     
     
-    if(!workspaceFields.isEmpty()) return failure("This function declares one or more workspace fields.  However, those have not yet been implemented.", clazz);
+   // if(!workspaceFields.isEmpty()) return failure("This function declares one or more workspace fields.  However, those have not yet been implemented.", clazz);
     if(outputField == null)  return failure("This function declares zero output fields.  A function must declare one output field.", clazz);
     
     // get function body.     
@@ -117,12 +118,21 @@ public class FunctionConverter {
     }
     
     Map<String, String> methods = MethodGrabbingVisitor.getMethods(cu, clazz);
-
+    List<String> imports = ImportGrabber.getMethods(cu);
     // return holder
     ValueReference[] ps = params.toArray(new ValueReference[params.size()]);
     WorkspaceReference[] works = workspaceFields.toArray(new WorkspaceReference[workspaceFields.size()]);
-    FunctionHolder fh = new FunctionHolder(template.scope(), template.nulls(), template.isBinaryCommutative(), template.name(), ps, outputField, works, methods);
-    return fh;
+    if(!methods.containsKey("eval")){
+      return failure("Failure finding eval method for function.", clazz);
+    }
+    
+    try{
+      FunctionHolder fh = new FunctionHolder(template.scope(), template.nulls(), template.isBinaryCommutative(), template.name(), ps, outputField, works, methods, imports);
+      return fh;
+    }catch(Exception ex){
+      return failure("Failure while creating function holder.", ex, clazz);
+    }
+    
   }
   
   
@@ -141,7 +151,7 @@ public class FunctionConverter {
       String body = IO.toString(is);
       
       //TODO: Hack to remove annotations so Janino doesn't choke.  Need to reconsider this problem...
-      body = body.replaceAll("@(?:Output|Param|FunctionTemplate\\([^\\\\]*?\\))", "");
+      body = body.replaceAll("@(?:Output|Param|Workspace|Override|FunctionTemplate\\([^\\\\]*?\\))", "");
       return new Parser(new Scanner(null, new StringReader(body))).parseCompilationUnit();
     }
     

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/7779a723/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 9d7fc35..d249f4d 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,6 +1,7 @@
 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;
@@ -14,6 +15,8 @@ import org.apache.drill.exec.expr.annotations.FunctionTemplate;
 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;
@@ -32,11 +35,12 @@ public class FunctionHolder {
   private String evalBody;
   private String addBody;
   private String setupBody;
+  private List<String> imports;
   private WorkspaceReference[] workspaceVars;
   private ValueReference[] parameters;
   private ValueReference returnValue;
   
-  public FunctionHolder(FunctionScope scope, NullHandling nullHandling, boolean isBinaryCommutative, String functionName, ValueReference[] parameters, ValueReference returnValue, WorkspaceReference[] workspaceVars, Map<String, String> methods) {
+  public FunctionHolder(FunctionScope scope, NullHandling nullHandling, boolean isBinaryCommutative, String functionName, ValueReference[] parameters, ValueReference returnValue, WorkspaceReference[] workspaceVars, Map<String, String> methods, List<String> imports) {
     super();
     this.scope = scope;
     this.nullHandling = nullHandling;
@@ -46,17 +50,66 @@ public class FunctionHolder {
     this.setupBody = methods.get("setup");
     this.addBody = methods.get("add");
     this.evalBody = methods.get("eval");
+    Preconditions.checkNotNull(evalBody);
     this.parameters = parameters;
     this.returnValue = returnValue;
+    this.imports = imports;
+  }
+  
+  public List<String> getImports() {
+    return imports;
   }
 
-  public HoldingContainer generateEvalBody(CodeGenerator<?> g, HoldingContainer[] inputVariables){
+  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 void addProtectedBlock(CodeGenerator<?> g, JBlock sub, String body, HoldingContainer[] inputVariables){
     
-    //g.getBlock().directStatement(String.format("//---- start of eval portion of %s function. ----//", functionName));
+    // create sub block with defined workspace variables.
+    JVar[] workspaceJVars = new JVar[workspaceVars.length];
+    for(int i =0 ; i < workspaceVars.length; i++){
+      workspaceJVars[i] = g.declareClassField("work", g.getModel()._ref(workspaceVars[i].type));
+    }
+
+    if(inputVariables != null){
+      for(int i =0; i < inputVariables.length; i++){
+        ValueReference parameter = parameters[i];
+        HoldingContainer inputVariable = inputVariables[i];
+        sub.decl(JMod.FINAL, inputVariable.getHolder().type(), parameter.name, inputVariable.getHolder());  
+      }
+    }
+
+    JVar[] internalVars = new JVar[workspaceJVars.length];
+    for(int i =0; i < workspaceJVars.length; i++){
+      internalVars[i] = sub.decl(JMod.FINAL, g.getModel()._ref(workspaceVars[i].type),  workspaceVars[i].name, workspaceJVars[i]);
+    }
     
-    JBlock sub = new JBlock(true, true);
+    Preconditions.checkNotNull(body);
+    sub.directStatement(body);
+    
+    // reassign workspace variables back to global space.
+    for(int i =0; i < workspaceJVars.length; i++){
+      sub.assign(workspaceJVars[i], internalVars[i]);
+    }
+  }
+
+  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);
     
     HoldingContainer out = null;
 
@@ -89,40 +142,16 @@ public class FunctionHolder {
     // add the subblock after the out declaration.
     g.getBlock().add(sub);
     
-    JVar[] workspaceJVars = new JVar[workspaceVars.length];
-    for(int i =0 ; i < workspaceVars.length; i++){
-      workspaceJVars[i] = g.declareClassField("work", g.getModel()._ref(workspaceVars[i].type));
-    }
     
-//    for(WorkspaceReference r : workspaceVars){
-//      g.declareClassField(, t)
-//    }
-//  
-//    g.declareClassField(prefix, t)
-    
-    
-    // locally name external blocks.
-    
-    // internal out value.
     JVar internalOutput = sub.decl(JMod.FINAL, g.getHolderType(returnValue.type), returnValue.name, JExpr._new(g.getHolderType(returnValue.type)));
-    
-    for(int i =0; i < inputVariables.length; i++){
-      
-      ValueReference parameter = parameters[i];
-      HoldingContainer inputVariable = inputVariables[i];
-      sub.decl(JMod.FINAL, inputVariable.getHolder().type(), parameter.name, inputVariable.getHolder());  
-    }
-    
-    
-    // add function body.
-    sub.directStatement(evalBody);
-    
+    addProtectedBlock(g, sub, evalBody, inputVariables);
     sub.assign(out.getHolder(), internalOutput);
 
-    //g.getBlock().directStatement(String.format("//---- end of eval portion of %s function. ----//\n", functionName));
     return out;
   }
   
+  
+  
   public boolean matches(FunctionCall call){
     if(!softCompare(call.getMajorType(), returnValue.type)) return false;
     if(call.args.size() != parameters.length) return false;
@@ -148,6 +177,8 @@ public class FunctionHolder {
     String name;
     public ValueReference(MajorType type, String name) {
       super();
+      Preconditions.checkNotNull(type);
+      Preconditions.checkNotNull(name);
       this.type = type;
       this.name = name;
     }
@@ -155,15 +186,18 @@ public class FunctionHolder {
     public String toString() {
       return "ValueReference [type=" + type + ", name=" + name + "]";
     }
-    
-    
   }
 
+  
   public static class WorkspaceReference{
     Class<?> type;
     String name;
+
+
     public WorkspaceReference(Class<?> type, String name) {
       super();
+      Preconditions.checkNotNull(type);
+      Preconditions.checkNotNull(name);
       this.type = type;
       this.name = name;
     }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/7779a723/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/ImportGrabber.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/ImportGrabber.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/ImportGrabber.java
new file mode 100644
index 0000000..8fc73c2
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/ImportGrabber.java
@@ -0,0 +1,55 @@
+package org.apache.drill.exec.expr.fn;
+
+import java.util.List;
+
+import org.codehaus.janino.Java;
+import org.codehaus.janino.Java.CompilationUnit.SingleStaticImportDeclaration;
+import org.codehaus.janino.Java.CompilationUnit.SingleTypeImportDeclaration;
+import org.codehaus.janino.Java.CompilationUnit.StaticImportOnDemandDeclaration;
+import org.codehaus.janino.Java.CompilationUnit.TypeImportOnDemandDeclaration;
+import org.codehaus.janino.util.Traverser;
+
+import com.google.common.collect.Lists;
+
+
+public class ImportGrabber{
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ImportGrabber.class);
+  
+  private List<String> imports = Lists.newArrayList();
+  private final ImportFinder finder = new ImportFinder();
+
+  private ImportGrabber() {
+  }
+  
+  public class ImportFinder extends Traverser{
+
+    @Override
+    public void traverseSingleTypeImportDeclaration(SingleTypeImportDeclaration stid) {
+      imports.add(stid.toString());
+    }
+
+    @Override
+    public void traverseSingleStaticImportDeclaration(SingleStaticImportDeclaration stid) {
+      imports.add(stid.toString());
+    }
+
+    @Override
+    public void traverseTypeImportOnDemandDeclaration(TypeImportOnDemandDeclaration tiodd) {
+      imports.add(tiodd.toString());
+    }
+
+    @Override
+    public void traverseStaticImportOnDemandDeclaration(StaticImportOnDemandDeclaration siodd) {
+      imports.add(siodd.toString());
+    }
+
+    
+  }
+  
+  public static List<String> getMethods(Java.CompilationUnit cu){
+    ImportGrabber visitor = new ImportGrabber();
+    cu.getPackageMemberTypeDeclarations()[0].accept(visitor.finder.comprehensiveVisitor());
+    return visitor.imports;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/7779a723/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/MethodGrabbingVisitor.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/MethodGrabbingVisitor.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/MethodGrabbingVisitor.java
index 57268ee..d46d008 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/MethodGrabbingVisitor.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/MethodGrabbingVisitor.java
@@ -28,12 +28,16 @@ public class MethodGrabbingVisitor{
 
     @Override
     public void traverseClassDeclaration(ClassDeclaration cd) {
+      logger.debug("Traversing: {}", cd.getClassName());
+      boolean prevCapture = captureMethods;
       captureMethods = c.getName().equals(cd.getClassName());
       super.traverseClassDeclaration(cd);
+      captureMethods = prevCapture;
     }
 
     @Override
     public void traverseMethodDeclarator(MethodDeclarator md) {
+      logger.debug(c.getName() + ": Found {}, include {}", md.name, captureMethods);
       if(captureMethods){
         StringWriter writer = new StringWriter();
         ModifiedUnparseVisitor v = new ModifiedUnparseVisitor(writer);

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/7779a723/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/Alternator.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/Alternator.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/Alternator.java
new file mode 100644
index 0000000..83f8368
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/Alternator.java
@@ -0,0 +1,49 @@
+package org.apache.drill.exec.expr.fn.impl;
+
+import org.apache.drill.common.expression.CallProvider;
+import org.apache.drill.common.expression.FunctionDefinition;
+import org.apache.drill.common.expression.NoArgValidator;
+import org.apache.drill.common.expression.OutputTypeDeterminer;
+import org.apache.drill.common.types.TypeProtos.MinorType;
+import org.apache.drill.common.types.Types;
+import org.apache.drill.exec.expr.DrillFunc;
+import org.apache.drill.exec.expr.annotations.FunctionTemplate;
+import org.apache.drill.exec.expr.annotations.FunctionTemplate.FunctionScope;
+import org.apache.drill.exec.expr.annotations.Output;
+import org.apache.drill.exec.expr.annotations.Workspace;
+import org.apache.drill.exec.expr.holders.LongHolder;
+import org.apache.drill.exec.record.RecordBatch;
+
+
+
+@FunctionTemplate(name = "alternate", scope = FunctionScope.SIMPLE)
+public class Alternator implements DrillFunc{
+
+  @Workspace int val;
+  @Output LongHolder out;
+  
+  public void setup(RecordBatch incoming) {
+    val = 0;
+  }
+
+
+  public void eval() {
+    out.value = val;
+    if(val == 0){
+      val = 1;
+    }else{
+      val = 0;
+    }
+  }
+  
+  public static class Provider implements CallProvider{
+
+    @Override
+    public FunctionDefinition[] getFunctionDefintions() {
+      return new FunctionDefinition[]{
+          FunctionDefinition.simple("alternate", NoArgValidator.VALIDATOR, new OutputTypeDeterminer.FixedType(Types.required(MinorType.BIGINT)), "alternate")
+      };
+    }
+    
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/7779a723/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/ValueVector.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/ValueVector.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/ValueVector.java
index 27089ac..b188d5b 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/ValueVector.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/ValueVector.java
@@ -116,7 +116,6 @@ public interface ValueVector extends Closeable {
      * @param index   Index of the value to get
      */
     public abstract Object getObject(int index);
-    
     public void reset();
   }
   

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/7779a723/sandbox/prototype/exec/java-exec/src/main/resources/drill-module.conf
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/resources/drill-module.conf b/sandbox/prototype/exec/java-exec/src/main/resources/drill-module.conf
index 3ce903d..a543197 100644
--- a/sandbox/prototype/exec/java-exec/src/main/resources/drill-module.conf
+++ b/sandbox/prototype/exec/java-exec/src/main/resources/drill-module.conf
@@ -1,6 +1,9 @@
 //  This file tells Drill to consider this module when class path scanning.  
 //  This file can also include any supplementary configuration information.  
 //  This file is in HOCON format, see https://github.com/typesafehub/config/blob/master/HOCON.md for more information.
+
+drill.logical.function.packages += "org.apache.drill.exec.expr.fn.impl"
+
 drill.exec: {
   cluster-id: "drillbits1"
   rpc: {

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/7779a723/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 df11aa7..e2bf25b 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
@@ -1,5 +1,6 @@
 package org.apache.drill.exec.physical.impl.filter;
 
+import static org.junit.Assert.*;
 import mockit.Injectable;
 import mockit.NonStrictExpectations;
 
@@ -46,7 +47,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()){
-      System.out.println(exec.getSelectionVector2().getCount());
+      assertEquals(50, exec.getSelectionVector2().getCount());
     }
   }
   

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/7779a723/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 a892c70..c9b367f 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
@@ -23,7 +23,7 @@
             @id:2,
             child: 1,
             pop:"filter",
-            expr: "true"
+            expr: "alternate()"
         },
         {
             @id: 3,


[23/53] [abbrv] git commit: Switch test log back to only utilize socket

Posted by ja...@apache.org.
Switch test log back to only utilize socket


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

Branch: refs/heads/master
Commit: 87ab0720e5d6bfb15e490fc1268f19d87a8464e8
Parents: b8db98a
Author: Jacques Nadeau <ja...@apache.org>
Authored: Wed May 22 08:06:28 2013 -0700
Committer: Jacques Nadeau <ja...@apache.org>
Committed: Wed May 22 08:06:28 2013 -0700

----------------------------------------------------------------------
 .../java-exec/src/test/resources/logback.xml    | 59 ++++++++------------
 1 file changed, 22 insertions(+), 37 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/87ab0720/sandbox/prototype/exec/java-exec/src/test/resources/logback.xml
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/resources/logback.xml b/sandbox/prototype/exec/java-exec/src/test/resources/logback.xml
index 3af6e10..206a469 100644
--- a/sandbox/prototype/exec/java-exec/src/test/resources/logback.xml
+++ b/sandbox/prototype/exec/java-exec/src/test/resources/logback.xml
@@ -1,43 +1,28 @@
 <?xml version="1.0" encoding="UTF-8" ?>
 <configuration>
-<!-- 
-  <appender name="SOCKET" class="de.huxhorn.lilith.logback.appender.ClassicMultiplexSocketAppender">
-    <Compressing>true</Compressing> 
-    <ReconnectionDelay>10000</ReconnectionDelay>
-    <IncludeCallerData>true</IncludeCallerData>
-    <RemoteHosts>localhost</RemoteHosts>
-  </appender>
--->
- <appender name="STDOUT" class="ch.qos.logback.core.ConsoleAppender">
-    <encoder>
-      <pattern>%d{HH:mm:ss.SSS} [%thread] %-5level %logger{36} - %msg%n</pattern>
-    </encoder>
-  </appender>
+	<appender name="SOCKET"
+		class="de.huxhorn.lilith.logback.appender.ClassicMultiplexSocketAppender">
+		<Compressing>true</Compressing>
+		<ReconnectionDelay>10000</ReconnectionDelay>
+		<IncludeCallerData>true</IncludeCallerData>
+		<RemoteHosts>localhost</RemoteHosts>
+	</appender>
 
-<!-- 
-  <appender name="FILE" class="ch.qos.logback.core.rolling.RollingFileAppender">
-    <file>/logs/test-common.log</file>
-    <encoder>
-      <pattern>%date %level [%thread] %logger{10} [%file:%line] %msg%n</pattern>
-    </encoder>
-    <rollingPolicy class="ch.qos.logback.core.rolling.TimeBasedRollingPolicy">
-	    <fileNamePattern>/logs/test-common.%d{yyyy-MM-dd}.log</fileNamePattern>
-	    <maxHistory>30</maxHistory>
-    </rollingPolicy>
-  </appender>
-  --> 
-  <logger name="org.apache.drill" additivity="false">
-    <level value="debug" />
-<!--     <appender-ref ref="SOCKET" /> -->
-    <appender-ref ref="STDOUT" />
-<!--     <appender-ref ref="FILE" /> -->
-  </logger>
+	<appender name="STDOUT" class="ch.qos.logback.core.ConsoleAppender">
+		<encoder>
+			<pattern>%d{HH:mm:ss.SSS} [%thread] %-5level %logger{36} - %msg%n
+			</pattern>
+		</encoder>
+	</appender>
 
-  <root>
-    <level value="error" />
-<!--     <appender-ref ref="SOCKET" /> -->
-    <appender-ref ref="STDOUT" />
-<!--     <appender-ref ref="FILE" /> -->
-  </root>
+	<logger name="org.apache.drill" additivity="false">
+		<level value="debug" />
+		<appender-ref ref="SOCKET" />
+	</logger>
+
+	<root>
+		<level value="error" />
+		<appender-ref ref="STDOUT" />
+	</root>
 
 </configuration>
\ No newline at end of file


[03/53] [abbrv] WIP fragmentation, physical plan, byte compiling, some vector work

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/StreamingRecordBatch.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/StreamingRecordBatch.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/StreamingRecordBatch.java
new file mode 100644
index 0000000..0fc7a1f
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/StreamingRecordBatch.java
@@ -0,0 +1,25 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.ops;
+
+/**
+ * Works on one incoming batch at a time.  Creates one output batch for each input batch.
+ */
+public class StreamingRecordBatch {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(StreamingRecordBatch.class);
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/exchange/ExchangeRecordBatch.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/exchange/ExchangeRecordBatch.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/exchange/ExchangeRecordBatch.java
new file mode 100644
index 0000000..07d7099
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/exchange/ExchangeRecordBatch.java
@@ -0,0 +1,22 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.ops.exchange;
+
+public class ExchangeRecordBatch {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ExchangeRecordBatch.class);
+}

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

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

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/filter/FilterRecordBatch.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/filter/FilterRecordBatch.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/filter/FilterRecordBatch.java
new file mode 100644
index 0000000..5bef612
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/filter/FilterRecordBatch.java
@@ -0,0 +1,109 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.ops.filter;
+
+import org.apache.drill.exec.ops.FilteringRecordBatchTransformer;
+import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.record.BatchSchema;
+import org.apache.drill.exec.record.InvalidValueAccessor;
+import org.apache.drill.exec.record.RecordBatch;
+import org.apache.drill.exec.record.vector.SelectionVector;
+import org.apache.drill.exec.record.vector.ValueVector;
+
+public abstract class FilterRecordBatch implements RecordBatch {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(FilterRecordBatch.class);
+
+  private RecordBatch incoming;
+  private SelectionVector selectionVector;
+  private BatchSchema schema;
+  private FilteringRecordBatchTransformer transformer;
+  private int outstanding;
+
+  public FilterRecordBatch(RecordBatch batch) {
+    this.incoming = batch;
+  }
+
+  @Override
+  public FragmentContext getContext() {
+    return incoming.getContext();
+  }
+
+  @Override
+  public BatchSchema getSchema() {
+    return schema;
+  }
+
+  @Override
+  public int getRecordCount() {
+    return 0;
+  }
+
+  @Override
+  public void kill() {
+    incoming.kill();
+  }
+
+  @Override
+  public <T extends ValueVector<T>> T getValueVector(int fieldId, Class<T> clazz) throws InvalidValueAccessor {
+    return null;
+  }
+
+  abstract int applyFilter(SelectionVector vector, int count);
+
+  /**
+   * Release all assets.
+   */
+  private void close() {
+
+  }
+
+  @Override
+  public IterOutcome next() {
+    while (true) {
+      IterOutcome o = incoming.next();
+      switch (o) {
+      case OK_NEW_SCHEMA:
+        transformer = incoming.getContext().getFilteringExpression(null);
+        schema = transformer.getSchema();
+        // fall through to ok.
+      case OK:
+
+      case NONE:
+      case STOP:
+        close();
+        return IterOutcome.STOP;
+      }
+
+      if (outstanding > 0) {
+        // move data to output location.
+
+        for (int i = incoming.getRecordCount() - outstanding; i < incoming.getRecordCount(); i++) {
+
+        }
+      }
+
+      // make sure the bit vector is as large as the current record batch.
+      if (selectionVector.size() < incoming.getRecordCount()) {
+        selectionVector.allocateNew(incoming.getRecordCount());
+      }
+
+      return null;
+    }
+
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/filter/SelectionVectorUpdater.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/filter/SelectionVectorUpdater.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/filter/SelectionVectorUpdater.java
new file mode 100644
index 0000000..218a19a
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/filter/SelectionVectorUpdater.java
@@ -0,0 +1,80 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.ops.filter;
+
+import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.memory.DirectBufferAllocator;
+import org.apache.drill.exec.record.vector.NullableInt32Vector;
+import org.apache.drill.exec.record.vector.UInt16Vector;
+import org.codehaus.janino.ExpressionEvaluator;
+
+public class SelectionVectorUpdater {
+  //static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(SelectionVectorUpdater.class);
+
+  // Add a selection vector to a record batch.
+  /**
+   * where a + b < 10
+   */
+
+  public static int applyToBatch(final int recordCount, final NullableInt32Vector a, final NullableInt32Vector b,
+      final UInt16Vector selectionVector) {
+    int selectionIndex = 0;
+    for (int i = 0; i < recordCount; i++) {
+      int isNotNull = a.isNull(i) * b.isNull(i);
+      if (isNotNull > 0 && a.get(i) + b.get(i) < 10) {
+        selectionVector.set(selectionIndex, (char) i);
+        selectionIndex++;
+      }
+    }
+    return selectionIndex;
+  }
+
+  public static void mai2n(String[] args) {
+    int size = 1024;
+    BufferAllocator allocator = new DirectBufferAllocator();
+    NullableInt32Vector a = new NullableInt32Vector(0, allocator);
+    NullableInt32Vector b = new NullableInt32Vector(1, allocator);
+    UInt16Vector select = new UInt16Vector(2, allocator);
+    a.allocateNew(size);
+    b.allocateNew(size);
+    select.allocateNew(size);
+    int r = 0;
+    for (int i = 0; i < 1500; i++) {
+      r += applyToBatch(size, a, b, select);
+    }
+
+    System.out.println(r);
+  }
+  
+public static void main(String[] args) throws Exception{
+  ExpressionEvaluator ee = new ExpressionEvaluator(
+      "c > d ? c : d",                     // expression
+      int.class,                           // expressionType
+      new String[] { "c", "d" },           // parameterNames
+      new Class[] { int.class, int.class } // parameterTypes
+  );
+  
+  Integer res = (Integer) ee.evaluate(
+      new Object[] {          // parameterValues
+          new Integer(10),
+          new Integer(11),
+      }
+  );
+  System.out.println("res = " + res);
+}
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/ExecPlanner.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/ExecPlanner.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/ExecPlanner.java
index 9554bf3..7b76810 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/ExecPlanner.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/ExecPlanner.java
@@ -17,11 +17,18 @@
  ******************************************************************************/
 package org.apache.drill.exec.planner;
 
+import org.apache.drill.common.physical.PhysicalPlan;
+import org.apache.drill.exec.exception.FragmentSetupException;
+import org.apache.drill.exec.foreman.QueryWorkUnit;
+import org.apache.drill.exec.ops.QueryContext;
+
 
 /**
  * Decides level of paralellization.
  * Generates smaller physical plans
  */
-public class ExecPlanner {
+public interface ExecPlanner {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ExecPlanner.class);
+  
+  public QueryWorkUnit getWorkUnit(QueryContext context, PhysicalPlan plan, int maxWidth) throws FragmentSetupException;
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/FragmentMaterializer.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/FragmentMaterializer.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/FragmentMaterializer.java
new file mode 100644
index 0000000..51b0691
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/FragmentMaterializer.java
@@ -0,0 +1,86 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.planner;
+
+import org.apache.drill.common.physical.pop.base.AbstractPhysicalVisitor;
+import org.apache.drill.common.physical.pop.base.AbstractStore;
+import org.apache.drill.common.physical.pop.base.Exchange;
+import org.apache.drill.common.physical.pop.base.PhysicalOperator;
+import org.apache.drill.common.physical.pop.base.Scan;
+import org.apache.drill.common.physical.pop.base.Store;
+import org.apache.drill.exec.exception.FragmentSetupException;
+
+public class FragmentMaterializer extends AbstractPhysicalVisitor<PhysicalOperator, FragmentMaterializer.IndexedFragmentNode, FragmentSetupException>{
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(FragmentMaterializer.class);
+
+  
+  @Override
+  public PhysicalOperator visitExchange(Exchange exchange, IndexedFragmentNode iNode) throws FragmentSetupException {
+    if(exchange == iNode.getNode().getSendingExchange()){
+      
+      // this is a sending exchange.
+      PhysicalOperator child = exchange.getChild();
+      return exchange.getSender(iNode.getMinorFragmentId(), child);
+      
+    }else{
+      // receiving exchange.
+      return exchange.getReceiver(iNode.getMinorFragmentId());
+    }
+  }
+
+  @Override
+  public PhysicalOperator visitScan(Scan<?> scan, IndexedFragmentNode iNode) throws FragmentSetupException {
+    return scan.getSpecificScan(iNode.getMinorFragmentId());
+  }
+
+  @Override
+  public PhysicalOperator visitStore(Store store, IndexedFragmentNode iNode) throws FragmentSetupException {
+    PhysicalOperator child = store.getChild();
+    return store.getSpecificStore(child, iNode.getMinorFragmentId());
+  }
+
+  @Override
+  public PhysicalOperator visitUnknown(PhysicalOperator op, IndexedFragmentNode iNode) throws FragmentSetupException {
+    return op;
+  }
+  
+  public static class IndexedFragmentNode{
+    final FragmentWrapper info;
+    final int minorFragmentId;
+    
+    public IndexedFragmentNode(int minorFragmentId, FragmentWrapper info) {
+      super();
+      this.info = info;
+      this.minorFragmentId = minorFragmentId;
+    }
+
+    public FragmentNode getNode() {
+      return info.getNode();
+    }
+
+    public int getMinorFragmentId() {
+      return minorFragmentId;
+    }
+
+    public FragmentWrapper getInfo() {
+      return info;
+    }
+    
+  }
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/FragmentNode.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/FragmentNode.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/FragmentNode.java
new file mode 100644
index 0000000..f53240e
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/FragmentNode.java
@@ -0,0 +1,138 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.planner;
+
+import java.util.Iterator;
+import java.util.List;
+
+import org.apache.drill.common.physical.pop.base.Exchange;
+import org.apache.drill.common.physical.pop.base.PhysicalOperator;
+import org.apache.drill.common.proto.CoordinationProtos.DrillbitEndpoint;
+import org.apache.drill.exec.exception.FragmentSetupException;
+
+import com.google.common.collect.Lists;
+
+public class FragmentNode implements Iterable<FragmentNode.ExchangeFragmentPair>{
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(FragmentNode.class);
+  
+  private PhysicalOperator root;
+  private Exchange sendingExchange;
+  private final List<ExchangeFragmentPair> receivingExchangePairs = Lists.newLinkedList();
+  private FragmentStats stats = new FragmentStats();
+  
+  public void addOperator(PhysicalOperator o){
+    if(root == null){
+      root = o;
+    }
+  }
+  
+  public void addSendExchange(Exchange e) throws FragmentSetupException{
+    if(sendingExchange != null) throw new FragmentSetupException("Fragment was trying to add a second SendExchange.  ");
+    sendingExchange = e;
+  }
+  
+  public void addReceiveExchange(Exchange e, FragmentNode fragment){
+    this.receivingExchangePairs.add(new ExchangeFragmentPair(e, fragment));
+  }
+
+  @Override
+  public Iterator<ExchangeFragmentPair> iterator() {
+    return this.receivingExchangePairs.iterator();
+  }
+
+  public List<ExchangeFragmentPair> getReceivingExchangePairs() {
+    return receivingExchangePairs;
+  }
+
+  public PhysicalOperator getRoot() {
+    return root;
+  }
+
+  public Exchange getSendingExchange() {
+    return sendingExchange;
+  }
+
+  public <T, V> T accept(FragmentVisitor<T, V> visitor, V extra){
+    return visitor.visit(this, extra);
+  }
+  
+  public FragmentStats getStats(){
+    return stats;
+  }
+  
+  public class ExchangeFragmentPair {
+    private Exchange exchange;
+    private FragmentNode node;
+    public ExchangeFragmentPair(Exchange exchange, FragmentNode node) {
+      super();
+      this.exchange = exchange;
+      this.node = node;
+    }
+    public Exchange getExchange() {
+      return exchange;
+    }
+    public FragmentNode getNode() {
+      return node;
+    }
+    @Override
+    public int hashCode() {
+      final int prime = 31;
+      int result = 1;
+      result = prime * result + FragmentNode.this.hashCode();
+      result = prime * result + ((exchange == null) ? 0 : exchange.hashCode());
+      result = prime * result + ((node == null) ? 0 : node.hashCode());
+      return result;
+    }
+    
+  }
+
+  @Override
+  public int hashCode() {
+    final int prime = 31;
+    int result = 1;
+    result = prime * result + ((receivingExchangePairs == null) ? 0 : receivingExchangePairs.hashCode());
+    result = prime * result + ((root == null) ? 0 : root.hashCode());
+    result = prime * result + ((sendingExchange == null) ? 0 : sendingExchange.hashCode());
+    result = prime * result + ((stats == null) ? 0 : stats.hashCode());
+    return result;
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (this == obj) return true;
+    if (obj == null) return false;
+    if (getClass() != obj.getClass()) return false;
+    FragmentNode other = (FragmentNode) obj;
+    if (receivingExchangePairs == null) {
+      if (other.receivingExchangePairs != null) return false;
+    } else if (!receivingExchangePairs.equals(other.receivingExchangePairs)) return false;
+    if (root == null) {
+      if (other.root != null) return false;
+    } else if (!root.equals(other.root)) return false;
+    if (sendingExchange == null) {
+      if (other.sendingExchange != null) return false;
+    } else if (!sendingExchange.equals(other.sendingExchange)) return false;
+    if (stats == null) {
+      if (other.stats != null) return false;
+    } else if (!stats.equals(other.stats)) return false;
+    return true;
+  }
+
+  
+ 
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/FragmentPlanningSet.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/FragmentPlanningSet.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/FragmentPlanningSet.java
new file mode 100644
index 0000000..3f7c3a9
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/FragmentPlanningSet.java
@@ -0,0 +1,61 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.planner;
+
+import java.util.Iterator;
+import java.util.Map;
+
+import org.apache.drill.common.proto.CoordinationProtos.DrillbitEndpoint;
+import org.apache.drill.exec.ops.QueryContext;
+
+public class FragmentPlanningSet implements Iterable<FragmentWrapper>{
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(FragmentPlanningSet.class);
+  
+  private Map<FragmentNode, FragmentWrapper> fragmentMap;
+  private int majorFragmentIdIndex = 0;
+  private QueryContext context;
+  
+  public FragmentPlanningSet(QueryContext context){
+    this.context = context;
+  }
+  
+  public void setStats(FragmentNode node, FragmentStats stats){
+    get(node).setStats(stats);
+  }
+
+  public void addAffinity(FragmentNode n, DrillbitEndpoint endpoint, float affinity){
+    get(n).addEndpointAffinity(endpoint, affinity);
+  }
+  
+  public void setWidth(FragmentNode n, int width){
+    get(n).setWidth(width);
+  }
+  
+  private FragmentWrapper get(FragmentNode node){
+    FragmentWrapper info = fragmentMap.get(node);
+    if(info == null) info = new FragmentWrapper(node, majorFragmentIdIndex++);
+    return info;
+  }
+
+  @Override
+  public Iterator<FragmentWrapper> iterator() {
+    return this.fragmentMap.values().iterator();
+  }
+  
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/FragmentRunnable.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/FragmentRunnable.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/FragmentRunnable.java
new file mode 100644
index 0000000..d551aa4
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/FragmentRunnable.java
@@ -0,0 +1,124 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.planner;
+
+import io.netty.buffer.ByteBuf;
+
+import java.util.concurrent.atomic.AtomicInteger;
+
+import org.apache.drill.exec.exception.FragmentSetupException;
+import org.apache.drill.exec.foreman.CancelableQuery;
+import org.apache.drill.exec.foreman.StatusProvider;
+import org.apache.drill.exec.metrics.SingleThreadNestedCounter;
+import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.ops.FragmentConverter;
+import org.apache.drill.exec.ops.FragmentRoot;
+import org.apache.drill.exec.proto.ExecProtos.FragmentStatus;
+import org.apache.drill.exec.proto.ExecProtos.FragmentStatus.FragmentState;
+import org.apache.drill.exec.proto.ExecProtos.PlanFragment;
+import org.apache.drill.exec.server.DrillbitContext;
+
+import com.yammer.metrics.MetricRegistry;
+import com.yammer.metrics.Timer;
+
+/**
+ * Responsible for running a single fragment on a single Drillbit. Listens/responds to status request and cancellation
+ * messages.
+ */
+public class FragmentRunnable implements Runnable, CancelableQuery, StatusProvider {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(FragmentRunnable.class);
+
+  private final AtomicInteger state = new AtomicInteger(FragmentState.AWAITING_ALLOCATION_VALUE);
+  private final FragmentRoot root;
+  private final FragmentContext context;
+
+  public FragmentRunnable(DrillbitContext dbContext, long fragmentId) throws FragmentSetupException {
+    PlanFragment fragment = dbContext.getCache().getFragment(fragmentId);
+    if (fragment == null) throw new FragmentSetupException(String.format("The provided fragment id [%d] was unknown.", fragmentId));
+    this.context = new FragmentContext(dbContext, fragment);
+    this.root = FragmentConverter.getFragment(this.context);
+  }
+
+  @Override
+  public FragmentStatus getStatus() {
+    return FragmentStatus.newBuilder() //
+        .setBatchesCompleted(context.batchesCompleted.get()) //
+        .setDataProcessed(context.dataProcessed.get()) //
+        .setMemoryUse(context.getAllocator().getAllocatedMemory()) //
+        .build();
+  }
+
+  @Override
+  public boolean cancel(long queryId) {
+    if (context.getFragment().getQueryId() == queryId) {
+      state.set(FragmentState.CANCELLED_VALUE);
+      return true;
+    }
+    return false;
+  }
+
+  private void fail(Throwable cause){
+    context.fail(cause);
+    state.set(FragmentState.FAILED_VALUE);
+  }
+  
+  @Override
+  public void run() {
+    if(!updateState(FragmentState.AWAITING_ALLOCATION, FragmentState.RUNNING, false)){
+      fail(new RuntimeException(String.format("Run was called when fragment was in %s state.  FragmentRunnables should only be started when they are currently in awaiting allocation state.", FragmentState.valueOf(state.get()))));
+      return;
+    }
+    
+    Timer.Context t = context.fragmentTime.time();
+    
+    // setup the query.
+    try{
+      root.setup();
+    }catch(FragmentSetupException e){
+      
+      context.fail(e);
+      return;
+    }
+    
+    // run the query.
+    try{
+      while(state.get() == FragmentState.RUNNING_VALUE){
+        if(!root.next()){
+          updateState(FragmentState.RUNNING, FragmentState.FINISHED, false);
+        }
+      }
+      t.stop();
+    }catch(Exception ex){
+      fail(ex);
+    }
+    
+  }
+
+  private boolean updateState(FragmentState current, FragmentState update, boolean exceptionOnFailure) {
+    boolean success = state.compareAndSet(current.getNumber(), update.getNumber());
+    if (!success && exceptionOnFailure) {
+      context.fail(new RuntimeException(String.format(
+          "State was different than expected.  Attempting to update state from %s to %s however current state was %s.",
+          current.name(), update.name(), FragmentState.valueOf(state.get()))));
+      return false;
+    }
+
+    return true;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/FragmentScheduler.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/FragmentScheduler.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/FragmentScheduler.java
new file mode 100644
index 0000000..168072a
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/FragmentScheduler.java
@@ -0,0 +1,32 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.planner;
+
+import org.apache.drill.exec.foreman.ResourceRequest;
+import org.apache.drill.exec.foreman.ResourceRequest.ResourceAllocation;
+
+import com.google.common.util.concurrent.ListenableFutureTask;
+
+public class FragmentScheduler {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(FragmentScheduler.class);
+  
+  public void getRunningResources(ResourceRequest resources, Runnable listener){
+    // request the resource.
+    
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/FragmentStats.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/FragmentStats.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/FragmentStats.java
new file mode 100644
index 0000000..512b5d0
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/FragmentStats.java
@@ -0,0 +1,63 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.planner;
+
+import org.apache.drill.common.physical.OperatorCost;
+
+public class FragmentStats {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(FragmentStats.class);
+  
+  private int maxWidth = Integer.MAX_VALUE;
+  private float networkCost; 
+  private float diskCost;
+  private float memoryCost;
+  private float cpuCost;
+  
+  public void addMaxWidth(int width){
+    maxWidth = Math.min(maxWidth, width);
+  }
+  
+  public void addCost(OperatorCost cost){
+    networkCost += cost.getNetwork();
+    diskCost += cost.getDisk();
+    memoryCost += cost.getMemory();
+    cpuCost += cost.getCpu();
+  }
+
+  public int getMaxWidth() {
+    return maxWidth;
+  }
+
+  public float getNetworkCost() {
+    return networkCost;
+  }
+
+  public float getDiskCost() {
+    return diskCost;
+  }
+
+  public float getMemoryCost() {
+    return memoryCost;
+  }
+
+  public float getCpuCost() {
+    return cpuCost;
+  }
+  
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/FragmentStatsCollector.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/FragmentStatsCollector.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/FragmentStatsCollector.java
new file mode 100644
index 0000000..a0dcde3
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/FragmentStatsCollector.java
@@ -0,0 +1,109 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.planner;
+
+import org.apache.drill.common.physical.pop.base.AbstractPhysicalVisitor;
+import org.apache.drill.common.physical.pop.base.Exchange;
+import org.apache.drill.common.physical.pop.base.PhysicalOperator;
+import org.apache.drill.common.physical.pop.base.Scan;
+import org.apache.drill.common.physical.pop.base.Store;
+import org.apache.drill.exec.planner.FragmentNode.ExchangeFragmentPair;
+
+import com.google.common.base.Preconditions;
+
+public class FragmentStatsCollector implements FragmentVisitor<Void, FragmentStats> {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(FragmentStatsCollector.class);
+
+  //private HashMap<FragmentNode, FragmentStats> nodeStats = Maps.newHashMap();
+  private final StatsCollector opCollector = new StatsCollector();
+  private final FragmentPlanningSet planningSet;
+  
+  public FragmentStatsCollector(FragmentPlanningSet planningSet){
+    this.planningSet = planningSet;
+  }
+  
+  @Override
+  public Void visit(FragmentNode n, FragmentStats stats) {
+    Preconditions.checkNotNull(stats);
+    Preconditions.checkNotNull(n);
+
+    n.getRoot().accept(opCollector, stats);
+
+    // sending exchange.
+    Exchange sending = n.getSendingExchange();
+    if (sending != null) {
+      stats.addCost(sending.getAggregateSendCost());
+      stats.addMaxWidth(sending.getMaxSendWidth());
+    }
+
+    // receivers...
+    for (ExchangeFragmentPair child : n) {
+      // add exchange receive cost.
+      Exchange receivingExchange = child.getExchange();
+      stats.addCost(receivingExchange.getAggregateReceiveCost());
+
+      FragmentStats childStats = new FragmentStats();
+      FragmentNode childNode = child.getNode();
+      childNode.accept(this, childStats);
+    }
+    
+    // store the stats for later use.
+    planningSet.setStats(n, stats);
+    
+    return null;
+  }
+
+  public void collectStats(FragmentNode rootFragment) {
+    FragmentStats s = new FragmentStats();
+    rootFragment.accept(this, s);
+  }
+
+  private class StatsCollector extends AbstractPhysicalVisitor<Void, FragmentStats, RuntimeException> {
+
+    @Override
+    public Void visitExchange(Exchange exchange, FragmentStats stats) throws RuntimeException {
+      // don't do anything here since we'll add the exchange costs elsewhere. We also don't want navigate across
+      // exchanges since they are separate fragments.
+      return null;
+    }
+
+    @Override
+    public Void visitScan(Scan<?> scan, FragmentStats stats) {
+      stats.addMaxWidth(scan.getReadEntries().size());
+      return super.visitScan(scan, stats);
+    }
+
+    @Override
+    public Void visitStore(Store store, FragmentStats stats) {
+      stats.addMaxWidth(store.getMaxWidth());
+      return super.visitStore(store, stats);
+    }
+
+    @Override
+    public Void visitUnknown(PhysicalOperator op, FragmentStats stats) {
+      stats.addCost(op.getCost());
+      for (PhysicalOperator child : op) {
+        child.accept(this, stats);
+      }
+      return null;
+    }
+
+  }
+
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/FragmentVisitor.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/FragmentVisitor.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/FragmentVisitor.java
new file mode 100644
index 0000000..12d2b9f
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/FragmentVisitor.java
@@ -0,0 +1,22 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.planner;
+
+public interface FragmentVisitor<T, V> {
+  public T visit(FragmentNode n, V extra);
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/FragmentWrapper.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/FragmentWrapper.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/FragmentWrapper.java
new file mode 100644
index 0000000..a1e4f81
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/FragmentWrapper.java
@@ -0,0 +1,127 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.planner;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ThreadLocalRandom;
+
+import org.apache.commons.lang.NotImplementedException;
+import org.apache.drill.common.physical.EndpointAffinity;
+import org.apache.drill.common.proto.CoordinationProtos.DrillbitEndpoint;
+import org.apache.drill.exec.planner.FragmentNode.ExchangeFragmentPair;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+
+public class FragmentWrapper {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(FragmentWrapper.class);
+
+  private final FragmentNode node;
+  private final int majorFragmentId;
+  private int width = -1;
+  private FragmentStats stats;
+  private boolean endpointsAssigned;
+  private Map<DrillbitEndpoint, EndpointAffinity> endpointAffinity = Maps.newHashMap();
+
+  // a list of assigned endpoints. Technically, there could repeated endpoints in this list if we'd like to assign the
+  // same fragment multiple times to the same endpoint.
+  private List<DrillbitEndpoint> endpoints = Lists.newLinkedList();
+
+  public FragmentWrapper(FragmentNode node, int majorFragmentId) {
+    this.majorFragmentId = majorFragmentId;
+    this.node = node;
+  }
+
+  public FragmentStats getStats() {
+    return stats;
+  }
+
+  public void setStats(FragmentStats stats) {
+    this.stats = stats;
+  }
+
+  public void addEndpointAffinity(DrillbitEndpoint endpoint, float affinity) {
+    Preconditions.checkState(!endpointsAssigned);
+    EndpointAffinity ea = endpointAffinity.get(endpoint);
+    if (ea == null) {
+      ea = new EndpointAffinity(endpoint);
+      endpointAffinity.put(endpoint, ea);
+    }
+
+    ea.addAffinity(affinity);
+    endpointAffinity.put(endpoint, ea);
+  }
+
+  public int getMajorFragmentId() {
+    return majorFragmentId;
+  }
+
+  public int getWidth() {
+    return width;
+  }
+
+  public void setWidth(int width) {
+    Preconditions.checkState(width == -1);
+    this.width = width;
+  }
+
+  public FragmentNode getNode() {
+    return node;
+  }
+
+  public void assignEndpoints(Collection<DrillbitEndpoint> allPossible) {
+    Preconditions.checkState(!endpointsAssigned);
+
+    endpointsAssigned = true;
+    
+    List<EndpointAffinity> values = Lists.newArrayList();
+    values.addAll(endpointAffinity.values());
+    
+    if(values.size() == 0){
+      final int div = allPossible.size();
+      int start = ThreadLocalRandom.current().nextInt(div);
+      // round robin with random start.
+      for(int i = start; i < start + width; i++){
+        endpoints.add(values.get(i % div).getEndpoint());
+      }
+    }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());
+      }
+    }
+
+    node.getSendingExchange().setupSenders(endpoints);
+    for (ExchangeFragmentPair e : node.getReceivingExchangePairs()) {
+      e.getExchange().setupReceivers(endpoints);
+    }
+  }
+
+  public DrillbitEndpoint getAssignedEndpoint(int minorFragmentId) {
+    Preconditions.checkState(endpointsAssigned);
+    return this.endpoints.get(minorFragmentId);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/FragmentingPhysicalVisitor.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/FragmentingPhysicalVisitor.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/FragmentingPhysicalVisitor.java
new file mode 100644
index 0000000..5f67617
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/FragmentingPhysicalVisitor.java
@@ -0,0 +1,71 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.planner;
+
+import org.apache.drill.common.physical.pop.base.AbstractPhysicalVisitor;
+import org.apache.drill.common.physical.pop.base.Exchange;
+import org.apache.drill.common.physical.pop.base.PhysicalOperator;
+import org.apache.drill.exec.exception.FragmentSetupException;
+
+/**
+ * Responsible for breaking a plan into its constituent Fragments.
+ */
+public class FragmentingPhysicalVisitor extends AbstractPhysicalVisitor<FragmentNode, FragmentNode, FragmentSetupException> {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(FragmentingPhysicalVisitor.class);
+
+  private FragmentNode rootFragment = new FragmentNode();
+  
+  public FragmentingPhysicalVisitor(){
+  }
+  
+  
+  @Override
+  public FragmentNode visitExchange(Exchange exchange, FragmentNode value) throws FragmentSetupException {
+//    logger.debug("Visiting Exchange {}", exchange);
+    if(value == null) throw new FragmentSetupException("The simple fragmenter was called without a FragmentBuilder value.  This will only happen if the initial call to SimpleFragmenter is by a Exchange node.  This should never happen since an Exchange node should never be the root node of a plan.");
+    FragmentNode next = getNextBuilder();
+    value.addReceiveExchange(exchange, next);
+    next.addSendExchange(exchange);
+    exchange.getChild().accept(this, getNextBuilder());
+    return value;
+  }
+  
+  @Override
+  public FragmentNode visitUnknown(PhysicalOperator op, FragmentNode value)  throws FragmentSetupException{
+//    logger.debug("Visiting Other {}", op);
+    value = ensureBuilder(value);
+    value.addOperator(op);
+    for(PhysicalOperator child : op){
+      child.accept(this, value);
+    }
+    return value;
+  }
+  
+  private FragmentNode ensureBuilder(FragmentNode value) throws FragmentSetupException{
+    if(value != null){
+      return value;
+    }else{
+      return rootFragment;
+    }
+  }
+  
+  public FragmentNode getNextBuilder(){
+    return new FragmentNode();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/MaterializedFragment.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/MaterializedFragment.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/MaterializedFragment.java
new file mode 100644
index 0000000..d1c85cb
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/MaterializedFragment.java
@@ -0,0 +1,69 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.planner;
+
+import org.apache.drill.common.physical.OperatorCost;
+import org.apache.drill.common.physical.pop.base.PhysicalOperator;
+import org.apache.drill.common.proto.CoordinationProtos.DrillbitEndpoint;
+
+public class MaterializedFragment {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(MaterializedFragment.class);
+
+  final PhysicalOperator root;
+  final DrillbitEndpoint endpoint;
+  final long queryId;
+  final int majorFragmentId;
+  final int minorFragmentId;
+  final OperatorCost cost;
+
+  public MaterializedFragment(PhysicalOperator root, DrillbitEndpoint endpoint, long queryId, int majorFragmentId,
+      int minorFragmentId, OperatorCost cost) {
+    super();
+    this.root = root;
+    this.endpoint = endpoint;
+    this.queryId = queryId;
+    this.majorFragmentId = majorFragmentId;
+    this.minorFragmentId = minorFragmentId;
+    this.cost = cost;
+  }
+
+  public PhysicalOperator getRoot() {
+    return root;
+  }
+
+  public DrillbitEndpoint getEndpoint() {
+    return endpoint;
+  }
+
+  public long getQueryId() {
+    return queryId;
+  }
+
+  public int getMajorFragmentId() {
+    return majorFragmentId;
+  }
+
+  public int getMinorFragmentId() {
+    return minorFragmentId;
+  }
+
+  public OperatorCost getCost() {
+    return cost;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/PhysicalPlanReader.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/PhysicalPlanReader.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/PhysicalPlanReader.java
new file mode 100644
index 0000000..ff31cd9
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/PhysicalPlanReader.java
@@ -0,0 +1,47 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.planner;
+
+import java.io.IOException;
+
+import org.apache.drill.common.physical.PhysicalPlan;
+import org.apache.drill.common.proto.CoordinationProtos.DrillbitEndpoint;
+import org.apache.drill.exec.server.DrillbitContext;
+import org.apache.drill.exec.store.StorageEngineRegistry;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.InjectableValues;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.ObjectReader;
+
+public class PhysicalPlanReader {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(PhysicalPlanReader.class);
+
+  private final ObjectReader reader;
+
+  public PhysicalPlanReader(ObjectMapper mapper, DrillbitEndpoint endpoint) {
+    InjectableValues injectables = new InjectableValues.Std() //
+        .addValue(DrillbitEndpoint.class, endpoint); //
+    this.reader = mapper.reader(PhysicalPlan.class).with(injectables);
+  }
+
+  public PhysicalPlan read(String json) throws JsonProcessingException, IOException {
+    return reader.readValue(json);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/ScanFinder.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/ScanFinder.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/ScanFinder.java
new file mode 100644
index 0000000..ff81d90
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/ScanFinder.java
@@ -0,0 +1,54 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.planner;
+
+import org.apache.drill.common.physical.pop.base.AbstractPhysicalVisitor;
+import org.apache.drill.common.physical.pop.base.Exchange;
+import org.apache.drill.common.physical.pop.base.PhysicalOperator;
+import org.apache.drill.common.physical.pop.base.Store;
+
+public class ScanFinder extends AbstractPhysicalVisitor<Boolean, Void, RuntimeException> {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ScanFinder.class);
+
+  private static final ScanFinder finder = new ScanFinder();
+  
+  private ScanFinder(){}
+  
+  @Override
+  public Boolean visitExchange(Exchange exchange, Void value) throws RuntimeException {
+    return false;
+  }
+
+  @Override
+  public Boolean visitStore(Store store, Void value) throws RuntimeException {
+    return true;
+  }
+
+  @Override
+  public Boolean visitUnknown(PhysicalOperator op, Void value) throws RuntimeException {
+    for(PhysicalOperator child : op){
+      if(child.accept(this,  null)) return true;
+    }
+    return false;
+  }
+  
+  public static boolean containsScan(PhysicalOperator op){
+    return op.accept(finder, null);
+  }
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/SimpleExecPlanner.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/SimpleExecPlanner.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/SimpleExecPlanner.java
new file mode 100644
index 0000000..d1c3add
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/SimpleExecPlanner.java
@@ -0,0 +1,54 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.planner;
+
+import java.util.List;
+
+import org.apache.drill.common.physical.PhysicalPlan;
+import org.apache.drill.common.physical.pop.base.PhysicalOperator;
+import org.apache.drill.exec.exception.FragmentSetupException;
+import org.apache.drill.exec.foreman.QueryWorkUnit;
+import org.apache.drill.exec.ops.QueryContext;
+import org.apache.drill.exec.proto.ExecProtos.PlanFragment;
+
+/**
+ * Parallelization is based on available nodes with source or target data.  Nodes that are "overloaded" are excluded from execution.
+ */
+public class SimpleExecPlanner implements ExecPlanner{
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(SimpleExecPlanner.class);
+  
+  private FragmentingPhysicalVisitor fragmenter = new FragmentingPhysicalVisitor();
+  private SimpleParallelizer parallelizer = new SimpleParallelizer();
+
+  @Override
+  public QueryWorkUnit getWorkUnit(QueryContext context, PhysicalPlan plan, int maxWidth) throws FragmentSetupException {
+    
+    // get the root physical operator and split the plan into sub fragments.
+    PhysicalOperator root = plan.getSortedOperators(false).iterator().next();
+    FragmentNode fragmentRoot = root.accept(fragmenter, null);
+    
+    // generate a planning set and collect stats.
+    FragmentPlanningSet planningSet = new FragmentPlanningSet(context);
+    FragmentStatsCollector statsCollector = new FragmentStatsCollector(planningSet);
+    statsCollector.collectStats(fragmentRoot);
+    
+    return parallelizer.getFragments(context, fragmentRoot, planningSet, maxWidth);
+    
+    
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/SimpleParallelizer.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/SimpleParallelizer.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/SimpleParallelizer.java
new file mode 100644
index 0000000..a52abaa
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/SimpleParallelizer.java
@@ -0,0 +1,147 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.planner;
+
+import java.util.Collection;
+import java.util.List;
+
+import org.apache.drill.common.physical.pop.base.PhysicalOperator;
+import org.apache.drill.common.proto.CoordinationProtos.DrillbitEndpoint;
+import org.apache.drill.exec.exception.FragmentSetupException;
+import org.apache.drill.exec.foreman.QueryWorkUnit;
+import org.apache.drill.exec.ops.QueryContext;
+import org.apache.drill.exec.planner.FragmentMaterializer.IndexedFragmentNode;
+import org.apache.drill.exec.proto.ExecProtos.PlanFragment;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.google.common.collect.Lists;
+
+public class SimpleParallelizer {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(SimpleParallelizer.class);
+
+  private final FragmentMaterializer materializer = new FragmentMaterializer();
+
+  /**
+   * Generate a set of assigned fragments based on the provided planningSet. Do not allow parallelization stages to go
+   * beyond the global max width.
+   * 
+   * @param context
+   *          The current QueryContext.
+   * @param planningSet
+   *          The set of queries with collected statistics that we'll work with.
+   * @param globalMaxWidth
+   *          The maximum level or paralellization any stage of the query can do. Note that while this might be the
+   *          number of active Drillbits, realistically, this could be well beyond that number of we want to do things
+   *          like speed results return.
+   * @return The list of generatoe PlanFragment protobuf objects to be assigned out to the individual nodes.
+   * @throws FragmentSetupException
+   */
+  public QueryWorkUnit getFragments(QueryContext context, FragmentNode rootNode, FragmentPlanningSet planningSet,
+      int globalMaxWidth) throws FragmentSetupException {
+    assignEndpoints(context.getActiveEndpoints(), planningSet, globalMaxWidth);
+    return generateWorkUnit(context.getQueryId(), context.getMapper(), rootNode, planningSet);
+  }
+
+  private QueryWorkUnit generateWorkUnit(long queryId, ObjectMapper mapper, FragmentNode rootNode,
+      FragmentPlanningSet planningSet) throws FragmentSetupException {
+
+    List<PlanFragment> fragments = Lists.newArrayList();
+
+    PlanFragment rootFragment = null;
+
+    // now we generate all the individual plan fragments and associated assignments. Note, we need all endpoints
+    // assigned before we can materialize, so we start a new loop here rather than utilizing the previous one.
+    for (FragmentWrapper info : planningSet) {
+
+      FragmentNode node = info.getNode();
+      FragmentStats stats = node.getStats();
+      PhysicalOperator abstractRoot = node.getRoot();
+      boolean isRootNode = rootNode == node;
+
+      if (isRootNode && info.getWidth() != 1)
+        throw new FragmentSetupException(
+            String
+                .format(
+                    "Failure while trying to setup fragment.  The root fragment must always have parallelization one.  In the current case, the width was set to %d.",
+                    info.getWidth()));
+      // a fragment is self driven if it doesn't rely on any other exchanges.
+      boolean selfDriven = node.getReceivingExchangePairs().size() == 0;
+
+      // Create a minorFragment for each major fragment.
+      for (int minorFragmentId = 0; minorFragmentId < info.getWidth(); minorFragmentId++) {
+        IndexedFragmentNode iNode = new IndexedFragmentNode(minorFragmentId, info);
+        PhysicalOperator root = abstractRoot.accept(materializer, iNode);
+
+        // get plan as JSON
+        String plan;
+        try {
+          plan = mapper.writeValueAsString(root);
+        } catch (JsonProcessingException e) {
+          throw new FragmentSetupException("Failure while trying to convert fragment into json.", e);
+        }
+
+        PlanFragment fragment = PlanFragment.newBuilder() //
+            .setCpuCost(stats.getCpuCost()) //
+            .setDiskCost(stats.getDiskCost()) //
+            .setMemoryCost(stats.getMemoryCost()) //
+            .setNetworkCost(stats.getNetworkCost()) //
+            .setFragmentJson(plan) //
+            .setMinorFragmentId(minorFragmentId) //
+            .setMajorFragmentId(info.getMajorFragmentId()).setQueryId(queryId) //
+            .setAssignment(info.getAssignedEndpoint(minorFragmentId)).setSelfDriven(selfDriven).build();
+
+        if (isRootNode) {
+          rootFragment = fragment;
+        } else {
+          fragments.add(fragment);
+        }
+      }
+    }
+
+    return new QueryWorkUnit(rootFragment, fragments);
+
+  }
+
+  private void assignEndpoints(Collection<DrillbitEndpoint> allNodes, FragmentPlanningSet planningSet,
+      int globalMaxWidth) {
+    // First we determine the amount of parallelization for a fragment. This will be between 1 and maxWidth based on
+    // cost. (Later could also be based on cluster operation.) then we decide endpoints based on affinity (later this
+    // could be based on endpoint load)
+    for (FragmentWrapper info : planningSet) {
+
+      FragmentStats stats = info.getStats();
+
+      // figure out width.
+      int width = Math.min(stats.getMaxWidth(), globalMaxWidth);
+      float diskCost = stats.getDiskCost();
+
+      // TODO: right now we'll just assume that each task is cost 1 so we'll set the breadth at the lesser of the number
+      // of tasks or the maximum width of the fragment.
+      if (diskCost < width) {
+        width = (int) diskCost;
+      }
+
+      if (width < 1) width = 1;
+      info.setWidth(width);
+
+      // figure out endpoint assignments. also informs the exchanges about their respective endpoints.
+      info.assignEndpoints(allNodes);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/pop/receiver/NWayOrderingReceiver.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/pop/receiver/NWayOrderingReceiver.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/pop/receiver/NWayOrderingReceiver.java
new file mode 100644
index 0000000..562d109
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/pop/receiver/NWayOrderingReceiver.java
@@ -0,0 +1,52 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.pop.receiver;
+
+import java.util.List;
+
+import org.apache.drill.common.physical.pop.base.AbstractReceiver;
+import org.apache.drill.common.physical.pop.base.PhysicalVisitor;
+import org.apache.drill.common.proto.CoordinationProtos.DrillbitEndpoint;
+
+import com.fasterxml.jackson.annotation.JsonTypeName;
+
+@JsonTypeName("nway-ordering-receiver")
+public class NWayOrderingReceiver extends AbstractReceiver{
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(NWayOrderingReceiver.class);
+
+  @Override
+  public List<DrillbitEndpoint> getProvidingEndpoints() {
+    return null;
+  }
+
+  @Override
+  public boolean supportsOutOfOrderExchange() {
+    return false;
+  }
+
+  @Override
+  public int getSenderCount() {
+    return 0;
+  }
+
+  @Override
+  public <T, X, E extends Throwable> T accept(PhysicalVisitor<T, X, E> physicalVisitor, X value) throws E {
+    return null;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/pop/receiver/RandomReceiver.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/pop/receiver/RandomReceiver.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/pop/receiver/RandomReceiver.java
new file mode 100644
index 0000000..487c645
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/pop/receiver/RandomReceiver.java
@@ -0,0 +1,55 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.pop.receiver;
+
+import java.util.Iterator;
+import java.util.List;
+
+import org.apache.drill.common.physical.pop.base.AbstractReceiver;
+import org.apache.drill.common.physical.pop.base.PhysicalOperator;
+import org.apache.drill.common.physical.pop.base.PhysicalVisitor;
+import org.apache.drill.common.proto.CoordinationProtos.DrillbitEndpoint;
+
+import com.fasterxml.jackson.annotation.JsonTypeName;
+
+@JsonTypeName("receiver-random")
+public class RandomReceiver extends AbstractReceiver{
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(RandomReceiver.class);
+
+  @Override
+  public List<DrillbitEndpoint> getProvidingEndpoints() {
+    return null;
+  }
+
+  @Override
+  public boolean supportsOutOfOrderExchange() {
+    return false;
+  }
+
+  @Override
+  public int getSenderCount() {
+    return 0;
+  }
+
+  @Override
+  public <T, X, E extends Throwable> T accept(PhysicalVisitor<T, X, E> physicalVisitor, X value) throws E {
+    return null;
+  }
+
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/pop/sender/HashPartitionSender.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/pop/sender/HashPartitionSender.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/pop/sender/HashPartitionSender.java
new file mode 100644
index 0000000..b0fb51c
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/pop/sender/HashPartitionSender.java
@@ -0,0 +1,49 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.pop.sender;
+
+import java.util.List;
+
+import org.apache.drill.common.physical.pop.base.AbstractSender;
+import org.apache.drill.common.physical.pop.base.PhysicalOperator;
+import org.apache.drill.common.physical.pop.base.PhysicalVisitor;
+import org.apache.drill.common.proto.CoordinationProtos.DrillbitEndpoint;
+
+import com.fasterxml.jackson.annotation.JsonTypeName;
+
+@JsonTypeName("hash-partition-sender")
+public class HashPartitionSender extends AbstractSender {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(HashPartitionSender.class);
+
+  public HashPartitionSender(PhysicalOperator child) {
+    super(child);
+  }
+
+
+  @Override
+  public List<DrillbitEndpoint> getDestinations() {
+    return null;
+  }
+
+  @Override
+  public <T, X, E extends Throwable> T accept(PhysicalVisitor<T, X, E> physicalVisitor, X value) throws E {
+    return null;
+  }
+
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/BatchSchema.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/BatchSchema.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/BatchSchema.java
index 1d32340..d3e4b23 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/BatchSchema.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/BatchSchema.java
@@ -32,10 +32,12 @@ import com.google.common.collect.Lists;
 public class BatchSchema implements Iterable<MaterializedField>{
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(BatchSchema.class);
   
-  private List<MaterializedField> fields = Lists.newArrayList();
+  private final List<MaterializedField> fields;
+  private final boolean hasSelectionVector;
   
-  private BatchSchema(List<MaterializedField> fields) {
+  private BatchSchema(boolean hasSelectionVector, List<MaterializedField> fields) {
     this.fields = fields;
+    this.hasSelectionVector = hasSelectionVector;
   }
 
   @Override
@@ -59,10 +61,13 @@ public class BatchSchema implements Iterable<MaterializedField>{
     private IntObjectOpenHashMap<MaterializedField> fields = new IntObjectOpenHashMap<MaterializedField>();
     private IntObjectOpenHashMap<MaterializedField> expectedFields = new IntObjectOpenHashMap<MaterializedField>();
     
+    private boolean hasSelectionVector;
+    
     public BatchSchemaBuilder(BatchSchema expected){
       for(MaterializedField f: expected){
         expectedFields.put(f.getFieldId(), f);
       }
+      hasSelectionVector = expected.hasSelectionVector;
     }
     
     public BatchSchemaBuilder(){
@@ -80,6 +85,10 @@ public class BatchSchema implements Iterable<MaterializedField>{
       addTypedField(fieldId, DataType.LATEBIND, nullable, mode, Void.class);
     }
     
+    public void setSelectionVector(boolean hasSelectionVector){
+      this.hasSelectionVector = hasSelectionVector;
+    }
+    
     private void setTypedField(short fieldId, DataType type, boolean nullable, ValueMode mode, Class<?> valueClass) throws SchemaChangeException{
       MaterializedField f = new MaterializedField(fieldId, type, nullable, mode, valueClass);
       if(expectedFields != null){
@@ -99,13 +108,13 @@ public class BatchSchema implements Iterable<MaterializedField>{
       setTypedField(fieldId, type, nullable, mode, valueClass);
     }
     
-    public void addVector(ValueVector<?> v){
-      
-    }
-    
-    public void replaceVector(ValueVector<?> oldVector, ValueVector<?> newVector){
-      
-    }
+//    public void addVector(ValueVector<?> v){
+//      
+//    }
+//    
+//    public void replaceVector(ValueVector<?> oldVector, ValueVector<?> newVector){
+//      
+//    }
     
     
     public BatchSchema buildAndClear() throws SchemaChangeException{
@@ -116,7 +125,7 @@ public class BatchSchema implements Iterable<MaterializedField>{
         if(f != null) fieldList.add(f);
       }
       Collections.sort(fieldList);
-      return new BatchSchema(fieldList);
+      return new BatchSchema(this.hasSelectionVector, fieldList);
     }
   }
   

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/MaterializedField.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/MaterializedField.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/MaterializedField.java
index 3cadf89..2e941a2 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/MaterializedField.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/MaterializedField.java
@@ -70,6 +70,10 @@ public class MaterializedField implements Comparable<MaterializedField>{
     check("valueMode", this.mode, expected.mode);
   }
 
+  public MaterializedField getNullableVersion(Class<?> valueClass){
+    return new MaterializedField(fieldId, type, true, mode, valueClass);
+  }
+  
   @Override
   public int compareTo(MaterializedField o) {
     return Integer.compare(this.fieldId, o.fieldId);

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/AbstractFixedValueVector.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/AbstractFixedValueVector.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/AbstractFixedValueVector.java
index 735493d..912e02d 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/AbstractFixedValueVector.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/AbstractFixedValueVector.java
@@ -19,7 +19,7 @@ package org.apache.drill.exec.record.vector;
 
 import io.netty.buffer.ByteBuf;
 
-import org.apache.drill.exec.BufferAllocator;
+import org.apache.drill.exec.memory.BufferAllocator;
 
 /**
  * Abstract class that fixed value vectors are derived from.

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/BaseValueVector.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/BaseValueVector.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/BaseValueVector.java
index 33a81e5..8d524b2 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/BaseValueVector.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/BaseValueVector.java
@@ -19,7 +19,7 @@ package org.apache.drill.exec.record.vector;
 
 import io.netty.buffer.ByteBuf;
 
-import org.apache.drill.exec.BufferAllocator;
+import org.apache.drill.exec.memory.BufferAllocator;
 import org.apache.drill.exec.record.DeadBuf;
 
 public abstract class BaseValueVector<T extends BaseValueVector<T>> implements ValueVector<T>{
@@ -37,7 +37,9 @@ public abstract class BaseValueVector<T extends BaseValueVector<T>> implements V
 
   public final void allocateNew(int valueCount){
     int allocationSize = getAllocationSize(valueCount);
-    resetAllocation(valueCount, allocator.buffer(allocationSize));
+    ByteBuf newBuf =  allocator.buffer(allocationSize);
+    newBuf.retain();
+    resetAllocation(valueCount, newBuf);
   }
 
   protected abstract int getAllocationSize(int valueCount);


Re: [04/53] [abbrv] WIP fragmentation, physical plan, byte compiling, some vector work

Posted by Ted Dunning <te...@gmail.com>.
On Fri, Jul 19, 2013 at 6:57 PM, <ja...@apache.org> wrote:

>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/compile/ClassBodyBuilder.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/compile/ClassBodyBuilder.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/compile/ClassBodyBuilder.java
> new file mode 100644
> index 0000000..ac44484
> --- /dev/null
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/compile/ClassBodyBuilder.java
> @@ -0,0 +1,247 @@
>
> +/*******************************************************************************
> + * Licensed to the Apache Software Foundation (ASF) under one
> + * or more contributor license agreements.  See the NOTICE file
> + * distributed with this work for additional information
> + * regarding copyright ownership.  The ASF licenses this file
> + * to you under the Apache License, Version 2.0 (the
> + * "License"); you may not use this file except in compliance
> + * with the License.  You may obtain a copy of the License at
> + *
> + * http://www.apache.org/licenses/LICENSE-2.0
> + *
> + * Unless required by applicable law or agreed to in writing, software
> + * distributed under the License is distributed on an "AS IS" BASIS,
> + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
> implied.
> + * See the License for the specific language governing permissions and
> + * limitations under the License.
> +
> ******************************************************************************/
> +package org.apache.drill.exec.compile;
> +
> +/*
> + * Janino - An embedded Java[TM] compiler
> + *
> + * Copyright (c) 2001-2010, Arno Unkrig
> + * All rights reserved.
> + *
> + * Redistribution and use in source and binary forms, with or without
> modification, are permitted provided that the
> + * following conditions are met:
> + *
> + *    1. Redistributions of source code must retain the above copyright
> notice, this list of conditions and the
> + *       following disclaimer.
> + *    2. Redistributions in binary form must reproduce the above
> copyright notice, this list of conditions and the
> + *       following disclaimer in the documentation and/or other materials
> provided with the distribution.
> + *    3. The name of the author may not be used to endorse or promote
> products derived from this software without
> + *       specific prior written permission.
> + *
> + * THIS SOFTWARE IS PROVIDED BY THE AUTHOR ``AS IS'' AND ANY EXPRESS OR
> IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED
> + * TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A
> PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL
> + * THE AUTHOR BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL,
> EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING,
> + * BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS
> OF USE, DATA, OR PROFITS; OR BUSINESS
> + * INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER
> IN CONTRACT, STRICT LIABILITY, OR TORT
> + * (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
> OF THIS SOFTWARE, EVEN IF ADVISED OF THE
> + * POSSIBILITY OF SUCH DAMAGE.
> + */
> +
> +import java.io.BufferedReader;
> +import java.io.IOException;
> +import java.io.PrintWriter;
> +import java.io.Reader;
> +import java.io.StringReader;
> +import java.io.StringWriter;
> +import java.nio.CharBuffer;
> +import java.util.ArrayList;
> +import java.util.List;
> +import java.util.regex.Matcher;
> +import java.util.regex.Pattern;
> +
> +import org.codehaus.commons.compiler.CompileException;
> +import org.codehaus.commons.compiler.IClassBodyEvaluator;
> +import org.codehaus.commons.io.MultiReader;
> +
> +public class ClassBodyBuilder {
> +
> +  private String[] optionalDefaultImports = null;
> +  private String className = IClassBodyEvaluator.DEFAULT_CLASS_NAME;
> +  private Class<?> optionalExtendedType = null;
> +  private Class<?>[] implementedTypes = new Class[0];
> +  private String[] imports = {};
> +  private String body;
> +  private boolean used = false;
> +
> +  public static ClassBodyBuilder newBuilder(){
> +    return new ClassBodyBuilder();
> +  }
> +
> +  private ClassBodyBuilder(){
> +  }
> +
> +  public ClassBodyBuilder setClassName(String className) {
> +    assertNotCooked();
> +    this.className = className;
> +    return this;
> +  }
> +
> +  public ClassBodyBuilder setDefaultImports(String...
> optionalDefaultImports) {
> +    assertNotCooked();
> +    this.optionalDefaultImports = optionalDefaultImports;
> +    return this;
> +  }
> +
> +  public ClassBodyBuilder setExtendedClass(Class<?> optionalExtendedType)
> {
> +    assertNotCooked();
> +    this.optionalExtendedType = optionalExtendedType;
> +    return this;
> +  }
> +
> +  public ClassBodyBuilder setImplementedInterfaces(Class<?>...
> implementedTypes) {
> +    assertNotCooked();
> +    this.implementedTypes = implementedTypes;
> +    return this;
> +  }
> +
> +  private void assertNotCooked() {
> +    assert !used;
> +  }
> +
> +  public ClassBodyBuilder setImports(String[] imports) {
> +    assertNotCooked();
> +    this.imports = imports;
> +    return this;
> +  }
> +
> +  public ClassBodyBuilder setBody(String body) {
> +    assertNotCooked();
> +    this.body = body;
> +    return this;
> +  }
> +
> +  public String build() throws CompileException, IOException {
> +    used = true;
> +    // Wrap the class body in a compilation unit.
> +    {
> +      StringWriter sw1 = new StringWriter();
> +      {
> +        PrintWriter pw = new PrintWriter(sw1);
> +
> +        // Break the class name up into package name and simple class
> name.
> +        String packageName; // null means default package.
> +        String simpleClassName;
> +        {
> +          int idx = this.className.lastIndexOf('.');
> +          if (idx == -1) {
> +            packageName = "";
> +            simpleClassName = this.className;
> +          } else {
> +            packageName = this.className.substring(0, idx);
> +            simpleClassName = this.className.substring(idx + 1);
> +          }
> +        }
> +
> +        // Print PACKAGE directive.
> +        if (!packageName.isEmpty()) {
> +          pw.print("package ");
> +          pw.print(packageName);
> +          pw.println(";");
> +        }
> +
> +        // Print default imports.
> +        if (this.optionalDefaultImports != null) {
> +          for (String defaultImport : this.optionalDefaultImports) {
> +            pw.print("import ");
> +            pw.print(defaultImport);
> +            pw.println(";");
> +          }
> +        }
> +
> +        // Print imports as declared in the document.
> +        for (String imporT : imports) {
> +          pw.print("import ");
> +          pw.print(imporT);
> +          pw.println(";");
> +        }
> +
> +        // Print the class declaration.
> +        pw.print("public class ");
> +        pw.print(simpleClassName);
> +        if (this.optionalExtendedType != null) {
> +          pw.print(" extends ");
> +          pw.print(this.optionalExtendedType.getCanonicalName());
> +        }
> +        if (this.implementedTypes.length > 0) {
> +          pw.print(" implements ");
> +          pw.print(this.implementedTypes[0].getName());
> +          for (int i = 1; i < this.implementedTypes.length; ++i) {
> +            pw.print(", ");
> +            pw.print(this.implementedTypes[i].getName());
> +          }
> +        }
> +        pw.println(" {");
> +        pw.close();
> +      }
> +
> +      StringWriter sw2 = new StringWriter();
> +      {
> +        PrintWriter pw = new PrintWriter(sw2);
> +        pw.println("}");
> +        pw.close();
> +      }
> +
> +      return sw1.toString() + body + sw2.toString();
> +
> +    }
> +
> +  }
> +
> +//  /**
> +//   * Heuristically parse IMPORT declarations at the beginning of the
> character stream produced by the given
> +//   * {@link Reader}. After this method returns, all characters up to
> and including that last IMPORT declaration have
> +//   * been read from the {@link Reader}.
> +//   * <p>
> +//   * This method does not handle comments and string literals
> correctly, i.e. if a pattern that looks like an IMPORT
> +//   * declaration appears within a comment or a string literal, it will
> be taken as an IMPORT declaration.
> +//   *
> +//   * @param r
> +//   *          A {@link Reader} that supports MARK, e.g. a {@link
> BufferedReader}
> +//   * @return The parsed imports, e.g. {@code "java.util.*", "static
> java.util.Map.Entry" }
> +//   */
> +//  protected static String[] parseImportDeclarations(Reader r) throws
> IOException {
> +//    final CharBuffer cb = CharBuffer.allocate(10000);
> +//    r.mark(cb.limit());
> +//    r.read(cb);
> +//    cb.rewind();
> +//
> +//    List<String> imports = new ArrayList<String>();
> +//    int afterLastImport = 0;
> +//    for (Matcher matcher = IMPORT_STATEMENT_PATTERN.matcher(cb);
> matcher.find();) {
> +//      imports.add(matcher.group(1));
> +//      afterLastImport = matcher.end();
> +//    }
> +//    r.reset();
> +//    r.skip(afterLastImport);
> +//    return imports.toArray(new String[imports.size()]);
> +//  }
> +//
> +//  private static final Pattern IMPORT_STATEMENT_PATTERN =
> Pattern.compile("\\bimport\\s+" + "(" + "(?:static\\s+)?"
> +//      +
> "[\\p{javaLowerCase}\\p{javaUpperCase}_\\$][\\p{javaLowerCase}\\p{javaUpperCase}\\d_\\$]*"
> +//      +
> "(?:\\.[\\p{javaLowerCase}\\p{javaUpperCase}_\\$][\\p{javaLowerCase}\\p{javaUpperCase}\\d_\\$]*)*"
> +//      + "(?:\\.\\*)?" + ");");
> +
> +//  @Override
> +//  public Object createInstance(Reader reader) throws CompileException,
> IOException {
> +//    this.cook(reader);
> +//    try {
> +//      return this.getClazz().newInstance();
> +//    } catch (InstantiationException ie) {
> +//      CompileException ce = new CompileException(
> +//          ("Class is abstract, an interface, an array class, a
> primitive type, or void; "
> +//              + "or has no zero-parameter constructor"), null);
> +//      ce.initCause(ie);
> +//      throw ce;
> +//    } catch (IllegalAccessException iae) {
> +//      CompileException ce = new CompileException("The class or its
> zero-parameter constructor is not accessible", null);
> +//      ce.initCause(iae);
> +//      throw ce;
> +//    }
> +//  }
> +
> +}
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/compile/ClassCompiler.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/compile/ClassCompiler.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/compile/ClassCompiler.java
> new file mode 100644
> index 0000000..0fb0115
> --- /dev/null
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/compile/ClassCompiler.java
> @@ -0,0 +1,29 @@
>
> +/*******************************************************************************
> + * Licensed to the Apache Software Foundation (ASF) under one
> + * or more contributor license agreements.  See the NOTICE file
> + * distributed with this work for additional information
> + * regarding copyright ownership.  The ASF licenses this file
> + * to you under the Apache License, Version 2.0 (the
> + * "License"); you may not use this file except in compliance
> + * with the License.  You may obtain a copy of the License at
> + *
> + * http://www.apache.org/licenses/LICENSE-2.0
> + *
> + * Unless required by applicable law or agreed to in writing, software
> + * distributed under the License is distributed on an "AS IS" BASIS,
> + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
> implied.
> + * See the License for the specific language governing permissions and
> + * limitations under the License.
> +
> ******************************************************************************/
> +package org.apache.drill.exec.compile;
> +
> +import java.io.IOException;
> +
> +import org.apache.drill.exec.exception.ClassTransformationException;
> +import org.codehaus.commons.compiler.CompileException;
> +
> +interface ClassCompiler {
> +
> +  public abstract byte[] getClassByteCode(String className, String
> sourcecode) throws CompileException, IOException, ClassNotFoundException,
> ClassTransformationException ;
> +
> +}
> \ No newline at end of file
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/compile/ClassTransformer.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/compile/ClassTransformer.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/compile/ClassTransformer.java
> new file mode 100644
> index 0000000..814b239
> --- /dev/null
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/compile/ClassTransformer.java
> @@ -0,0 +1,210 @@
>
> +/*******************************************************************************
> + * Licensed to the Apache Software Foundation (ASF) under one
> + * or more contributor license agreements.  See the NOTICE file
> + * distributed with this work for additional information
> + * regarding copyright ownership.  The ASF licenses this file
> + * to you under the Apache License, Version 2.0 (the
> + * "License"); you may not use this file except in compliance
> + * with the License.  You may obtain a copy of the License at
> + *
> + * http://www.apache.org/licenses/LICENSE-2.0
> + *
> + * Unless required by applicable law or agreed to in writing, software
> + * distributed under the License is distributed on an "AS IS" BASIS,
> + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
> implied.
> + * See the License for the specific language governing permissions and
> + * limitations under the License.
> +
> ******************************************************************************/
> +package org.apache.drill.exec.compile;
> +
> +import java.io.File;
> +import java.io.IOException;
> +import java.io.PrintWriter;
> +import java.lang.reflect.Modifier;
> +import java.net.URL;
> +import java.util.Iterator;
> +import java.util.concurrent.ExecutionException;
> +import java.util.concurrent.TimeUnit;
> +import java.util.concurrent.atomic.AtomicLong;
> +
> +import org.apache.drill.exec.exception.ClassTransformationException;
> +import org.codehaus.commons.compiler.CompileException;
> +import org.objectweb.asm.ClassAdapter;
> +import org.objectweb.asm.ClassReader;
> +import org.objectweb.asm.ClassVisitor;
> +import org.objectweb.asm.ClassWriter;
> +import org.objectweb.asm.MethodVisitor;
> +import org.objectweb.asm.commons.EmptyVisitor;
> +import org.objectweb.asm.commons.RemappingClassAdapter;
> +import org.objectweb.asm.commons.RemappingMethodAdapter;
> +import org.objectweb.asm.commons.SimpleRemapper;
> +import org.objectweb.asm.tree.ClassNode;
> +import org.objectweb.asm.tree.FieldNode;
> +import org.objectweb.asm.tree.MethodNode;
> +import org.objectweb.asm.util.TraceClassVisitor;
> +
> +import com.google.common.cache.CacheBuilder;
> +import com.google.common.cache.CacheLoader;
> +import com.google.common.cache.LoadingCache;
> +import com.google.common.io.Resources;
> +
> +public class ClassTransformer {
> +  static final org.slf4j.Logger logger =
> org.slf4j.LoggerFactory.getLogger(ClassTransformer.class);
> +
> +  private AtomicLong index = new AtomicLong(0);
> +  private AtomicLong interfaceIndex = new AtomicLong(0);
> +  private LoadingCache<String, byte[]> byteCode =
> CacheBuilder.newBuilder()
> +      .maximumSize(10000)
> +      .expireAfterWrite(10, TimeUnit.MINUTES)
> +      .build(new ClassBytesCacheLoader());
> +
> +
> +  private class ClassBytesCacheLoader extends CacheLoader<String, byte[]>{
> +    public byte[] load(String path) throws ClassTransformationException,
> IOException {
> +      URL u = this.getClass().getResource(path);
> +      if (u == null) throw new
> ClassTransformationException(String.format("Unable to find TemplateClass at
> path %s",path));
> +      return Resources.toByteArray(u);
> +    }
> +  };
> +
> +  private byte[] getClassByteCodeFromPath(String path) throws
> ClassTransformationException, IOException {
> +    try{
> +      return byteCode.get(path);
> +    } catch (ExecutionException e) {
> +      Throwable c = e.getCause();
> +      if(c instanceof ClassTransformationException) throw
> (ClassTransformationException) c;
> +      if(c instanceof IOException) throw (IOException) c;
> +      throw new ClassTransformationException(c);
> +    }
> +  }
> +
> +
> +  @SuppressWarnings("unchecked")
> +  public <T, I> T getImplementationClass(QueryClassLoader classLoader,
> +      TemplateClassDefinition<T, I> templateDefinition, String
> internalClassBody, I initObject)
> +      throws ClassTransformationException {
> +    final String implClassName =
> templateDefinition.getTemplateClassName() +
> interfaceIndex.getAndIncrement();
> +    final String materializedClassName = "org.apache.drill.generated."
> +        + templateDefinition.getExternalInterface().getSimpleName() +
> index.getAndIncrement();
> +    // final String materializedClassName =
> templateDefinition.getTemplateClassName();
> +    try {
> +
> +      // Get Implementation Class
> +      String classBody = ClassBodyBuilder.newBuilder() //
> +          .setClassName(implClassName) //
> +
>  .setImplementedInterfaces(templateDefinition.getInternalInterface()) //
> +          .setBody(internalClassBody) //
> +          .build();
> +      final byte[] implementationClass =
> classLoader.getClassByteCode(implClassName, classBody);
> +
> +      // Get Template Class
> +      final String templateClassName =
> templateDefinition.getTemplateClassName().replaceAll("\\.", File.separator);
> +      final String templateClassPath = File.separator + templateClassName
> + ".class";
> +      final byte[] templateClass =
> getClassByteCodeFromPath(templateClassPath);
> +
> +      // Generate Merge Class
> +      ClassNode impl = getClassNodeFromByteCode(implementationClass);
> +      // traceClassToSystemOut(implementationClass);
> +      // traceClassToSystemOut(templateClass);
> +      ClassWriter cw = new ClassWriter(0);
> +      MergeAdapter adapter = new MergeAdapter(cw, impl);
> +      ClassReader tReader = new ClassReader(templateClass);
> +      tReader.accept(adapter, 0);
> +
> +      byte[] outputClass = cw.toByteArray();
> +
> +      cw = new ClassWriter(0);
> +      RemappingClassAdapter r = new RemappingClassAdapter(cw, new
> SimpleRemapper(templateClassName,
> +          materializedClassName.replace('.', '/')));
> +      new ClassReader(outputClass).accept(r, 0);
> +      outputClass = cw.toByteArray();
> +      // traceClassToSystemOut(outputClass);
> +
> +      // Load the class
> +      classLoader.injectByteCode(materializedClassName, outputClass);
> +      Class<?> c = classLoader.findClass(materializedClassName);
> +      if (templateDefinition.getExternalInterface().isAssignableFrom(c)) {
> +        return (T) c.newInstance();
> +      } else {
> +        throw new ClassTransformationException("The requested class did
> not implement the expected interface.");
> +      }
> +
> +    } catch (CompileException | IOException | ClassNotFoundException |
> InstantiationException | IllegalAccessException e) {
> +      throw new ClassTransformationException("Failure generating
> transformation classes.", e);
> +    }
> +
> +  }
> +
> +  private ClassNode getClassNodeFromByteCode(byte[] bytes) {
> +    ClassReader iReader = new ClassReader(bytes);
> +    ClassNode impl = new ClassNode();
> +    iReader.accept(impl, 0);
> +    return impl;
> +  }
> +
> +  private void traceClassToSystemOut(byte[] bytecode) {
> +    TraceClassVisitor tcv = new TraceClassVisitor(new EmptyVisitor(), new
> PrintWriter(System.out));
> +    ClassReader cr = new ClassReader(bytecode);
> +    cr.accept(tcv, 0);
> +  }
> +
> +  public class MergeAdapter extends ClassAdapter {
> +    private ClassNode classToMerge;
> +    private String cname;
> +
> +    public MergeAdapter(ClassVisitor cv, ClassNode cn) {
> +      super(cv);
> +      this.classToMerge = cn;
> +    }
> +
> +    // visit the class
> +    public void visit(int version, int access, String name, String
> signature, String superName, String[] interfaces) {
> +      // use the access and names of the impl class.
> +      super.visit(version, access ^ Modifier.ABSTRACT | Modifier.FINAL,
> name, signature, superName, interfaces);
> +      this.cname = name;
> +    }
> +
> +    @Override
> +    public MethodVisitor visitMethod(int access, String arg1, String
> arg2, String arg3, String[] arg4) {
> +      // finalize all methods.
> +
> +      // skip all abstract methods as they should have implementations.
> +      if ((access & Modifier.ABSTRACT) != 0) {
> +        // logger.debug("Skipping copy of '{}()' since it is abstract.",
> arg1);
> +        return null;
> +      }
> +
> +      // if( (access & Modifier.PUBLIC) == 0){
> +      // access = access ^ Modifier.PUBLIC ^ Modifier.PROTECTED |
> Modifier.PRIVATE;
> +      // }
> +      if (!arg1.equals("<init>")) {
> +        access = access | Modifier.FINAL;
> +      }
> +      return super.visitMethod(access, arg1, arg2, arg3, arg4);
> +    }
> +
> +    @SuppressWarnings("unchecked")
> +    public void visitEnd() {
> +      for (Iterator<?> it = classToMerge.fields.iterator();
> it.hasNext();) {
> +        ((FieldNode) it.next()).accept(this);
> +      }
> +      for (Iterator<?> it = classToMerge.methods.iterator();
> it.hasNext();) {
> +        MethodNode mn = (MethodNode) it.next();
> +
> +        // skip the init.
> +        if (mn.name.equals("<init>")) continue;
> +
> +        String[] exceptions = new String[mn.exceptions.size()];
> +        mn.exceptions.toArray(exceptions);
> +        MethodVisitor mv = cv.visitMethod(mn.access | Modifier.FINAL,
> mn.name, mn.desc, mn.signature, exceptions);
> +        mn.instructions.resetLabels();
> +        // mn.accept(new RemappingMethodAdapter(mn.access, mn.desc, mv,
> new
> +        //
> SimpleRemapper("org.apache.drill.exec.compile.ExampleTemplate", "Bunky")));
> +        mn.accept(new RemappingMethodAdapter(mn.access, mn.desc, mv, new
> SimpleRemapper(cname.replace('.', '/'),
> +            classToMerge.name.replace('.', '/'))));
> +      }
> +      super.visitEnd();
> +    }
> +  }
> +
> +}
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/compile/JDKClassCompiler.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/compile/JDKClassCompiler.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/compile/JDKClassCompiler.java
> new file mode 100644
> index 0000000..15e87fe
> --- /dev/null
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/compile/JDKClassCompiler.java
> @@ -0,0 +1,177 @@
>
> +/*******************************************************************************
> + * Licensed to the Apache Software Foundation (ASF) under one
> + * or more contributor license agreements.  See the NOTICE file
> + * distributed with this work for additional information
> + * regarding copyright ownership.  The ASF licenses this file
> + * to you under the Apache License, Version 2.0 (the
> + * "License"); you may not use this file except in compliance
> + * with the License.  You may obtain a copy of the License at
> + *
> + * http://www.apache.org/licenses/LICENSE-2.0
> + *
> + * Unless required by applicable law or agreed to in writing, software
> + * distributed under the License is distributed on an "AS IS" BASIS,
> + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
> implied.
> + * See the License for the specific language governing permissions and
> + * limitations under the License.
> +
> ******************************************************************************/
> +package org.apache.drill.exec.compile;
> +
> +import java.io.IOException;
> +import java.io.Reader;
> +import java.io.StringReader;
> +import java.net.URI;
> +import java.net.URISyntaxException;
> +import java.util.ArrayList;
> +import java.util.Collection;
> +import java.util.Collections;
> +import java.util.List;
> +
> +import javax.tools.Diagnostic;
> +import javax.tools.DiagnosticListener;
> +import javax.tools.JavaCompiler;
> +import javax.tools.JavaFileManager;
> +import javax.tools.JavaFileObject;
> +import javax.tools.JavaCompiler.CompilationTask;
> +import javax.tools.JavaFileObject.Kind;
> +import javax.tools.SimpleJavaFileObject;
> +import javax.tools.StandardLocation;
> +import javax.tools.ToolProvider;
> +
> +import org.codehaus.commons.compiler.CompileException;
> +import org.codehaus.commons.compiler.Location;
> +import org.codehaus.commons.compiler.jdk.ByteArrayJavaFileManager;
> +import
> org.codehaus.commons.compiler.jdk.ByteArrayJavaFileManager.ByteArrayJavaFileObject;
> +
> +import com.beust.jcommander.internal.Lists;
> +
> +class JDKClassCompiler implements ClassCompiler {
> +  static final org.slf4j.Logger logger =
> org.slf4j.LoggerFactory.getLogger(JDKClassCompiler.class);
> +
> +  private boolean debugLines;
> +  private boolean debugVars;
> +  private boolean debugSource;
> +  private Collection<String> compilerOptions = new ArrayList<String>();
> +  private DiagnosticListener<JavaFileObject> listener;
> +  private final JavaCompiler compiler;
> +  private final JavaFileManager fileManager;
> +
> +  public JDKClassCompiler() {
> +    this.compiler = ToolProvider.getSystemJavaCompiler();
> +    if (compiler == null) {
> +      throw new UnsupportedOperationException(
> +          "JDK Java compiler not available - probably you're running a
> JRE, not a JDK");
> +    }
> +    this.fileManager = new
> ByteArrayJavaFileManager<JavaFileManager>(compiler.getStandardFileManager(null,
> null, null));
> +    this.listener = new DiagListener();
> +  }
> +
> +  private JavaFileObject getCompilationUnit(final String s) {
> +
> +    final URI uri;
> +    try {
> +      uri = new URI("drill-class-compiler");
> +    } catch (URISyntaxException use) {
> +      throw new RuntimeException(use);
> +    }
> +    JavaFileObject javaFileObject = new SimpleJavaFileObject(uri,
> Kind.SOURCE) {
> +
> +      @Override
> +      public boolean isNameCompatible(String simpleName, Kind kind) {
> +        return true;
> +      }
> +
> +      @Override
> +      public Reader openReader(boolean ignoreEncodingErrors) throws
> IOException {
> +        return new StringReader(s);
> +      }
> +
> +      @Override
> +      public CharSequence getCharContent(boolean ignoreEncodingErrors)
> throws IOException {
> +        return s;
> +      }
> +
> +    };
> +
> +    return javaFileObject;
> +  }
> +
> +  private List<String> getOptions() {
> +    List<String> opts = Lists.newArrayList(compilerOptions);
> +    String option = this.debugSource ? "-g:source" + (this.debugLines ?
> ",lines" : "")
> +        + (this.debugVars ? ",vars" : "") : this.debugLines ? "-g:lines"
> + (this.debugVars ? ",vars" : "")
> +        : this.debugVars ? "-g:vars" : "-g:none";
> +    opts.add(option);
> +    return opts;
> +  }
> +
> +  /* (non-Javadoc)
> +   * @see
> org.apache.drill.exec.compile.ClassCompiler#getClassByteCode(java.lang.String,
> java.lang.String)
> +   */
> +  @Override
> +  public byte[] getClassByteCode(final String className, final String
> sourcecode) throws CompileException, IOException,
> +      ClassNotFoundException {
> +
> +    // Create one Java source file in memory, which will be compiled
> later.
> +    JavaFileObject compilationUnit = getCompilationUnit(sourcecode);
> +
> +    //logger.debug("Compiling the following source code\n{}", sourcecode);
> +    // Run the compiler.
> +    try {
> +      CompilationTask task = compiler.getTask(null, fileManager,
> listener, getOptions(), null, Collections.singleton(compilationUnit));
> +      long n0 = System.nanoTime();
> +      if(!task.call()){
> +        throw new CompileException("Compilation failed", null);
> +      }
> +      long n1 = (System.nanoTime() - n0)/1000/1000;
> +
> +    } catch (RuntimeException rte) {
> +
> +      // Unwrap the compilation exception and throw it.
> +      Throwable cause = rte.getCause();
> +      if (cause != null) {
> +        cause = cause.getCause();
> +        if (cause instanceof CompileException) throw (CompileException)
> cause;
> +        if (cause instanceof IOException) throw (IOException) cause;
> +      }
> +      throw rte;
> +    }
> +
> +    JavaFileObject classFileObject =
> fileManager.getJavaFileForInput(StandardLocation.CLASS_OUTPUT, className,
> Kind.CLASS);
> +
> +    if (classFileObject == null) {
> +      throw new ClassNotFoundException(className + ": Class file not
> created by compilation");
> +    }
> +
> +    if (!(classFileObject instanceof ByteArrayJavaFileObject))
> +      throw new UnsupportedOperationException("Only supports byte array
> based java file objects.");
> +
> +    ByteArrayJavaFileObject bajfo = (ByteArrayJavaFileObject)
> classFileObject;
> +    return bajfo.toByteArray();
> +
> +  }
> +
> +  private class DiagListener implements
> DiagnosticListener<JavaFileObject> {
> +    @Override
> +    public void report(Diagnostic<? extends JavaFileObject> diagnostic) {
> +      System.err.println("*** " + diagnostic.toString() + " *** " +
> diagnostic.getCode());
> +
> +      Location loc = new Location( //
> +          diagnostic.getSource().toString(), //
> +          (short) diagnostic.getLineNumber(), //
> +          (short) diagnostic.getColumnNumber() //
> +      );
> +      String code = diagnostic.getCode();
> +      String message = diagnostic.getMessage(null) + " (" + code + ")";
> +
> +      // Wrap the exception in a RuntimeException, because "report()"
> does not declare checked
> +      // exceptions.
> +      throw new RuntimeException(new CompileException(message, loc));
> +    }
> +  }
> +
> +
> +
> +
> +
> +}
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/compile/JaninoClassCompiler.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/compile/JaninoClassCompiler.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/compile/JaninoClassCompiler.java
> new file mode 100644
> index 0000000..86fe58b
> --- /dev/null
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/compile/JaninoClassCompiler.java
> @@ -0,0 +1,62 @@
>
> +/*******************************************************************************
> + * Licensed to the Apache Software Foundation (ASF) under one
> + * or more contributor license agreements.  See the NOTICE file
> + * distributed with this work for additional information
> + * regarding copyright ownership.  The ASF licenses this file
> + * to you under the Apache License, Version 2.0 (the
> + * "License"); you may not use this file except in compliance
> + * with the License.  You may obtain a copy of the License at
> + *
> + * http://www.apache.org/licenses/LICENSE-2.0
> + *
> + * Unless required by applicable law or agreed to in writing, software
> + * distributed under the License is distributed on an "AS IS" BASIS,
> + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
> implied.
> + * See the License for the specific language governing permissions and
> + * limitations under the License.
> +
> ******************************************************************************/
> +package org.apache.drill.exec.compile;
> +
> +import java.io.IOException;
> +import java.io.StringReader;
> +
> +import org.apache.drill.exec.exception.ClassTransformationException;
> +import org.codehaus.commons.compiler.CompileException;
> +import org.codehaus.janino.ClassLoaderIClassLoader;
> +import org.codehaus.janino.IClassLoader;
> +import org.codehaus.janino.Java;
> +import org.codehaus.janino.Parser;
> +import org.codehaus.janino.Scanner;
> +import org.codehaus.janino.UnitCompiler;
> +import org.codehaus.janino.util.ClassFile;
> +
> +public class JaninoClassCompiler implements ClassCompiler{
> +  static final org.slf4j.Logger logger =
> org.slf4j.LoggerFactory.getLogger(JaninoClassCompiler.class);
> +
> +  private IClassLoader compilationClassLoader;
> +
> +  private boolean debugLines;
> +  private boolean debugVars;
> +  private boolean debugSource;
> +
> +  public JaninoClassCompiler(ClassLoader parentClassLoader) {
> +    this.compilationClassLoader = new
> ClassLoaderIClassLoader(parentClassLoader);
> +  }
> +
> +  public byte[] getClassByteCode(final String className, final String
> code) throws CompileException, IOException, ClassNotFoundException,
> ClassTransformationException {
> +    StringReader reader = new StringReader(code);
> +    Scanner scanner = new Scanner((String) null, reader);
> +    Java.CompilationUnit compilationUnit = new
> Parser(scanner).parseCompilationUnit();
> +    ClassFile[] classFiles = new UnitCompiler(compilationUnit,
> compilationClassLoader).compileUnit(this.debugSource,
> +        this.debugLines, this.debugVars);
> +    if (classFiles.length != 1)
> +      throw new ClassTransformationException("Only one class file should
> have been generated from source code.");
> +    return classFiles[0].toByteArray();
> +  }
> +
> +  public void setDebuggingInformation(boolean debugSource, boolean
> debugLines, boolean debugVars) {
> +    this.debugSource = debugSource;
> +    this.debugLines = debugLines;
> +    this.debugVars = debugVars;
> +  }
> +}
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/compile/QueryClassLoader.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/compile/QueryClassLoader.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/compile/QueryClassLoader.java
> new file mode 100644
> index 0000000..60aa8f3
> --- /dev/null
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/compile/QueryClassLoader.java
> @@ -0,0 +1,80 @@
>
> +/*******************************************************************************
> + * Licensed to the Apache Software Foundation (ASF) under one
> + * or more contributor license agreements.  See the NOTICE file
> + * distributed with this work for additional information
> + * regarding copyright ownership.  The ASF licenses this file
> + * to you under the Apache License, Version 2.0 (the
> + * "License"); you may not use this file except in compliance
> + * with the License.  You may obtain a copy of the License at
> + *
> + * http://www.apache.org/licenses/LICENSE-2.0
> + *
> + * Unless required by applicable law or agreed to in writing, software
> + * distributed under the License is distributed on an "AS IS" BASIS,
> + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
> implied.
> + * See the License for the specific language governing permissions and
> + * limitations under the License.
> +
> ******************************************************************************/
> +package org.apache.drill.exec.compile;
> +
> +import java.io.IOException;
> +import java.io.InputStream;
> +import java.io.OutputStream;
> +import java.net.URL;
> +import java.net.URLClassLoader;
> +import java.util.concurrent.ConcurrentMap;
> +
> +import javax.tools.JavaCompiler;
> +import javax.tools.JavaFileManager;
> +import javax.tools.JavaFileObject;
> +import javax.tools.JavaFileObject.Kind;
> +import javax.tools.StandardLocation;
> +import javax.tools.ToolProvider;
> +
> +import org.apache.drill.exec.exception.ClassTransformationException;
> +import org.codehaus.commons.compiler.CompileException;
> +import org.codehaus.commons.compiler.jdk.ByteArrayJavaFileManager;
> +import
> org.codehaus.commons.compiler.jdk.ByteArrayJavaFileManager.ByteArrayJavaFileObject;
> +
> +import com.google.common.collect.MapMaker;
> +import com.google.common.io.ByteStreams;
> +
> +public class QueryClassLoader extends URLClassLoader {
> +
> +  static final org.slf4j.Logger logger =
> org.slf4j.LoggerFactory.getLogger(QueryClassLoader.class);
> +
> +  private final ClassCompiler classCompiler;
> +  private ConcurrentMap<String, byte[]> customClasses = new
> MapMaker().concurrencyLevel(4).makeMap();
> +
> +  public QueryClassLoader(boolean useJanino) {
> +    super(new URL[0]);
> +    if (useJanino) {
> +      this.classCompiler = new JaninoClassCompiler(this);
> +    } else {
> +      this.classCompiler = new JDKClassCompiler();
> +    }
> +  }
> +
> +  public void injectByteCode(String className, byte[] classBytes) throws
> IOException {
> +    if(customClasses.containsKey(className)) throw new
> IOException(String.format("The class defined {} has already been loaded.",
> className));
> +    customClasses.put(className, classBytes);
> +  }
> +
> +  @Override
> +  protected Class<?> findClass(String className) throws
> ClassNotFoundException {
> +    byte[] ba = customClasses.get(className);
> +    if(ba != null){
> +      return this.defineClass(className, ba, 0, ba.length);
> +    }else{
> +      return super.findClass(className);
> +    }
> +  }
> +
> +  public byte[] getClassByteCode(final String className, final String
> sourcecode) throws CompileException, IOException,
> +      ClassNotFoundException, ClassTransformationException {
> +    byte[] bc = classCompiler.getClassByteCode(className, sourcecode);
> +    return bc;
> +
> +  }
> +
> +}
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/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
> new file mode 100644
> index 0000000..fee4c97
> --- /dev/null
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/compile/TemplateClassDefinition.java
> @@ -0,0 +1,58 @@
>
> +/*******************************************************************************
> + * Licensed to the Apache Software Foundation (ASF) under one
> + * or more contributor license agreements.  See the NOTICE file
> + * distributed with this work for additional information
> + * regarding copyright ownership.  The ASF licenses this file
> + * to you under the Apache License, Version 2.0 (the
> + * "License"); you may not use this file except in compliance
> + * with the License.  You may obtain a copy of the License at
> + *
> + * http://www.apache.org/licenses/LICENSE-2.0
> + *
> + * Unless required by applicable law or agreed to in writing, software
> + * distributed under the License is distributed on an "AS IS" BASIS,
> + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
> implied.
> + * See the License for the specific language governing permissions and
> + * limitations under the License.
> +
> ******************************************************************************/
> +package org.apache.drill.exec.compile;
> +
> +import java.lang.reflect.Method;
> +
> +
> +public class TemplateClassDefinition<T, I>{
> +
> +  private final Class<T> externalInterface;
> +  private final String templateClassName;
> +  private final Class<?> internalInterface;
> +  private final Class<I> constructorObject;
> +
> +  public TemplateClassDefinition(Class<T> externalInterface, String
> templateClassName, Class<?> internalInterface, Class<I> constructorObject) {
> +    super();
> +    this.externalInterface = externalInterface;
> +    this.templateClassName = templateClassName;
> +    this.internalInterface = internalInterface;
> +    this.constructorObject = constructorObject;
> +  }
> +
> +  public Class<T> getExternalInterface() {
> +    return externalInterface;
> +  }
> +
> +
> +  public Class<?> getInternalInterface() {
> +    return internalInterface;
> +  }
> +
> +  public String getTemplateClassName() {
> +    return templateClassName;
> +  }
> +
> +  public Class<I> getConstructorObject() {
> +    return constructorObject;
> +  }
> +
> +
> +
> +
> +}
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/coord/ClusterCoordinator.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/coord/ClusterCoordinator.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/coord/ClusterCoordinator.java
> index d7ea8fa..d3580b5 100644
> ---
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/coord/ClusterCoordinator.java
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/coord/ClusterCoordinator.java
> @@ -17,11 +17,11 @@
>
> ******************************************************************************/
>  package org.apache.drill.exec.coord;
>
> -import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
> -
>  import java.io.Closeable;
>  import java.util.Collection;
>
> +import org.apache.drill.common.proto.CoordinationProtos.DrillbitEndpoint;
> +
>  /**
>   * Pluggable interface built to manage cluster coordination. Allows
> Drillbit or DrillClient to register its capabilities
>   * as well as understand other node's existence and capabilities.
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/coord/DrillServiceInstanceHelper.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/coord/DrillServiceInstanceHelper.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/coord/DrillServiceInstanceHelper.java
> index 289aa3c..ce0fb92 100644
> ---
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/coord/DrillServiceInstanceHelper.java
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/coord/DrillServiceInstanceHelper.java
> @@ -17,9 +17,9 @@
>
> ******************************************************************************/
>  package org.apache.drill.exec.coord;
>
> +import
> org.apache.drill.common.proto.CoordinationProtos.DrillServiceInstance;
> +import org.apache.drill.common.proto.CoordinationProtos.DrillbitEndpoint;
>  import org.apache.drill.exec.ExecConstants;
> -import
> org.apache.drill.exec.proto.CoordinationProtos.DrillServiceInstance;
> -import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
>
>  import com.netflix.curator.x.discovery.ServiceInstance;
>  import com.netflix.curator.x.discovery.ServiceInstanceBuilder;
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/coord/ZKClusterCoordinator.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/coord/ZKClusterCoordinator.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/coord/ZKClusterCoordinator.java
> index 3ad08e1..85c573d 100644
> ---
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/coord/ZKClusterCoordinator.java
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/coord/ZKClusterCoordinator.java
> @@ -17,6 +17,17 @@
>
> ******************************************************************************/
>  package org.apache.drill.exec.coord;
>
> +import static com.google.common.base.Throwables.propagate;
> +import static com.google.common.collect.Collections2.transform;
> +
> +import java.io.IOException;
> +import java.util.Collection;
> +import java.util.Collections;
> +
> +import org.apache.drill.common.config.DrillConfig;
> +import org.apache.drill.common.proto.CoordinationProtos.DrillbitEndpoint;
> +import org.apache.drill.exec.ExecConstants;
> +
>  import com.google.common.base.Function;
>  import com.netflix.curator.RetryPolicy;
>  import com.netflix.curator.framework.CuratorFramework;
> @@ -28,16 +39,6 @@ import
> com.netflix.curator.x.discovery.ServiceDiscoveryBuilder;
>  import com.netflix.curator.x.discovery.ServiceInstance;
>  import com.netflix.curator.x.discovery.details.ServiceCache;
>  import com.netflix.curator.x.discovery.details.ServiceCacheListener;
> -import org.apache.drill.common.config.DrillConfig;
> -import org.apache.drill.exec.ExecConstants;
> -import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
> -
> -import java.io.IOException;
> -import java.util.Collection;
> -import java.util.Collections;
> -
> -import static com.google.common.base.Throwables.propagate;
> -import static com.google.common.collect.Collections2.transform;
>
>  /**
>   * Manages cluster coordination utilizing zookeeper. *
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/exception/ClassTransformationException.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/exception/ClassTransformationException.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/exception/ClassTransformationException.java
> new file mode 100644
> index 0000000..13ec95e
> --- /dev/null
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/exception/ClassTransformationException.java
> @@ -0,0 +1,47 @@
>
> +/*******************************************************************************
> + * Licensed to the Apache Software Foundation (ASF) under one
> + * or more contributor license agreements.  See the NOTICE file
> + * distributed with this work for additional information
> + * regarding copyright ownership.  The ASF licenses this file
> + * to you under the Apache License, Version 2.0 (the
> + * "License"); you may not use this file except in compliance
> + * with the License.  You may obtain a copy of the License at
> + *
> + * http://www.apache.org/licenses/LICENSE-2.0
> + *
> + * Unless required by applicable law or agreed to in writing, software
> + * distributed under the License is distributed on an "AS IS" BASIS,
> + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
> implied.
> + * See the License for the specific language governing permissions and
> + * limitations under the License.
> +
> ******************************************************************************/
> +package org.apache.drill.exec.exception;
> +
> +import org.apache.drill.common.exceptions.DrillException;
> +
> +public class ClassTransformationException extends DrillException{
> +  static final org.slf4j.Logger logger =
> org.slf4j.LoggerFactory.getLogger(ClassTransformationException.class);
> +
> +  public ClassTransformationException() {
> +    super();
> +  }
> +
> +  public ClassTransformationException(String message, Throwable cause,
> boolean enableSuppression,
> +      boolean writableStackTrace) {
> +    super(message, cause, enableSuppression, writableStackTrace);
> +  }
> +
> +  public ClassTransformationException(String message, Throwable cause) {
> +    super(message, cause);
> +  }
> +
> +  public ClassTransformationException(String message) {
> +    super(message);
> +  }
> +
> +  public ClassTransformationException(Throwable cause) {
> +    super(cause);
> +  }
> +
> +
> +}
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/exception/FragmentSetupException.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/exception/FragmentSetupException.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/exception/FragmentSetupException.java
> new file mode 100644
> index 0000000..c273463
> --- /dev/null
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/exception/FragmentSetupException.java
> @@ -0,0 +1,42 @@
>
> +/*******************************************************************************
> + * Licensed to the Apache Software Foundation (ASF) under one
> + * or more contributor license agreements.  See the NOTICE file
> + * distributed with this work for additional information
> + * regarding copyright ownership.  The ASF licenses this file
> + * to you under the Apache License, Version 2.0 (the
> + * "License"); you may not use this file except in compliance
> + * with the License.  You may obtain a copy of the License at
> + *
> + * http://www.apache.org/licenses/LICENSE-2.0
> + *
> + * Unless required by applicable law or agreed to in writing, software
> + * distributed under the License is distributed on an "AS IS" BASIS,
> + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
> implied.
> + * See the License for the specific language governing permissions and
> + * limitations under the License.
> +
> ******************************************************************************/
> +package org.apache.drill.exec.exception;
> +
> +public class FragmentSetupException extends ExecutionSetupException{
> +  static final org.slf4j.Logger logger =
> org.slf4j.LoggerFactory.getLogger(FragmentSetupException.class);
> +
> +  public FragmentSetupException() {
> +    super();
> +  }
> +
> +  public FragmentSetupException(String message, Throwable cause, boolean
> enableSuppression, boolean writableStackTrace) {
> +    super(message, cause, enableSuppression, writableStackTrace);
> +  }
> +
> +  public FragmentSetupException(String message, Throwable cause) {
> +    super(message, cause);
> +  }
> +
> +  public FragmentSetupException(String message) {
> +    super(message);
> +  }
> +
> +  public FragmentSetupException(Throwable cause) {
> +    super(cause);
> +  }
> +}
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/foreman/CancelableQuery.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/foreman/CancelableQuery.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/foreman/CancelableQuery.java
> new file mode 100644
> index 0000000..30e7a63
> --- /dev/null
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/foreman/CancelableQuery.java
> @@ -0,0 +1,22 @@
>
> +/*******************************************************************************
> + * Licensed to the Apache Software Foundation (ASF) under one
> + * or more contributor license agreements.  See the NOTICE file
> + * distributed with this work for additional information
> + * regarding copyright ownership.  The ASF licenses this file
> + * to you under the Apache License, Version 2.0 (the
> + * "License"); you may not use this file except in compliance
> + * with the License.  You may obtain a copy of the License at
> + *
> + * http://www.apache.org/licenses/LICENSE-2.0
> + *
> + * Unless required by applicable law or agreed to in writing, software
> + * distributed under the License is distributed on an "AS IS" BASIS,
> + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
> implied.
> + * See the License for the specific language governing permissions and
> + * limitations under the License.
> +
> ******************************************************************************/
> +package org.apache.drill.exec.foreman;
> +
> +public interface CancelableQuery {
> +  public boolean cancel(long queryid);
> +}
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/foreman/ExecutionPlanner.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/foreman/ExecutionPlanner.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/foreman/ExecutionPlanner.java
> new file mode 100644
> index 0000000..4e4ec77
> --- /dev/null
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/foreman/ExecutionPlanner.java
> @@ -0,0 +1,24 @@
>
> +/*******************************************************************************
> + * Licensed to the Apache Software Foundation (ASF) under one
> + * or more contributor license agreements.  See the NOTICE file
> + * distributed with this work for additional information
> + * regarding copyright ownership.  The ASF licenses this file
> + * to you under the Apache License, Version 2.0 (the
> + * "License"); you may not use this file except in compliance
> + * with the License.  You may obtain a copy of the License at
> + *
> + * http://www.apache.org/licenses/LICENSE-2.0
> + *
> + * Unless required by applicable law or agreed to in writing, software
> + * distributed under the License is distributed on an "AS IS" BASIS,
> + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
> implied.
> + * See the License for the specific language governing permissions and
> + * limitations under the License.
> +
> ******************************************************************************/
> +package org.apache.drill.exec.foreman;
> +
> +public class ExecutionPlanner {
> +  static final org.slf4j.Logger logger =
> org.slf4j.LoggerFactory.getLogger(ExecutionPlanner.class);
> +
> +
> +}
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/foreman/Foreman.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/foreman/Foreman.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/foreman/Foreman.java
> new file mode 100644
> index 0000000..f138171
> --- /dev/null
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/foreman/Foreman.java
> @@ -0,0 +1,39 @@
>
> +/*******************************************************************************
> + * Licensed to the Apache Software Foundation (ASF) under one
> + * or more contributor license agreements.  See the NOTICE file
> + * distributed with this work for additional information
> + * regarding copyright ownership.  The ASF licenses this file
> + * to you under the Apache License, Version 2.0 (the
> + * "License"); you may not use this file except in compliance
> + * with the License.  You may obtain a copy of the License at
> + *
> + * http://www.apache.org/licenses/LICENSE-2.0
> + *
> + * Unless required by applicable law or agreed to in writing, software
> + * distributed under the License is distributed on an "AS IS" BASIS,
> + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
> implied.
> + * See the License for the specific language governing permissions and
> + * limitations under the License.
> +
> ******************************************************************************/
> +package org.apache.drill.exec.foreman;
> +
> +
> +public class Foreman extends Thread{
> +  static final org.slf4j.Logger logger =
> org.slf4j.LoggerFactory.getLogger(Foreman.class);
> +
> +  public Foreman(){
> +
> +  }
> +
> +  public void doWork(QueryWorkUnit work){
> +    // generate fragment structure.
> +    // store fragments in distributed grid.
> +    // generate any codegen required and store in grid.
> +    // drop
> +    // do get on the result set you're looking for.  Do the initial get
> on the result node you're looking for.  This will return either data or a
> metadata record set
> +  }
> +
> +  public boolean checkStatus(long queryId){
> +    return false;
> +  }
> +}
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/foreman/QueryWorkUnit.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/foreman/QueryWorkUnit.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/foreman/QueryWorkUnit.java
> new file mode 100644
> index 0000000..bdf4a1e
> --- /dev/null
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/foreman/QueryWorkUnit.java
> @@ -0,0 +1,54 @@
>
> +/*******************************************************************************
> + * Licensed to the Apache Software Foundation (ASF) under one
> + * or more contributor license agreements.  See the NOTICE file
> + * distributed with this work for additional information
> + * regarding copyright ownership.  The ASF licenses this file
> + * to you under the Apache License, Version 2.0 (the
> + * "License"); you may not use this file except in compliance
> + * with the License.  You may obtain a copy of the License at
> + *
> + * http://www.apache.org/licenses/LICENSE-2.0
> + *
> + * Unless required by applicable law or agreed to in writing, software
> + * distributed under the License is distributed on an "AS IS" BASIS,
> + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
> implied.
> + * See the License for the specific language governing permissions and
> + * limitations under the License.
> +
> ******************************************************************************/
> +package org.apache.drill.exec.foreman;
> +
> +import java.util.Collection;
> +import java.util.List;
> +
> +import org.apache.drill.exec.proto.ExecProtos.PlanFragment;
> +
> +import com.google.common.base.Preconditions;
> +
> +public class QueryWorkUnit {
> +  static final org.slf4j.Logger logger =
> org.slf4j.LoggerFactory.getLogger(QueryWorkUnit.class);
> +
> +  private PlanFragment rootFragment; // for local
> +  private List<PlanFragment> fragments;
> +
> +  public QueryWorkUnit(PlanFragment rootFragment, List<PlanFragment>
> fragments) {
> +    super();
> +    Preconditions.checkNotNull(rootFragment);
> +    Preconditions.checkNotNull(fragments);
> +    this.rootFragment = rootFragment;
> +    this.fragments = fragments;
> +  }
> +
> +  public PlanFragment getRootFragment() {
> +    return rootFragment;
> +  }
> +
> +  public List<PlanFragment> getFragments() {
> +    return fragments;
> +  }
> +
> +
> +
> +
> +
> +
> +}
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/foreman/ResourceRequest.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/foreman/ResourceRequest.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/foreman/ResourceRequest.java
> new file mode 100644
> index 0000000..96d7d1e
> --- /dev/null
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/foreman/ResourceRequest.java
> @@ -0,0 +1,30 @@
>
> +/*******************************************************************************
> + * Licensed to the Apache Software Foundation (ASF) under one
> + * or more contributor license agreements.  See the NOTICE file
> + * distributed with this work for additional information
> + * regarding copyright ownership.  The ASF licenses this file
> + * to you under the Apache License, Version 2.0 (the
> + * "License"); you may not use this file except in compliance
> + * with the License.  You may obtain a copy of the License at
> + *
> + * http://www.apache.org/licenses/LICENSE-2.0
> + *
> + * Unless required by applicable law or agreed to in writing, software
> + * distributed under the License is distributed on an "AS IS" BASIS,
> + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
> implied.
> + * See the License for the specific language governing permissions and
> + * limitations under the License.
> +
> ******************************************************************************/
> +package org.apache.drill.exec.foreman;
> +
> +public class ResourceRequest {
> +  static final org.slf4j.Logger logger =
> org.slf4j.LoggerFactory.getLogger(ResourceRequest.class);
> +
> +  public long memoryMin;
> +  public long memoryDesired;
> +
> +
> +  public static class ResourceAllocation {
> +    public long memory;
> +  }
> +}
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/foreman/StatusProvider.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/foreman/StatusProvider.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/foreman/StatusProvider.java
> new file mode 100644
> index 0000000..fee6172
> --- /dev/null
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/foreman/StatusProvider.java
> @@ -0,0 +1,24 @@
>
> +/*******************************************************************************
> + * Licensed to the Apache Software Foundation (ASF) under one
> + * or more contributor license agreements.  See the NOTICE file
> + * distributed with this work for additional information
> + * regarding copyright ownership.  The ASF licenses this file
> + * to you under the Apache License, Version 2.0 (the
> + * "License"); you may not use this file except in compliance
> + * with the License.  You may obtain a copy of the License at
> + *
> + * http://www.apache.org/licenses/LICENSE-2.0
> + *
> + * Unless required by applicable law or agreed to in writing, software
> + * distributed under the License is distributed on an "AS IS" BASIS,
> + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
> implied.
> + * See the License for the specific language governing permissions and
> + * limitations under the License.
> +
> ******************************************************************************/
> +package org.apache.drill.exec.foreman;
> +
> +import org.apache.drill.exec.proto.ExecProtos.FragmentStatus;
> +
> +public interface StatusProvider {
> +  public FragmentStatus getStatus();
> +}
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/memory/BufferAllocator.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/memory/BufferAllocator.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/memory/BufferAllocator.java
> new file mode 100644
> index 0000000..2b3f574
> --- /dev/null
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/memory/BufferAllocator.java
> @@ -0,0 +1,58 @@
>
> +/*******************************************************************************
> + * Licensed to the Apache Software Foundation (ASF) under one
> + * or more contributor license agreements.  See the NOTICE file
> + * distributed with this work for additional information
> + * regarding copyright ownership.  The ASF licenses this file
> + * to you under the Apache License, Version 2.0 (the
> + * "License"); you may not use this file except in compliance
> + * with the License.  You may obtain a copy of the License at
> + *
> + * http://www.apache.org/licenses/LICENSE-2.0
> + *
> + * Unless required by applicable law or agreed to in writing, software
> + * distributed under the License is distributed on an "AS IS" BASIS,
> + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
> implied.
> + * See the License for the specific language governing permissions and
> + * limitations under the License.
> +
> ******************************************************************************/
> +package org.apache.drill.exec.memory;
> +
> +import io.netty.buffer.ByteBuf;
> +import io.netty.buffer.ByteBufAllocator;
> +import io.netty.buffer.PooledByteBufAllocator;
> +
> +import java.io.Closeable;
> +
> +import org.apache.drill.exec.server.DrillbitContext;
> +
> +/**
> + * Wrapper class to deal with byte buffer allocation. Ensures users only
> use designated methods.  Also allows inser
> + */
> +public abstract class BufferAllocator implements Closeable{
> +  static final org.slf4j.Logger logger =
> org.slf4j.LoggerFactory.getLogger(BufferAllocator.class);
> +
> +  /**
> +   * Allocate a new or reused buffer of the provided size.  Note that the
> buffer may technically be larger than the requested size for rounding
> purposes.  However, the buffers capacity will be set to the configured size.
> +   * @param size The size in bytes.
> +   * @return A new ByteBuf.
> +   */
> +  public abstract ByteBuf buffer(int size);
> +
> +  public abstract ByteBufAllocator getUnderlyingAllocator();
> +
> +  public abstract BufferAllocator getChildAllocator(long
> initialReservation, long maximumReservation);
> +
> +  /**
> +   * Close and release all buffers generated from this buffer pool.
> +   */
> +  @Override
> +  public abstract void close();
> +
> +  public static BufferAllocator getAllocator(DrillbitContext context){
> +    // TODO: support alternative allocators (including a debugging
> allocator that records all allocation locations for each buffer).
> +    return new DirectBufferAllocator();
> +  }
> +
> +  public abstract long getAllocatedMemory();
> +
> +}
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/memory/DirectBufferAllocator.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/memory/DirectBufferAllocator.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/memory/DirectBufferAllocator.java
> new file mode 100644
> index 0000000..8c5b003
> --- /dev/null
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/memory/DirectBufferAllocator.java
> @@ -0,0 +1,58 @@
>
> +/*******************************************************************************
> + * Licensed to the Apache Software Foundation (ASF) under one
> + * or more contributor license agreements.  See the NOTICE file
> + * distributed with this work for additional information
> + * regarding copyright ownership.  The ASF licenses this file
> + * to you under the Apache License, Version 2.0 (the
> + * "License"); you may not use this file except in compliance
> + * with the License.  You may obtain a copy of the License at
> + *
> + * http://www.apache.org/licenses/LICENSE-2.0
> + *
> + * Unless required by applicable law or agreed to in writing, software
> + * distributed under the License is distributed on an "AS IS" BASIS,
> + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
> implied.
> + * See the License for the specific language governing permissions and
> + * limitations under the License.
> +
> ******************************************************************************/
> +package org.apache.drill.exec.memory;
> +
> +import io.netty.buffer.ByteBuf;
> +import io.netty.buffer.ByteBufAllocator;
> +import io.netty.buffer.PooledByteBufAllocator;
> +
> +public class DirectBufferAllocator extends BufferAllocator{
> +  static final org.slf4j.Logger logger =
> org.slf4j.LoggerFactory.getLogger(DirectBufferAllocator.class);
> +
> +  private final PooledByteBufAllocator buffer = new
> PooledByteBufAllocator(true);
> +
> +  @Override
> +  public ByteBuf buffer(int size) {
> +    // TODO: wrap it
> +    return buffer.directBuffer(size);
> +  }
> +
> +  @Override
> +  public long getAllocatedMemory() {
> +    return 0;
> +  }
> +
> +  @Override
> +  public ByteBufAllocator getUnderlyingAllocator() {
> +    return buffer;
> +  }
> +
> +
> +
> +  @Override
> +  public BufferAllocator getChildAllocator(long initialReservation, long
> maximumReservation) {
> +    //TODO: Add child account allocator.
> +    return this;
> +  }
> +
> +  @Override
> +  public void close() {
> +    // TODO: collect all buffers and release them away using a weak
> hashmap so we don't impact pool work
> +  }
> +
> +}
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/metrics/SingleThreadNestedCounter.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/metrics/SingleThreadNestedCounter.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/metrics/SingleThreadNestedCounter.java
> new file mode 100644
> index 0000000..6b89c12
> --- /dev/null
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/metrics/SingleThreadNestedCounter.java
> @@ -0,0 +1,55 @@
>
> +/*******************************************************************************
> + * Licensed to the Apache Software Foundation (ASF) under one
> + * or more contributor license agreements.  See the NOTICE file
> + * distributed with this work for additional information
> + * regarding copyright ownership.  The ASF licenses this file
> + * to you under the Apache License, Version 2.0 (the
> + * "License"); you may not use this file except in compliance
> + * with the License.  You may obtain a copy of the License at
> + *
> + * http://www.apache.org/licenses/LICENSE-2.0
> + *
> + * Unless required by applicable law or agreed to in writing, software
> + * distributed under the License is distributed on an "AS IS" BASIS,
> + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
> implied.
> + * See the License for the specific language governing permissions and
> + * limitations under the License.
> +
> ******************************************************************************/
> +package org.apache.drill.exec.metrics;
> +
> +import org.apache.drill.exec.server.DrillbitContext;
> +
> +import com.yammer.metrics.Counter;
> +
> +/**
> + * Wraps a parent counter so that local in thread metrics can be
> collected while collecting for a global counter.
> + */
> +public class SingleThreadNestedCounter {
> +  static final org.slf4j.Logger logger =
> org.slf4j.LoggerFactory.getLogger(SingleThreadNestedCounter.class);
> +
> +  private volatile long count;
> +  private final Counter counter;
> +
> +
> +  public SingleThreadNestedCounter(DrillbitContext context, String name) {
> +    super();
> +    this.counter = context.getMetrics().counter(name);
> +  }
> +
> +  public long inc(long n){
> +    counter.inc(n);
> +    count+= n;
> +    return count;
> +  }
> +
> +  public long dec(long n){
> +    counter.dec(n);
> +    count -= n;
> +    return count;
> +  }
> +
> +  public long get(){
> +    return count;
> +  }
> +
> +}
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/BatchIterator.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/BatchIterator.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/BatchIterator.java
> deleted file mode 100644
> index 2ebbef5..0000000
> ---
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/BatchIterator.java
> +++ /dev/null
> @@ -1,32 +0,0 @@
>
> -/*******************************************************************************
> - * Licensed to the Apache Software Foundation (ASF) under one
> - * or more contributor license agreements.  See the NOTICE file
> - * distributed with this work for additional information
> - * regarding copyright ownership.  The ASF licenses this file
> - * to you under the Apache License, Version 2.0 (the
> - * "License"); you may not use this file except in compliance
> - * with the License.  You may obtain a copy of the License at
> - *
> - * http://www.apache.org/licenses/LICENSE-2.0
> - *
> - * Unless required by applicable law or agreed to in writing, software
> - * distributed under the License is distributed on an "AS IS" BASIS,
> - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
> implied.
> - * See the License for the specific language governing permissions and
> - * limitations under the License.
> -
> ******************************************************************************/
> -package org.apache.drill.exec.ops;
> -
> -import org.apache.drill.exec.record.RecordBatch;
> -
> -import parquet.schema.MessageType;
> -
> -public interface BatchIterator {
> -  static enum IterOutcome{NONE, FULL_NEW_SCHEMA, FULL,
> PARTIAL_NEW_SCHEMA, PARTIAL, STOP}
> -  public RecordBatch getBatch();
> -  public FragmentContext getContext();
> -  public MessageType getSchema();
> -  public void kill(QueryOutcome outcome);
> -  public IterOutcome next();
> -
> -}
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/FilteringRecordBatchTransformer.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/FilteringRecordBatchTransformer.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/FilteringRecordBatchTransformer.java
> new file mode 100644
> index 0000000..f626cea
> --- /dev/null
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/FilteringRecordBatchTransformer.java
> @@ -0,0 +1,58 @@
>
> +/*******************************************************************************
> + * Licensed to the Apache Software Foundation (ASF) under one
> + * or more contributor license agreements.  See the NOTICE file
> + * distributed with this work for additional information
> + * regarding copyright ownership.  The ASF licenses this file
> + * to you under the Apache License, Version 2.0 (the
> + * "License"); you may not use this file except in compliance
> + * with the License.  You may obtain a copy of the License at
> + *
> + * http://www.apache.org/licenses/LICENSE-2.0
> + *
> + * Unless required by applicable law or agreed to in writing, software
> + * distributed under the License is distributed on an "AS IS" BASIS,
> + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
> implied.
> + * See the License for the specific language governing permissions and
> + * limitations under the License.
> +
> ******************************************************************************/
> +package org.apache.drill.exec.ops;
> +
> +import org.apache.drill.exec.record.BatchSchema;
> +import org.apache.drill.exec.record.RecordBatch;
> +import org.apache.drill.exec.record.vector.SelectionVector;
> +
> +public abstract class FilteringRecordBatchTransformer {
> +  static final org.slf4j.Logger logger =
> org.slf4j.LoggerFactory.getLogger(FilteringRecordBatchTransformer.class);
> +
> +  final RecordBatch incoming;
> +  final SelectionVector selectionVector;
> +  final BatchSchema schema;
> +
> +  public FilteringRecordBatchTransformer(RecordBatch incoming,
> OutputMutator output, SelectionVector selectionVector) {
> +    super();
> +    this.incoming = incoming;
> +    this.selectionVector = selectionVector;
> +    this.schema = innerSetup();
> +  }
> +
> +  public abstract BatchSchema innerSetup();
> +
> +  /**
> +   * Applies the filter to the selection index.  Ignores any values in
> the selection vector, instead creating a.
> +   * @return
> +   */
> +  public abstract int apply();
> +
> +  /**
> +   * Applies the filter to the selection index.  Utilizes the existing
> selection index and only evaluates on those records.
> +   * @return
> +   */
> +  public abstract int applyWithSelection();
> +
> +  public BatchSchema getSchema() {
> +    return schema;
> +  }
> +
> +
> +
> +}
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/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 be1081f..0cf17e9 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
> @@ -17,18 +17,42 @@
>
> ******************************************************************************/
>  package org.apache.drill.exec.ops;
>
> -import org.apache.drill.common.logical.StorageEngineConfig;
> +import org.apache.drill.common.expression.LogicalExpression;
> +import org.apache.drill.exec.memory.BufferAllocator;
> +import org.apache.drill.exec.metrics.SingleThreadNestedCounter;
> +import org.apache.drill.exec.planner.FragmentRunnable;
> +import org.apache.drill.exec.proto.ExecProtos.PlanFragment;
>  import org.apache.drill.exec.rpc.bit.BitCom;
>  import org.apache.drill.exec.server.DrillbitContext;
> -import org.apache.drill.exec.store.StorageEngine;
>
> +import com.yammer.metrics.MetricRegistry;
> +import com.yammer.metrics.Timer;
> +
> +/**
> + * Contextual objects required for execution of a particular fragment.
> + */
>  public class FragmentContext {
>    static final org.slf4j.Logger logger =
> org.slf4j.LoggerFactory.getLogger(FragmentContext.class);
>
> +  private final static String METRIC_TIMER_FRAGMENT_TIME =
> MetricRegistry.name(FragmentRunnable.class, "completionTimes");
> +  private final static String METRIC_BATCHES_COMPLETED =
> MetricRegistry.name(FragmentRunnable.class, "batchesCompleted");
> +  private final static String METRIC_RECORDS_COMPLETED =
> MetricRegistry.name(FragmentRunnable.class, "recordsCompleted");
> +  private final static String METRIC_DATA_PROCESSED =
> MetricRegistry.name(FragmentRunnable.class, "dataProcessed");
> +
>    private final DrillbitContext context;
> -
> -  public FragmentContext(DrillbitContext context) {
> -    this.context = context;
> +  private final PlanFragment fragment;
> +  public final SingleThreadNestedCounter batchesCompleted;
> +  public final SingleThreadNestedCounter recordsCompleted;
> +  public final SingleThreadNestedCounter dataProcessed;
> +  public final Timer fragmentTime;
> +
> +  public FragmentContext(DrillbitContext dbContext, PlanFragment
> fragment) {
> +    this.fragmentTime =
> dbContext.getMetrics().timer(METRIC_TIMER_FRAGMENT_TIME);
> +    this.batchesCompleted = new SingleThreadNestedCounter(dbContext,
> METRIC_BATCHES_COMPLETED);
> +    this.recordsCompleted = new SingleThreadNestedCounter(dbContext,
> METRIC_RECORDS_COMPLETED);
> +    this.dataProcessed = new SingleThreadNestedCounter(dbContext,
> METRIC_DATA_PROCESSED);
> +    this.context = dbContext;
> +    this.fragment = fragment;
>    }
>
>    public void fail(Throwable cause) {
> @@ -39,10 +63,21 @@ public class FragmentContext {
>      return context;
>    }
>
> -  public StorageEngine getStorageEngine(StorageEngineConfig config){
> +  public PlanFragment getFragment() {
> +    return fragment;
> +  }
> +
> +  public BufferAllocator getAllocator(){
> +    // TODO: A local query allocator to ensure memory limits and
> accurately gauge memory usage.
> +    return context.getAllocator();
> +  }
> +
> +
> +  public FilteringRecordBatchTransformer
> getFilteringExpression(LogicalExpression expr){
>      return null;
>    }
>
> +
>    public BitCom getCommunicator(){
>      return null;
>    }
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/FragmentConverter.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/FragmentConverter.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/FragmentConverter.java
> new file mode 100644
> index 0000000..3c75648
> --- /dev/null
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/FragmentConverter.java
> @@ -0,0 +1,30 @@
>
> +/*******************************************************************************
> + * Licensed to the Apache Software Foundation (ASF) under one
> + * or more contributor license agreements.  See the NOTICE file
> + * distributed with this work for additional information
> + * regarding copyright ownership.  The ASF licenses this file
> + * to you under the Apache License, Version 2.0 (the
> + * "License"); you may not use this file except in compliance
> + * with the License.  You may obtain a copy of the License at
> + *
> + * http://www.apache.org/licenses/LICENSE-2.0
> + *
> + * Unless required by applicable law or agreed to in writing, software
> + * distributed under the License is distributed on an "AS IS" BASIS,
> + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
> implied.
> + * See the License for the specific language governing permissions and
> + * limitations under the License.
> +
> ******************************************************************************/
> +package org.apache.drill.exec.ops;
> +
> +import org.apache.drill.exec.proto.ExecProtos.PlanFragment;
> +
> +public class FragmentConverter {
> +  static final org.slf4j.Logger logger =
> org.slf4j.LoggerFactory.getLogger(FragmentConverter.class);
> +
> +  public static FragmentRoot getFragment(FragmentContext context){
> +    PlanFragment m = context.getFragment();
> +
> +    return null;
> +  }
> +}
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/FragmentRoot.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/FragmentRoot.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/FragmentRoot.java
> new file mode 100644
> index 0000000..ddacb41
> --- /dev/null
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/FragmentRoot.java
> @@ -0,0 +1,37 @@
>
> +/*******************************************************************************
> + * Licensed to the Apache Software Foundation (ASF) under one
> + * or more contributor license agreements.  See the NOTICE file
> + * distributed with this work for additional information
> + * regarding copyright ownership.  The ASF licenses this file
> + * to you under the Apache License, Version 2.0 (the
> + * "License"); you may not use this file except in compliance
> + * with the License.  You may obtain a copy of the License at
> + *
> + * http://www.apache.org/licenses/LICENSE-2.0
> + *
> + * Unless required by applicable law or agreed to in writing, software
> + * distributed under the License is distributed on an "AS IS" BASIS,
> + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
> implied.
> + * See the License for the specific language governing permissions and
> + * limitations under the License.
> +
> ******************************************************************************/
> +package org.apache.drill.exec.ops;
> +
> +import org.apache.drill.exec.exception.FragmentSetupException;
> +
> +/**
> + * A FragmentRoot is a node which is the last processing node in a query
> plan. FragmentTerminals include Exchange
> + * output nodes and storage nodes.  They are there driving force behind
> the completion of a query.
> + */
> +public interface FragmentRoot {
> +  static final org.slf4j.Logger logger =
> org.slf4j.LoggerFactory.getLogger(FragmentRoot.class);
> +
> +  /**
> +   * Do the next batch of work.
> +   * @return Whether or not additional batches of work are necessary.
> +   */
> +  public boolean next();
> +
> +
> +  public void setup() throws FragmentSetupException;
> +}
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/OperatorFactory.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/OperatorFactory.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/OperatorFactory.java
> new file mode 100644
> index 0000000..8d4e807
> --- /dev/null
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/OperatorFactory.java
> @@ -0,0 +1,22 @@
>
> +/*******************************************************************************
> + * Licensed to the Apache Software Foundation (ASF) under one
> + * or more contributor license agreements.  See the NOTICE file
> + * distributed with this work for additional information
> + * regarding copyright ownership.  The ASF licenses this file
> + * to you under the Apache License, Version 2.0 (the
> + * "License"); you may not use this file except in compliance
> + * with the License.  You may obtain a copy of the License at
> + *
> + * http://www.apache.org/licenses/LICENSE-2.0
> + *
> + * Unless required by applicable law or agreed to in writing, software
> + * distributed under the License is distributed on an "AS IS" BASIS,
> + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
> implied.
> + * See the License for the specific language governing permissions and
> + * limitations under the License.
> +
> ******************************************************************************/
> +package org.apache.drill.exec.ops;
> +
> +public class OperatorFactory {
> +  static final org.slf4j.Logger logger =
> org.slf4j.LoggerFactory.getLogger(OperatorFactory.class);
> +}
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/QueryContext.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/QueryContext.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/QueryContext.java
> new file mode 100644
> index 0000000..fe37e70
> --- /dev/null
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/QueryContext.java
> @@ -0,0 +1,51 @@
>
> +/*******************************************************************************
> + * Licensed to the Apache Software Foundation (ASF) under one
> + * or more contributor license agreements.  See the NOTICE file
> + * distributed with this work for additional information
> + * regarding copyright ownership.  The ASF licenses this file
> + * to you under the Apache License, Version 2.0 (the
> + * "License"); you may not use this file except in compliance
> + * with the License.  You may obtain a copy of the License at
> + *
> + * http://www.apache.org/licenses/LICENSE-2.0
> + *
> + * Unless required by applicable law or agreed to in writing, software
> + * distributed under the License is distributed on an "AS IS" BASIS,
> + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
> implied.
> + * See the License for the specific language governing permissions and
> + * limitations under the License.
> +
> ******************************************************************************/
> +package org.apache.drill.exec.ops;
> +
> +import java.util.Collection;
> +
> +import org.apache.drill.common.proto.CoordinationProtos.DrillbitEndpoint;
> +import org.apache.drill.exec.server.DrillbitContext;
> +
> +import com.fasterxml.jackson.databind.ObjectMapper;
> +
> +public class QueryContext {
> +  static final org.slf4j.Logger logger =
> org.slf4j.LoggerFactory.getLogger(QueryContext.class);
> +
> +  private long queryId;
> +  private DrillbitContext drillbitContext;
> +
> +  public QueryContext(long queryId, DrillbitContext drllbitContext) {
> +    super();
> +    this.queryId = queryId;
> +    this.drillbitContext = drllbitContext;
> +  }
> +
> +  public long getQueryId() {
> +    return queryId;
> +  }
> +
> +  public ObjectMapper getMapper(){
> +    return drillbitContext.getConfig().getMapper();
> +  }
> +
> +  public Collection<DrillbitEndpoint> getActiveEndpoints(){
> +    return drillbitContext.getBits();
> +  }
> +
> +}
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/QueryOutcome.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/QueryOutcome.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/QueryOutcome.java
> deleted file mode 100644
> index b737f7c..0000000
> ---
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/QueryOutcome.java
> +++ /dev/null
> @@ -1,22 +0,0 @@
>
> -/*******************************************************************************
> - * Licensed to the Apache Software Foundation (ASF) under one
> - * or more contributor license agreements.  See the NOTICE file
> - * distributed with this work for additional information
> - * regarding copyright ownership.  The ASF licenses this file
> - * to you under the Apache License, Version 2.0 (the
> - * "License"); you may not use this file except in compliance
> - * with the License.  You may obtain a copy of the License at
> - *
> - * http://www.apache.org/licenses/LICENSE-2.0
> - *
> - * Unless required by applicable law or agreed to in writing, software
> - * distributed under the License is distributed on an "AS IS" BASIS,
> - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
> implied.
> - * See the License for the specific language governing permissions and
> - * limitations under the License.
> -
> ******************************************************************************/
> -package org.apache.drill.exec.ops;
> -
> -public class QueryOutcome {
> -  static final org.slf4j.Logger logger =
> org.slf4j.LoggerFactory.getLogger(QueryOutcome.class);
> -}
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/ScanBatch.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/ScanBatch.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/ScanBatch.java
> index 88b8af2..b46804f 100644
> ---
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/ScanBatch.java
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/ScanBatch.java
> @@ -33,7 +33,7 @@ import
> com.carrotsearch.hppc.procedures.IntObjectProcedure;
>  /**
>   * Record batch used for a particular scan. Operators against one or more
>   */
> -public class ScanBatch implements RecordBatch {
> +public abstract class ScanBatch implements RecordBatch {
>    static final org.slf4j.Logger logger =
> org.slf4j.LoggerFactory.getLogger(ScanBatch.class);
>
>    private IntObjectOpenHashMap<ValueVector<?>> fields = new
> IntObjectOpenHashMap<ValueVector<?>>();
>
>

[04/53] [abbrv] WIP fragmentation, physical plan, byte compiling, some vector work

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/compile/ClassBodyBuilder.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/compile/ClassBodyBuilder.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/compile/ClassBodyBuilder.java
new file mode 100644
index 0000000..ac44484
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/compile/ClassBodyBuilder.java
@@ -0,0 +1,247 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.compile;
+
+/*
+ * Janino - An embedded Java[TM] compiler
+ *
+ * Copyright (c) 2001-2010, Arno Unkrig
+ * All rights reserved.
+ *
+ * Redistribution and use in source and binary forms, with or without modification, are permitted provided that the
+ * following conditions are met:
+ *
+ *    1. Redistributions of source code must retain the above copyright notice, this list of conditions and the
+ *       following disclaimer.
+ *    2. Redistributions in binary form must reproduce the above copyright notice, this list of conditions and the
+ *       following disclaimer in the documentation and/or other materials provided with the distribution.
+ *    3. The name of the author may not be used to endorse or promote products derived from this software without
+ *       specific prior written permission.
+ *
+ * THIS SOFTWARE IS PROVIDED BY THE AUTHOR ``AS IS'' AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED
+ * TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL
+ * THE AUTHOR BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING,
+ * BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS
+ * INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+ * (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE
+ * POSSIBILITY OF SUCH DAMAGE.
+ */
+
+import java.io.BufferedReader;
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.io.Reader;
+import java.io.StringReader;
+import java.io.StringWriter;
+import java.nio.CharBuffer;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+import org.codehaus.commons.compiler.CompileException;
+import org.codehaus.commons.compiler.IClassBodyEvaluator;
+import org.codehaus.commons.io.MultiReader;
+
+public class ClassBodyBuilder {
+
+  private String[] optionalDefaultImports = null;
+  private String className = IClassBodyEvaluator.DEFAULT_CLASS_NAME;
+  private Class<?> optionalExtendedType = null;
+  private Class<?>[] implementedTypes = new Class[0];
+  private String[] imports = {};
+  private String body;
+  private boolean used = false;
+  
+  public static ClassBodyBuilder newBuilder(){
+    return new ClassBodyBuilder();
+  }
+  
+  private ClassBodyBuilder(){
+  }
+  
+  public ClassBodyBuilder setClassName(String className) {
+    assertNotCooked();
+    this.className = className;
+    return this;
+  }
+
+  public ClassBodyBuilder setDefaultImports(String... optionalDefaultImports) {
+    assertNotCooked();
+    this.optionalDefaultImports = optionalDefaultImports;
+    return this;
+  }
+
+  public ClassBodyBuilder setExtendedClass(Class<?> optionalExtendedType) {
+    assertNotCooked();
+    this.optionalExtendedType = optionalExtendedType;
+    return this;
+  }
+
+  public ClassBodyBuilder setImplementedInterfaces(Class<?>... implementedTypes) {
+    assertNotCooked();
+    this.implementedTypes = implementedTypes;
+    return this;
+  }
+
+  private void assertNotCooked() {
+    assert !used;
+  }
+  
+  public ClassBodyBuilder setImports(String[] imports) {
+    assertNotCooked();
+    this.imports = imports;
+    return this;
+  }
+
+  public ClassBodyBuilder setBody(String body) {
+    assertNotCooked();
+    this.body = body;
+    return this;
+  }
+
+  public String build() throws CompileException, IOException {
+    used = true;
+    // Wrap the class body in a compilation unit.
+    {
+      StringWriter sw1 = new StringWriter();
+      {
+        PrintWriter pw = new PrintWriter(sw1);
+
+        // Break the class name up into package name and simple class name.
+        String packageName; // null means default package.
+        String simpleClassName;
+        {
+          int idx = this.className.lastIndexOf('.');
+          if (idx == -1) {
+            packageName = "";
+            simpleClassName = this.className;
+          } else {
+            packageName = this.className.substring(0, idx);
+            simpleClassName = this.className.substring(idx + 1);
+          }
+        }
+
+        // Print PACKAGE directive.
+        if (!packageName.isEmpty()) {
+          pw.print("package ");
+          pw.print(packageName);
+          pw.println(";");
+        }
+
+        // Print default imports.
+        if (this.optionalDefaultImports != null) {
+          for (String defaultImport : this.optionalDefaultImports) {
+            pw.print("import ");
+            pw.print(defaultImport);
+            pw.println(";");
+          }
+        }
+
+        // Print imports as declared in the document.
+        for (String imporT : imports) {
+          pw.print("import ");
+          pw.print(imporT);
+          pw.println(";");
+        }
+
+        // Print the class declaration.
+        pw.print("public class ");
+        pw.print(simpleClassName);
+        if (this.optionalExtendedType != null) {
+          pw.print(" extends ");
+          pw.print(this.optionalExtendedType.getCanonicalName());
+        }
+        if (this.implementedTypes.length > 0) {
+          pw.print(" implements ");
+          pw.print(this.implementedTypes[0].getName());
+          for (int i = 1; i < this.implementedTypes.length; ++i) {
+            pw.print(", ");
+            pw.print(this.implementedTypes[i].getName());
+          }
+        }
+        pw.println(" {");
+        pw.close();
+      }
+
+      StringWriter sw2 = new StringWriter();
+      {
+        PrintWriter pw = new PrintWriter(sw2);
+        pw.println("}");
+        pw.close();
+      }
+
+      return sw1.toString() + body + sw2.toString();
+
+    }
+
+  }
+
+//  /**
+//   * Heuristically parse IMPORT declarations at the beginning of the character stream produced by the given
+//   * {@link Reader}. After this method returns, all characters up to and including that last IMPORT declaration have
+//   * been read from the {@link Reader}.
+//   * <p>
+//   * This method does not handle comments and string literals correctly, i.e. if a pattern that looks like an IMPORT
+//   * declaration appears within a comment or a string literal, it will be taken as an IMPORT declaration.
+//   * 
+//   * @param r
+//   *          A {@link Reader} that supports MARK, e.g. a {@link BufferedReader}
+//   * @return The parsed imports, e.g. {@code "java.util.*", "static java.util.Map.Entry" }
+//   */
+//  protected static String[] parseImportDeclarations(Reader r) throws IOException {
+//    final CharBuffer cb = CharBuffer.allocate(10000);
+//    r.mark(cb.limit());
+//    r.read(cb);
+//    cb.rewind();
+//
+//    List<String> imports = new ArrayList<String>();
+//    int afterLastImport = 0;
+//    for (Matcher matcher = IMPORT_STATEMENT_PATTERN.matcher(cb); matcher.find();) {
+//      imports.add(matcher.group(1));
+//      afterLastImport = matcher.end();
+//    }
+//    r.reset();
+//    r.skip(afterLastImport);
+//    return imports.toArray(new String[imports.size()]);
+//  }
+//
+//  private static final Pattern IMPORT_STATEMENT_PATTERN = Pattern.compile("\\bimport\\s+" + "(" + "(?:static\\s+)?"
+//      + "[\\p{javaLowerCase}\\p{javaUpperCase}_\\$][\\p{javaLowerCase}\\p{javaUpperCase}\\d_\\$]*"
+//      + "(?:\\.[\\p{javaLowerCase}\\p{javaUpperCase}_\\$][\\p{javaLowerCase}\\p{javaUpperCase}\\d_\\$]*)*"
+//      + "(?:\\.\\*)?" + ");");
+
+//  @Override
+//  public Object createInstance(Reader reader) throws CompileException, IOException {
+//    this.cook(reader);
+//    try {
+//      return this.getClazz().newInstance();
+//    } catch (InstantiationException ie) {
+//      CompileException ce = new CompileException(
+//          ("Class is abstract, an interface, an array class, a primitive type, or void; "
+//              + "or has no zero-parameter constructor"), null);
+//      ce.initCause(ie);
+//      throw ce;
+//    } catch (IllegalAccessException iae) {
+//      CompileException ce = new CompileException("The class or its zero-parameter constructor is not accessible", null);
+//      ce.initCause(iae);
+//      throw ce;
+//    }
+//  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/compile/ClassCompiler.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/compile/ClassCompiler.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/compile/ClassCompiler.java
new file mode 100644
index 0000000..0fb0115
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/compile/ClassCompiler.java
@@ -0,0 +1,29 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.compile;
+
+import java.io.IOException;
+
+import org.apache.drill.exec.exception.ClassTransformationException;
+import org.codehaus.commons.compiler.CompileException;
+
+interface ClassCompiler {
+
+  public abstract byte[] getClassByteCode(String className, String sourcecode) throws CompileException, IOException, ClassNotFoundException, ClassTransformationException ;
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/compile/ClassTransformer.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/compile/ClassTransformer.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/compile/ClassTransformer.java
new file mode 100644
index 0000000..814b239
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/compile/ClassTransformer.java
@@ -0,0 +1,210 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.compile;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.lang.reflect.Modifier;
+import java.net.URL;
+import java.util.Iterator;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+
+import org.apache.drill.exec.exception.ClassTransformationException;
+import org.codehaus.commons.compiler.CompileException;
+import org.objectweb.asm.ClassAdapter;
+import org.objectweb.asm.ClassReader;
+import org.objectweb.asm.ClassVisitor;
+import org.objectweb.asm.ClassWriter;
+import org.objectweb.asm.MethodVisitor;
+import org.objectweb.asm.commons.EmptyVisitor;
+import org.objectweb.asm.commons.RemappingClassAdapter;
+import org.objectweb.asm.commons.RemappingMethodAdapter;
+import org.objectweb.asm.commons.SimpleRemapper;
+import org.objectweb.asm.tree.ClassNode;
+import org.objectweb.asm.tree.FieldNode;
+import org.objectweb.asm.tree.MethodNode;
+import org.objectweb.asm.util.TraceClassVisitor;
+
+import com.google.common.cache.CacheBuilder;
+import com.google.common.cache.CacheLoader;
+import com.google.common.cache.LoadingCache;
+import com.google.common.io.Resources;
+
+public class ClassTransformer {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ClassTransformer.class);
+
+  private AtomicLong index = new AtomicLong(0);
+  private AtomicLong interfaceIndex = new AtomicLong(0);
+  private LoadingCache<String, byte[]> byteCode = CacheBuilder.newBuilder()
+      .maximumSize(10000)
+      .expireAfterWrite(10, TimeUnit.MINUTES)
+      .build(new ClassBytesCacheLoader());
+  
+  
+  private class ClassBytesCacheLoader extends CacheLoader<String, byte[]>{
+    public byte[] load(String path) throws ClassTransformationException, IOException {
+      URL u = this.getClass().getResource(path);
+      if (u == null) throw new ClassTransformationException(String.format("Unable to find TemplateClass at path %s",path));
+      return Resources.toByteArray(u);              
+    }
+  };
+  
+  private byte[] getClassByteCodeFromPath(String path) throws ClassTransformationException, IOException {
+    try{
+      return byteCode.get(path);
+    } catch (ExecutionException e) {
+      Throwable c = e.getCause();
+      if(c instanceof ClassTransformationException) throw (ClassTransformationException) c;
+      if(c instanceof IOException) throw (IOException) c;
+      throw new ClassTransformationException(c);
+    }
+  }
+
+  
+  @SuppressWarnings("unchecked")
+  public <T, I> T getImplementationClass(QueryClassLoader classLoader,
+      TemplateClassDefinition<T, I> templateDefinition, String internalClassBody, I initObject)
+      throws ClassTransformationException {
+    final String implClassName = templateDefinition.getTemplateClassName() + interfaceIndex.getAndIncrement();
+    final String materializedClassName = "org.apache.drill.generated."
+        + templateDefinition.getExternalInterface().getSimpleName() + index.getAndIncrement();
+    // final String materializedClassName = templateDefinition.getTemplateClassName();
+    try {
+
+      // Get Implementation Class
+      String classBody = ClassBodyBuilder.newBuilder() //
+          .setClassName(implClassName) //
+          .setImplementedInterfaces(templateDefinition.getInternalInterface()) //
+          .setBody(internalClassBody) //
+          .build();
+      final byte[] implementationClass = classLoader.getClassByteCode(implClassName, classBody);
+
+      // Get Template Class
+      final String templateClassName = templateDefinition.getTemplateClassName().replaceAll("\\.", File.separator);
+      final String templateClassPath = File.separator + templateClassName + ".class";
+      final byte[] templateClass = getClassByteCodeFromPath(templateClassPath);
+
+      // Generate Merge Class
+      ClassNode impl = getClassNodeFromByteCode(implementationClass);
+      // traceClassToSystemOut(implementationClass);
+      // traceClassToSystemOut(templateClass);
+      ClassWriter cw = new ClassWriter(0);
+      MergeAdapter adapter = new MergeAdapter(cw, impl);
+      ClassReader tReader = new ClassReader(templateClass);
+      tReader.accept(adapter, 0);
+
+      byte[] outputClass = cw.toByteArray();
+
+      cw = new ClassWriter(0);
+      RemappingClassAdapter r = new RemappingClassAdapter(cw, new SimpleRemapper(templateClassName,
+          materializedClassName.replace('.', '/')));
+      new ClassReader(outputClass).accept(r, 0);
+      outputClass = cw.toByteArray();
+      // traceClassToSystemOut(outputClass);
+
+      // Load the class
+      classLoader.injectByteCode(materializedClassName, outputClass);
+      Class<?> c = classLoader.findClass(materializedClassName);
+      if (templateDefinition.getExternalInterface().isAssignableFrom(c)) {
+        return (T) c.newInstance();
+      } else {
+        throw new ClassTransformationException("The requested class did not implement the expected interface.");
+      }
+
+    } catch (CompileException | IOException | ClassNotFoundException | InstantiationException | IllegalAccessException e) {
+      throw new ClassTransformationException("Failure generating transformation classes.", e);
+    }
+
+  }
+
+  private ClassNode getClassNodeFromByteCode(byte[] bytes) {
+    ClassReader iReader = new ClassReader(bytes);
+    ClassNode impl = new ClassNode();
+    iReader.accept(impl, 0);
+    return impl;
+  }
+
+  private void traceClassToSystemOut(byte[] bytecode) {
+    TraceClassVisitor tcv = new TraceClassVisitor(new EmptyVisitor(), new PrintWriter(System.out));
+    ClassReader cr = new ClassReader(bytecode);
+    cr.accept(tcv, 0);
+  }
+
+  public class MergeAdapter extends ClassAdapter {
+    private ClassNode classToMerge;
+    private String cname;
+
+    public MergeAdapter(ClassVisitor cv, ClassNode cn) {
+      super(cv);
+      this.classToMerge = cn;
+    }
+
+    // visit the class
+    public void visit(int version, int access, String name, String signature, String superName, String[] interfaces) {
+      // use the access and names of the impl class.
+      super.visit(version, access ^ Modifier.ABSTRACT | Modifier.FINAL, name, signature, superName, interfaces);
+      this.cname = name;
+    }
+
+    @Override
+    public MethodVisitor visitMethod(int access, String arg1, String arg2, String arg3, String[] arg4) {
+      // finalize all methods.
+
+      // skip all abstract methods as they should have implementations.
+      if ((access & Modifier.ABSTRACT) != 0) {
+        // logger.debug("Skipping copy of '{}()' since it is abstract.", arg1);
+        return null;
+      }
+
+      // if( (access & Modifier.PUBLIC) == 0){
+      // access = access ^ Modifier.PUBLIC ^ Modifier.PROTECTED | Modifier.PRIVATE;
+      // }
+      if (!arg1.equals("<init>")) {
+        access = access | Modifier.FINAL;
+      }
+      return super.visitMethod(access, arg1, arg2, arg3, arg4);
+    }
+
+    @SuppressWarnings("unchecked")
+    public void visitEnd() {
+      for (Iterator<?> it = classToMerge.fields.iterator(); it.hasNext();) {
+        ((FieldNode) it.next()).accept(this);
+      }
+      for (Iterator<?> it = classToMerge.methods.iterator(); it.hasNext();) {
+        MethodNode mn = (MethodNode) it.next();
+
+        // skip the init.
+        if (mn.name.equals("<init>")) continue;
+
+        String[] exceptions = new String[mn.exceptions.size()];
+        mn.exceptions.toArray(exceptions);
+        MethodVisitor mv = cv.visitMethod(mn.access | Modifier.FINAL, mn.name, mn.desc, mn.signature, exceptions);
+        mn.instructions.resetLabels();
+        // mn.accept(new RemappingMethodAdapter(mn.access, mn.desc, mv, new
+        // SimpleRemapper("org.apache.drill.exec.compile.ExampleTemplate", "Bunky")));
+        mn.accept(new RemappingMethodAdapter(mn.access, mn.desc, mv, new SimpleRemapper(cname.replace('.', '/'),
+            classToMerge.name.replace('.', '/'))));
+      }
+      super.visitEnd();
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/compile/JDKClassCompiler.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/compile/JDKClassCompiler.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/compile/JDKClassCompiler.java
new file mode 100644
index 0000000..15e87fe
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/compile/JDKClassCompiler.java
@@ -0,0 +1,177 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.compile;
+
+import java.io.IOException;
+import java.io.Reader;
+import java.io.StringReader;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+
+import javax.tools.Diagnostic;
+import javax.tools.DiagnosticListener;
+import javax.tools.JavaCompiler;
+import javax.tools.JavaFileManager;
+import javax.tools.JavaFileObject;
+import javax.tools.JavaCompiler.CompilationTask;
+import javax.tools.JavaFileObject.Kind;
+import javax.tools.SimpleJavaFileObject;
+import javax.tools.StandardLocation;
+import javax.tools.ToolProvider;
+
+import org.codehaus.commons.compiler.CompileException;
+import org.codehaus.commons.compiler.Location;
+import org.codehaus.commons.compiler.jdk.ByteArrayJavaFileManager;
+import org.codehaus.commons.compiler.jdk.ByteArrayJavaFileManager.ByteArrayJavaFileObject;
+
+import com.beust.jcommander.internal.Lists;
+
+class JDKClassCompiler implements ClassCompiler {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(JDKClassCompiler.class);
+
+  private boolean debugLines;
+  private boolean debugVars;
+  private boolean debugSource;
+  private Collection<String> compilerOptions = new ArrayList<String>();
+  private DiagnosticListener<JavaFileObject> listener;
+  private final JavaCompiler compiler;
+  private final JavaFileManager fileManager;
+  
+  public JDKClassCompiler() {
+    this.compiler = ToolProvider.getSystemJavaCompiler();
+    if (compiler == null) {
+      throw new UnsupportedOperationException(
+          "JDK Java compiler not available - probably you're running a JRE, not a JDK");
+    }
+    this.fileManager = new ByteArrayJavaFileManager<JavaFileManager>(compiler.getStandardFileManager(null, null, null));
+    this.listener = new DiagListener();
+  }
+
+  private JavaFileObject getCompilationUnit(final String s) {
+
+    final URI uri;
+    try {
+      uri = new URI("drill-class-compiler");
+    } catch (URISyntaxException use) {
+      throw new RuntimeException(use);
+    }
+    JavaFileObject javaFileObject = new SimpleJavaFileObject(uri, Kind.SOURCE) {
+
+      @Override
+      public boolean isNameCompatible(String simpleName, Kind kind) {
+        return true;
+      }
+
+      @Override
+      public Reader openReader(boolean ignoreEncodingErrors) throws IOException {
+        return new StringReader(s);
+      }
+
+      @Override
+      public CharSequence getCharContent(boolean ignoreEncodingErrors) throws IOException {
+        return s;
+      }
+
+    };
+
+    return javaFileObject;
+  }
+
+  private List<String> getOptions() {
+    List<String> opts = Lists.newArrayList(compilerOptions);
+    String option = this.debugSource ? "-g:source" + (this.debugLines ? ",lines" : "")
+        + (this.debugVars ? ",vars" : "") : this.debugLines ? "-g:lines" + (this.debugVars ? ",vars" : "")
+        : this.debugVars ? "-g:vars" : "-g:none";
+    opts.add(option);
+    return opts;
+  }
+
+  /* (non-Javadoc)
+   * @see org.apache.drill.exec.compile.ClassCompiler#getClassByteCode(java.lang.String, java.lang.String)
+   */
+  @Override
+  public byte[] getClassByteCode(final String className, final String sourcecode) throws CompileException, IOException,
+      ClassNotFoundException {
+
+    // Create one Java source file in memory, which will be compiled later.
+    JavaFileObject compilationUnit = getCompilationUnit(sourcecode);
+
+    //logger.debug("Compiling the following source code\n{}", sourcecode);
+    // Run the compiler.
+    try {
+      CompilationTask task = compiler.getTask(null, fileManager, listener, getOptions(), null, Collections.singleton(compilationUnit));
+      long n0 = System.nanoTime();
+      if(!task.call()){
+        throw new CompileException("Compilation failed", null);
+      }
+      long n1 = (System.nanoTime() - n0)/1000/1000;
+      
+    } catch (RuntimeException rte) {
+      
+      // Unwrap the compilation exception and throw it.
+      Throwable cause = rte.getCause();
+      if (cause != null) {
+        cause = cause.getCause();
+        if (cause instanceof CompileException) throw (CompileException) cause;
+        if (cause instanceof IOException) throw (IOException) cause;
+      }
+      throw rte;
+    }
+
+    JavaFileObject classFileObject = fileManager.getJavaFileForInput(StandardLocation.CLASS_OUTPUT, className, Kind.CLASS);
+
+    if (classFileObject == null) {
+      throw new ClassNotFoundException(className + ": Class file not created by compilation");
+    }
+
+    if (!(classFileObject instanceof ByteArrayJavaFileObject))
+      throw new UnsupportedOperationException("Only supports byte array based java file objects.");
+
+    ByteArrayJavaFileObject bajfo = (ByteArrayJavaFileObject) classFileObject;
+    return bajfo.toByteArray();
+
+  }
+
+  private class DiagListener implements DiagnosticListener<JavaFileObject> {
+    @Override
+    public void report(Diagnostic<? extends JavaFileObject> diagnostic) {
+      System.err.println("*** " + diagnostic.toString() + " *** " + diagnostic.getCode());
+
+      Location loc = new Location( //
+          diagnostic.getSource().toString(), //
+          (short) diagnostic.getLineNumber(), //
+          (short) diagnostic.getColumnNumber() //
+      );
+      String code = diagnostic.getCode();
+      String message = diagnostic.getMessage(null) + " (" + code + ")";
+
+      // Wrap the exception in a RuntimeException, because "report()" does not declare checked
+      // exceptions.
+      throw new RuntimeException(new CompileException(message, loc));
+    }
+  }
+
+
+  
+  
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/compile/JaninoClassCompiler.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/compile/JaninoClassCompiler.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/compile/JaninoClassCompiler.java
new file mode 100644
index 0000000..86fe58b
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/compile/JaninoClassCompiler.java
@@ -0,0 +1,62 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.compile;
+
+import java.io.IOException;
+import java.io.StringReader;
+
+import org.apache.drill.exec.exception.ClassTransformationException;
+import org.codehaus.commons.compiler.CompileException;
+import org.codehaus.janino.ClassLoaderIClassLoader;
+import org.codehaus.janino.IClassLoader;
+import org.codehaus.janino.Java;
+import org.codehaus.janino.Parser;
+import org.codehaus.janino.Scanner;
+import org.codehaus.janino.UnitCompiler;
+import org.codehaus.janino.util.ClassFile;
+
+public class JaninoClassCompiler implements ClassCompiler{
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(JaninoClassCompiler.class);
+
+  private IClassLoader compilationClassLoader;
+
+  private boolean debugLines;
+  private boolean debugVars;
+  private boolean debugSource;
+
+  public JaninoClassCompiler(ClassLoader parentClassLoader) {
+    this.compilationClassLoader = new ClassLoaderIClassLoader(parentClassLoader);
+  }
+
+  public byte[] getClassByteCode(final String className, final String code) throws CompileException, IOException, ClassNotFoundException, ClassTransformationException {
+    StringReader reader = new StringReader(code);
+    Scanner scanner = new Scanner((String) null, reader);
+    Java.CompilationUnit compilationUnit = new Parser(scanner).parseCompilationUnit();
+    ClassFile[] classFiles = new UnitCompiler(compilationUnit, compilationClassLoader).compileUnit(this.debugSource,
+        this.debugLines, this.debugVars);
+    if (classFiles.length != 1)
+      throw new ClassTransformationException("Only one class file should have been generated from source code.");
+    return classFiles[0].toByteArray();
+  }
+
+  public void setDebuggingInformation(boolean debugSource, boolean debugLines, boolean debugVars) {
+    this.debugSource = debugSource;
+    this.debugLines = debugLines;
+    this.debugVars = debugVars;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/compile/QueryClassLoader.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/compile/QueryClassLoader.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/compile/QueryClassLoader.java
new file mode 100644
index 0000000..60aa8f3
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/compile/QueryClassLoader.java
@@ -0,0 +1,80 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.compile;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.net.URL;
+import java.net.URLClassLoader;
+import java.util.concurrent.ConcurrentMap;
+
+import javax.tools.JavaCompiler;
+import javax.tools.JavaFileManager;
+import javax.tools.JavaFileObject;
+import javax.tools.JavaFileObject.Kind;
+import javax.tools.StandardLocation;
+import javax.tools.ToolProvider;
+
+import org.apache.drill.exec.exception.ClassTransformationException;
+import org.codehaus.commons.compiler.CompileException;
+import org.codehaus.commons.compiler.jdk.ByteArrayJavaFileManager;
+import org.codehaus.commons.compiler.jdk.ByteArrayJavaFileManager.ByteArrayJavaFileObject;
+
+import com.google.common.collect.MapMaker;
+import com.google.common.io.ByteStreams;
+
+public class QueryClassLoader extends URLClassLoader {
+
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(QueryClassLoader.class);
+
+  private final ClassCompiler classCompiler;
+  private ConcurrentMap<String, byte[]> customClasses = new MapMaker().concurrencyLevel(4).makeMap();
+
+  public QueryClassLoader(boolean useJanino) {
+    super(new URL[0]);
+    if (useJanino) {
+      this.classCompiler = new JaninoClassCompiler(this);
+    } else {
+      this.classCompiler = new JDKClassCompiler();
+    }
+  }
+
+  public void injectByteCode(String className, byte[] classBytes) throws IOException {
+    if(customClasses.containsKey(className)) throw new IOException(String.format("The class defined {} has already been loaded.", className));
+    customClasses.put(className, classBytes);
+  }
+
+  @Override
+  protected Class<?> findClass(String className) throws ClassNotFoundException {
+    byte[] ba = customClasses.get(className);
+    if(ba != null){
+      return this.defineClass(className, ba, 0, ba.length);
+    }else{
+      return super.findClass(className);
+    }
+  }
+
+  public byte[] getClassByteCode(final String className, final String sourcecode) throws CompileException, IOException,
+      ClassNotFoundException, ClassTransformationException {
+    byte[] bc = classCompiler.getClassByteCode(className, sourcecode);
+    return bc;
+
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/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
new file mode 100644
index 0000000..fee4c97
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/compile/TemplateClassDefinition.java
@@ -0,0 +1,58 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.compile;
+
+import java.lang.reflect.Method;
+
+
+public class TemplateClassDefinition<T, I>{
+  
+  private final Class<T> externalInterface;
+  private final String templateClassName;
+  private final Class<?> internalInterface;
+  private final Class<I> constructorObject;
+  
+  public TemplateClassDefinition(Class<T> externalInterface, String templateClassName, Class<?> internalInterface, Class<I> constructorObject) {
+    super();
+    this.externalInterface = externalInterface;
+    this.templateClassName = templateClassName; 
+    this.internalInterface = internalInterface;
+    this.constructorObject = constructorObject;
+  }
+
+  public Class<T> getExternalInterface() {
+    return externalInterface;
+  }
+
+  
+  public Class<?> getInternalInterface() {
+    return internalInterface;
+  }
+
+  public String getTemplateClassName() {
+    return templateClassName;
+  }
+
+  public Class<I> getConstructorObject() {
+    return constructorObject;
+  }
+  
+  
+  
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/coord/ClusterCoordinator.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/coord/ClusterCoordinator.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/coord/ClusterCoordinator.java
index d7ea8fa..d3580b5 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/coord/ClusterCoordinator.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/coord/ClusterCoordinator.java
@@ -17,11 +17,11 @@
  ******************************************************************************/
 package org.apache.drill.exec.coord;
 
-import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
-
 import java.io.Closeable;
 import java.util.Collection;
 
+import org.apache.drill.common.proto.CoordinationProtos.DrillbitEndpoint;
+
 /**
  * Pluggable interface built to manage cluster coordination. Allows Drillbit or DrillClient to register its capabilities
  * as well as understand other node's existence and capabilities.

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/coord/DrillServiceInstanceHelper.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/coord/DrillServiceInstanceHelper.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/coord/DrillServiceInstanceHelper.java
index 289aa3c..ce0fb92 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/coord/DrillServiceInstanceHelper.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/coord/DrillServiceInstanceHelper.java
@@ -17,9 +17,9 @@
  ******************************************************************************/
 package org.apache.drill.exec.coord;
 
+import org.apache.drill.common.proto.CoordinationProtos.DrillServiceInstance;
+import org.apache.drill.common.proto.CoordinationProtos.DrillbitEndpoint;
 import org.apache.drill.exec.ExecConstants;
-import org.apache.drill.exec.proto.CoordinationProtos.DrillServiceInstance;
-import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
 
 import com.netflix.curator.x.discovery.ServiceInstance;
 import com.netflix.curator.x.discovery.ServiceInstanceBuilder;

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/coord/ZKClusterCoordinator.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/coord/ZKClusterCoordinator.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/coord/ZKClusterCoordinator.java
index 3ad08e1..85c573d 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/coord/ZKClusterCoordinator.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/coord/ZKClusterCoordinator.java
@@ -17,6 +17,17 @@
  ******************************************************************************/
 package org.apache.drill.exec.coord;
 
+import static com.google.common.base.Throwables.propagate;
+import static com.google.common.collect.Collections2.transform;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.Collections;
+
+import org.apache.drill.common.config.DrillConfig;
+import org.apache.drill.common.proto.CoordinationProtos.DrillbitEndpoint;
+import org.apache.drill.exec.ExecConstants;
+
 import com.google.common.base.Function;
 import com.netflix.curator.RetryPolicy;
 import com.netflix.curator.framework.CuratorFramework;
@@ -28,16 +39,6 @@ import com.netflix.curator.x.discovery.ServiceDiscoveryBuilder;
 import com.netflix.curator.x.discovery.ServiceInstance;
 import com.netflix.curator.x.discovery.details.ServiceCache;
 import com.netflix.curator.x.discovery.details.ServiceCacheListener;
-import org.apache.drill.common.config.DrillConfig;
-import org.apache.drill.exec.ExecConstants;
-import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
-
-import java.io.IOException;
-import java.util.Collection;
-import java.util.Collections;
-
-import static com.google.common.base.Throwables.propagate;
-import static com.google.common.collect.Collections2.transform;
 
 /**
  * Manages cluster coordination utilizing zookeeper. *

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/exception/ClassTransformationException.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/exception/ClassTransformationException.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/exception/ClassTransformationException.java
new file mode 100644
index 0000000..13ec95e
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/exception/ClassTransformationException.java
@@ -0,0 +1,47 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.exception;
+
+import org.apache.drill.common.exceptions.DrillException;
+
+public class ClassTransformationException extends DrillException{
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ClassTransformationException.class);
+
+  public ClassTransformationException() {
+    super();
+  }
+
+  public ClassTransformationException(String message, Throwable cause, boolean enableSuppression,
+      boolean writableStackTrace) {
+    super(message, cause, enableSuppression, writableStackTrace);
+  }
+
+  public ClassTransformationException(String message, Throwable cause) {
+    super(message, cause);
+  }
+
+  public ClassTransformationException(String message) {
+    super(message);
+  }
+
+  public ClassTransformationException(Throwable cause) {
+    super(cause);
+  }
+  
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/exception/FragmentSetupException.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/exception/FragmentSetupException.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/exception/FragmentSetupException.java
new file mode 100644
index 0000000..c273463
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/exception/FragmentSetupException.java
@@ -0,0 +1,42 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.exception;
+
+public class FragmentSetupException extends ExecutionSetupException{
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(FragmentSetupException.class);
+
+  public FragmentSetupException() {
+    super();
+  }
+
+  public FragmentSetupException(String message, Throwable cause, boolean enableSuppression, boolean writableStackTrace) {
+    super(message, cause, enableSuppression, writableStackTrace);
+  }
+
+  public FragmentSetupException(String message, Throwable cause) {
+    super(message, cause);
+  }
+
+  public FragmentSetupException(String message) {
+    super(message);
+  }
+
+  public FragmentSetupException(Throwable cause) {
+    super(cause);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/foreman/CancelableQuery.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/foreman/CancelableQuery.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/foreman/CancelableQuery.java
new file mode 100644
index 0000000..30e7a63
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/foreman/CancelableQuery.java
@@ -0,0 +1,22 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.foreman;
+
+public interface CancelableQuery {
+  public boolean cancel(long queryid);
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/foreman/ExecutionPlanner.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/foreman/ExecutionPlanner.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/foreman/ExecutionPlanner.java
new file mode 100644
index 0000000..4e4ec77
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/foreman/ExecutionPlanner.java
@@ -0,0 +1,24 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.foreman;
+
+public class ExecutionPlanner {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ExecutionPlanner.class);
+  
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/foreman/Foreman.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/foreman/Foreman.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/foreman/Foreman.java
new file mode 100644
index 0000000..f138171
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/foreman/Foreman.java
@@ -0,0 +1,39 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.foreman;
+
+
+public class Foreman extends Thread{
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(Foreman.class);
+  
+  public Foreman(){
+    
+  }
+  
+  public void doWork(QueryWorkUnit work){
+    // generate fragment structure. 
+    // store fragments in distributed grid.
+    // generate any codegen required and store in grid.
+    // drop 
+    // do get on the result set you're looking for.  Do the initial get on the result node you're looking for.  This will return either data or a metadata record set
+  }
+
+  public boolean checkStatus(long queryId){
+    return false;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/foreman/QueryWorkUnit.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/foreman/QueryWorkUnit.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/foreman/QueryWorkUnit.java
new file mode 100644
index 0000000..bdf4a1e
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/foreman/QueryWorkUnit.java
@@ -0,0 +1,54 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.foreman;
+
+import java.util.Collection;
+import java.util.List;
+
+import org.apache.drill.exec.proto.ExecProtos.PlanFragment;
+
+import com.google.common.base.Preconditions;
+
+public class QueryWorkUnit {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(QueryWorkUnit.class);
+  
+  private PlanFragment rootFragment; // for local
+  private List<PlanFragment> fragments;
+  
+  public QueryWorkUnit(PlanFragment rootFragment, List<PlanFragment> fragments) {
+    super();
+    Preconditions.checkNotNull(rootFragment);
+    Preconditions.checkNotNull(fragments);
+    this.rootFragment = rootFragment;
+    this.fragments = fragments;
+  }
+
+  public PlanFragment getRootFragment() {
+    return rootFragment;
+  }
+
+  public List<PlanFragment> getFragments() {
+    return fragments;
+  }
+  
+  
+  
+  
+  
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/foreman/ResourceRequest.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/foreman/ResourceRequest.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/foreman/ResourceRequest.java
new file mode 100644
index 0000000..96d7d1e
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/foreman/ResourceRequest.java
@@ -0,0 +1,30 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.foreman;
+
+public class ResourceRequest {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ResourceRequest.class);
+  
+  public long memoryMin;
+  public long memoryDesired;
+  
+
+  public static class ResourceAllocation {
+    public long memory;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/foreman/StatusProvider.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/foreman/StatusProvider.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/foreman/StatusProvider.java
new file mode 100644
index 0000000..fee6172
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/foreman/StatusProvider.java
@@ -0,0 +1,24 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.foreman;
+
+import org.apache.drill.exec.proto.ExecProtos.FragmentStatus;
+
+public interface StatusProvider {
+  public FragmentStatus getStatus();
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/memory/BufferAllocator.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/memory/BufferAllocator.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/memory/BufferAllocator.java
new file mode 100644
index 0000000..2b3f574
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/memory/BufferAllocator.java
@@ -0,0 +1,58 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.memory;
+
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.ByteBufAllocator;
+import io.netty.buffer.PooledByteBufAllocator;
+
+import java.io.Closeable;
+
+import org.apache.drill.exec.server.DrillbitContext;
+
+/**
+ * Wrapper class to deal with byte buffer allocation. Ensures users only use designated methods.  Also allows inser 
+ */
+public abstract class BufferAllocator implements Closeable{
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(BufferAllocator.class);
+  
+  /**
+   * Allocate a new or reused buffer of the provided size.  Note that the buffer may technically be larger than the requested size for rounding purposes.  However, the buffers capacity will be set to the configured size.
+   * @param size The size in bytes.
+   * @return A new ByteBuf.
+   */
+  public abstract ByteBuf buffer(int size);
+  
+  public abstract ByteBufAllocator getUnderlyingAllocator();
+  
+  public abstract BufferAllocator getChildAllocator(long initialReservation, long maximumReservation);
+  
+  /**
+   * Close and release all buffers generated from this buffer pool.
+   */
+  @Override
+  public abstract void close(); 
+  
+  public static BufferAllocator getAllocator(DrillbitContext context){
+    // TODO: support alternative allocators (including a debugging allocator that records all allocation locations for each buffer).
+    return new DirectBufferAllocator();
+  }
+  
+  public abstract long getAllocatedMemory();
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/memory/DirectBufferAllocator.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/memory/DirectBufferAllocator.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/memory/DirectBufferAllocator.java
new file mode 100644
index 0000000..8c5b003
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/memory/DirectBufferAllocator.java
@@ -0,0 +1,58 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.memory;
+
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.ByteBufAllocator;
+import io.netty.buffer.PooledByteBufAllocator;
+
+public class DirectBufferAllocator extends BufferAllocator{
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DirectBufferAllocator.class);
+
+  private final PooledByteBufAllocator buffer = new PooledByteBufAllocator(true);
+  
+  @Override
+  public ByteBuf buffer(int size) {
+    // TODO: wrap it
+    return buffer.directBuffer(size);
+  }
+
+  @Override
+  public long getAllocatedMemory() {
+    return 0;
+  }
+
+  @Override
+  public ByteBufAllocator getUnderlyingAllocator() {
+    return buffer;
+  }
+
+  
+
+  @Override
+  public BufferAllocator getChildAllocator(long initialReservation, long maximumReservation) {
+    //TODO: Add child account allocator.
+    return this;
+  }
+
+  @Override
+  public void close() {
+    // TODO: collect all buffers and release them away using a weak hashmap so we don't impact pool work
+  }
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/metrics/SingleThreadNestedCounter.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/metrics/SingleThreadNestedCounter.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/metrics/SingleThreadNestedCounter.java
new file mode 100644
index 0000000..6b89c12
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/metrics/SingleThreadNestedCounter.java
@@ -0,0 +1,55 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.metrics;
+
+import org.apache.drill.exec.server.DrillbitContext;
+
+import com.yammer.metrics.Counter;
+
+/**
+ * Wraps a parent counter so that local in thread metrics can be collected while collecting for a global counter.
+ */
+public class SingleThreadNestedCounter {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(SingleThreadNestedCounter.class);
+  
+  private volatile long count;
+  private final Counter counter;
+  
+  
+  public SingleThreadNestedCounter(DrillbitContext context, String name) {
+    super();
+    this.counter = context.getMetrics().counter(name);
+  }
+
+  public long inc(long n){
+    counter.inc(n);
+    count+= n;
+    return count;
+  }
+  
+  public long dec(long n){
+    counter.dec(n);
+    count -= n;
+    return count;
+  }
+  
+  public long get(){
+    return count;
+  }
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/BatchIterator.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/BatchIterator.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/BatchIterator.java
deleted file mode 100644
index 2ebbef5..0000000
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/BatchIterator.java
+++ /dev/null
@@ -1,32 +0,0 @@
-/*******************************************************************************
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- ******************************************************************************/
-package org.apache.drill.exec.ops;
-
-import org.apache.drill.exec.record.RecordBatch;
-
-import parquet.schema.MessageType;
-
-public interface BatchIterator {
-  static enum IterOutcome{NONE, FULL_NEW_SCHEMA, FULL, PARTIAL_NEW_SCHEMA, PARTIAL, STOP}
-  public RecordBatch getBatch();
-  public FragmentContext getContext();
-  public MessageType getSchema();
-  public void kill(QueryOutcome outcome);
-  public IterOutcome next();
-  
-}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/FilteringRecordBatchTransformer.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/FilteringRecordBatchTransformer.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/FilteringRecordBatchTransformer.java
new file mode 100644
index 0000000..f626cea
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/FilteringRecordBatchTransformer.java
@@ -0,0 +1,58 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.ops;
+
+import org.apache.drill.exec.record.BatchSchema;
+import org.apache.drill.exec.record.RecordBatch;
+import org.apache.drill.exec.record.vector.SelectionVector;
+
+public abstract class FilteringRecordBatchTransformer {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(FilteringRecordBatchTransformer.class);
+  
+  final RecordBatch incoming;
+  final SelectionVector selectionVector;
+  final BatchSchema schema;
+  
+  public FilteringRecordBatchTransformer(RecordBatch incoming, OutputMutator output, SelectionVector selectionVector) {
+    super();
+    this.incoming = incoming;
+    this.selectionVector = selectionVector;
+    this.schema = innerSetup();
+  }
+
+  public abstract BatchSchema innerSetup();
+  
+  /**
+   * Applies the filter to the selection index.  Ignores any values in the selection vector, instead creating a.
+   * @return
+   */
+  public abstract int apply();
+  
+  /**
+   * Applies the filter to the selection index.  Utilizes the existing selection index and only evaluates on those records.
+   * @return
+   */
+  public abstract int applyWithSelection();
+
+  public BatchSchema getSchema() {
+    return schema;
+  }
+  
+  
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/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 be1081f..0cf17e9 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
@@ -17,18 +17,42 @@
  ******************************************************************************/
 package org.apache.drill.exec.ops;
 
-import org.apache.drill.common.logical.StorageEngineConfig;
+import org.apache.drill.common.expression.LogicalExpression;
+import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.metrics.SingleThreadNestedCounter;
+import org.apache.drill.exec.planner.FragmentRunnable;
+import org.apache.drill.exec.proto.ExecProtos.PlanFragment;
 import org.apache.drill.exec.rpc.bit.BitCom;
 import org.apache.drill.exec.server.DrillbitContext;
-import org.apache.drill.exec.store.StorageEngine;
 
+import com.yammer.metrics.MetricRegistry;
+import com.yammer.metrics.Timer;
+
+/**
+ * Contextual objects required for execution of a particular fragment.  
+ */
 public class FragmentContext {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(FragmentContext.class);
 
+  private final static String METRIC_TIMER_FRAGMENT_TIME = MetricRegistry.name(FragmentRunnable.class, "completionTimes");
+  private final static String METRIC_BATCHES_COMPLETED = MetricRegistry.name(FragmentRunnable.class, "batchesCompleted");
+  private final static String METRIC_RECORDS_COMPLETED = MetricRegistry.name(FragmentRunnable.class, "recordsCompleted");
+  private final static String METRIC_DATA_PROCESSED = MetricRegistry.name(FragmentRunnable.class, "dataProcessed");
+
   private final DrillbitContext context;
-  
-  public FragmentContext(DrillbitContext context) {
-    this.context = context;
+  private final PlanFragment fragment;
+  public final SingleThreadNestedCounter batchesCompleted;
+  public final SingleThreadNestedCounter recordsCompleted;
+  public final SingleThreadNestedCounter dataProcessed;
+  public final Timer fragmentTime;
+
+  public FragmentContext(DrillbitContext dbContext, PlanFragment fragment) {
+    this.fragmentTime = dbContext.getMetrics().timer(METRIC_TIMER_FRAGMENT_TIME);
+    this.batchesCompleted = new SingleThreadNestedCounter(dbContext, METRIC_BATCHES_COMPLETED);
+    this.recordsCompleted = new SingleThreadNestedCounter(dbContext, METRIC_RECORDS_COMPLETED);
+    this.dataProcessed = new SingleThreadNestedCounter(dbContext, METRIC_DATA_PROCESSED);
+    this.context = dbContext;
+    this.fragment = fragment;
   }
 
   public void fail(Throwable cause) {
@@ -39,10 +63,21 @@ public class FragmentContext {
     return context;
   }
   
-  public StorageEngine getStorageEngine(StorageEngineConfig config){
+  public PlanFragment getFragment() {
+    return fragment;
+  }
+  
+  public BufferAllocator getAllocator(){
+    // TODO: A local query allocator to ensure memory limits and accurately gauge memory usage.
+    return context.getAllocator();
+  }
+
+  
+  public FilteringRecordBatchTransformer getFilteringExpression(LogicalExpression expr){
     return null;
   }
   
+  
   public BitCom getCommunicator(){
     return null;
   }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/FragmentConverter.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/FragmentConverter.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/FragmentConverter.java
new file mode 100644
index 0000000..3c75648
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/FragmentConverter.java
@@ -0,0 +1,30 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.ops;
+
+import org.apache.drill.exec.proto.ExecProtos.PlanFragment;
+
+public class FragmentConverter {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(FragmentConverter.class);
+  
+  public static FragmentRoot getFragment(FragmentContext context){
+    PlanFragment m = context.getFragment();
+    
+    return null;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/FragmentRoot.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/FragmentRoot.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/FragmentRoot.java
new file mode 100644
index 0000000..ddacb41
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/FragmentRoot.java
@@ -0,0 +1,37 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.ops;
+
+import org.apache.drill.exec.exception.FragmentSetupException;
+
+/**
+ * A FragmentRoot is a node which is the last processing node in a query plan. FragmentTerminals include Exchange
+ * output nodes and storage nodes.  They are there driving force behind the completion of a query.
+ */
+public interface FragmentRoot {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(FragmentRoot.class);
+  
+  /**
+   * Do the next batch of work.  
+   * @return Whether or not additional batches of work are necessary.
+   */
+  public boolean next();
+  
+  
+  public void setup() throws FragmentSetupException;
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/OperatorFactory.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/OperatorFactory.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/OperatorFactory.java
new file mode 100644
index 0000000..8d4e807
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/OperatorFactory.java
@@ -0,0 +1,22 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.ops;
+
+public class OperatorFactory {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(OperatorFactory.class);
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/QueryContext.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/QueryContext.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/QueryContext.java
new file mode 100644
index 0000000..fe37e70
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/QueryContext.java
@@ -0,0 +1,51 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.ops;
+
+import java.util.Collection;
+
+import org.apache.drill.common.proto.CoordinationProtos.DrillbitEndpoint;
+import org.apache.drill.exec.server.DrillbitContext;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+
+public class QueryContext {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(QueryContext.class);
+  
+  private long queryId;
+  private DrillbitContext drillbitContext;
+  
+  public QueryContext(long queryId, DrillbitContext drllbitContext) {
+    super();
+    this.queryId = queryId;
+    this.drillbitContext = drllbitContext;
+  }
+  
+  public long getQueryId() {
+    return queryId;
+  }
+  
+  public ObjectMapper getMapper(){
+    return drillbitContext.getConfig().getMapper();
+  }
+  
+  public Collection<DrillbitEndpoint> getActiveEndpoints(){
+    return drillbitContext.getBits();
+  }
+  
+}

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

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/ScanBatch.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/ScanBatch.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/ScanBatch.java
index 88b8af2..b46804f 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/ScanBatch.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/ScanBatch.java
@@ -33,7 +33,7 @@ import com.carrotsearch.hppc.procedures.IntObjectProcedure;
 /**
  * Record batch used for a particular scan. Operators against one or more
  */
-public class ScanBatch implements RecordBatch {
+public abstract class ScanBatch implements RecordBatch {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ScanBatch.class);
 
   private IntObjectOpenHashMap<ValueVector<?>> fields = new IntObjectOpenHashMap<ValueVector<?>>();


Re: [48/53] [abbrv] git commit: Working Project merge build

Posted by Ted Dunning <te...@gmail.com>.
On Fri, Jul 19, 2013 at 6:58 PM, <ja...@apache.org> wrote:

> Working Project merge build
>
>
> Project: http://git-wip-us.apache.org/repos/asf/incubator-drill/repo
> Commit:
> http://git-wip-us.apache.org/repos/asf/incubator-drill/commit/3d41be41
> Tree: http://git-wip-us.apache.org/repos/asf/incubator-drill/tree/3d41be41
> Diff: http://git-wip-us.apache.org/repos/asf/incubator-drill/diff/3d41be41
>
> Branch: refs/heads/master
> Commit: 3d41be41d6a1c9dab7256a1044370348848a1ec3
> Parents: 80b1d24
> Author: Jacques Nadeau <ja...@apache.org>
> Authored: Tue Jul 16 10:45:25 2013 -0700
> Committer: Jacques Nadeau <ja...@apache.org>
> Committed: Fri Jul 19 14:53:31 2013 -0700
>
> ----------------------------------------------------------------------
>  .../common/src/main/protobuf/Types.proto        |   1 +
>  sandbox/prototype/exec/java-exec/pom.xml        |  30 -
>  .../templates/FixedValueVectors.java            |  31 +
>  .../templates/NullableValueVectors.java         |  28 +
>  .../templates/RepeatedValueVectors.java         |  38 +-
>  .../ValueVectors/templates/TypeHelper.java      |   6 +-
>  .../templates/VariableLengthVectors.java        |  29 +
>  .../drill/exec/expr/EvaluationVisitor.java      |   5 +-
>  .../exec/expr/ValueVectorReadExpression.java    |   1 -
>  .../exec/expr/ValueVectorWriteExpression.java   |   1 -
>  .../drill/exec/expr/fn/FunctionHolder.java      |  15 +-
>  .../exec/physical/config/MockRecordReader.java  |   6 +-
>  .../drill/exec/physical/config/MockScanPOP.java |   2 +-
>  .../exec/physical/impl/FilterRecordBatch.java   |  30 +-
>  .../impl/FilteringRecordBatchTransformer.java   |   6 +-
>  .../drill/exec/physical/impl/OutputMutator.java |  14 +-
>  .../drill/exec/physical/impl/ScanBatch.java     |  84 +-
>  .../drill/exec/physical/impl/VectorHolder.java  |  12 +-
>  .../exec/physical/impl/WireRecordBatch.java     |  19 +-
>  .../impl/filter/EvaluationPredicate.java        |   2 +-
>  .../physical/impl/filter/ExampleFilter.java     |  10 +-
>  .../physical/impl/filter/FilterTemplate.java    |   2 +-
>  .../impl/project/ProjectRecordBatch.java        |  43 +-
>  .../exec/physical/impl/project/Projector.java   |   3 +-
>  .../impl/project/ProjectorTemplate.java         |  11 +-
>  .../physical/impl/project/TransferPairing.java  |  35 -
>  .../drill/exec/record/MaterializedField.java    |   5 +-
>  .../apache/drill/exec/record/RecordBatch.java   |   8 +-
>  .../drill/exec/record/RecordBatchLoader.java    |  32 +-
>  .../apache/drill/exec/record/TransferPair.java  |   8 +
>  .../apache/drill/exec/record/WritableBatch.java |   8 +-
>  .../exec/record/selection/SelectionVector2.java |  49 ++
>  .../exec/record/selection/SelectionVector4.java |  41 +
>  .../drill/exec/record/vector/BitUtil.java       | 108 ---
>  .../drill/exec/record/vector/BufBitSet.java     | 847 -------------------
>  .../exec/record/vector/NullValueException.java  |   9 -
>  .../exec/record/vector/NullableFixed8.java      |  43 -
>  .../exec/record/vector/SelectionVector.java     |  49 --
>  .../exec/record/vector/SelectionVector2.java    |  49 --
>  .../exec/record/vector/SelectionVector4.java    |  41 -
>  .../drill/exec/store/JSONRecordReader.java      |  19 +-
>  .../drill/exec/vector/AllocationHelper.java     |  15 +
>  .../drill/exec/vector/BaseValueVector.java      |   2 -
>  .../org/apache/drill/exec/vector/BitVector.java |  27 +
>  .../apache/drill/exec/vector/ValueVector.java   |   5 +-
>  .../apache/drill/exec/expr/ExpressionTest.java  |  11 +-
>  .../exec/physical/impl/SimpleRootExec.java      |   4 +-
>  .../physical/impl/TestSimpleFragmentRun.java    |  58 +-
>  .../impl/project/TestSimpleProjection.java      |  17 +-
>  .../exec/record/vector/TestOpenBitSet.java      | 361 --------
>  .../exec/record/vector/TestValueVector.java     |  55 +-
>  .../drill/exec/store/JSONRecordReaderTest.java  |  41 +-
>  52 files changed, 482 insertions(+), 1894 deletions(-)
> ----------------------------------------------------------------------
>
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/3d41be41/sandbox/prototype/common/src/main/protobuf/Types.proto
> ----------------------------------------------------------------------
> diff --git a/sandbox/prototype/common/src/main/protobuf/Types.proto
> b/sandbox/prototype/common/src/main/protobuf/Types.proto
> index 58399de..17e371d 100644
> --- a/sandbox/prototype/common/src/main/protobuf/Types.proto
> +++ b/sandbox/prototype/common/src/main/protobuf/Types.proto
> @@ -15,6 +15,7 @@ enum MinorType {
>      BIGINT = 6;   //  eight byte signed integer
>      DECIMAL4 = 7;   //  a decimal supporting precision between 1 and 8 (4
> bits for decimal location, 1 sign)
>      DECIMAL8 = 8;   //  a decimal supporting precision between 9 and 18
> (5 bits for decimal location, 1 sign)
> +    DECIMAL12 = 9;   //  a decimal supporting precision between
>      DECIMAL16 = 10;   //  a decimal supporting precision between 19 and
> 37 (6 bits for decimal location, 1 sign)
>      MONEY = 11;   //  signed decimal with two digit precision
>      DATE = 12;   //  days since 4713bc
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/3d41be41/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 f2f0bff..0290be1 100644
> --- a/sandbox/prototype/exec/java-exec/pom.xml
> +++ b/sandbox/prototype/exec/java-exec/pom.xml
> @@ -179,7 +179,6 @@
>                  <exec executable="protoc">
>                    <arg value="--java_out=${target.gen.source.path}" />
>                    <arg value="--proto_path=${proto.cas.path}" />
> -<<<<<<< HEAD
>                    <arg
> value="--proto_path=${project.basedir}/../../common/src/main/protobuf/" />
>
> <arg line="${proto.files}" />
>                                                                 </exec>
> @@ -226,34 +225,5 @@
>                         <!-- </plugin> -->
>                 </plugins>
>         </build>
> -=======
> -                  <arg
> -
>  value="--proto_path=${project.basedir}/../../common/src/main/protobuf/" />
> -                  <arg line="${proto.files}" />
> -                </exec>
> -              </tasks>
> -              <sourceRoot>${target.gen.source.path}</sourceRoot>
> -            </configuration>
> -            <goals>
> -              <goal>run</goal>
> -            </goals>
> -          </execution>
> -        </executions>
> -      </plugin>
> -      <!-- <plugin> -->
> -      <!-- <groupId>com.github.igor-petruk.protobuf</groupId> -->
> -      <!-- <artifactId>protobuf-maven-plugin</artifactId> -->
> -      <!-- <version>0.6.2</version> -->
> -      <!-- <executions> -->
> -      <!-- <execution> -->
> -      <!-- <goals> -->
> -      <!-- <goal>run</goal> -->
> -      <!-- </goals> -->
> -      <!-- </execution> -->
> -      <!-- </executions> -->
> -      <!-- </plugin> -->
> -    </plugins>
> -  </build>
> ->>>>>>> Build working
>
>  </project>
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/3d41be41/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/FixedValueVectors.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/FixedValueVectors.java
> b/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/FixedValueVectors.java
> index 7583d9f..e0c8406 100644
> ---
> a/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/FixedValueVectors.java
> +++
> b/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/FixedValueVectors.java
> @@ -9,6 +9,7 @@ package org.apache.drill.exec.vector;
>  import io.netty.buffer.ByteBuf;
>
>  import org.apache.drill.exec.memory.BufferAllocator;
> +import org.apache.drill.exec.record.TransferPair;
>  import org.apache.drill.exec.proto.UserBitShared.FieldMetadata;
>  import org.apache.drill.exec.record.DeadBuf;
>  import org.apache.drill.exec.record.MaterializedField;
> @@ -49,6 +50,8 @@ public final class ${minor.class}Vector extends
> BaseDataValueVector implements F
>    public Mutator getMutator(){
>      return mutator;
>    }
> +
> +
>
>    /**
>     * Allocate a new buffer that supports setting at least the provided
> number of values.  May actually be sized bigger depending on underlying
> buffer rounding size. Must be called prior to using the ValueVector.
> @@ -87,6 +90,33 @@ public final class ${minor.class}Vector extends
> BaseDataValueVector implements F
>      assert metadata.getBufferLength() == loaded;
>    }
>
> +  public TransferPair getTransferPair(){
> +    return new TransferImpl();
> +  }
> +
> +  public void transferTo(${minor.class}Vector target){
> +    target.data = data;
> +    target.data.retain();
> +    target.recordCount = recordCount;
> +    clear();
> +  }
> +
> +  private class TransferImpl implements TransferPair{
> +    ${minor.class}Vector to;
> +
> +    public TransferImpl(){
> +      this.to = new ${minor.class}Vector(getField(), allocator);
> +    }
> +
> +    public ${minor.class}Vector getTo(){
> +      return to;
> +    }
> +
> +    public void transfer(){
> +      transferTo(to);
> +    }
> +  }
> +
>    public final class Accessor extends BaseValueVector.BaseAccessor{
>
>      public int getRecordCount() {
> @@ -184,6 +214,7 @@ public final class ${minor.class}Vector extends
> BaseDataValueVector implements F
>
>
>
> +
>
>   }
>  }
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/3d41be41/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/NullableValueVectors.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/NullableValueVectors.java
> b/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/NullableValueVectors.java
> index 3232f87..d2d436a 100644
> ---
> a/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/NullableValueVectors.java
> +++
> b/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/NullableValueVectors.java
> @@ -17,6 +17,7 @@ import org.apache.drill.exec.proto.SchemaDefProtos;
>  import org.apache.drill.exec.proto.UserBitShared.FieldMetadata;
>  import org.apache.drill.exec.record.DeadBuf;
>  import org.apache.drill.exec.record.MaterializedField;
> +import org.apache.drill.exec.record.TransferPair;
>  import org.apache.drill.exec.vector.BaseValueVector;
>  import org.apache.drill.exec.vector.BitVector;
>  import org.apache.drill.exec.vector.UInt2Vector;
> @@ -146,6 +147,33 @@ public final class Nullable${minor.class}Vector
> extends BaseValueVector implemen
>
>    </#if>
>
> +  public TransferPair getTransferPair(){
> +    return new TransferImpl();
> +  }
> +
> +  public void transferTo(Nullable${minor.class}Vector target){
> +    bits.transferTo(target.bits);
> +    values.transferTo(target.values);
> +    target.recordCount = recordCount;
> +    clear();
> +  }
> +
> +  private class TransferImpl implements TransferPair{
> +    Nullable${minor.class}Vector to;
> +
> +    public TransferImpl(){
> +      this.to = new Nullable${minor.class}Vector(getField(), allocator);
> +    }
> +
> +    public Nullable${minor.class}Vector getTo(){
> +      return to;
> +    }
> +
> +    public void transfer(){
> +      transferTo(to);
> +    }
> +  }
> +
>    public Accessor getAccessor(){
>      return accessor;
>    }
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/3d41be41/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/RepeatedValueVectors.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/RepeatedValueVectors.java
> b/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/RepeatedValueVectors.java
> index 363e4c8..1976340 100644
> ---
> a/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/RepeatedValueVectors.java
> +++
> b/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/RepeatedValueVectors.java
> @@ -7,11 +7,6 @@ import org.apache.drill.exec.vector.UInt4Vector;
>  <@pp.changeOutputFile name="Repeated${minor.class}Vector.java" />
>  package org.apache.drill.exec.vector;
>
> -
> -
> -
> -
> -
>  import static com.google.common.base.Preconditions.checkArgument;
>  import static com.google.common.base.Preconditions.checkState;
>  import io.netty.buffer.ByteBuf;
> @@ -25,6 +20,7 @@ import org.apache.drill.exec.proto.SchemaDefProtos;
>  import org.apache.drill.exec.proto.UserBitShared.FieldMetadata;
>  import org.apache.drill.exec.record.DeadBuf;
>  import org.apache.drill.exec.record.MaterializedField;
> +import org.apache.drill.exec.record.TransferPair;
>
>  @SuppressWarnings("unused")
>  /**
> @@ -65,6 +61,35 @@ import org.apache.drill.exec.record.MaterializedField;
>      return counts.getBufferSize() + offsets.getBufferSize() +
> values.getBufferSize();
>    }
>
> +  public TransferPair getTransferPair(){
> +    return new TransferImpl();
> +  }
> +
> +  public void transferTo(Repeated${minor.class}Vector target){
> +    counts.transferTo(target.counts);
> +    offsets.transferTo(target.offsets);
> +    values.transferTo(target.values);
> +    target.parentValueCount = parentValueCount;
> +    target.childValueCount = childValueCount;
> +    clear();
> +  }
> +
> +  private class TransferImpl implements TransferPair{
> +    Repeated${minor.class}Vector to;
> +
> +    public TransferImpl(){
> +      this.to = new Repeated${minor.class}Vector(getField(), allocator);
> +    }
> +
> +    public Repeated${minor.class}Vector getTo(){
> +      return to;
> +    }
> +
> +    public void transfer(){
> +      transferTo(to);
> +    }
> +  }
> +
>    <#if type.major == "VarLen">
>    @Override
>    public FieldMetadata getMetadata() {
> @@ -84,9 +109,6 @@ import org.apache.drill.exec.record.MaterializedField;
>      mutator.reset();
>      accessor.reset();
>    }
> -
> -
> -
>
>    @Override
>    public int load(int dataBytes, int parentValueCount, int
> childValueCount, ByteBuf buf){
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/3d41be41/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/TypeHelper.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/TypeHelper.java
> b/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/TypeHelper.java
> index b03b842..b73cd21 100644
> ---
> a/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/TypeHelper.java
> +++
> b/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/TypeHelper.java
> @@ -18,9 +18,9 @@
>  package org.apache.drill.exec.vector;
>
>  import org.apache.drill.exec.memory.BufferAllocator;
> -import org.apache.drill.exec.proto.SchemaDefProtos.DataMode;
> -import org.apache.drill.exec.proto.SchemaDefProtos.MajorType;
> -import org.apache.drill.exec.proto.SchemaDefProtos.MinorType;
> +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.exec.record.MaterializedField;
>
>  public class TypeHelper {
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/3d41be41/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/VariableLengthVectors.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/VariableLengthVectors.java
> b/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/VariableLengthVectors.java
> index c615258..9d723ab 100644
> ---
> a/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/VariableLengthVectors.java
> +++
> b/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/VariableLengthVectors.java
> @@ -19,6 +19,7 @@ import org.apache.drill.exec.proto.SchemaDefProtos;
>  import org.apache.drill.exec.proto.UserBitShared.FieldMetadata;
>  import org.apache.drill.exec.record.DeadBuf;
>  import org.apache.drill.exec.record.MaterializedField;
> +import org.apache.drill.exec.record.TransferPair;
>  import org.apache.drill.exec.vector.ByteHolder;
>
>  /**
> @@ -102,6 +103,34 @@ public final class ${minor.class}Vector extends
> BaseDataValueVector implements V
>      return new ByteBuf[]{offsetVector.data, this.data};
>    }
>
> +  public TransferPair getTransferPair(){
> +    return new TransferImpl();
> +  }
> +
> +  public void transferTo(${minor.class}Vector target){
> +    this.offsetVector.transferTo(target.offsetVector);
> +    target.data = data;
> +    target.data.retain();
> +    target.recordCount = recordCount;
> +    clear();
> +  }
> +
> +  private class TransferImpl implements TransferPair{
> +    ${minor.class}Vector to;
> +
> +    public TransferImpl(){
> +      this.to = new ${minor.class}Vector(getField(), allocator);
> +    }
> +
> +    public ${minor.class}Vector getTo(){
> +      return to;
> +    }
> +
> +    public void transfer(){
> +      transferTo(to);
> +    }
> +  }
> +
>    public void allocateNew(int totalBytes, int valueCount) {
>      clear();
>      assert totalBytes >= 0;
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/3d41be41/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 04f78d0..6b0e499 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
> @@ -15,15 +15,14 @@ import
> org.apache.drill.exec.expr.CodeGenerator.HoldingContainer;
>  import org.apache.drill.exec.expr.fn.FunctionHolder;
>  import org.apache.drill.exec.expr.fn.FunctionImplementationRegistry;
>  import
> org.apache.drill.exec.physical.impl.filter.SelectionVectorPopulationExpression;
> -import org.apache.drill.exec.record.vector.SelectionVector2;
> -import org.apache.drill.exec.record.vector.TypeHelper;
> +import org.apache.drill.exec.record.selection.SelectionVector2;
> +import org.apache.drill.exec.vector.TypeHelper;
>
>  import com.google.common.base.Preconditions;
>  import com.sun.codemodel.JBlock;
>  import com.sun.codemodel.JClass;
>  import com.sun.codemodel.JConditional;
>  import com.sun.codemodel.JExpr;
> -import com.sun.codemodel.JExpression;
>  import com.sun.codemodel.JType;
>  import com.sun.codemodel.JVar;
>
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/3d41be41/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 c8d906d..b1888d6 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
> @@ -5,7 +5,6 @@ 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.RecordBatch.TypedFieldId;
> -import org.apache.drill.exec.record.vector.ValueVector;
>
>  public class ValueVectorReadExpression implements LogicalExpression{
>    static final org.slf4j.Logger logger =
> org.slf4j.LoggerFactory.getLogger(ValueVectorReadExpression.class);
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/3d41be41/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 96d9d05..417b975 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
> @@ -5,7 +5,6 @@ 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.common.types.Types;
> -import org.apache.drill.exec.record.vector.ValueVector;
>
>  public class ValueVectorWriteExpression implements LogicalExpression {
>    static final org.slf4j.Logger logger =
> org.slf4j.LoggerFactory.getLogger(ValueVectorWriteExpression.class);
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/3d41be41/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 f7ecd02..9d7fc35 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,6 +1,5 @@
>  package org.apache.drill.exec.expr.fn;
>
> -import java.lang.reflect.Type;
>  import java.util.Arrays;
>  import java.util.Map;
>
> @@ -92,14 +91,16 @@ public class FunctionHolder {
>
>      JVar[] workspaceJVars = new JVar[workspaceVars.length];
>      for(int i =0 ; i < workspaceVars.length; i++){
> -      workspaceJVars[i] = g.declareClassField("work",
> g.getModel()._ref(workspaceVars[i].type)));
> +      workspaceJVars[i] = g.declareClassField("work",
> g.getModel()._ref(workspaceVars[i].type));
>      }
>
> -    for(WorkspaceReference r : workspaceVars){
> -      g.declareClassField(, t)
> -    }
> -
> -    g.declareClassField(prefix, t)
> +//    for(WorkspaceReference r : workspaceVars){
> +//      g.declareClassField(, t)
> +//    }
> +//
> +//    g.declareClassField(prefix, t)
> +
> +
>      // locally name external blocks.
>
>      // internal out value.
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/3d41be41/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockRecordReader.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockRecordReader.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockRecordReader.java
> index 7b76d05..d710d78 100644
> ---
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockRecordReader.java
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockRecordReader.java
> @@ -28,8 +28,6 @@ import
> org.apache.drill.exec.physical.config.MockScanPOP.MockColumn;
>  import org.apache.drill.exec.physical.config.MockScanPOP.MockScanEntry;
>  import org.apache.drill.exec.physical.impl.OutputMutator;
>  import org.apache.drill.exec.record.MaterializedField;
> -import org.apache.drill.exec.record.vector.TypeHelper;
> -import org.apache.drill.exec.record.vector.ValueVector;
>  import org.apache.drill.exec.store.RecordReader;
>  import org.apache.drill.exec.vector.FixedWidthVector;
>  import org.apache.drill.exec.vector.NonRepeatedMutator;
> @@ -60,12 +58,12 @@ public class MockRecordReader implements RecordReader {
>      return x;
>    }
>
> -  private ValueVector<?> getVector(String name, MajorType type, int
> length) {
> +  private ValueVector getVector(String name, MajorType type, int length) {
>      assert context != null : "Context shouldn't be null.";
>      if(type.getMode() != DataMode.REQUIRED) throw new
> UnsupportedOperationException();
>
>      MaterializedField f = MaterializedField.create(new SchemaPath(name,
> ExpressionPosition.UNKNOWN), type);
> -    ValueVector<?> v;
> +    ValueVector v;
>      v = TypeHelper.getNewVector(f, context.getAllocator());
>      if(v instanceof FixedWidthVector){
>        ((FixedWidthVector)v).allocateNew(length);
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/3d41be41/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockScanPOP.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockScanPOP.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockScanPOP.java
> index 3802ce2..151d541 100644
> ---
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockScanPOP.java
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockScanPOP.java
> @@ -32,7 +32,7 @@ import
> org.apache.drill.exec.physical.base.PhysicalOperator;
>  import org.apache.drill.exec.physical.base.Scan;
>  import org.apache.drill.exec.physical.base.Size;
>  import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
> -import org.apache.drill.exec.record.vector.TypeHelper;
> +import org.apache.drill.exec.vector.TypeHelper;
>
>  import com.fasterxml.jackson.annotation.JsonCreator;
>  import com.fasterxml.jackson.annotation.JsonIgnore;
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/3d41be41/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/FilterRecordBatch.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/FilterRecordBatch.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/FilterRecordBatch.java
> index 1ddc3f9..acabe30 100644
> ---
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/FilterRecordBatch.java
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/FilterRecordBatch.java
> @@ -20,24 +20,17 @@ package org.apache.drill.exec.physical.impl;
>  import org.apache.drill.common.expression.SchemaPath;
>  import org.apache.drill.exec.ops.FragmentContext;
>  import org.apache.drill.exec.record.BatchSchema;
> -import org.apache.drill.exec.record.InvalidValueAccessor;
>  import org.apache.drill.exec.record.RecordBatch;
> -import org.apache.drill.exec.record.RecordBatch.TypedFieldId;
>  import org.apache.drill.exec.record.WritableBatch;
> -import org.apache.drill.exec.record.vector.SelectionVector;
> -<<<<<<< HEAD
> +import org.apache.drill.exec.record.selection.SelectionVector2;
> +import org.apache.drill.exec.record.selection.SelectionVector4;
>  import org.apache.drill.exec.vector.ValueVector;
> -=======
> -import org.apache.drill.exec.record.vector.SelectionVector2;
> -import org.apache.drill.exec.record.vector.SelectionVector4;
> -import org.apache.drill.exec.record.vector.ValueVector;
> ->>>>>>> Build working
>
>  public abstract class FilterRecordBatch implements RecordBatch {
>    static final org.slf4j.Logger logger =
> org.slf4j.LoggerFactory.getLogger(FilterRecordBatch.class);
>
>    private RecordBatch incoming;
> -  private SelectionVector selectionVector;
> +  private SelectionVector2 selectionVector;
>    private BatchSchema schema;
>    private FilteringRecordBatchTransformer transformer;
>    private int outstanding;
> @@ -83,20 +76,15 @@ public abstract class FilterRecordBatch implements
> RecordBatch {
>    }
>
>    @Override
> -  public <T extends ValueVector<T>> T getValueVectorById(int fieldId,
> Class<?> vvClass) {
> +  public <T extends ValueVector> T getValueVectorById(int fieldId,
> Class<?> vvClass) {
>      return null;
>    }
>
> -  @Override
> -<<<<<<< HEAD
> -  public <T extends ValueVector> T getValueVector(int fieldId, Class<T>
> clazz) throws InvalidValueAccessor {
> -=======
>    public WritableBatch getWritableBatch() {
> ->>>>>>> Build working
>      return null;
>    }
>
> -  abstract int applyFilter(SelectionVector vector, int count);
> +  abstract int applyFilter(SelectionVector2 vector, int count);
>
>    /**
>     * Release all assets.
> @@ -130,10 +118,10 @@ public abstract class FilterRecordBatch implements
> RecordBatch {
>          }
>        }
>
> -      // make sure the bit vector is as large as the current record batch.
> -      if (selectionVector.capacity() < incoming.getRecordCount()) {
> -        selectionVector.allocateNew(incoming.getRecordCount());
> -      }
> +//      // make sure the bit vector is as large as the current record
> batch.
> +//      if (selectionVector.capacity() < incoming.getRecordCount()) {
> +//        selectionVector.allocateNew(incoming.getRecordCount());
> +//      }
>
>        return null;
>      }
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/3d41be41/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/FilteringRecordBatchTransformer.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/FilteringRecordBatchTransformer.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/FilteringRecordBatchTransformer.java
> index 191521a..0813481 100644
> ---
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/FilteringRecordBatchTransformer.java
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/FilteringRecordBatchTransformer.java
> @@ -19,16 +19,16 @@ package org.apache.drill.exec.physical.impl;
>
>  import org.apache.drill.exec.record.BatchSchema;
>  import org.apache.drill.exec.record.RecordBatch;
> -import org.apache.drill.exec.record.vector.SelectionVector;
> +import org.apache.drill.exec.record.selection.SelectionVector2;
>
>  public abstract class FilteringRecordBatchTransformer {
>    static final org.slf4j.Logger logger =
> org.slf4j.LoggerFactory.getLogger(FilteringRecordBatchTransformer.class);
>
>    final RecordBatch incoming;
> -  final SelectionVector selectionVector;
> +  final SelectionVector2 selectionVector;
>    final BatchSchema schema;
>
> -  public FilteringRecordBatchTransformer(RecordBatch incoming,
> OutputMutator output, SelectionVector selectionVector) {
> +  public FilteringRecordBatchTransformer(RecordBatch incoming,
> OutputMutator output, SelectionVector2 selectionVector) {
>      super();
>      this.incoming = incoming;
>      this.selectionVector = selectionVector;
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/3d41be41/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/OutputMutator.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/OutputMutator.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/OutputMutator.java
> index 5247d08..7e72683 100644
> ---
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/OutputMutator.java
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/OutputMutator.java
> @@ -18,19 +18,11 @@
>  package org.apache.drill.exec.physical.impl;
>
>  import org.apache.drill.exec.exception.SchemaChangeException;
> -<<<<<<< HEAD
> -import org.apache.drill.exec.vector.ValueVector;
> -
> -public interface OutputMutator {
> -  public void removeField(int fieldId) throws SchemaChangeException;
> -  public void addField(int fieldId, ValueVector vector) throws
> SchemaChangeException ;
> -=======
>  import org.apache.drill.exec.record.MaterializedField;
> -import org.apache.drill.exec.record.vector.ValueVector;
> +import org.apache.drill.exec.vector.ValueVector;
>
>  public interface OutputMutator {
>    public void removeField(MaterializedField field) throws
> SchemaChangeException;
> -  public void addField(ValueVector<?> vector) throws
> SchemaChangeException ;
> ->>>>>>> Build working
> -  public void setNewSchema() throws SchemaChangeException ;
> +  public void addField(ValueVector vector) throws SchemaChangeException ;
> +  public void setNewSchema() throws SchemaChangeException;
>  }
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/3d41be41/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScanBatch.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScanBatch.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScanBatch.java
> index be80349..5688bb1 100644
> ---
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScanBatch.java
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScanBatch.java
> @@ -18,37 +18,27 @@
>  package org.apache.drill.exec.physical.impl;
>
>  import java.util.Iterator;
> -<<<<<<< HEAD
> -=======
>  import java.util.List;
>  import java.util.Map;
> ->>>>>>> Build working
>
>  import org.apache.drill.common.exceptions.ExecutionSetupException;
>  import org.apache.drill.common.expression.SchemaPath;
>  import org.apache.drill.exec.exception.SchemaChangeException;
>  import org.apache.drill.exec.ops.FragmentContext;
>  import org.apache.drill.exec.record.BatchSchema;
> +import org.apache.drill.exec.record.InvalidValueAccessor;
>  import org.apache.drill.exec.record.MaterializedField;
>  import org.apache.drill.exec.record.RecordBatch;
>  import org.apache.drill.exec.record.SchemaBuilder;
>  import org.apache.drill.exec.record.WritableBatch;
> -<<<<<<< HEAD
> -=======
> -import org.apache.drill.exec.record.vector.SelectionVector2;
> -import org.apache.drill.exec.record.vector.SelectionVector4;
> -import org.apache.drill.exec.record.vector.ValueVector;
> ->>>>>>> Build working
> +import org.apache.drill.exec.record.selection.SelectionVector2;
> +import org.apache.drill.exec.record.selection.SelectionVector4;
>  import org.apache.drill.exec.store.RecordReader;
>  import org.apache.drill.exec.vector.ValueVector;
>
> -<<<<<<< HEAD
> -import com.carrotsearch.hppc.IntObjectOpenHashMap;
> -import com.carrotsearch.hppc.procedures.IntObjectProcedure;
> -=======
>  import com.beust.jcommander.internal.Lists;
>  import com.beust.jcommander.internal.Maps;
> ->>>>>>> Build working
> +import com.carrotsearch.hppc.procedures.IntObjectProcedure;
>
>  /**
>   * Record batch used for a particular scan. Operators against one or more
> @@ -56,14 +46,10 @@ import com.beust.jcommander.internal.Maps;
>  public class ScanBatch implements RecordBatch {
>    static final org.slf4j.Logger logger =
> org.slf4j.LoggerFactory.getLogger(ScanBatch.class);
>
> -<<<<<<< HEAD
> -  private IntObjectOpenHashMap<ValueVector> fields = new
> IntObjectOpenHashMap<ValueVector>();
> -=======
> -  final List<ValueVector<?>> vectors = Lists.newLinkedList();
> -  final Map<MaterializedField, ValueVector<?>> fieldVectorMap =
> Maps.newHashMap();
> -
> +  final List<ValueVector> vectors = Lists.newLinkedList();
> +  final Map<MaterializedField, ValueVector> fieldVectorMap =
> Maps.newHashMap();
> +
>    private VectorHolder holder = new VectorHolder(vectors);
> ->>>>>>> Build working
>    private BatchSchema schema;
>    private int recordCount;
>    private boolean schemaChanged = true;
> @@ -72,11 +58,11 @@ public class ScanBatch implements RecordBatch {
>    private RecordReader currentReader;
>    private final Mutator mutator = new Mutator();
>
> -  public ScanBatch(FragmentContext context, Iterator<RecordReader>
> readers)
> -      throws ExecutionSetupException {
> +  public ScanBatch(FragmentContext context, Iterator<RecordReader>
> readers) throws ExecutionSetupException {
>      this.context = context;
>      this.readers = readers;
> -    if (!readers.hasNext()) throw new ExecutionSetupException("A scan
> batch must contain at least one reader.");
> +    if (!readers.hasNext())
> +      throw new ExecutionSetupException("A scan batch must contain at
> least one reader.");
>      this.currentReader = readers.next();
>      this.currentReader.setup(mutator);
>    }
> @@ -107,33 +93,11 @@ public class ScanBatch implements RecordBatch {
>    }
>
>    private void releaseAssets() {
> -<<<<<<< HEAD
> -    fields.forEach(new IntObjectProcedure<ValueVector>() {
> -      @Override
> -      public void apply(int key, ValueVector value) {
> -        value.close();
> -      }
> -    });
> -  }
> -
> -  @SuppressWarnings("unchecked")
> -  @Override
> -  public <T extends ValueVector> T getValueVector(int fieldId, Class<T>
> clazz) throws InvalidValueAccessor {
> -    if (fields.containsKey(fieldId)) throw new
> InvalidValueAccessor(String.format("Unknown value accesor for field id
> %d."));
> -    ValueVector vector = this.fields.lget();
> -    if (vector.getClass().isAssignableFrom(clazz)) {
> -      return (T) vector;
> -    } else {
> -      throw new InvalidValueAccessor(String.format(
> -          "You requested a field accessor of type %s for field id %d but
> the actual type was %s.",
> -          clazz.getCanonicalName(), fieldId,
> vector.getClass().getCanonicalName()));
> -=======
> -    for(ValueVector<?> v : vectors){
> +    for (ValueVector v : vectors) {
>        v.close();
> ->>>>>>> Build working
>      }
>    }
> -
> +
>    @Override
>    public IterOutcome next() {
>      while ((recordCount = currentReader.next()) == 0) {
> @@ -161,7 +125,6 @@ public class ScanBatch implements RecordBatch {
>      }
>    }
>
> -
>    @Override
>    public SelectionVector2 getSelectionVector2() {
>      throw new UnsupportedOperationException();
> @@ -178,37 +141,24 @@ public class ScanBatch implements RecordBatch {
>    }
>
>    @Override
> -  public <T extends ValueVector<T>> T getValueVectorById(int fieldId,
> Class<?> clazz) {
> +  public <T extends ValueVector> T getValueVectorById(int fieldId,
> Class<?> clazz) {
>      return holder.getValueVector(fieldId, clazz);
>    }
>
> -
>    private class Mutator implements OutputMutator {
>      private SchemaBuilder builder = BatchSchema.newBuilder();
> -
> -    public void removeField(MaterializedField field) throws
> SchemaChangeException {
> -      schemaChanged();
> -<<<<<<< HEAD
> -      ValueVector v = fields.remove(fieldId);
> -      if (v == null) throw new SchemaChangeException("Failure attempting
> to remove an unknown field.");
> -      v.close();
> -    }
>
> -    public void addField(int fieldId, ValueVector vector) {
> +    public void removeField(MaterializedField field) throws
> SchemaChangeException {
>        schemaChanged();
> -      ValueVector v = fields.put(fieldId, vector);
> -      vector.getField();
> -=======
> -      ValueVector<?> vector = fieldVectorMap.remove(field);
> +      ValueVector vector = fieldVectorMap.remove(field);
>        if (vector == null) throw new SchemaChangeException("Failure
> attempting to remove an unknown field.");
>        vectors.remove(vector);
>        vector.close();
>      }
>
> -    public void addField(ValueVector<?> vector) {
> +    public void addField(ValueVector vector) {
>        vectors.add(vector);
>        fieldVectorMap.put(vector.getField(), vector);
> ->>>>>>> Build working
>        builder.addField(vector.getField());
>      }
>
> @@ -224,5 +174,5 @@ public class ScanBatch implements RecordBatch {
>    public WritableBatch getWritableBatch() {
>      return WritableBatch.get(this.getRecordCount(), vectors);
>    }
> -
> +
>  }
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/3d41be41/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/VectorHolder.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/VectorHolder.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/VectorHolder.java
> index 4209daa..65a7365 100644
> ---
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/VectorHolder.java
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/VectorHolder.java
> @@ -4,29 +4,29 @@ import java.util.List;
>
>  import org.apache.drill.common.expression.SchemaPath;
>  import org.apache.drill.exec.record.RecordBatch.TypedFieldId;
> -import org.apache.drill.exec.record.vector.ValueVector;
> +import org.apache.drill.exec.vector.ValueVector;
>
>  public class VectorHolder {
>    static final org.slf4j.Logger logger =
> org.slf4j.LoggerFactory.getLogger(VectorHolder.class);
>
> -  private List<ValueVector<?>> vectors;
> +  private List<ValueVector> vectors;
>
> -  public VectorHolder(List<ValueVector<?>> vectors) {
> +  public VectorHolder(List<ValueVector> vectors) {
>      super();
>      this.vectors = vectors;
>    }
>
>    public TypedFieldId getValueVector(SchemaPath path) {
>      for(int i =0; i < vectors.size(); i++){
> -      ValueVector<?> vv = vectors.get(i);
> +      ValueVector vv = vectors.get(i);
>        if(vv.getField().matches(path)) return new
> TypedFieldId(vv.getField().getType(), i);
>      }
>      return null;
>    }
>
>    @SuppressWarnings("unchecked")
> -  public <T extends ValueVector<T>> T getValueVector(int fieldId,
> Class<?> clazz) {
> -    ValueVector<?> v = vectors.get(fieldId);
> +  public <T extends ValueVector> T getValueVector(int fieldId, Class<?>
> clazz) {
> +    ValueVector v = vectors.get(fieldId);
>      assert v != null;
>      if (v.getClass() != clazz){
>        logger.warn(String.format(
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/3d41be41/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/WireRecordBatch.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/WireRecordBatch.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/WireRecordBatch.java
> index 7b2a533..d2b8bfd 100644
> ---
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/WireRecordBatch.java
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/WireRecordBatch.java
> @@ -17,26 +17,20 @@
>
> ******************************************************************************/
>  package org.apache.drill.exec.physical.impl;
>
> -<<<<<<< HEAD
> -=======
>  import org.apache.drill.common.expression.SchemaPath;
> ->>>>>>> Build working
>  import org.apache.drill.exec.exception.SchemaChangeException;
>  import org.apache.drill.exec.ops.FragmentContext;
>  import org.apache.drill.exec.proto.UserBitShared.RecordBatchDef;
>  import org.apache.drill.exec.record.BatchSchema;
> +import org.apache.drill.exec.record.InvalidValueAccessor;
>  import org.apache.drill.exec.record.RawFragmentBatch;
>  import org.apache.drill.exec.record.RawFragmentBatchProvider;
>  import org.apache.drill.exec.record.RecordBatch;
>  import org.apache.drill.exec.record.RecordBatchLoader;
>  import org.apache.drill.exec.record.WritableBatch;
> -<<<<<<< HEAD
> +import org.apache.drill.exec.record.selection.SelectionVector2;
> +import org.apache.drill.exec.record.selection.SelectionVector4;
>  import org.apache.drill.exec.vector.ValueVector;
> -=======
> -import org.apache.drill.exec.record.vector.SelectionVector2;
> -import org.apache.drill.exec.record.vector.SelectionVector4;
> -import org.apache.drill.exec.record.vector.ValueVector;
> ->>>>>>> Build working
>
>  public class WireRecordBatch implements RecordBatch{
>    static final org.slf4j.Logger logger =
> org.slf4j.LoggerFactory.getLogger(WireRecordBatch.class);
> @@ -90,14 +84,9 @@ public class WireRecordBatch implements RecordBatch{
>    }
>
>    @Override
> -<<<<<<< HEAD
> -  public <T extends ValueVector> T getValueVector(int fieldId, Class<T>
> clazz) throws InvalidValueAccessor {
> -=======
> -  public <T extends ValueVector<T>> T getValueVectorById(int fieldId,
> Class<?> clazz) {
> ->>>>>>> Build working
> +  public <T extends ValueVector> T getValueVectorById(int fieldId,
> Class<?> clazz){
>      return batchLoader.getValueVector(fieldId, clazz);
>    }
> -
>
>    @Override
>    public IterOutcome next() {
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/3d41be41/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/filter/EvaluationPredicate.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/filter/EvaluationPredicate.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/filter/EvaluationPredicate.java
> index 3176c41..862e42c 100644
> ---
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/filter/EvaluationPredicate.java
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/filter/EvaluationPredicate.java
> @@ -1,6 +1,6 @@
>  package org.apache.drill.exec.physical.impl.filter;
>
> -import org.apache.drill.exec.record.vector.SelectionVector2;
> +import org.apache.drill.exec.record.selection.SelectionVector2;
>
>  public class EvaluationPredicate {
>    private SelectionVector2 vector;
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/3d41be41/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/filter/ExampleFilter.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/filter/ExampleFilter.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/filter/ExampleFilter.java
> index c4a4032..85f598f 100644
> ---
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/filter/ExampleFilter.java
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/filter/ExampleFilter.java
> @@ -22,13 +22,11 @@ import org.apache.drill.common.expression.SchemaPath;
>  import org.apache.drill.exec.exception.SchemaChangeException;
>  import org.apache.drill.exec.ops.FragmentContext;
>  import org.apache.drill.exec.record.BatchSchema;
> -import org.apache.drill.exec.record.InvalidValueAccessor;
>  import org.apache.drill.exec.record.RecordBatch;
> -import org.apache.drill.exec.record.RecordBatch.TypedFieldId;
>  import org.apache.drill.exec.record.WritableBatch;
> -import org.apache.drill.exec.record.vector.SelectionVector2;
> -import org.apache.drill.exec.record.vector.SelectionVector4;
> -import org.apache.drill.exec.record.vector.ValueVector;
> +import org.apache.drill.exec.record.selection.SelectionVector2;
> +import org.apache.drill.exec.record.selection.SelectionVector4;
> +import org.apache.drill.exec.vector.ValueVector;
>
>  public class ExampleFilter implements RecordBatch {
>    static final org.slf4j.Logger logger =
> org.slf4j.LoggerFactory.getLogger(ExampleFilter.class);
> @@ -83,7 +81,7 @@ public class ExampleFilter implements RecordBatch {
>    }
>
>    @Override
> -  public <T extends ValueVector<T>> T getValueVectorById(int fieldId,
> Class<?> vvClass) {
> +  public <T extends ValueVector> T getValueVectorById(int fieldId,
> Class<?> vvClass) {
>      return null;
>    }
>
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/3d41be41/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/filter/FilterTemplate.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/filter/FilterTemplate.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/filter/FilterTemplate.java
> index d2983ad..216bfec 100644
> ---
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/filter/FilterTemplate.java
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/filter/FilterTemplate.java
> @@ -1,7 +1,7 @@
>  package org.apache.drill.exec.physical.impl.filter;
>
>  import org.apache.drill.exec.record.RecordBatch;
> -import org.apache.drill.exec.record.vector.SelectionVector2;
> +import org.apache.drill.exec.record.selection.SelectionVector2;
>
>  public abstract class FilterTemplate {
>    static final org.slf4j.Logger logger =
> org.slf4j.LoggerFactory.getLogger(FilterTemplate.class);
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/3d41be41/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/ProjectRecordBatch.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/ProjectRecordBatch.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/ProjectRecordBatch.java
> index 7a3fec1..3d1e3f7 100644
> ---
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/ProjectRecordBatch.java
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/ProjectRecordBatch.java
> @@ -28,12 +28,14 @@ import
> org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
>  import org.apache.drill.exec.record.MaterializedField;
>  import org.apache.drill.exec.record.RecordBatch;
>  import org.apache.drill.exec.record.SchemaBuilder;
> +import org.apache.drill.exec.record.TransferPair;
>  import org.apache.drill.exec.record.WritableBatch;
> -import org.apache.drill.exec.record.vector.SelectionVector2;
> -import org.apache.drill.exec.record.vector.SelectionVector4;
> -import org.apache.drill.exec.record.vector.TypeHelper;
> -import org.apache.drill.exec.record.vector.ValueVector;
> -
> +import org.apache.drill.exec.record.selection.SelectionVector2;
> +import org.apache.drill.exec.record.selection.SelectionVector4;
> +import org.apache.drill.exec.vector.AllocationHelper;
> +import org.apache.drill.exec.vector.NonRepeatedMutator;
> +import org.apache.drill.exec.vector.TypeHelper;
> +import org.apache.drill.exec.vector.ValueVector;
>
>  import com.google.common.base.Preconditions;
>  import com.google.common.collect.Lists;
> @@ -46,8 +48,8 @@ public class ProjectRecordBatch implements RecordBatch{
>    private final FragmentContext context;
>    private BatchSchema outSchema;
>    private Projector projector;
> -  private List<ValueVector<?>> allocationVectors;
> -  private List<ValueVector<?>> outputVectors;
> +  private List<ValueVector> allocationVectors;
> +  private List<ValueVector> outputVectors;
>    private VectorHolder vh;
>
>
> @@ -95,7 +97,7 @@ public class ProjectRecordBatch implements RecordBatch{
>    }
>
>    @Override
> -  public <T extends ValueVector<T>> T getValueVectorById(int fieldId,
> Class<?> clazz) {
> +  public <T extends ValueVector> T getValueVectorById(int fieldId,
> Class<?> clazz) {
>      return vh.getValueVector(fieldId, clazz);
>    }
>
> @@ -121,12 +123,17 @@ public class ProjectRecordBatch implements
> RecordBatch{
>        // fall through.
>      case OK:
>        int recordCount = incoming.getRecordCount();
> -      for(ValueVector<?> v : this.allocationVectors){
> -        v.allocateNew(recordCount);
> +      for(ValueVector v : this.allocationVectors){
> +        AllocationHelper.allocate(v, recordCount, 50);
>        }
>        projector.projectRecords(recordCount, 0);
> -      for(ValueVector<?> v : this.outputVectors){
> -        v.setRecordCount(recordCount);
> +      for(ValueVector v : this.outputVectors){
> +        ValueVector.Mutator m = v.getMutator();
> +        if(m instanceof NonRepeatedMutator){
> +          ((NonRepeatedMutator) m).setValueCount(recordCount);
> +        }else{
> +          throw new UnsupportedOperationException();
> +        }
>        }
>        return upstream; // change if upstream changed, otherwise normal.
>      default:
> @@ -138,7 +145,7 @@ public class ProjectRecordBatch implements RecordBatch{
>    private Projector createNewProjector() throws SchemaChangeException{
>      this.allocationVectors = Lists.newArrayList();
>      if(outputVectors != null){
> -      for(ValueVector<?> v : outputVectors){
> +      for(ValueVector v : outputVectors){
>          v.close();
>        }
>      }
> @@ -146,7 +153,7 @@ public class ProjectRecordBatch implements RecordBatch{
>      this.vh = new VectorHolder(outputVectors);
>      final List<NamedExpression> exprs = pop.getExprs();
>      final ErrorCollector collector = new ErrorCollectorImpl();
> -    final List<TransferPairing<?>> transfers = Lists.newArrayList();
> +    final List<TransferPair> transfers = Lists.newArrayList();
>
>      final CodeGenerator<Projector> cg = new
> CodeGenerator<Projector>(Projector.TEMPLATE_DEFINITION,
> context.getFunctionRegistry());
>
> @@ -161,15 +168,15 @@ public class ProjectRecordBatch implements
> RecordBatch{
>        // add value vector to transfer if direct reference and this is
> allowed, otherwise, add to evaluation stack.
>        if(expr instanceof ValueVectorReadExpression &&
> incoming.getSchema().getSelectionVector() == SelectionVectorMode.NONE){
>          ValueVectorReadExpression vectorRead =
> (ValueVectorReadExpression) expr;
> -        ValueVector<?> vvIn =
> incoming.getValueVectorById(vectorRead.getFieldId(),
> TypeHelper.getValueVectorClass(vectorRead.getMajorType()));
> +        ValueVector vvIn =
> incoming.getValueVectorById(vectorRead.getFieldId(),
> TypeHelper.getValueVectorClass(vectorRead.getMajorType().getMinorType(),
> vectorRead.getMajorType().getMode()));
>          Preconditions.checkNotNull(incoming);
>
> -        TransferPairing<?> tp = vvIn.getTransferPair(outputField);
> +        TransferPair tp = vvIn.getTransferPair();
>          transfers.add(tp);
>          outputVectors.add(tp.getTo());
>        }else{
>          // need to do evaluation.
> -        ValueVector<?> vector = TypeHelper.getNewVector(outputField,
> context.getAllocator());
> +        ValueVector vector = TypeHelper.getNewVector(outputField,
> context.getAllocator());
>          allocationVectors.add(vector);
>          outputVectors.add(vector);
>          ValueVectorWriteExpression write = new
> ValueVectorWriteExpression(outputVectors.size() - 1, expr);
> @@ -179,7 +186,7 @@ public class ProjectRecordBatch implements RecordBatch{
>      }
>
>      SchemaBuilder bldr =
> BatchSchema.newBuilder().setSelectionVectorMode(SelectionVectorMode.NONE);
> -    for(ValueVector<?> v : outputVectors){
> +    for(ValueVector v : outputVectors){
>        bldr.addField(v.getField());
>      }
>      this.outSchema = bldr.build();
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/3d41be41/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 8f22b40..2787f0c 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
> @@ -6,10 +6,11 @@ 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.record.RecordBatch;
> +import org.apache.drill.exec.record.TransferPair;
>
>  public interface Projector {
>
> -  public abstract void setup(FragmentContext context, RecordBatch
> incoming,  RecordBatch outgoing, List<TransferPairing<?>> transfers)
>  throws SchemaChangeException;
> +  public abstract void setup(FragmentContext context, RecordBatch
> incoming,  RecordBatch outgoing, List<TransferPair> transfers)  throws
> SchemaChangeException;
>
>
>    public abstract int projectRecords(int recordCount, int
> firstOutputIndex);
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/3d41be41/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/ProjectorTemplate.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/ProjectorTemplate.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/ProjectorTemplate.java
> index 1163fef..486c7b0 100644
> ---
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/ProjectorTemplate.java
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/ProjectorTemplate.java
> @@ -6,15 +6,16 @@ import
> org.apache.drill.exec.exception.SchemaChangeException;
>  import org.apache.drill.exec.ops.FragmentContext;
>  import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
>  import org.apache.drill.exec.record.RecordBatch;
> -import org.apache.drill.exec.record.vector.SelectionVector2;
> -import org.apache.drill.exec.record.vector.SelectionVector4;
> +import org.apache.drill.exec.record.TransferPair;
> +import org.apache.drill.exec.record.selection.SelectionVector2;
> +import org.apache.drill.exec.record.selection.SelectionVector4;
>
>  import com.google.common.collect.ImmutableList;
>
>  public abstract class ProjectorTemplate implements Projector {
>    static final org.slf4j.Logger logger =
> org.slf4j.LoggerFactory.getLogger(ProjectorTemplate.class);
>
> -  private ImmutableList<TransferPairing<?>> transfers;
> +  private ImmutableList<TransferPair> transfers;
>    private SelectionVector2 vector2;
>    private SelectionVector4 vector4;
>    private SelectionVectorMode svMode;
> @@ -39,7 +40,7 @@ public abstract class ProjectorTemplate implements
> Projector {
>
>      case NONE:
>
> -      for(TransferPairing<?> t : transfers){
> +      for(TransferPair t : transfers){
>          t.transfer();
>        }
>        final int countN = recordCount;
> @@ -55,7 +56,7 @@ public abstract class ProjectorTemplate implements
> Projector {
>    }
>
>    @Override
> -  public final void setup(FragmentContext context, RecordBatch incoming,
> RecordBatch outgoing, List<TransferPairing<?>> transfers)  throws
> SchemaChangeException{
> +  public final void setup(FragmentContext context, RecordBatch incoming,
> RecordBatch outgoing, List<TransferPair> transfers)  throws
> SchemaChangeException{
>
>      this.svMode = incoming.getSchema().getSelectionVector();
>      switch(svMode){
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/3d41be41/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/TransferPairing.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/TransferPairing.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/TransferPairing.java
> deleted file mode 100644
> index 2b4ac81..0000000
> ---
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/TransferPairing.java
> +++ /dev/null
> @@ -1,35 +0,0 @@
> -package org.apache.drill.exec.physical.impl.project;
> -
> -import org.apache.drill.exec.record.vector.ValueVector;
> -
> -public class TransferPairing<T extends ValueVector<T>> {
> -
> -  static final org.slf4j.Logger logger =
> org.slf4j.LoggerFactory.getLogger(TransferPairing.class);
> -
> -  final T from;
> -  final T to;
> -
> -  protected TransferPairing(T from, T to) {
> -    super();
> -    this.from = from;
> -    this.to = to;
> -  }
> -
> -  public void transfer(){
> -    from.transferTo(to);
> -  }
> -
> -  public static <T extends ValueVector<T>> TransferPairing<T>
> getTransferPairing(T from, T to){
> -    return new TransferPairing<T>(from, to);
> -  }
> -
> -  public T getFrom() {
> -    return from;
> -  }
> -
> -  public T getTo() {
> -    return to;
> -  }
> -
> -
> -}
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/3d41be41/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/MaterializedField.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/MaterializedField.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/MaterializedField.java
> index 6cf7087..732dab3 100644
> ---
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/MaterializedField.java
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/MaterializedField.java
> @@ -26,7 +26,7 @@ import
> org.apache.drill.common.types.TypeProtos.MajorType;
>  import org.apache.drill.exec.proto.SchemaDefProtos.FieldDef;
>  import org.apache.drill.exec.proto.SchemaDefProtos.NamePart;
>  import org.apache.drill.exec.proto.SchemaDefProtos.NamePart.Type;
> -import org.apache.drill.exec.record.vector.TypeHelper;
> +import org.apache.drill.exec.vector.TypeHelper;
>
>  public class MaterializedField{
>    private final FieldDef def;
> @@ -113,7 +113,8 @@ public class MaterializedField{
>        throw new UnsupportedOperationException();
>      }
>      return new
> MaterializedField(def.toBuilder().setMajorType(mt.toBuilder().setMode(newDataMode).build()).build());
> -
> +  }
> +
>    public Class<?> getValueClass() {
>      return TypeHelper.getValueVectorClass(getType().getMinorType(),
> getDataMode());
>    }
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/3d41be41/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatch.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatch.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatch.java
> index 0edb93e..650a148 100644
> ---
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatch.java
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatch.java
> @@ -20,9 +20,9 @@ package org.apache.drill.exec.record;
>  import org.apache.drill.common.expression.SchemaPath;
>  import org.apache.drill.common.types.TypeProtos.MajorType;
>  import org.apache.drill.exec.ops.FragmentContext;
> -import org.apache.drill.exec.record.vector.SelectionVector2;
> -import org.apache.drill.exec.record.vector.SelectionVector4;
> -import org.apache.drill.exec.record.vector.ValueVector;
> +import org.apache.drill.exec.record.selection.SelectionVector2;
> +import org.apache.drill.exec.record.selection.SelectionVector4;
> +import org.apache.drill.exec.vector.ValueVector;
>
>  /**
>   * A record batch contains a set of field values for a particular range
> of records. In the case of a record batch
> @@ -88,7 +88,7 @@ public interface RecordBatch {
>    public abstract TypedFieldId getValueVectorId(SchemaPath path);
>
>
> -  public abstract <T extends ValueVector<T>> T getValueVectorById(int
> fieldId, Class<?> clazz);
> +  public abstract <T extends ValueVector> T getValueVectorById(int
> fieldId, Class<?> clazz);
>
>    /**
>     * Update the data in each Field reading interface for the next range
> of records. Once a RecordBatch returns an
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/3d41be41/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatchLoader.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatchLoader.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatchLoader.java
> index c3db9f0..e2a1648 100644
> ---
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatchLoader.java
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatchLoader.java
> @@ -30,19 +30,17 @@ import
> org.apache.drill.exec.proto.SchemaDefProtos.FieldDef;
>  import org.apache.drill.exec.proto.UserBitShared.FieldMetadata;
>  import org.apache.drill.exec.proto.UserBitShared.RecordBatchDef;
>  import org.apache.drill.exec.record.RecordBatch.TypedFieldId;
> -import org.apache.drill.exec.record.vector.TypeHelper;
> -import org.apache.drill.exec.record.vector.ValueVector;
> +import org.apache.drill.exec.vector.TypeHelper;
> +import org.apache.drill.exec.vector.ValueVector;
>
>  import com.beust.jcommander.internal.Lists;
>  import com.beust.jcommander.internal.Maps;
> -import com.carrotsearch.hppc.IntObjectOpenHashMap;
> -import com.carrotsearch.hppc.cursors.IntObjectCursor;
>  import com.google.common.collect.ImmutableList;
>
> -public class RecordBatchLoader implements Iterable<ValueVector<?>>{
> +public class RecordBatchLoader implements Iterable<ValueVector>{
>    static final org.slf4j.Logger logger =
> org.slf4j.LoggerFactory.getLogger(RecordBatchLoader.class);
>
> -  private List<ValueVector<?>> vectors = Lists.newArrayList();
> +  private List<ValueVector> vectors = Lists.newArrayList();
>    private final BufferAllocator allocator;
>    private int recordCount;
>    private BatchSchema schema;
> @@ -67,19 +65,19 @@ public class RecordBatchLoader implements
> Iterable<ValueVector<?>>{
>      this.recordCount = def.getRecordCount();
>      boolean schemaChanged = false;
>
> -    Map<MaterializedField, ValueVector<?>> oldFields = Maps.newHashMap();
> -    for(ValueVector<?> v : this.vectors){
> +    Map<MaterializedField, ValueVector> oldFields = Maps.newHashMap();
> +    for(ValueVector v : this.vectors){
>        oldFields.put(v.getField(), v);
>      }
>
> -    List<ValueVector<?>> newVectors = Lists.newArrayList();
> +    List<ValueVector> newVectors = Lists.newArrayList();
>
>      List<FieldMetadata> fields = def.getFieldList();
>
>      int bufOffset = 0;
>      for (FieldMetadata fmd : fields) {
>        FieldDef fieldDef = fmd.getDef();
> -      ValueVector<?> v = oldFields.remove(fieldDef);
> +      ValueVector v = oldFields.remove(fieldDef);
>        if(v != null){
>          newVectors.add(v);
>          continue;
> @@ -90,21 +88,19 @@ public class RecordBatchLoader implements
> Iterable<ValueVector<?>>{
>        MaterializedField m = new MaterializedField(fieldDef);
>        v = TypeHelper.getNewVector(m, allocator);
>        v.load(fmd, buf.slice(bufOffset, fmd.getBufferLength()));
> -      newVectors.put(fieldDef.getFieldId(), v);
> -      v.setTo(fmd, buf.slice(bufOffset, fmd.getBufferLength()));
>        newVectors.add(v);
>      }
>
>      if(!oldFields.isEmpty()){
>        schemaChanged = true;
> -      for(ValueVector<?> v : oldFields.values()){
> +      for(ValueVector v : oldFields.values()){
>          v.close();
>        }
>      }
>
>      // rebuild the schema.
>      SchemaBuilder b = BatchSchema.newBuilder();
> -    for(ValueVector<?> v : newVectors){
> +    for(ValueVector v : newVectors){
>        b.addField(v.getField());
>      }
>      b.setSelectionVectorMode(BatchSchema.SelectionVectorMode.NONE);
> @@ -116,15 +112,15 @@ public class RecordBatchLoader implements
> Iterable<ValueVector<?>>{
>
>    public TypedFieldId getValueVector(SchemaPath path) {
>      for(int i =0; i < vectors.size(); i++){
> -      ValueVector<?> vv = vectors.get(i);
> +      ValueVector vv = vectors.get(i);
>        if(vv.getField().matches(path)) return new
> TypedFieldId(vv.getField().getType(), i);
>      }
>      return null;
>    }
>
>    @SuppressWarnings("unchecked")
> -  public <T extends ValueVector<T>> T getValueVector(int fieldId,
> Class<?> clazz) {
> -    ValueVector<?> v = vectors.get(fieldId);
> +  public <T extends ValueVector> T getValueVector(int fieldId, Class<?>
> clazz) {
> +    ValueVector v = vectors.get(fieldId);
>      assert v != null;
>      if (v.getClass() != clazz){
>        logger.warn(String.format(
> @@ -145,7 +141,7 @@ public class RecordBatchLoader implements
> Iterable<ValueVector<?>>{
>    }
>
>    @Override
> -  public Iterator<ValueVector<?>> iterator() {
> +  public Iterator<ValueVector> iterator() {
>      return this.vectors.iterator();
>    }
>
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/3d41be41/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/TransferPair.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/TransferPair.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/TransferPair.java
> new file mode 100644
> index 0000000..a90e2d8
> --- /dev/null
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/TransferPair.java
> @@ -0,0 +1,8 @@
> +package org.apache.drill.exec.record;
> +
> +import org.apache.drill.exec.vector.ValueVector;
> +
> +public interface TransferPair {
> +  public void transfer();
> +  public ValueVector getTo();
> +}
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/3d41be41/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/WritableBatch.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/WritableBatch.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/WritableBatch.java
> index 2e1754c..8335e91 100644
> ---
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/WritableBatch.java
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/WritableBatch.java
> @@ -25,8 +25,6 @@ import
> org.apache.drill.exec.proto.UserBitShared.FieldMetadata;
>  import org.apache.drill.exec.proto.UserBitShared.RecordBatchDef;
>  import org.apache.drill.exec.vector.ValueVector;
>
> -import com.carrotsearch.hppc.IntObjectOpenHashMap;
> -import com.carrotsearch.hppc.procedures.IntObjectProcedure;
>  import com.google.common.collect.Lists;
>
>  /**
> @@ -58,20 +56,20 @@ public class WritableBatch {
>      return buffers;
>    }
>
> -  public static WritableBatch get(int recordCount, List<ValueVector<?>>
> vectors){
> +  public static WritableBatch get(int recordCount, List<ValueVector>
> vectors){
>
>      List<ByteBuf> buffers = Lists.newArrayList();
>      List<FieldMetadata> metadata = Lists.newArrayList();
>
>
> -    for(ValueVector<?> vv : vectors){
> +    for(ValueVector vv : vectors){
>        metadata.add(vv.getMetadata());
>        for(ByteBuf b : vv.getBuffers()){
>          buffers.add(b);
>          b.retain();
>        }
>        // allocate new buffer to release hold on old buffer.
> -      vv.allocateNew(vv.capacity());
> +      vv.clear();
>      }
>
>      RecordBatchDef batchDef =
> RecordBatchDef.newBuilder().addAllField(metadata).setRecordCount(recordCount).build();
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/3d41be41/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/selection/SelectionVector2.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/selection/SelectionVector2.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/selection/SelectionVector2.java
> new file mode 100644
> index 0000000..cdc136e
> --- /dev/null
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/selection/SelectionVector2.java
> @@ -0,0 +1,49 @@
>
> +/*******************************************************************************
> + * Licensed to the Apache Software Foundation (ASF) under one
> + * or more contributor license agreements.  See the NOTICE file
> + * distributed with this work for additional information
> + * regarding copyright ownership.  The ASF licenses this file
> + * to you under the Apache License, Version 2.0 (the
> + * "License"); you may not use this file except in compliance
> + * with the License.  You may obtain a copy of the License at
> + *
> + * http://www.apache.org/licenses/LICENSE-2.0
> + *
> + * Unless required by applicable law or agreed to in writing, software
> + * distributed under the License is distributed on an "AS IS" BASIS,
> + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
> implied.
> + * See the License for the specific language governing permissions and
> + * limitations under the License.
> +
> ******************************************************************************/
> +package org.apache.drill.exec.record.selection;
> +
> +import io.netty.buffer.ByteBuf;
> +
> +import org.apache.drill.exec.memory.BufferAllocator;
> +import org.apache.drill.exec.record.DeadBuf;
> +
> +/**
> + * A selection vector that fronts, at most, a
> + */
> +public class SelectionVector2{
> +  static final org.slf4j.Logger logger =
> org.slf4j.LoggerFactory.getLogger(SelectionVector2.class);
> +
> +  private final BufferAllocator allocator;
> +  private ByteBuf buffer = DeadBuf.DEAD_BUFFER;
> +
> +  public SelectionVector2(BufferAllocator allocator) {
> +    this.allocator = allocator;
> +  }
> +
> +  public int getCount(){
> +    return -1;
> +  }
> +
> +  public int getIndex(int directIndex){
> +    return buffer.getChar(directIndex);
> +  }
> +
> +  public void setIndex(int directIndex, char value){
> +    buffer.setChar(directIndex, value);
> +  }
> +}
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/3d41be41/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/selection/SelectionVector4.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/selection/SelectionVector4.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/selection/SelectionVector4.java
> new file mode 100644
> index 0000000..1f3874f
> --- /dev/null
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/selection/SelectionVector4.java
> @@ -0,0 +1,41 @@
>
> +/*******************************************************************************
> + * Licensed to the Apache Software Foundation (ASF) under one
> + * or more contributor license agreements.  See the NOTICE file
> + * distributed with this work for additional information
> + * regarding copyright ownership.  The ASF licenses this file
> + * to you under the Apache License, Version 2.0 (the
> + * "License"); you may not use this file except in compliance
> + * with the License.  You may obtain a copy of the License at
> + *
> + * http://www.apache.org/licenses/LICENSE-2.0
> + *
> + * Unless required by applicable law or agreed to in writing, software
> + * distributed under the License is distributed on an "AS IS" BASIS,
> + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
> implied.
> + * See the License for the specific language governing permissions and
> + * limitations under the License.
> +
> ******************************************************************************/
> +
> +package org.apache.drill.exec.record.selection;
> +
> +import io.netty.buffer.ByteBuf;
> +
> +import org.apache.drill.exec.memory.BufferAllocator;
> +import org.apache.drill.exec.record.DeadBuf;
> +
> +public class SelectionVector4 {
> +  static final org.slf4j.Logger logger =
> org.slf4j.LoggerFactory.getLogger(SelectionVector4.class);
> +
> +  private final BufferAllocator allocator;
> +  private ByteBuf buffer = DeadBuf.DEAD_BUFFER;
> +
> +  public SelectionVector4(BufferAllocator allocator) {
> +    this.allocator = allocator;
> +  }
> +
> +  public int getCount(){
> +    return -1;
> +  }
> +
> +
> +}
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/3d41be41/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/BitUtil.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/BitUtil.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/BitUtil.java
> deleted file mode 100644
> index 4a3ae2f..0000000
> ---
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/BitUtil.java
> +++ /dev/null
> @@ -1,108 +0,0 @@
>
> -/*******************************************************************************
> - * Licensed to the Apache Software Foundation (ASF) under one
> - * or more contributor license agreements.  See the NOTICE file
> - * distributed with this work for additional information
> - * regarding copyright ownership.  The ASF licenses this file
> - * to you under the Apache License, Version 2.0 (the
> - * "License"); you may not use this file except in compliance
> - * with the License.  You may obtain a copy of the License at
> - *
> - * http://www.apache.org/licenses/LICENSE-2.0
> - *
> - * Unless required by applicable law or agreed to in writing, software
> - * distributed under the License is distributed on an "AS IS" BASIS,
> - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
> implied.
> - * See the License for the specific language governing permissions and
> - * limitations under the License.
> -
> ******************************************************************************/
> -package org.apache.drill.exec.record.vector;  // from
> org.apache.solr.util rev 555343
> -
> -import io.netty.buffer.ByteBuf;
> -
> -
> -/**  A variety of high efficiency bit twiddling routines.
> - * @lucene.internal
> - */
> -public final class BitUtil {
> -
> -  private BitUtil() {} // no instance
> -
> -  // The pop methods used to rely on bit-manipulation tricks for speed
> but it
> -  // turns out that it is faster to use the Long.bitCount method (which
> is an
> -  // intrinsic since Java 6u18) in a naive loop, see LUCENE-2221
> -
> -  /** Returns the number of set bits in an array of longs. */
> -  public static long pop_array(ByteBuf arr, int wordOffset, int numWords)
> {
> -    long popCount = 0;
> -    for (int i = wordOffset, end = wordOffset + numWords; i < end; i+=8) {
> -      popCount += Long.bitCount(arr.getLong(i));
> -    }
> -    return popCount;
> -  }
> -
> -  /** Returns the popcount or cardinality of the two sets after an
> intersection.
> -   *  Neither array is modified. */
> -  public static long pop_intersect(ByteBuf arr1, ByteBuf arr2, int
> wordOffset, int numWords) {
> -    long popCount = 0;
> -    for (int i = wordOffset, end = wordOffset + numWords; i < end; i+=8) {
> -      popCount += Long.bitCount(arr1.getLong(i) & arr2.getLong(i));
> -    }
> -    return popCount;
> -  }
> -
> -   /** Returns the popcount or cardinality of the union of two sets.
> -    *  Neither array is modified. */
> -   public static long pop_union(ByteBuf arr1, ByteBuf arr2, int
> wordOffset, int numWords) {
> -     long popCount = 0;
> -     for (int i = wordOffset, end = wordOffset + numWords; i < end; i+=8)
> {
> -       popCount += Long.bitCount(arr1.getLong(i) | arr2.getLong(i));
> -     }
> -     return popCount;
> -   }
> -
> -  /** Returns the popcount or cardinality of A & ~B.
> -   *  Neither array is modified. */
> -  public static long pop_andnot(ByteBuf arr1, ByteBuf arr2, int
> wordOffset, int numWords) {
> -    long popCount = 0;
> -    for (int i = wordOffset, end = wordOffset + numWords; i < end; i+=8) {
> -      popCount += Long.bitCount(arr1.getLong(i) & ~arr2.getLong(i));
> -    }
> -    return popCount;
> -  }
> -
> -  /** Returns the popcount or cardinality of A ^ B
> -    * Neither array is modified. */
> -  public static long pop_xor(ByteBuf arr1, ByteBuf arr2, int wordOffset,
> int numWords) {
> -    long popCount = 0;
> -    for (int i = wordOffset, end = wordOffset + numWords; i < end; i+=8) {
> -      popCount += Long.bitCount(arr1.getLong(i) ^ arr2.getLong(i));
> -    }
> -    return popCount;
> -  }
> -
> -  /** returns the next highest power of two, or the current value if it's
> already a power of two or zero*/
> -  public static int nextHighestPowerOfTwo(int v) {
> -    v--;
> -    v |= v >> 1;
> -    v |= v >> 2;
> -    v |= v >> 4;
> -    v |= v >> 8;
> -    v |= v >> 16;
> -    v++;
> -    return v;
> -  }
> -
> -  /** returns the next highest power of two, or the current value if it's
> already a power of two or zero*/
> -   public static long nextHighestPowerOfTwo(long v) {
> -    v--;
> -    v |= v >> 1;
> -    v |= v >> 2;
> -    v |= v >> 4;
> -    v |= v >> 8;
> -    v |= v >> 16;
> -    v |= v >> 32;
> -    v++;
> -    return v;
> -  }
> -
> -}
> \ No newline at end of file
>
>

[48/53] [abbrv] git commit: Working Project merge build

Posted by ja...@apache.org.
Working Project merge build


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

Branch: refs/heads/master
Commit: 3d41be41d6a1c9dab7256a1044370348848a1ec3
Parents: 80b1d24
Author: Jacques Nadeau <ja...@apache.org>
Authored: Tue Jul 16 10:45:25 2013 -0700
Committer: Jacques Nadeau <ja...@apache.org>
Committed: Fri Jul 19 14:53:31 2013 -0700

----------------------------------------------------------------------
 .../common/src/main/protobuf/Types.proto        |   1 +
 sandbox/prototype/exec/java-exec/pom.xml        |  30 -
 .../templates/FixedValueVectors.java            |  31 +
 .../templates/NullableValueVectors.java         |  28 +
 .../templates/RepeatedValueVectors.java         |  38 +-
 .../ValueVectors/templates/TypeHelper.java      |   6 +-
 .../templates/VariableLengthVectors.java        |  29 +
 .../drill/exec/expr/EvaluationVisitor.java      |   5 +-
 .../exec/expr/ValueVectorReadExpression.java    |   1 -
 .../exec/expr/ValueVectorWriteExpression.java   |   1 -
 .../drill/exec/expr/fn/FunctionHolder.java      |  15 +-
 .../exec/physical/config/MockRecordReader.java  |   6 +-
 .../drill/exec/physical/config/MockScanPOP.java |   2 +-
 .../exec/physical/impl/FilterRecordBatch.java   |  30 +-
 .../impl/FilteringRecordBatchTransformer.java   |   6 +-
 .../drill/exec/physical/impl/OutputMutator.java |  14 +-
 .../drill/exec/physical/impl/ScanBatch.java     |  84 +-
 .../drill/exec/physical/impl/VectorHolder.java  |  12 +-
 .../exec/physical/impl/WireRecordBatch.java     |  19 +-
 .../impl/filter/EvaluationPredicate.java        |   2 +-
 .../physical/impl/filter/ExampleFilter.java     |  10 +-
 .../physical/impl/filter/FilterTemplate.java    |   2 +-
 .../impl/project/ProjectRecordBatch.java        |  43 +-
 .../exec/physical/impl/project/Projector.java   |   3 +-
 .../impl/project/ProjectorTemplate.java         |  11 +-
 .../physical/impl/project/TransferPairing.java  |  35 -
 .../drill/exec/record/MaterializedField.java    |   5 +-
 .../apache/drill/exec/record/RecordBatch.java   |   8 +-
 .../drill/exec/record/RecordBatchLoader.java    |  32 +-
 .../apache/drill/exec/record/TransferPair.java  |   8 +
 .../apache/drill/exec/record/WritableBatch.java |   8 +-
 .../exec/record/selection/SelectionVector2.java |  49 ++
 .../exec/record/selection/SelectionVector4.java |  41 +
 .../drill/exec/record/vector/BitUtil.java       | 108 ---
 .../drill/exec/record/vector/BufBitSet.java     | 847 -------------------
 .../exec/record/vector/NullValueException.java  |   9 -
 .../exec/record/vector/NullableFixed8.java      |  43 -
 .../exec/record/vector/SelectionVector.java     |  49 --
 .../exec/record/vector/SelectionVector2.java    |  49 --
 .../exec/record/vector/SelectionVector4.java    |  41 -
 .../drill/exec/store/JSONRecordReader.java      |  19 +-
 .../drill/exec/vector/AllocationHelper.java     |  15 +
 .../drill/exec/vector/BaseValueVector.java      |   2 -
 .../org/apache/drill/exec/vector/BitVector.java |  27 +
 .../apache/drill/exec/vector/ValueVector.java   |   5 +-
 .../apache/drill/exec/expr/ExpressionTest.java  |  11 +-
 .../exec/physical/impl/SimpleRootExec.java      |   4 +-
 .../physical/impl/TestSimpleFragmentRun.java    |  58 +-
 .../impl/project/TestSimpleProjection.java      |  17 +-
 .../exec/record/vector/TestOpenBitSet.java      | 361 --------
 .../exec/record/vector/TestValueVector.java     |  55 +-
 .../drill/exec/store/JSONRecordReaderTest.java  |  41 +-
 52 files changed, 482 insertions(+), 1894 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/3d41be41/sandbox/prototype/common/src/main/protobuf/Types.proto
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/protobuf/Types.proto b/sandbox/prototype/common/src/main/protobuf/Types.proto
index 58399de..17e371d 100644
--- a/sandbox/prototype/common/src/main/protobuf/Types.proto
+++ b/sandbox/prototype/common/src/main/protobuf/Types.proto
@@ -15,6 +15,7 @@ enum MinorType {
     BIGINT = 6;   //  eight byte signed integer
     DECIMAL4 = 7;   //  a decimal supporting precision between 1 and 8 (4 bits for decimal location, 1 sign)
     DECIMAL8 = 8;   //  a decimal supporting precision between 9 and 18 (5 bits for decimal location, 1 sign)
+    DECIMAL12 = 9;   //  a decimal supporting precision between 
     DECIMAL16 = 10;   //  a decimal supporting precision between 19 and 37 (6 bits for decimal location, 1 sign)
     MONEY = 11;   //  signed decimal with two digit precision
     DATE = 12;   //  days since 4713bc 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/3d41be41/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 f2f0bff..0290be1 100644
--- a/sandbox/prototype/exec/java-exec/pom.xml
+++ b/sandbox/prototype/exec/java-exec/pom.xml
@@ -179,7 +179,6 @@
                 <exec executable="protoc">
                   <arg value="--java_out=${target.gen.source.path}" />
                   <arg value="--proto_path=${proto.cas.path}" />
-<<<<<<< HEAD
                   <arg value="--proto_path=${project.basedir}/../../common/src/main/protobuf/" />
 									<arg line="${proto.files}" />
 								</exec>
@@ -226,34 +225,5 @@
 			<!-- </plugin> -->
 		</plugins>
 	</build>
-=======
-                  <arg
-                    value="--proto_path=${project.basedir}/../../common/src/main/protobuf/" />
-                  <arg line="${proto.files}" />
-                </exec>
-              </tasks>
-              <sourceRoot>${target.gen.source.path}</sourceRoot>
-            </configuration>
-            <goals>
-              <goal>run</goal>
-            </goals>
-          </execution>
-        </executions>
-      </plugin>
-      <!-- <plugin> -->
-      <!-- <groupId>com.github.igor-petruk.protobuf</groupId> -->
-      <!-- <artifactId>protobuf-maven-plugin</artifactId> -->
-      <!-- <version>0.6.2</version> -->
-      <!-- <executions> -->
-      <!-- <execution> -->
-      <!-- <goals> -->
-      <!-- <goal>run</goal> -->
-      <!-- </goals> -->
-      <!-- </execution> -->
-      <!-- </executions> -->
-      <!-- </plugin> -->
-    </plugins>
-  </build>
->>>>>>> Build working
 
 </project>

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/3d41be41/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/FixedValueVectors.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/FixedValueVectors.java b/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/FixedValueVectors.java
index 7583d9f..e0c8406 100644
--- a/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/FixedValueVectors.java
+++ b/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/FixedValueVectors.java
@@ -9,6 +9,7 @@ package org.apache.drill.exec.vector;
 import io.netty.buffer.ByteBuf;
 
 import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.record.TransferPair;
 import org.apache.drill.exec.proto.UserBitShared.FieldMetadata;
 import org.apache.drill.exec.record.DeadBuf;
 import org.apache.drill.exec.record.MaterializedField;
@@ -49,6 +50,8 @@ public final class ${minor.class}Vector extends BaseDataValueVector implements F
   public Mutator getMutator(){
     return mutator;
   }
+  
+
 
   /**
    * Allocate a new buffer that supports setting at least the provided number of values.  May actually be sized bigger depending on underlying buffer rounding size. Must be called prior to using the ValueVector.
@@ -87,6 +90,33 @@ public final class ${minor.class}Vector extends BaseDataValueVector implements F
     assert metadata.getBufferLength() == loaded;
   }
   
+  public TransferPair getTransferPair(){
+    return new TransferImpl();
+  }
+  
+  public void transferTo(${minor.class}Vector target){
+    target.data = data;
+    target.data.retain();
+    target.recordCount = recordCount;
+    clear();
+  }
+  
+  private class TransferImpl implements TransferPair{
+    ${minor.class}Vector to;
+    
+    public TransferImpl(){
+      this.to = new ${minor.class}Vector(getField(), allocator);
+    }
+    
+    public ${minor.class}Vector getTo(){
+      return to;
+    }
+    
+    public void transfer(){
+      transferTo(to);
+    }
+  }
+  
   public final class Accessor extends BaseValueVector.BaseAccessor{
 
     public int getRecordCount() {
@@ -184,6 +214,7 @@ public final class ${minor.class}Vector extends BaseDataValueVector implements F
 
 
 
+
   
  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/3d41be41/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/NullableValueVectors.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/NullableValueVectors.java b/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/NullableValueVectors.java
index 3232f87..d2d436a 100644
--- a/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/NullableValueVectors.java
+++ b/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/NullableValueVectors.java
@@ -17,6 +17,7 @@ import org.apache.drill.exec.proto.SchemaDefProtos;
 import org.apache.drill.exec.proto.UserBitShared.FieldMetadata;
 import org.apache.drill.exec.record.DeadBuf;
 import org.apache.drill.exec.record.MaterializedField;
+import org.apache.drill.exec.record.TransferPair;
 import org.apache.drill.exec.vector.BaseValueVector;
 import org.apache.drill.exec.vector.BitVector;
 import org.apache.drill.exec.vector.UInt2Vector;
@@ -146,6 +147,33 @@ public final class Nullable${minor.class}Vector extends BaseValueVector implemen
   
   </#if>
   
+  public TransferPair getTransferPair(){
+    return new TransferImpl();
+  }
+  
+  public void transferTo(Nullable${minor.class}Vector target){
+    bits.transferTo(target.bits);
+    values.transferTo(target.values);
+    target.recordCount = recordCount;
+    clear();
+  }
+  
+  private class TransferImpl implements TransferPair{
+    Nullable${minor.class}Vector to;
+    
+    public TransferImpl(){
+      this.to = new Nullable${minor.class}Vector(getField(), allocator);
+    }
+    
+    public Nullable${minor.class}Vector getTo(){
+      return to;
+    }
+    
+    public void transfer(){
+      transferTo(to);
+    }
+  }
+  
   public Accessor getAccessor(){
     return accessor;
   }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/3d41be41/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/RepeatedValueVectors.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/RepeatedValueVectors.java b/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/RepeatedValueVectors.java
index 363e4c8..1976340 100644
--- a/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/RepeatedValueVectors.java
+++ b/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/RepeatedValueVectors.java
@@ -7,11 +7,6 @@ import org.apache.drill.exec.vector.UInt4Vector;
 <@pp.changeOutputFile name="Repeated${minor.class}Vector.java" />
 package org.apache.drill.exec.vector;
 
-
-
-
-
-
 import static com.google.common.base.Preconditions.checkArgument;
 import static com.google.common.base.Preconditions.checkState;
 import io.netty.buffer.ByteBuf;
@@ -25,6 +20,7 @@ import org.apache.drill.exec.proto.SchemaDefProtos;
 import org.apache.drill.exec.proto.UserBitShared.FieldMetadata;
 import org.apache.drill.exec.record.DeadBuf;
 import org.apache.drill.exec.record.MaterializedField;
+import org.apache.drill.exec.record.TransferPair;
 
 @SuppressWarnings("unused")
 /**
@@ -65,6 +61,35 @@ import org.apache.drill.exec.record.MaterializedField;
     return counts.getBufferSize() + offsets.getBufferSize() + values.getBufferSize();
   }
   
+  public TransferPair getTransferPair(){
+    return new TransferImpl();
+  }
+  
+  public void transferTo(Repeated${minor.class}Vector target){
+    counts.transferTo(target.counts);
+    offsets.transferTo(target.offsets);
+    values.transferTo(target.values);
+    target.parentValueCount = parentValueCount;
+    target.childValueCount = childValueCount;
+    clear();
+  }
+  
+  private class TransferImpl implements TransferPair{
+    Repeated${minor.class}Vector to;
+    
+    public TransferImpl(){
+      this.to = new Repeated${minor.class}Vector(getField(), allocator);
+    }
+    
+    public Repeated${minor.class}Vector getTo(){
+      return to;
+    }
+    
+    public void transfer(){
+      transferTo(to);
+    }
+  }
+  
   <#if type.major == "VarLen">
   @Override
   public FieldMetadata getMetadata() {
@@ -84,9 +109,6 @@ import org.apache.drill.exec.record.MaterializedField;
     mutator.reset();
     accessor.reset();
   }
-
-  
-
   
   @Override
   public int load(int dataBytes, int parentValueCount, int childValueCount, ByteBuf buf){

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/3d41be41/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/TypeHelper.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/TypeHelper.java b/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/TypeHelper.java
index b03b842..b73cd21 100644
--- a/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/TypeHelper.java
+++ b/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/TypeHelper.java
@@ -18,9 +18,9 @@
 package org.apache.drill.exec.vector;
 
 import org.apache.drill.exec.memory.BufferAllocator;
-import org.apache.drill.exec.proto.SchemaDefProtos.DataMode;
-import org.apache.drill.exec.proto.SchemaDefProtos.MajorType;
-import org.apache.drill.exec.proto.SchemaDefProtos.MinorType;
+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.exec.record.MaterializedField;
 
 public class TypeHelper {

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/3d41be41/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/VariableLengthVectors.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/VariableLengthVectors.java b/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/VariableLengthVectors.java
index c615258..9d723ab 100644
--- a/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/VariableLengthVectors.java
+++ b/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/VariableLengthVectors.java
@@ -19,6 +19,7 @@ import org.apache.drill.exec.proto.SchemaDefProtos;
 import org.apache.drill.exec.proto.UserBitShared.FieldMetadata;
 import org.apache.drill.exec.record.DeadBuf;
 import org.apache.drill.exec.record.MaterializedField;
+import org.apache.drill.exec.record.TransferPair;
 import org.apache.drill.exec.vector.ByteHolder;
 
 /**
@@ -102,6 +103,34 @@ public final class ${minor.class}Vector extends BaseDataValueVector implements V
     return new ByteBuf[]{offsetVector.data, this.data};
   }
   
+  public TransferPair getTransferPair(){
+    return new TransferImpl();
+  }
+  
+  public void transferTo(${minor.class}Vector target){
+    this.offsetVector.transferTo(target.offsetVector);
+    target.data = data;
+    target.data.retain();
+    target.recordCount = recordCount;
+    clear();
+  }
+  
+  private class TransferImpl implements TransferPair{
+    ${minor.class}Vector to;
+    
+    public TransferImpl(){
+      this.to = new ${minor.class}Vector(getField(), allocator);
+    }
+    
+    public ${minor.class}Vector getTo(){
+      return to;
+    }
+    
+    public void transfer(){
+      transferTo(to);
+    }
+  }
+  
   public void allocateNew(int totalBytes, int valueCount) {
     clear();
     assert totalBytes >= 0;

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/3d41be41/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 04f78d0..6b0e499 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
@@ -15,15 +15,14 @@ import org.apache.drill.exec.expr.CodeGenerator.HoldingContainer;
 import org.apache.drill.exec.expr.fn.FunctionHolder;
 import org.apache.drill.exec.expr.fn.FunctionImplementationRegistry;
 import org.apache.drill.exec.physical.impl.filter.SelectionVectorPopulationExpression;
-import org.apache.drill.exec.record.vector.SelectionVector2;
-import org.apache.drill.exec.record.vector.TypeHelper;
+import org.apache.drill.exec.record.selection.SelectionVector2;
+import org.apache.drill.exec.vector.TypeHelper;
 
 import com.google.common.base.Preconditions;
 import com.sun.codemodel.JBlock;
 import com.sun.codemodel.JClass;
 import com.sun.codemodel.JConditional;
 import com.sun.codemodel.JExpr;
-import com.sun.codemodel.JExpression;
 import com.sun.codemodel.JType;
 import com.sun.codemodel.JVar;
 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/3d41be41/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 c8d906d..b1888d6 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
@@ -5,7 +5,6 @@ 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.RecordBatch.TypedFieldId;
-import org.apache.drill.exec.record.vector.ValueVector;
 
 public class ValueVectorReadExpression implements LogicalExpression{
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ValueVectorReadExpression.class);

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/3d41be41/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 96d9d05..417b975 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
@@ -5,7 +5,6 @@ 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.common.types.Types;
-import org.apache.drill.exec.record.vector.ValueVector;
 
 public class ValueVectorWriteExpression implements LogicalExpression {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ValueVectorWriteExpression.class);

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/3d41be41/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 f7ecd02..9d7fc35 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,6 +1,5 @@
 package org.apache.drill.exec.expr.fn;
 
-import java.lang.reflect.Type;
 import java.util.Arrays;
 import java.util.Map;
 
@@ -92,14 +91,16 @@ public class FunctionHolder {
     
     JVar[] workspaceJVars = new JVar[workspaceVars.length];
     for(int i =0 ; i < workspaceVars.length; i++){
-      workspaceJVars[i] = g.declareClassField("work", g.getModel()._ref(workspaceVars[i].type)));
+      workspaceJVars[i] = g.declareClassField("work", g.getModel()._ref(workspaceVars[i].type));
     }
     
-    for(WorkspaceReference r : workspaceVars){
-      g.declareClassField(, t)
-    }
-  
-    g.declareClassField(prefix, t)
+//    for(WorkspaceReference r : workspaceVars){
+//      g.declareClassField(, t)
+//    }
+//  
+//    g.declareClassField(prefix, t)
+    
+    
     // locally name external blocks.
     
     // internal out value.

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/3d41be41/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockRecordReader.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockRecordReader.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockRecordReader.java
index 7b76d05..d710d78 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockRecordReader.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockRecordReader.java
@@ -28,8 +28,6 @@ import org.apache.drill.exec.physical.config.MockScanPOP.MockColumn;
 import org.apache.drill.exec.physical.config.MockScanPOP.MockScanEntry;
 import org.apache.drill.exec.physical.impl.OutputMutator;
 import org.apache.drill.exec.record.MaterializedField;
-import org.apache.drill.exec.record.vector.TypeHelper;
-import org.apache.drill.exec.record.vector.ValueVector;
 import org.apache.drill.exec.store.RecordReader;
 import org.apache.drill.exec.vector.FixedWidthVector;
 import org.apache.drill.exec.vector.NonRepeatedMutator;
@@ -60,12 +58,12 @@ public class MockRecordReader implements RecordReader {
     return x;
   }
 
-  private ValueVector<?> getVector(String name, MajorType type, int length) {
+  private ValueVector getVector(String name, MajorType type, int length) {
     assert context != null : "Context shouldn't be null.";
     if(type.getMode() != DataMode.REQUIRED) throw new UnsupportedOperationException();
     
     MaterializedField f = MaterializedField.create(new SchemaPath(name, ExpressionPosition.UNKNOWN), type);
-    ValueVector<?> v;
+    ValueVector v;
     v = TypeHelper.getNewVector(f, context.getAllocator());
     if(v instanceof FixedWidthVector){
       ((FixedWidthVector)v).allocateNew(length);  

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/3d41be41/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockScanPOP.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockScanPOP.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockScanPOP.java
index 3802ce2..151d541 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockScanPOP.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockScanPOP.java
@@ -32,7 +32,7 @@ import org.apache.drill.exec.physical.base.PhysicalOperator;
 import org.apache.drill.exec.physical.base.Scan;
 import org.apache.drill.exec.physical.base.Size;
 import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
-import org.apache.drill.exec.record.vector.TypeHelper;
+import org.apache.drill.exec.vector.TypeHelper;
 
 import com.fasterxml.jackson.annotation.JsonCreator;
 import com.fasterxml.jackson.annotation.JsonIgnore;

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/3d41be41/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/FilterRecordBatch.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/FilterRecordBatch.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/FilterRecordBatch.java
index 1ddc3f9..acabe30 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/FilterRecordBatch.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/FilterRecordBatch.java
@@ -20,24 +20,17 @@ package org.apache.drill.exec.physical.impl;
 import org.apache.drill.common.expression.SchemaPath;
 import org.apache.drill.exec.ops.FragmentContext;
 import org.apache.drill.exec.record.BatchSchema;
-import org.apache.drill.exec.record.InvalidValueAccessor;
 import org.apache.drill.exec.record.RecordBatch;
-import org.apache.drill.exec.record.RecordBatch.TypedFieldId;
 import org.apache.drill.exec.record.WritableBatch;
-import org.apache.drill.exec.record.vector.SelectionVector;
-<<<<<<< HEAD
+import org.apache.drill.exec.record.selection.SelectionVector2;
+import org.apache.drill.exec.record.selection.SelectionVector4;
 import org.apache.drill.exec.vector.ValueVector;
-=======
-import org.apache.drill.exec.record.vector.SelectionVector2;
-import org.apache.drill.exec.record.vector.SelectionVector4;
-import org.apache.drill.exec.record.vector.ValueVector;
->>>>>>> Build working
 
 public abstract class FilterRecordBatch implements RecordBatch {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(FilterRecordBatch.class);
 
   private RecordBatch incoming;
-  private SelectionVector selectionVector;
+  private SelectionVector2 selectionVector;
   private BatchSchema schema;
   private FilteringRecordBatchTransformer transformer;
   private int outstanding;
@@ -83,20 +76,15 @@ public abstract class FilterRecordBatch implements RecordBatch {
   }
 
   @Override
-  public <T extends ValueVector<T>> T getValueVectorById(int fieldId, Class<?> vvClass) {
+  public <T extends ValueVector> T getValueVectorById(int fieldId, Class<?> vvClass) {
     return null;
   }
 
-  @Override
-<<<<<<< HEAD
-  public <T extends ValueVector> T getValueVector(int fieldId, Class<T> clazz) throws InvalidValueAccessor {
-=======
   public WritableBatch getWritableBatch() {
->>>>>>> Build working
     return null;
   }
 
-  abstract int applyFilter(SelectionVector vector, int count);
+  abstract int applyFilter(SelectionVector2 vector, int count);
 
   /**
    * Release all assets.
@@ -130,10 +118,10 @@ public abstract class FilterRecordBatch implements RecordBatch {
         }
       }
 
-      // make sure the bit vector is as large as the current record batch.
-      if (selectionVector.capacity() < incoming.getRecordCount()) {
-        selectionVector.allocateNew(incoming.getRecordCount());
-      }
+//      // make sure the bit vector is as large as the current record batch.
+//      if (selectionVector.capacity() < incoming.getRecordCount()) {
+//        selectionVector.allocateNew(incoming.getRecordCount());
+//      }
 
       return null;
     }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/3d41be41/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/FilteringRecordBatchTransformer.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/FilteringRecordBatchTransformer.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/FilteringRecordBatchTransformer.java
index 191521a..0813481 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/FilteringRecordBatchTransformer.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/FilteringRecordBatchTransformer.java
@@ -19,16 +19,16 @@ package org.apache.drill.exec.physical.impl;
 
 import org.apache.drill.exec.record.BatchSchema;
 import org.apache.drill.exec.record.RecordBatch;
-import org.apache.drill.exec.record.vector.SelectionVector;
+import org.apache.drill.exec.record.selection.SelectionVector2;
 
 public abstract class FilteringRecordBatchTransformer {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(FilteringRecordBatchTransformer.class);
   
   final RecordBatch incoming;
-  final SelectionVector selectionVector;
+  final SelectionVector2 selectionVector;
   final BatchSchema schema;
   
-  public FilteringRecordBatchTransformer(RecordBatch incoming, OutputMutator output, SelectionVector selectionVector) {
+  public FilteringRecordBatchTransformer(RecordBatch incoming, OutputMutator output, SelectionVector2 selectionVector) {
     super();
     this.incoming = incoming;
     this.selectionVector = selectionVector;

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/3d41be41/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/OutputMutator.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/OutputMutator.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/OutputMutator.java
index 5247d08..7e72683 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/OutputMutator.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/OutputMutator.java
@@ -18,19 +18,11 @@
 package org.apache.drill.exec.physical.impl;
 
 import org.apache.drill.exec.exception.SchemaChangeException;
-<<<<<<< HEAD
-import org.apache.drill.exec.vector.ValueVector;
-
-public interface OutputMutator {
-  public void removeField(int fieldId) throws SchemaChangeException;
-  public void addField(int fieldId, ValueVector vector) throws SchemaChangeException ;
-=======
 import org.apache.drill.exec.record.MaterializedField;
-import org.apache.drill.exec.record.vector.ValueVector;
+import org.apache.drill.exec.vector.ValueVector;
 
 public interface OutputMutator {
   public void removeField(MaterializedField field) throws SchemaChangeException;
-  public void addField(ValueVector<?> vector) throws SchemaChangeException ;
->>>>>>> Build working
-  public void setNewSchema() throws SchemaChangeException ;
+  public void addField(ValueVector vector) throws SchemaChangeException ;
+  public void setNewSchema() throws SchemaChangeException;
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/3d41be41/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScanBatch.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScanBatch.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScanBatch.java
index be80349..5688bb1 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScanBatch.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScanBatch.java
@@ -18,37 +18,27 @@
 package org.apache.drill.exec.physical.impl;
 
 import java.util.Iterator;
-<<<<<<< HEAD
-=======
 import java.util.List;
 import java.util.Map;
->>>>>>> Build working
 
 import org.apache.drill.common.exceptions.ExecutionSetupException;
 import org.apache.drill.common.expression.SchemaPath;
 import org.apache.drill.exec.exception.SchemaChangeException;
 import org.apache.drill.exec.ops.FragmentContext;
 import org.apache.drill.exec.record.BatchSchema;
+import org.apache.drill.exec.record.InvalidValueAccessor;
 import org.apache.drill.exec.record.MaterializedField;
 import org.apache.drill.exec.record.RecordBatch;
 import org.apache.drill.exec.record.SchemaBuilder;
 import org.apache.drill.exec.record.WritableBatch;
-<<<<<<< HEAD
-=======
-import org.apache.drill.exec.record.vector.SelectionVector2;
-import org.apache.drill.exec.record.vector.SelectionVector4;
-import org.apache.drill.exec.record.vector.ValueVector;
->>>>>>> Build working
+import org.apache.drill.exec.record.selection.SelectionVector2;
+import org.apache.drill.exec.record.selection.SelectionVector4;
 import org.apache.drill.exec.store.RecordReader;
 import org.apache.drill.exec.vector.ValueVector;
 
-<<<<<<< HEAD
-import com.carrotsearch.hppc.IntObjectOpenHashMap;
-import com.carrotsearch.hppc.procedures.IntObjectProcedure;
-=======
 import com.beust.jcommander.internal.Lists;
 import com.beust.jcommander.internal.Maps;
->>>>>>> Build working
+import com.carrotsearch.hppc.procedures.IntObjectProcedure;
 
 /**
  * Record batch used for a particular scan. Operators against one or more
@@ -56,14 +46,10 @@ import com.beust.jcommander.internal.Maps;
 public class ScanBatch implements RecordBatch {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ScanBatch.class);
 
-<<<<<<< HEAD
-  private IntObjectOpenHashMap<ValueVector> fields = new IntObjectOpenHashMap<ValueVector>();
-=======
-  final List<ValueVector<?>> vectors = Lists.newLinkedList();
-  final Map<MaterializedField, ValueVector<?>> fieldVectorMap = Maps.newHashMap();
-  
+  final List<ValueVector> vectors = Lists.newLinkedList();
+  final Map<MaterializedField, ValueVector> fieldVectorMap = Maps.newHashMap();
+
   private VectorHolder holder = new VectorHolder(vectors);
->>>>>>> Build working
   private BatchSchema schema;
   private int recordCount;
   private boolean schemaChanged = true;
@@ -72,11 +58,11 @@ public class ScanBatch implements RecordBatch {
   private RecordReader currentReader;
   private final Mutator mutator = new Mutator();
 
-  public ScanBatch(FragmentContext context, Iterator<RecordReader> readers)
-      throws ExecutionSetupException {
+  public ScanBatch(FragmentContext context, Iterator<RecordReader> readers) throws ExecutionSetupException {
     this.context = context;
     this.readers = readers;
-    if (!readers.hasNext()) throw new ExecutionSetupException("A scan batch must contain at least one reader.");
+    if (!readers.hasNext())
+      throw new ExecutionSetupException("A scan batch must contain at least one reader.");
     this.currentReader = readers.next();
     this.currentReader.setup(mutator);
   }
@@ -107,33 +93,11 @@ public class ScanBatch implements RecordBatch {
   }
 
   private void releaseAssets() {
-<<<<<<< HEAD
-    fields.forEach(new IntObjectProcedure<ValueVector>() {
-      @Override
-      public void apply(int key, ValueVector value) {
-        value.close();
-      }
-    });
-  }
-
-  @SuppressWarnings("unchecked")
-  @Override
-  public <T extends ValueVector> T getValueVector(int fieldId, Class<T> clazz) throws InvalidValueAccessor {
-    if (fields.containsKey(fieldId)) throw new InvalidValueAccessor(String.format("Unknown value accesor for field id %d."));
-    ValueVector vector = this.fields.lget();
-    if (vector.getClass().isAssignableFrom(clazz)) {
-      return (T) vector;
-    } else {
-      throw new InvalidValueAccessor(String.format(
-          "You requested a field accessor of type %s for field id %d but the actual type was %s.",
-          clazz.getCanonicalName(), fieldId, vector.getClass().getCanonicalName()));
-=======
-    for(ValueVector<?> v : vectors){
+    for (ValueVector v : vectors) {
       v.close();
->>>>>>> Build working
     }
   }
-  
+
   @Override
   public IterOutcome next() {
     while ((recordCount = currentReader.next()) == 0) {
@@ -161,7 +125,6 @@ public class ScanBatch implements RecordBatch {
     }
   }
 
-  
   @Override
   public SelectionVector2 getSelectionVector2() {
     throw new UnsupportedOperationException();
@@ -178,37 +141,24 @@ public class ScanBatch implements RecordBatch {
   }
 
   @Override
-  public <T extends ValueVector<T>> T getValueVectorById(int fieldId, Class<?> clazz) {
+  public <T extends ValueVector> T getValueVectorById(int fieldId, Class<?> clazz) {
     return holder.getValueVector(fieldId, clazz);
   }
 
-
   private class Mutator implements OutputMutator {
     private SchemaBuilder builder = BatchSchema.newBuilder();
-    
-    public void removeField(MaterializedField field) throws SchemaChangeException {
-      schemaChanged();
-<<<<<<< HEAD
-      ValueVector v = fields.remove(fieldId);
-      if (v == null) throw new SchemaChangeException("Failure attempting to remove an unknown field.");
-      v.close();
-    }
 
-    public void addField(int fieldId, ValueVector vector) {
+    public void removeField(MaterializedField field) throws SchemaChangeException {
       schemaChanged();
-      ValueVector v = fields.put(fieldId, vector);
-      vector.getField();
-=======
-      ValueVector<?> vector = fieldVectorMap.remove(field);
+      ValueVector vector = fieldVectorMap.remove(field);
       if (vector == null) throw new SchemaChangeException("Failure attempting to remove an unknown field.");
       vectors.remove(vector);
       vector.close();
     }
 
-    public void addField(ValueVector<?> vector) {
+    public void addField(ValueVector vector) {
       vectors.add(vector);
       fieldVectorMap.put(vector.getField(), vector);
->>>>>>> Build working
       builder.addField(vector.getField());
     }
 
@@ -224,5 +174,5 @@ public class ScanBatch implements RecordBatch {
   public WritableBatch getWritableBatch() {
     return WritableBatch.get(this.getRecordCount(), vectors);
   }
-  
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/3d41be41/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/VectorHolder.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/VectorHolder.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/VectorHolder.java
index 4209daa..65a7365 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/VectorHolder.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/VectorHolder.java
@@ -4,29 +4,29 @@ import java.util.List;
 
 import org.apache.drill.common.expression.SchemaPath;
 import org.apache.drill.exec.record.RecordBatch.TypedFieldId;
-import org.apache.drill.exec.record.vector.ValueVector;
+import org.apache.drill.exec.vector.ValueVector;
 
 public class VectorHolder {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(VectorHolder.class);
   
-  private List<ValueVector<?>> vectors;
+  private List<ValueVector> vectors;
 
-  public VectorHolder(List<ValueVector<?>> vectors) {
+  public VectorHolder(List<ValueVector> vectors) {
     super();
     this.vectors = vectors;
   }
   
   public TypedFieldId getValueVector(SchemaPath path) {
     for(int i =0; i < vectors.size(); i++){
-      ValueVector<?> vv = vectors.get(i);
+      ValueVector vv = vectors.get(i);
       if(vv.getField().matches(path)) return new TypedFieldId(vv.getField().getType(), i); 
     }
     return null;
   }
   
   @SuppressWarnings("unchecked")
-  public <T extends ValueVector<T>> T getValueVector(int fieldId, Class<?> clazz) {
-    ValueVector<?> v = vectors.get(fieldId);
+  public <T extends ValueVector> T getValueVector(int fieldId, Class<?> clazz) {
+    ValueVector v = vectors.get(fieldId);
     assert v != null;
     if (v.getClass() != clazz){
       logger.warn(String.format(

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/3d41be41/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/WireRecordBatch.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/WireRecordBatch.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/WireRecordBatch.java
index 7b2a533..d2b8bfd 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/WireRecordBatch.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/WireRecordBatch.java
@@ -17,26 +17,20 @@
  ******************************************************************************/
 package org.apache.drill.exec.physical.impl;
 
-<<<<<<< HEAD
-=======
 import org.apache.drill.common.expression.SchemaPath;
->>>>>>> Build working
 import org.apache.drill.exec.exception.SchemaChangeException;
 import org.apache.drill.exec.ops.FragmentContext;
 import org.apache.drill.exec.proto.UserBitShared.RecordBatchDef;
 import org.apache.drill.exec.record.BatchSchema;
+import org.apache.drill.exec.record.InvalidValueAccessor;
 import org.apache.drill.exec.record.RawFragmentBatch;
 import org.apache.drill.exec.record.RawFragmentBatchProvider;
 import org.apache.drill.exec.record.RecordBatch;
 import org.apache.drill.exec.record.RecordBatchLoader;
 import org.apache.drill.exec.record.WritableBatch;
-<<<<<<< HEAD
+import org.apache.drill.exec.record.selection.SelectionVector2;
+import org.apache.drill.exec.record.selection.SelectionVector4;
 import org.apache.drill.exec.vector.ValueVector;
-=======
-import org.apache.drill.exec.record.vector.SelectionVector2;
-import org.apache.drill.exec.record.vector.SelectionVector4;
-import org.apache.drill.exec.record.vector.ValueVector;
->>>>>>> Build working
 
 public class WireRecordBatch implements RecordBatch{
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(WireRecordBatch.class);
@@ -90,14 +84,9 @@ public class WireRecordBatch implements RecordBatch{
   }
 
   @Override
-<<<<<<< HEAD
-  public <T extends ValueVector> T getValueVector(int fieldId, Class<T> clazz) throws InvalidValueAccessor {
-=======
-  public <T extends ValueVector<T>> T getValueVectorById(int fieldId, Class<?> clazz) {
->>>>>>> Build working
+  public <T extends ValueVector> T getValueVectorById(int fieldId, Class<?> clazz){
     return batchLoader.getValueVector(fieldId, clazz);
   }
-
   
   @Override
   public IterOutcome next() {

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/3d41be41/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/filter/EvaluationPredicate.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/filter/EvaluationPredicate.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/filter/EvaluationPredicate.java
index 3176c41..862e42c 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/filter/EvaluationPredicate.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/filter/EvaluationPredicate.java
@@ -1,6 +1,6 @@
 package org.apache.drill.exec.physical.impl.filter;
 
-import org.apache.drill.exec.record.vector.SelectionVector2;
+import org.apache.drill.exec.record.selection.SelectionVector2;
 
 public class EvaluationPredicate {
   private SelectionVector2 vector;

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/3d41be41/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/filter/ExampleFilter.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/filter/ExampleFilter.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/filter/ExampleFilter.java
index c4a4032..85f598f 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/filter/ExampleFilter.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/filter/ExampleFilter.java
@@ -22,13 +22,11 @@ import org.apache.drill.common.expression.SchemaPath;
 import org.apache.drill.exec.exception.SchemaChangeException;
 import org.apache.drill.exec.ops.FragmentContext;
 import org.apache.drill.exec.record.BatchSchema;
-import org.apache.drill.exec.record.InvalidValueAccessor;
 import org.apache.drill.exec.record.RecordBatch;
-import org.apache.drill.exec.record.RecordBatch.TypedFieldId;
 import org.apache.drill.exec.record.WritableBatch;
-import org.apache.drill.exec.record.vector.SelectionVector2;
-import org.apache.drill.exec.record.vector.SelectionVector4;
-import org.apache.drill.exec.record.vector.ValueVector;
+import org.apache.drill.exec.record.selection.SelectionVector2;
+import org.apache.drill.exec.record.selection.SelectionVector4;
+import org.apache.drill.exec.vector.ValueVector;
 
 public class ExampleFilter implements RecordBatch {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ExampleFilter.class);
@@ -83,7 +81,7 @@ public class ExampleFilter implements RecordBatch {
   }
 
   @Override
-  public <T extends ValueVector<T>> T getValueVectorById(int fieldId, Class<?> vvClass) {
+  public <T extends ValueVector> T getValueVectorById(int fieldId, Class<?> vvClass) {
     return null;
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/3d41be41/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/filter/FilterTemplate.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/filter/FilterTemplate.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/filter/FilterTemplate.java
index d2983ad..216bfec 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/filter/FilterTemplate.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/filter/FilterTemplate.java
@@ -1,7 +1,7 @@
 package org.apache.drill.exec.physical.impl.filter;
 
 import org.apache.drill.exec.record.RecordBatch;
-import org.apache.drill.exec.record.vector.SelectionVector2;
+import org.apache.drill.exec.record.selection.SelectionVector2;
 
 public abstract class FilterTemplate {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(FilterTemplate.class);

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/3d41be41/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/ProjectRecordBatch.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/ProjectRecordBatch.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/ProjectRecordBatch.java
index 7a3fec1..3d1e3f7 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/ProjectRecordBatch.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/ProjectRecordBatch.java
@@ -28,12 +28,14 @@ import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
 import org.apache.drill.exec.record.MaterializedField;
 import org.apache.drill.exec.record.RecordBatch;
 import org.apache.drill.exec.record.SchemaBuilder;
+import org.apache.drill.exec.record.TransferPair;
 import org.apache.drill.exec.record.WritableBatch;
-import org.apache.drill.exec.record.vector.SelectionVector2;
-import org.apache.drill.exec.record.vector.SelectionVector4;
-import org.apache.drill.exec.record.vector.TypeHelper;
-import org.apache.drill.exec.record.vector.ValueVector;
-
+import org.apache.drill.exec.record.selection.SelectionVector2;
+import org.apache.drill.exec.record.selection.SelectionVector4;
+import org.apache.drill.exec.vector.AllocationHelper;
+import org.apache.drill.exec.vector.NonRepeatedMutator;
+import org.apache.drill.exec.vector.TypeHelper;
+import org.apache.drill.exec.vector.ValueVector;
 
 import com.google.common.base.Preconditions;
 import com.google.common.collect.Lists;
@@ -46,8 +48,8 @@ public class ProjectRecordBatch implements RecordBatch{
   private final FragmentContext context;
   private BatchSchema outSchema;
   private Projector projector;
-  private List<ValueVector<?>> allocationVectors;
-  private List<ValueVector<?>> outputVectors;
+  private List<ValueVector> allocationVectors;
+  private List<ValueVector> outputVectors;
   private VectorHolder vh;
   
   
@@ -95,7 +97,7 @@ public class ProjectRecordBatch implements RecordBatch{
   }
 
   @Override
-  public <T extends ValueVector<T>> T getValueVectorById(int fieldId, Class<?> clazz) {
+  public <T extends ValueVector> T getValueVectorById(int fieldId, Class<?> clazz) {
     return vh.getValueVector(fieldId, clazz);
   }
 
@@ -121,12 +123,17 @@ public class ProjectRecordBatch implements RecordBatch{
       // fall through.
     case OK:
       int recordCount = incoming.getRecordCount();
-      for(ValueVector<?> v : this.allocationVectors){
-        v.allocateNew(recordCount);
+      for(ValueVector v : this.allocationVectors){
+        AllocationHelper.allocate(v, recordCount, 50);
       }
       projector.projectRecords(recordCount, 0);
-      for(ValueVector<?> v : this.outputVectors){
-        v.setRecordCount(recordCount);
+      for(ValueVector v : this.outputVectors){
+        ValueVector.Mutator m = v.getMutator();
+        if(m instanceof NonRepeatedMutator){
+          ((NonRepeatedMutator) m).setValueCount(recordCount);
+        }else{
+          throw new UnsupportedOperationException();
+        }
       }
       return upstream; // change if upstream changed, otherwise normal.
     default:
@@ -138,7 +145,7 @@ public class ProjectRecordBatch implements RecordBatch{
   private Projector createNewProjector() throws SchemaChangeException{
     this.allocationVectors = Lists.newArrayList();
     if(outputVectors != null){
-      for(ValueVector<?> v : outputVectors){
+      for(ValueVector v : outputVectors){
         v.close();
       }
     }
@@ -146,7 +153,7 @@ public class ProjectRecordBatch implements RecordBatch{
     this.vh = new VectorHolder(outputVectors);
     final List<NamedExpression> exprs = pop.getExprs();
     final ErrorCollector collector = new ErrorCollectorImpl();
-    final List<TransferPairing<?>> transfers = Lists.newArrayList();
+    final List<TransferPair> transfers = Lists.newArrayList();
     
     final CodeGenerator<Projector> cg = new CodeGenerator<Projector>(Projector.TEMPLATE_DEFINITION, context.getFunctionRegistry());
     
@@ -161,15 +168,15 @@ public class ProjectRecordBatch implements RecordBatch{
       // add value vector to transfer if direct reference and this is allowed, otherwise, add to evaluation stack.
       if(expr instanceof ValueVectorReadExpression && incoming.getSchema().getSelectionVector() == SelectionVectorMode.NONE){
         ValueVectorReadExpression vectorRead = (ValueVectorReadExpression) expr;
-        ValueVector<?> vvIn = incoming.getValueVectorById(vectorRead.getFieldId(), TypeHelper.getValueVectorClass(vectorRead.getMajorType()));
+        ValueVector vvIn = incoming.getValueVectorById(vectorRead.getFieldId(), TypeHelper.getValueVectorClass(vectorRead.getMajorType().getMinorType(), vectorRead.getMajorType().getMode()));
         Preconditions.checkNotNull(incoming);
 
-        TransferPairing<?> tp = vvIn.getTransferPair(outputField);
+        TransferPair tp = vvIn.getTransferPair();
         transfers.add(tp);
         outputVectors.add(tp.getTo());
       }else{
         // need to do evaluation.
-        ValueVector<?> vector = TypeHelper.getNewVector(outputField, context.getAllocator());
+        ValueVector vector = TypeHelper.getNewVector(outputField, context.getAllocator());
         allocationVectors.add(vector);
         outputVectors.add(vector);
         ValueVectorWriteExpression write = new ValueVectorWriteExpression(outputVectors.size() - 1, expr);
@@ -179,7 +186,7 @@ public class ProjectRecordBatch implements RecordBatch{
     }
     
     SchemaBuilder bldr = BatchSchema.newBuilder().setSelectionVectorMode(SelectionVectorMode.NONE);
-    for(ValueVector<?> v : outputVectors){
+    for(ValueVector v : outputVectors){
       bldr.addField(v.getField());
     }
     this.outSchema = bldr.build();

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/3d41be41/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 8f22b40..2787f0c 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
@@ -6,10 +6,11 @@ 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.record.RecordBatch;
+import org.apache.drill.exec.record.TransferPair;
 
 public interface Projector {
 
-  public abstract void setup(FragmentContext context, RecordBatch incoming,  RecordBatch outgoing, List<TransferPairing<?>> transfers)  throws SchemaChangeException;
+  public abstract void setup(FragmentContext context, RecordBatch incoming,  RecordBatch outgoing, List<TransferPair> transfers)  throws SchemaChangeException;
 
   
   public abstract int projectRecords(int recordCount, int firstOutputIndex);

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/3d41be41/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/ProjectorTemplate.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/ProjectorTemplate.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/ProjectorTemplate.java
index 1163fef..486c7b0 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/ProjectorTemplate.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/ProjectorTemplate.java
@@ -6,15 +6,16 @@ import org.apache.drill.exec.exception.SchemaChangeException;
 import org.apache.drill.exec.ops.FragmentContext;
 import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
 import org.apache.drill.exec.record.RecordBatch;
-import org.apache.drill.exec.record.vector.SelectionVector2;
-import org.apache.drill.exec.record.vector.SelectionVector4;
+import org.apache.drill.exec.record.TransferPair;
+import org.apache.drill.exec.record.selection.SelectionVector2;
+import org.apache.drill.exec.record.selection.SelectionVector4;
 
 import com.google.common.collect.ImmutableList;
 
 public abstract class ProjectorTemplate implements Projector {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ProjectorTemplate.class);
   
-  private ImmutableList<TransferPairing<?>> transfers;
+  private ImmutableList<TransferPair> transfers;
   private SelectionVector2 vector2;
   private SelectionVector4 vector4;
   private SelectionVectorMode svMode;
@@ -39,7 +40,7 @@ public abstract class ProjectorTemplate implements Projector {
       
     case NONE:
       
-      for(TransferPairing<?> t : transfers){
+      for(TransferPair t : transfers){
         t.transfer();
       }
       final int countN = recordCount;
@@ -55,7 +56,7 @@ public abstract class ProjectorTemplate implements Projector {
   }
 
   @Override
-  public final void setup(FragmentContext context, RecordBatch incoming, RecordBatch outgoing, List<TransferPairing<?>> transfers)  throws SchemaChangeException{
+  public final void setup(FragmentContext context, RecordBatch incoming, RecordBatch outgoing, List<TransferPair> transfers)  throws SchemaChangeException{
 
     this.svMode = incoming.getSchema().getSelectionVector(); 
     switch(svMode){

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/3d41be41/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/TransferPairing.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/TransferPairing.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/TransferPairing.java
deleted file mode 100644
index 2b4ac81..0000000
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/TransferPairing.java
+++ /dev/null
@@ -1,35 +0,0 @@
-package org.apache.drill.exec.physical.impl.project;
-
-import org.apache.drill.exec.record.vector.ValueVector;
-
-public class TransferPairing<T extends ValueVector<T>> {
-  
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TransferPairing.class);
-  
-  final T from;
-  final T to;
-  
-  protected TransferPairing(T from, T to) {
-    super();
-    this.from = from;
-    this.to = to;
-  }
-
-  public void transfer(){
-    from.transferTo(to);
-  }
-  
-  public static <T extends ValueVector<T>> TransferPairing<T> getTransferPairing(T from, T to){
-    return new TransferPairing<T>(from, to);
-  }
-
-  public T getFrom() {
-    return from;
-  }
-
-  public T getTo() {
-    return to;
-  }
-  
-  
-}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/3d41be41/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/MaterializedField.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/MaterializedField.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/MaterializedField.java
index 6cf7087..732dab3 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/MaterializedField.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/MaterializedField.java
@@ -26,7 +26,7 @@ import org.apache.drill.common.types.TypeProtos.MajorType;
 import org.apache.drill.exec.proto.SchemaDefProtos.FieldDef;
 import org.apache.drill.exec.proto.SchemaDefProtos.NamePart;
 import org.apache.drill.exec.proto.SchemaDefProtos.NamePart.Type;
-import org.apache.drill.exec.record.vector.TypeHelper;
+import org.apache.drill.exec.vector.TypeHelper;
 
 public class MaterializedField{
   private final FieldDef def;
@@ -113,7 +113,8 @@ public class MaterializedField{
       throw new UnsupportedOperationException();
     }
     return new MaterializedField(def.toBuilder().setMajorType(mt.toBuilder().setMode(newDataMode).build()).build());
-
+  }
+  
   public Class<?> getValueClass() {
     return TypeHelper.getValueVectorClass(getType().getMinorType(), getDataMode());
   }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/3d41be41/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatch.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatch.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatch.java
index 0edb93e..650a148 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatch.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatch.java
@@ -20,9 +20,9 @@ package org.apache.drill.exec.record;
 import org.apache.drill.common.expression.SchemaPath;
 import org.apache.drill.common.types.TypeProtos.MajorType;
 import org.apache.drill.exec.ops.FragmentContext;
-import org.apache.drill.exec.record.vector.SelectionVector2;
-import org.apache.drill.exec.record.vector.SelectionVector4;
-import org.apache.drill.exec.record.vector.ValueVector;
+import org.apache.drill.exec.record.selection.SelectionVector2;
+import org.apache.drill.exec.record.selection.SelectionVector4;
+import org.apache.drill.exec.vector.ValueVector;
 
 /**
  * A record batch contains a set of field values for a particular range of records. In the case of a record batch
@@ -88,7 +88,7 @@ public interface RecordBatch {
   public abstract TypedFieldId getValueVectorId(SchemaPath path);
   
   
-  public abstract <T extends ValueVector<T>> T getValueVectorById(int fieldId, Class<?> clazz);
+  public abstract <T extends ValueVector> T getValueVectorById(int fieldId, Class<?> clazz);
 
   /**
    * Update the data in each Field reading interface for the next range of records. Once a RecordBatch returns an

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/3d41be41/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatchLoader.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatchLoader.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatchLoader.java
index c3db9f0..e2a1648 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatchLoader.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatchLoader.java
@@ -30,19 +30,17 @@ import org.apache.drill.exec.proto.SchemaDefProtos.FieldDef;
 import org.apache.drill.exec.proto.UserBitShared.FieldMetadata;
 import org.apache.drill.exec.proto.UserBitShared.RecordBatchDef;
 import org.apache.drill.exec.record.RecordBatch.TypedFieldId;
-import org.apache.drill.exec.record.vector.TypeHelper;
-import org.apache.drill.exec.record.vector.ValueVector;
+import org.apache.drill.exec.vector.TypeHelper;
+import org.apache.drill.exec.vector.ValueVector;
 
 import com.beust.jcommander.internal.Lists;
 import com.beust.jcommander.internal.Maps;
-import com.carrotsearch.hppc.IntObjectOpenHashMap;
-import com.carrotsearch.hppc.cursors.IntObjectCursor;
 import com.google.common.collect.ImmutableList;
 
-public class RecordBatchLoader implements Iterable<ValueVector<?>>{
+public class RecordBatchLoader implements Iterable<ValueVector>{
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(RecordBatchLoader.class);
 
-  private List<ValueVector<?>> vectors = Lists.newArrayList();
+  private List<ValueVector> vectors = Lists.newArrayList();
   private final BufferAllocator allocator;
   private int recordCount; 
   private BatchSchema schema;
@@ -67,19 +65,19 @@ public class RecordBatchLoader implements Iterable<ValueVector<?>>{
     this.recordCount = def.getRecordCount();
     boolean schemaChanged = false;
 
-    Map<MaterializedField, ValueVector<?>> oldFields = Maps.newHashMap();
-    for(ValueVector<?> v : this.vectors){
+    Map<MaterializedField, ValueVector> oldFields = Maps.newHashMap();
+    for(ValueVector v : this.vectors){
       oldFields.put(v.getField(), v);
     }
     
-    List<ValueVector<?>> newVectors = Lists.newArrayList();
+    List<ValueVector> newVectors = Lists.newArrayList();
 
     List<FieldMetadata> fields = def.getFieldList();
     
     int bufOffset = 0;
     for (FieldMetadata fmd : fields) {
       FieldDef fieldDef = fmd.getDef();
-      ValueVector<?> v = oldFields.remove(fieldDef);
+      ValueVector v = oldFields.remove(fieldDef);
       if(v != null){
         newVectors.add(v);
         continue;
@@ -90,21 +88,19 @@ public class RecordBatchLoader implements Iterable<ValueVector<?>>{
       MaterializedField m = new MaterializedField(fieldDef);
       v = TypeHelper.getNewVector(m, allocator);
       v.load(fmd, buf.slice(bufOffset, fmd.getBufferLength()));
-      newVectors.put(fieldDef.getFieldId(), v);
-      v.setTo(fmd, buf.slice(bufOffset, fmd.getBufferLength()));
       newVectors.add(v);
     }
     
     if(!oldFields.isEmpty()){
       schemaChanged = true;
-      for(ValueVector<?> v : oldFields.values()){
+      for(ValueVector v : oldFields.values()){
         v.close();
       }
     }
     
     // rebuild the schema.
     SchemaBuilder b = BatchSchema.newBuilder();
-    for(ValueVector<?> v : newVectors){
+    for(ValueVector v : newVectors){
       b.addField(v.getField());
     }
     b.setSelectionVectorMode(BatchSchema.SelectionVectorMode.NONE);
@@ -116,15 +112,15 @@ public class RecordBatchLoader implements Iterable<ValueVector<?>>{
 
   public TypedFieldId getValueVector(SchemaPath path) {
     for(int i =0; i < vectors.size(); i++){
-      ValueVector<?> vv = vectors.get(i);
+      ValueVector vv = vectors.get(i);
       if(vv.getField().matches(path)) return new TypedFieldId(vv.getField().getType(), i); 
     }
     return null;
   }
   
   @SuppressWarnings("unchecked")
-  public <T extends ValueVector<T>> T getValueVector(int fieldId, Class<?> clazz) {
-    ValueVector<?> v = vectors.get(fieldId);
+  public <T extends ValueVector> T getValueVector(int fieldId, Class<?> clazz) {
+    ValueVector v = vectors.get(fieldId);
     assert v != null;
     if (v.getClass() != clazz){
       logger.warn(String.format(
@@ -145,7 +141,7 @@ public class RecordBatchLoader implements Iterable<ValueVector<?>>{
   }
 
   @Override
-  public Iterator<ValueVector<?>> iterator() {
+  public Iterator<ValueVector> iterator() {
     return this.vectors.iterator();
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/3d41be41/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/TransferPair.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/TransferPair.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/TransferPair.java
new file mode 100644
index 0000000..a90e2d8
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/TransferPair.java
@@ -0,0 +1,8 @@
+package org.apache.drill.exec.record;
+
+import org.apache.drill.exec.vector.ValueVector;
+
+public interface TransferPair {
+  public void transfer();
+  public ValueVector getTo();
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/3d41be41/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/WritableBatch.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/WritableBatch.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/WritableBatch.java
index 2e1754c..8335e91 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/WritableBatch.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/WritableBatch.java
@@ -25,8 +25,6 @@ import org.apache.drill.exec.proto.UserBitShared.FieldMetadata;
 import org.apache.drill.exec.proto.UserBitShared.RecordBatchDef;
 import org.apache.drill.exec.vector.ValueVector;
 
-import com.carrotsearch.hppc.IntObjectOpenHashMap;
-import com.carrotsearch.hppc.procedures.IntObjectProcedure;
 import com.google.common.collect.Lists;
 
 /**
@@ -58,20 +56,20 @@ public class WritableBatch {
     return buffers;
   }
 
-  public static WritableBatch get(int recordCount, List<ValueVector<?>> vectors){
+  public static WritableBatch get(int recordCount, List<ValueVector> vectors){
     
     List<ByteBuf> buffers = Lists.newArrayList();
     List<FieldMetadata> metadata = Lists.newArrayList();
     
 
-    for(ValueVector<?> vv : vectors){
+    for(ValueVector vv : vectors){
       metadata.add(vv.getMetadata());
       for(ByteBuf b : vv.getBuffers()){
         buffers.add(b);
         b.retain();
       }
       // allocate new buffer to release hold on old buffer.
-      vv.allocateNew(vv.capacity());
+      vv.clear();
     }
 
     RecordBatchDef batchDef = RecordBatchDef.newBuilder().addAllField(metadata).setRecordCount(recordCount).build();

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/3d41be41/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/selection/SelectionVector2.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/selection/SelectionVector2.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/selection/SelectionVector2.java
new file mode 100644
index 0000000..cdc136e
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/selection/SelectionVector2.java
@@ -0,0 +1,49 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.record.selection;
+
+import io.netty.buffer.ByteBuf;
+
+import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.record.DeadBuf;
+
+/**
+ * A selection vector that fronts, at most, a
+ */
+public class SelectionVector2{
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(SelectionVector2.class);
+
+  private final BufferAllocator allocator;
+  private ByteBuf buffer = DeadBuf.DEAD_BUFFER;
+
+  public SelectionVector2(BufferAllocator allocator) {
+    this.allocator = allocator;
+  }
+
+  public int getCount(){
+    return -1;
+  }
+
+  public int getIndex(int directIndex){
+    return buffer.getChar(directIndex);
+  }
+
+  public void setIndex(int directIndex, char value){
+    buffer.setChar(directIndex, value);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/3d41be41/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/selection/SelectionVector4.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/selection/SelectionVector4.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/selection/SelectionVector4.java
new file mode 100644
index 0000000..1f3874f
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/selection/SelectionVector4.java
@@ -0,0 +1,41 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+
+package org.apache.drill.exec.record.selection;
+
+import io.netty.buffer.ByteBuf;
+
+import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.record.DeadBuf;
+
+public class SelectionVector4 {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(SelectionVector4.class);
+
+  private final BufferAllocator allocator;
+  private ByteBuf buffer = DeadBuf.DEAD_BUFFER;
+
+  public SelectionVector4(BufferAllocator allocator) {
+    this.allocator = allocator;
+  }
+
+  public int getCount(){
+    return -1;
+  }
+
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/3d41be41/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/BitUtil.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/BitUtil.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/BitUtil.java
deleted file mode 100644
index 4a3ae2f..0000000
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/BitUtil.java
+++ /dev/null
@@ -1,108 +0,0 @@
-/*******************************************************************************
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- ******************************************************************************/
-package org.apache.drill.exec.record.vector;  // from org.apache.solr.util rev 555343
-
-import io.netty.buffer.ByteBuf;
-
-
-/**  A variety of high efficiency bit twiddling routines.
- * @lucene.internal
- */
-public final class BitUtil {
-
-  private BitUtil() {} // no instance
-
-  // The pop methods used to rely on bit-manipulation tricks for speed but it
-  // turns out that it is faster to use the Long.bitCount method (which is an
-  // intrinsic since Java 6u18) in a naive loop, see LUCENE-2221
-
-  /** Returns the number of set bits in an array of longs. */
-  public static long pop_array(ByteBuf arr, int wordOffset, int numWords) {
-    long popCount = 0;
-    for (int i = wordOffset, end = wordOffset + numWords; i < end; i+=8) {
-      popCount += Long.bitCount(arr.getLong(i));
-    }
-    return popCount;
-  }
-
-  /** Returns the popcount or cardinality of the two sets after an intersection.
-   *  Neither array is modified. */
-  public static long pop_intersect(ByteBuf arr1, ByteBuf arr2, int wordOffset, int numWords) {
-    long popCount = 0;
-    for (int i = wordOffset, end = wordOffset + numWords; i < end; i+=8) {
-      popCount += Long.bitCount(arr1.getLong(i) & arr2.getLong(i));
-    }
-    return popCount;
-  }
-
-   /** Returns the popcount or cardinality of the union of two sets.
-    *  Neither array is modified. */
-   public static long pop_union(ByteBuf arr1, ByteBuf arr2, int wordOffset, int numWords) {
-     long popCount = 0;
-     for (int i = wordOffset, end = wordOffset + numWords; i < end; i+=8) {
-       popCount += Long.bitCount(arr1.getLong(i) | arr2.getLong(i));
-     }
-     return popCount;
-   }
-
-  /** Returns the popcount or cardinality of A & ~B.
-   *  Neither array is modified. */
-  public static long pop_andnot(ByteBuf arr1, ByteBuf arr2, int wordOffset, int numWords) {
-    long popCount = 0;
-    for (int i = wordOffset, end = wordOffset + numWords; i < end; i+=8) {
-      popCount += Long.bitCount(arr1.getLong(i) & ~arr2.getLong(i));
-    }
-    return popCount;
-  }
-
-  /** Returns the popcount or cardinality of A ^ B
-    * Neither array is modified. */
-  public static long pop_xor(ByteBuf arr1, ByteBuf arr2, int wordOffset, int numWords) {
-    long popCount = 0;
-    for (int i = wordOffset, end = wordOffset + numWords; i < end; i+=8) {
-      popCount += Long.bitCount(arr1.getLong(i) ^ arr2.getLong(i));
-    }
-    return popCount;
-  }
-
-  /** returns the next highest power of two, or the current value if it's already a power of two or zero*/
-  public static int nextHighestPowerOfTwo(int v) {
-    v--;
-    v |= v >> 1;
-    v |= v >> 2;
-    v |= v >> 4;
-    v |= v >> 8;
-    v |= v >> 16;
-    v++;
-    return v;
-  }
-
-  /** returns the next highest power of two, or the current value if it's already a power of two or zero*/
-   public static long nextHighestPowerOfTwo(long v) {
-    v--;
-    v |= v >> 1;
-    v |= v >> 2;
-    v |= v >> 4;
-    v |= v >> 8;
-    v |= v >> 16;
-    v |= v >> 32;
-    v++;
-    return v;
-  }
-
-}
\ No newline at end of file


Re: [42/53] [abbrv] Types transition

Posted by Ted Dunning <te...@gmail.com>.
On Fri, Jul 19, 2013 at 6:58 PM, <ja...@apache.org> wrote:

>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/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
> new file mode 100644
> index 0000000..bc44490
> --- /dev/null
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/ExpressionTreeMaterializer.java
> @@ -0,0 +1,128 @@
>
> +/*******************************************************************************
> + * Licensed to the Apache Software Foundation (ASF) under one
> + * or more contributor license agreements.  See the NOTICE file
> + * distributed with this work for additional information
> + * regarding copyright ownership.  The ASF licenses this file
> + * to you under the Apache License, Version 2.0 (the
> + * "License"); you may not use this file except in compliance
> + * with the License.  You may obtain a copy of the License at
> + *
> + * http://www.apache.org/licenses/LICENSE-2.0
> + *
> + * Unless required by applicable law or agreed to in writing, software
> + * distributed under the License is distributed on an "AS IS" BASIS,
> + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
> implied.
> + * See the License for the specific language governing permissions and
> + * limitations under the License.
> +
> ******************************************************************************/
> +
> +package org.apache.drill.exec.expr;
> +
> +import java.util.List;
> +
> +import org.apache.drill.common.expression.ErrorCollector;
> +import org.apache.drill.common.expression.ExpressionValidator;
> +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;
> +import org.apache.drill.common.expression.visitors.SimpleExprVisitor;
> +import org.apache.drill.exec.record.NullExpression;
> +import org.apache.drill.exec.record.RecordBatch;
> +import org.apache.drill.exec.record.RecordBatch.TypedFieldId;
> +
> +import com.google.common.collect.Lists;
> +
> +public class ExpressionTreeMaterializer {
> +
> +  static final org.slf4j.Logger logger =
> org.slf4j.LoggerFactory.getLogger(ExpressionTreeMaterializer.class);
> +
> +
> +
> +  private ExpressionTreeMaterializer() {
> +  };
> +
> +  public static LogicalExpression materialize(LogicalExpression expr,
> RecordBatch batch, ErrorCollector errorCollector) {
> +    return expr.accept(new MaterializeVisitor(batch, errorCollector),
> null);
> +  }
> +
> +  private static class MaterializeVisitor extends
> SimpleExprVisitor<LogicalExpression> {
> +    private final ErrorCollector errorCollector;
> +    private final RecordBatch batch;
> +    private ExpressionValidator validator = new ExpressionValidator();
> +
> +    public MaterializeVisitor(RecordBatch batch, ErrorCollector
> errorCollector) {
> +      this.batch = batch;
> +      this.errorCollector = errorCollector;
> +    }
> +
> +    private LogicalExpression validateNewExpr(LogicalExpression newExpr) {
> +      newExpr.accept(validator, errorCollector);
> +      return newExpr;
> +    }
> +
> +    @Override
> +    public LogicalExpression visitUnknown(LogicalExpression e, Void
> value) throws RuntimeException {
> +      throw new UnsupportedOperationException(String.format("Expression
> tree materializer does not currently support materializing nodes of type
> %s.", e.getClass().getCanonicalName()));
> +    }
> +
> +    @Override
> +    public LogicalExpression visitFunctionCall(FunctionCall call) {
> +      List<LogicalExpression> args = Lists.newArrayList();
> +      for (int i = 0; i < call.args.size(); ++i) {
> +        LogicalExpression newExpr = call.args.get(i).accept(this, null);
> +        args.add(newExpr);
> +      }
> +
> +      return validateNewExpr(new FunctionCall(call.getDefinition(), args,
> call.getPosition()));
> +    }
> +
> +    @Override
> +    public LogicalExpression visitIfExpression(IfExpression ifExpr) {
> +      List<IfExpression.IfCondition> conditions =
> Lists.newArrayList(ifExpr.iterator());
> +      LogicalExpression newElseExpr = ifExpr.elseExpression.accept(this,
> null);
> +
> +      for (int i = 0; i < conditions.size(); ++i) {
> +        IfExpression.IfCondition condition = conditions.get(i);
> +
> +        LogicalExpression newCondition = condition.condition.accept(this,
> null);
> +        LogicalExpression newExpr = condition.expression.accept(this,
> null);
> +        conditions.set(i, new IfExpression.IfCondition(newCondition,
> newExpr));
> +      }
> +
> +      return
> validateNewExpr(IfExpression.newBuilder().setElse(newElseExpr).addConditions(conditions).build());
> +    }
> +
> +    @Override
> +    public LogicalExpression visitSchemaPath(SchemaPath path) {
> +      logger.debug("Visiting schema path {}", path);
> +      TypedFieldId tfId = batch.getValueVector(path);
> +      if (tfId == null) {
> +        return NullExpression.INSTANCE;
> +      } else {
> +        return new ValueVectorReadExpression(tfId);
> +      }
> +    }
> +
> +    @Override
> +    public LogicalExpression
> visitLongConstant(ValueExpressions.LongExpression intExpr) {
> +      return intExpr;
> +    }
> +
> +    @Override
> +    public LogicalExpression
> visitDoubleConstant(ValueExpressions.DoubleExpression dExpr) {
> +      return dExpr;
> +    }
> +
> +    @Override
> +    public LogicalExpression
> visitBooleanConstant(ValueExpressions.BooleanExpression e) {
> +      return e;
> +    }
> +
> +    @Override
> +    public LogicalExpression
> visitQuotedStringConstant(ValueExpressions.QuotedString e) {
> +      return e;
> +    }
> +  }
> +}
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/SingleClassStringWriter.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/SingleClassStringWriter.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/SingleClassStringWriter.java
> new file mode 100644
> index 0000000..813a32f
> --- /dev/null
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/SingleClassStringWriter.java
> @@ -0,0 +1,40 @@
> +package org.apache.drill.exec.expr;
> +
> +import java.io.IOException;
> +import java.io.OutputStream;
> +import java.io.StringWriter;
> +import java.io.Writer;
> +
> +import com.google.common.base.Preconditions;
> +import com.sun.codemodel.CodeWriter;
> +import com.sun.codemodel.JPackage;
> +
> +public class SingleClassStringWriter extends CodeWriter{
> +  static final org.slf4j.Logger logger =
> org.slf4j.LoggerFactory.getLogger(SingleClassStringWriter.class);
> +
> +  private boolean used;
> +  private StringWriter writer = new StringWriter();
> +
> +  @Override
> +  public OutputStream openBinary(JPackage pkg, String fileName) throws
> IOException {
> +    throw new UnsupportedOperationException();
> +  }
> +
> +
> +  @Override
> +  public Writer openSource(JPackage pkg, String fileName) throws
> IOException {
> +    Preconditions.checkArgument(!used, "The SingleClassStringWriter can
> only output once src file.");
> +    used = true;
> +    return writer;
> +  }
> +
> +  @Override
> +  public void close() throws IOException {
> +  }
> +
> +  public StringBuffer getCode(){
> +    return writer.getBuffer();
> +  }
> +
> +
> +}
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/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
> new file mode 100644
> index 0000000..c8d906d
> --- /dev/null
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/ValueVectorReadExpression.java
> @@ -0,0 +1,41 @@
> +package org.apache.drill.exec.expr;
> +
> +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.RecordBatch.TypedFieldId;
> +import org.apache.drill.exec.record.vector.ValueVector;
> +
> +public class ValueVectorReadExpression implements LogicalExpression{
> +  static final org.slf4j.Logger logger =
> org.slf4j.LoggerFactory.getLogger(ValueVectorReadExpression.class);
> +
> +  private final MajorType type;
> +  private final int fieldId;
> +
> +  public ValueVectorReadExpression(TypedFieldId tfId) {
> +    this.type = tfId.getType();
> +    this.fieldId = tfId.getFieldId();
> +  }
> +
> +  @Override
> +  public MajorType getMajorType() {
> +    return type;
> +  }
> +
> +  @Override
> +  public <T, V, E extends Exception> T accept(ExprVisitor<T, V, E>
> visitor, V value) throws E {
> +    return visitor.visitUnknown(this, value);
> +  }
> +
> +  public int getFieldId() {
> +    return fieldId;
> +  }
> +
> +  @Override
> +  public ExpressionPosition getPosition() {
> +    return ExpressionPosition.UNKNOWN;
> +  }
> +
> +
> +}
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/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
> new file mode 100644
> index 0000000..96d9d05
> --- /dev/null
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/ValueVectorWriteExpression.java
> @@ -0,0 +1,45 @@
> +package org.apache.drill.exec.expr;
> +
> +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.common.types.Types;
> +import org.apache.drill.exec.record.vector.ValueVector;
> +
> +public class ValueVectorWriteExpression implements LogicalExpression {
> +  static final org.slf4j.Logger logger =
> org.slf4j.LoggerFactory.getLogger(ValueVectorWriteExpression.class);
> +
> +  private final int fieldId;
> +  private final LogicalExpression child;
> +
> +  public ValueVectorWriteExpression(int fieldId, LogicalExpression child){
> +    this.fieldId = fieldId;
> +    this.child = child;
> +  }
> +
> +  public int getFieldId() {
> +    return fieldId;
> +  }
> +
> +  @Override
> +  public MajorType getMajorType() {
> +    return Types.NULL;
> +  }
> +
> +  @Override
> +  public <T, V, E extends Exception> T accept(ExprVisitor<T, V, E>
> visitor, V value) throws E {
> +    return visitor.visitUnknown(this, value);
> +  }
> +
> +  @Override
> +  public ExpressionPosition getPosition() {
> +    return ExpressionPosition.UNKNOWN;
> +  }
> +
> +  public LogicalExpression getChild() {
> +    return child;
> +  }
> +
> +
> +}
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/annotations/FunctionTemplate.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/annotations/FunctionTemplate.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/annotations/FunctionTemplate.java
> new file mode 100644
> index 0000000..9e32750
> --- /dev/null
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/annotations/FunctionTemplate.java
> @@ -0,0 +1,25 @@
> +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;
> +
> +
> +@Retention(RetentionPolicy.RUNTIME)
> +@Target({ElementType.TYPE})
> +public @interface FunctionTemplate {
> +
> +  String name();
> +  FunctionScope scope();
> +  NullHandling nulls() default NullHandling.INTERNAL;
> +  boolean isBinaryCommutative() default false;
> +
> +  public static enum NullHandling {
> +    INTERNAL, NULL_IF_NULL;
> +  }
> +
> +  public static enum FunctionScope{
> +    SIMPLE, AGGREGATE, RUNNING;
> +  }
> +}
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/annotations/Output.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/annotations/Output.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/annotations/Output.java
> new file mode 100644
> index 0000000..2d77dfc
> --- /dev/null
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/annotations/Output.java
> @@ -0,0 +1,18 @@
> +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;
> +
> +/**
> + * Describes the field will provide output from the given function.
> + */
> +@Retention(RetentionPolicy.RUNTIME)
> +@Target({ElementType.FIELD})
> +public @interface Output {
> +  static final org.slf4j.Logger logger =
> org.slf4j.LoggerFactory.getLogger(Output.class);
> +
> +
> +
> +}
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/annotations/Param.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/annotations/Param.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/annotations/Param.java
> new file mode 100644
> index 0000000..9ba7611
> --- /dev/null
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/annotations/Param.java
> @@ -0,0 +1,18 @@
> +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.FIELD})
> +public @interface Param {
> +  static final org.slf4j.Logger logger =
> org.slf4j.LoggerFactory.getLogger(Param.class);
> +}
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/annotations/WorkSpace.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/annotations/WorkSpace.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/annotations/WorkSpace.java
> new file mode 100644
> index 0000000..cc106a7
> --- /dev/null
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/annotations/WorkSpace.java
> @@ -0,0 +1,14 @@
> +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;
> +
> +/**
> + * Describes the field will provide output from the given function.
> + */
> +@Retention(RetentionPolicy.RUNTIME)
> +@Target({ElementType.FIELD})
> +public @interface WorkSpace {
> +}
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/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
> new file mode 100644
> index 0000000..7276e7d
> --- /dev/null
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/FunctionConverter.java
> @@ -0,0 +1,186 @@
> +package org.apache.drill.exec.expr.fn;
> +
> +import java.io.File;
> +import java.io.IOException;
> +import java.io.InputStream;
> +import java.io.StringReader;
> +import java.lang.reflect.Field;
> +import java.net.URL;
> +import java.util.List;
> +import java.util.Map;
> +
> +import org.apache.drill.common.types.TypeProtos.MajorType;
> +import org.apache.drill.exec.expr.DrillFunc;
> +import org.apache.drill.exec.expr.annotations.FunctionTemplate;
> +import org.apache.drill.exec.expr.annotations.Output;
> +import org.apache.drill.exec.expr.annotations.Param;
> +import org.apache.drill.exec.expr.annotations.WorkSpace;
> +import org.apache.drill.exec.expr.fn.FunctionHolder.ValueReference;
> +import org.apache.drill.exec.expr.holders.ValueHolder;
> +import org.codehaus.commons.compiler.CompileException;
> +import org.codehaus.janino.Java;
> +import org.codehaus.janino.Java.CompilationUnit;
> +import org.codehaus.janino.Parser;
> +import org.codehaus.janino.Scanner;
> +import org.mortbay.util.IO;
> +
> +import com.beust.jcommander.internal.Lists;
> +import com.google.common.io.InputSupplier;
> +import com.google.common.io.Resources;
> +
> +/**
> + * Converts FunctionCalls to Java Expressions.
> + */
> +public class FunctionConverter {
> +  static final org.slf4j.Logger logger =
> org.slf4j.LoggerFactory.getLogger(FunctionConverter.class);
> +
> +
> +  public <T extends DrillFunc> FunctionHolder getHolder(Class<T> clazz){
> +    FunctionTemplate template =
> clazz.getAnnotation(FunctionTemplate.class);
> +    if(template == null){
> +      return failure("Class does not declare FunctionTemplate
> annotation.", clazz);
> +    }
> +
> +    // start by getting field information.
> +    List<ValueReference> params = Lists.newArrayList();
> +    List<String> workspaceFields = Lists.newArrayList();
> +    ValueReference outputField = null;
> +
> +
> +    for(Field field : clazz.getDeclaredFields()){
> +
> +      Param param = field.getAnnotation(Param.class);
> +      Output output = field.getAnnotation(Output.class);
> +      WorkSpace workspace = field.getAnnotation(WorkSpace.class);
> +
> +      int i =0;
> +      if(param != null) i++;
> +      if(output != null) i++;
> +      if(workspace != null) i++;
> +      if(i == 0){
> +        return failure("The field must be either a @Param, @Output or
> @WorkSpace field.", clazz, field);
> +      }else if(i > 1){
> +        return failure("The field must be only one of @Param, @Output or
> @WorkSpace.  It currently has more than one of these annotations.", clazz,
> field);
> +      }
> +
> +
> +
> +      if(param != null || output != null){
> +
> +        // check that param and output are value holders.
> +        if(!ValueHolder.class.isAssignableFrom(field.getType())){
> +          return failure(String.format("The field doesn't holds value of
> type %s which does not implement the ValueHolder interface.  All fields of
> type @Param or @Output must extend this interface..", field.getType()),
> clazz, field);
> +        }
> +
> +        // get the type field from the value holder.
> +        MajorType type = null;
> +        try{
> +          type = getStaticFieldValue("TYPE", field.getType(),
> MajorType.class);
> +        }catch(Exception e){
> +          return failure("Failure while trying to access the
> ValueHolder's TYPE static variable.  All ValueHolders must contain a static
> TYPE variable that defines their MajorType.", e, clazz, field.getName());
> +        }
> +
> +
> +        ValueReference p = new ValueReference(type, field.getName());
> +        if(param != null){
> +          params.add(p);
> +        }else{
> +          if(outputField != null){
> +            return failure("You've declared more than one @Output field.
>  You must declare one and only @Output field per Function class.", clazz,
> field);
> +          }else{
> +            outputField = p;
> +
> +          }
> +
> +        }
> +
> +      }else{
> +        // workspace work.
> +        workspaceFields.add(field.getName());
> +      }
> +
> +    }
> +
> +
> +    if(!workspaceFields.isEmpty()) return failure("This function declares
> one or more workspace fields.  However, those have not yet been
> implemented.", clazz);
> +    if(outputField == null)  return failure("This function declares zero
> output fields.  A function must declare one output field.", clazz);
> +
> +    // get function body.
> +
> +    CompilationUnit cu;
> +    try {
> +      cu = getClassBody(clazz);
> +    } catch (CompileException | IOException e) {
> +      return failure("Failure while getting class body.", e, clazz);
> +    }
> +
> +    Map<String, String> methods = MethodGrabbingVisitor.getMethods(cu,
> clazz);
> +
> +    // return holder
> +    ValueReference[] ps = params.toArray(new
> ValueReference[params.size()]);
> +    FunctionHolder fh = new FunctionHolder(template.scope(),
> template.nulls(), template.isBinaryCommutative(), template.name(), ps,
> outputField, methods);
> +    return fh;
> +  }
> +
> +
> +
> +  private Java.CompilationUnit getClassBody(Class<?> c) throws
> CompileException, IOException{
> +    String path = c.getName();
> +    path = path.replaceFirst("\\$.*", "");
> +    path = path.replace(".", File.separator);
> +    path = "/" + path + ".java";
> +    URL u = Resources.getResource(FunctionConverter.class, path);
> +    InputSupplier<InputStream> supplier =
> Resources.newInputStreamSupplier(u);
> +    try(InputStream is = supplier.getInput()){
> +      if(is == null){
> +        throw new IOException(String.format("Failure trying to located
> source code for Class %s, tried to read on classpath location %s",
> c.getName(), path));
> +      }
> +      String body = IO.toString(is);
> +
> +      //TODO: Hack to remove annotations so Janino doesn't choke.  Need
> to reconsider this problem...
> +      body =
> body.replaceAll("@(?:Output|Param|FunctionTemplate\\([^\\\\]*?\\))", "");
> +      return new Parser(new Scanner(null, new
> StringReader(body))).parseCompilationUnit();
> +    }
> +
> +  }
> +
> +  @SuppressWarnings("unchecked")
> +  private <T> T getStaticFieldValue(String fieldName, Class<?> valueType,
> Class<T> c) throws NoSuchFieldException, SecurityException,
> IllegalArgumentException, IllegalAccessException{
> +      Field f = valueType.getDeclaredField(fieldName);
> +      Object val = f.get(null);
> +      return (T) val;
> +  }
> +
> +  private static FunctionHolder failure(String message, Throwable t,
> Class<?> clazz, String fieldName){
> +    logger.warn("Failure loading function class {}, field {}. " +
> message, clazz.getName(), fieldName, t);
> +    return null;
> +  }
> +
> +  private FunctionHolder failure(String message, Class<?> clazz, String
> fieldName){
> +    logger.warn("Failure loading function class {}, field {}. " +
> message, clazz.getName(), fieldName);
> +    return null;
> +  }
> +
> +  private FunctionHolder failure(String message, Class<?> clazz){
> +    logger.warn("Failure loading function class {}. " + message,
> clazz.getName());
> +    return null;
> +  }
> +
> +  private FunctionHolder failure(String message, Throwable t, Class<?>
> clazz){
> +    logger.warn("Failure loading function class {}. " + message, t,
> clazz.getName());
> +    return null;
> +  }
> +
> +  private FunctionHolder failure(String message, Class<?> clazz, Field
> field){
> +    return failure(message, clazz, field.getName());
> +  }
> +
> +  public static void main(String[] args) throws Exception{
> +
> +    URL u = Resources.getResource(FunctionConverter.class,
> "/org/apache/drill/exec/expr/fn/impl/MathFunctions.java");
> +    InputStream is = Resources.newInputStreamSupplier(u).getInput();
> +    String s = IO.toString(is);
> +    System.out.println(s);
> +  }
> +
> +}
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/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
> new file mode 100644
> index 0000000..460169f
> --- /dev/null
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/FunctionHolder.java
> @@ -0,0 +1,158 @@
> +package org.apache.drill.exec.expr.fn;
> +
> +import java.util.Arrays;
> +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;
> +import org.apache.drill.exec.expr.CodeGenerator.HoldingContainer;
> +import org.apache.drill.exec.expr.annotations.FunctionTemplate;
> +import
> org.apache.drill.exec.expr.annotations.FunctionTemplate.FunctionScope;
> +import
> org.apache.drill.exec.expr.annotations.FunctionTemplate.NullHandling;
> +
> +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 {
> +
> +  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 ValueReference[] parameters;
> +  private ValueReference returnValue;
> +
> +  public FunctionHolder(FunctionScope scope, NullHandling nullHandling,
> boolean isBinaryCommutative, String functionName, ValueReference[]
> parameters, ValueReference returnValue, Map<String, String> methods) {
> +    super();
> +    this.scope = scope;
> +    this.nullHandling = nullHandling;
> +    this.isBinaryCommutative = isBinaryCommutative;
> +    this.functionName = functionName;
> +    this.setupBody = methods.get("setup");
> +    this.addBody = methods.get("add");
> +    this.evalBody = methods.get("eval");
> +    this.parameters = parameters;
> +    this.returnValue = returnValue;
> +  }
> +
> +  public 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);
> +
> +
> +
> +    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, false);
> +        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(sub);
> +
> +    // locally name external blocks.
> +
> +    // internal out value.
> +    JVar internalOutput = sub.decl(JMod.FINAL,
> g.getHolderType(returnValue.type), returnValue.name,
> JExpr._new(g.getHolderType(returnValue.type)));
> +
> +    for(int i =0; i < inputVariables.length; i++){
> +
> +      ValueReference parameter = parameters[i];
> +      HoldingContainer inputVariable = inputVariables[i];
> +      sub.decl(JMod.FINAL, inputVariable.getHolder().type(),
> parameter.name, inputVariable.getHolder());
> +    }
> +
> +    // add function body.
> +    sub.directStatement(evalBody);
> +
> +    sub.assign(out.getHolder(), internalOutput);
> +
> +    g.getBlock().directStatement(String.format("//---- end of eval
> portion of %s function. ----//\n", functionName));
> +    return out;
> +  }
> +
> +  public boolean matches(FunctionCall call){
> +    if(!softCompare(call.getMajorType(), returnValue.type)) return false;
> +    if(call.args.size() != parameters.length) return false;
> +    for(int i =0; i < parameters.length; i++){
> +      ValueReference param = parameters[i];
> +      LogicalExpression arg = call.args.get(i);
> +      if(!softCompare(param.type, arg.getMajorType())) return false;
> +    }
> +
> +    return true;
> +  }
> +
> +  private boolean softCompare(MajorType a, MajorType b){
> +    return Types.softEquals(a, b, nullHandling ==
> NullHandling.NULL_IF_NULL);
> +  }
> +
> +  public String getFunctionName() {
> +    return functionName;
> +  }
> +
> +  public static class ValueReference{
> +    MajorType type;
> +    String name;
> +    public ValueReference(MajorType type, String name) {
> +      super();
> +      this.type = type;
> +      this.name = name;
> +    }
> +    @Override
> +    public String toString() {
> +      return "ValueReference [type=" + type + ", name=" + name + "]";
> +    }
> +
> +
> +  }
> +
> +  @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/ce0da88d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/FunctionImplementationRegistry.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/FunctionImplementationRegistry.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/FunctionImplementationRegistry.java
> new file mode 100644
> index 0000000..5130f2b
> --- /dev/null
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/FunctionImplementationRegistry.java
> @@ -0,0 +1,43 @@
> +package org.apache.drill.exec.expr.fn;
> +
> +import java.util.Set;
> +
> +import org.apache.drill.common.config.DrillConfig;
> +import org.apache.drill.common.expression.FunctionCall;
> +import org.apache.drill.common.util.PathScanner;
> +import org.apache.drill.exec.ExecConstants;
> +import org.apache.drill.exec.expr.DrillFunc;
> +
> +import com.google.common.collect.ArrayListMultimap;
> +
> +public class FunctionImplementationRegistry {
> +  static final org.slf4j.Logger logger =
> org.slf4j.LoggerFactory.getLogger(FunctionImplementationRegistry.class);
> +
> +  private ArrayListMultimap<String, FunctionHolder> methods =
> ArrayListMultimap.create();
> +
> +  public FunctionImplementationRegistry(DrillConfig config){
> +    FunctionConverter converter = new FunctionConverter();
> +    Set<Class<? extends DrillFunc>> providerClasses =
> PathScanner.scanForImplementations(DrillFunc.class,
> config.getStringList(ExecConstants.FUNCTION_PACKAGES));
> +    for (Class<? extends DrillFunc> clazz : providerClasses) {
> +      FunctionHolder holder = converter.getHolder(clazz);
> +      if(holder != null){
> +        methods.put(holder.getFunctionName(), holder);
> +        logger.debug("Registering function {}", holder);
> +      }else{
> +        logger.debug("Unable to initialize function for class {}",
> clazz.getName());
> +      }
> +    }
> +  }
> +
> +  public FunctionHolder getFunction(FunctionCall call){
> +    for(FunctionHolder h : methods.get(call.getDefinition().getName())){
> +      if(h.matches(call)){
> +        return h;
> +      }
> +    }
> +    throw new UnsupportedOperationException(String.format("Unable to find
> matching function implementation for call %s.", call));
> +  }
> +
> +
> +
> +}
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/MethodGrabbingVisitor.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/MethodGrabbingVisitor.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/MethodGrabbingVisitor.java
> new file mode 100644
> index 0000000..22b9046
> --- /dev/null
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/MethodGrabbingVisitor.java
> @@ -0,0 +1,52 @@
> +package org.apache.drill.exec.expr.fn;
> +
> +import java.io.StringWriter;
> +import java.util.Map;
> +
> +import org.codehaus.janino.Java;
> +import org.codehaus.janino.Java.ClassDeclaration;
> +import org.codehaus.janino.Java.MethodDeclarator;
> +import org.codehaus.janino.util.Traverser;
> +
> +import com.beust.jcommander.internal.Maps;
> +
> +public class MethodGrabbingVisitor{
> +  static final org.slf4j.Logger logger =
> org.slf4j.LoggerFactory.getLogger(MethodGrabbingVisitor.class);
> +
> +  private Class<?> c;
> +  private Map<String, String> methods = Maps.newHashMap();
> +  private ClassFinder classFinder = new ClassFinder();
> +  private boolean captureMethods = false;
> +
> +  private MethodGrabbingVisitor(Class<?> c) {
> +    super();
> +    this.c = c;
> +  }
> +
> +  public class ClassFinder extends Traverser{
> +
> +    @Override
> +    public void traverseClassDeclaration(ClassDeclaration cd) {
> +      captureMethods = c.getName().equals(cd.getClassName());
> +      super.traverseClassDeclaration(cd);
> +    }
> +
> +    @Override
> +    public void traverseMethodDeclarator(MethodDeclarator md) {
> +      if(captureMethods){
> +        StringWriter writer = new StringWriter();
> +        ModifiedUnparseVisitor v = new ModifiedUnparseVisitor(writer);
> +        md.accept(v);
> +        methods.put(md.name, writer.getBuffer().toString());
> +      }
> +    }
> +  }
> +
> +
> +  public static Map<String, String> getMethods(Java.CompilationUnit cu,
> Class<?> c){
> +    MethodGrabbingVisitor visitor = new MethodGrabbingVisitor(c);
> +
>  cu.getPackageMemberTypeDeclarations()[0].accept(visitor.classFinder.comprehensiveVisitor());
> +    return visitor.methods;
> +  }
> +
> +}
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/ModifiedUnparseVisitor.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/ModifiedUnparseVisitor.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/ModifiedUnparseVisitor.java
> new file mode 100644
> index 0000000..72b2008
> --- /dev/null
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/ModifiedUnparseVisitor.java
> @@ -0,0 +1,891 @@
> +package org.apache.drill.exec.expr.fn;
> +
> +
> +/*
> + *
> + * Modified so that we can avoid printing some things.
> + *
> + * Janino - An embedded Java[TM] compiler
> + *
> + * Copyright (c) 2001-2010, Arno Unkrig
> + * All rights reserved.
> + *
> + * Redistribution and use in source and binary forms, with or without
> modification, are permitted provided that the
> + * following conditions are met:
> + *
> + *    1. Redistributions of source code must retain the above copyright
> notice, this list of conditions and the
> + *       following disclaimer.
> + *    2. Redistributions in binary form must reproduce the above
> copyright notice, this list of conditions and the
> + *       following disclaimer in the documentation and/or other materials
> provided with the distribution.
> + *    3. The name of the author may not be used to endorse or promote
> products derived from this software without
> + *       specific prior written permission.
> + *
> + * THIS SOFTWARE IS PROVIDED BY THE AUTHOR ``AS IS'' AND ANY EXPRESS OR
> IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED
> + * TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A
> PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL
> + * THE AUTHOR BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL,
> EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING,
> + * BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS
> OF USE, DATA, OR PROFITS; OR BUSINESS
> + * INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER
> IN CONTRACT, STRICT LIABILITY, OR TORT
> + * (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
> OF THIS SOFTWARE, EVEN IF ADVISED OF THE
> + * POSSIBILITY OF SUCH DAMAGE.
> + */
> +
> +
> +import java.io.BufferedReader;
> +import java.io.BufferedWriter;
> +import java.io.FileReader;
> +import java.io.IOException;
> +import java.io.OutputStreamWriter;
> +import java.io.PrintWriter;
> +import java.io.StringReader;
> +import java.io.Writer;
> +import java.util.HashMap;
> +import java.util.HashSet;
> +import java.util.Iterator;
> +import java.util.List;
> +import java.util.Map;
> +import java.util.Set;
> +
> +import org.codehaus.janino.JaninoRuntimeException;
> +import org.codehaus.janino.Java;
> +import org.codehaus.janino.Mod;
> +import org.codehaus.janino.Parser;
> +import org.codehaus.janino.Scanner;
> +import org.codehaus.janino.Visitor.ComprehensiveVisitor;
> +import org.codehaus.janino.util.AutoIndentWriter;
> +
> +/**
> + * A visitor that unparses (un-compiles) an AST to a {@link Writer}. See
> + * {@link #main(String[])} for a usage example.
> + */
> +public class ModifiedUnparseVisitor implements ComprehensiveVisitor {
> +    protected final AutoIndentWriter aiw;
> +    protected final PrintWriter      pw;
> +
> +    /**
> +     * Testing of parsing/unparsing.
> +     * <p>
> +     * Reads compilation units from the files named on the command line
> +     * and unparses them to {@link System#out}.
> +     */
> +    public static void main(String[] args) throws Exception {
> +        Writer w = new BufferedWriter(new OutputStreamWriter(System.out));
> +        for (int i = 0; i < args.length; ++i) {
> +            String fileName = args[i];
> +
> +            // Parse each compilation unit.
> +            FileReader r = new FileReader(fileName);
> +            Java.CompilationUnit cu;
> +            try {
> +                cu = new Parser(new Scanner(fileName,
> r)).parseCompilationUnit();
> +            } finally {
> +                r.close();
> +            }
> +
> +            // Unparse each compilation unit.
> +            ModifiedUnparseVisitor.unparse(cu, w);
> +        }
> +        w.flush();
> +    }
> +
> +    /**
> +     * Unparse the given {@link Java.CompilationUnit} to the given {@link
> Writer}.
> +     */
> +    public static void unparse(Java.CompilationUnit cu, Writer w) {
> +        ModifiedUnparseVisitor uv = new ModifiedUnparseVisitor(w);
> +        uv.unparseCompilationUnit(cu);
> +        uv.close();
> +    }
> +
> +    public ModifiedUnparseVisitor(Writer w) {
> +        this.aiw = new AutoIndentWriter(w);
> +        this.pw = new PrintWriter(this.aiw, true);
> +    }
> +
> +    /**
> +     * Flushes all generated code and closes the {@link Writer} that was
> passed
> +     * to {@link #ModifiedUnparseVisitor(Writer)}.
> +     */
> +    public void close() {
> +        this.pw.close();
> +    }
> +
> +    public void unparseCompilationUnit(Java.CompilationUnit cu) {
> +        if (cu.optionalPackageDeclaration != null) {
> +            this.pw.println();
> +            this.pw.println("package " +
> cu.optionalPackageDeclaration.packageName + ';');
> +        }
> +        if (!cu.importDeclarations.isEmpty()) {
> +            this.pw.println();
> +            for (Iterator it = cu.importDeclarations.iterator();
> it.hasNext();) {
> +                ((Java.CompilationUnit.ImportDeclaration)
> it.next()).accept(this);
> +            }
> +        }
> +        for (Iterator it = cu.packageMemberTypeDeclarations.iterator();
> it.hasNext();) {
> +            this.pw.println();
> +
>  this.unparseTypeDeclaration((Java.PackageMemberTypeDeclaration) it.next());
> +            this.pw.println();
> +        }
> +    }
> +
> +    public void
> visitSingleTypeImportDeclaration(Java.CompilationUnit.SingleTypeImportDeclaration
> stid) {
> +        this.pw.println("import " + Java.join(stid.identifiers, ".") +
> ';');
> +    }
> +    public void
> visitTypeImportOnDemandDeclaration(Java.CompilationUnit.TypeImportOnDemandDeclaration
> tiodd) {
> +        this.pw.println("import " + Java.join(tiodd.identifiers, ".") +
> ".*;");
> +    }
> +    public void
> visitSingleStaticImportDeclaration(Java.CompilationUnit.SingleStaticImportDeclaration
> ssid) {
> +        this.pw.println("import static " + Java.join(ssid.identifiers,
> ".") + ';');
> +    }
> +    public void
> visitStaticImportOnDemandDeclaration(Java.CompilationUnit.StaticImportOnDemandDeclaration
> siodd) {
> +        this.pw.println("import static " + Java.join(siodd.identifiers,
> ".") + ".*;");
> +    }
> +
> +    public void visitLocalClassDeclaration(Java.LocalClassDeclaration
> lcd) {
> +        this.unparseNamedClassDeclaration(lcd);
> +    }
> +    public void visitMemberClassDeclaration(Java.MemberClassDeclaration
> mcd) {
> +        this.unparseNamedClassDeclaration(mcd);
> +    }
> +    public void
> visitMemberInterfaceDeclaration(Java.MemberInterfaceDeclaration mid) {
> +        this.unparseInterfaceDeclaration(mid);
> +    }
> +    public void
> visitPackageMemberClassDeclaration(Java.PackageMemberClassDeclaration pmcd)
> {
> +        this.unparseNamedClassDeclaration(pmcd);
> +    }
> +    public void
> visitPackageMemberInterfaceDeclaration(Java.PackageMemberInterfaceDeclaration
> pmid) {
> +        this.unparseInterfaceDeclaration(pmid);
> +    }
> +    public void visitConstructorDeclarator(Java.ConstructorDeclarator cd)
> {
> +        this.unparseDocComment(cd);
> +        this.unparseModifiers(cd.modifiers);
> +        Java.ClassDeclaration declaringClass = cd.getDeclaringClass();
> +        this.pw.print(
> +            declaringClass instanceof Java.NamedClassDeclaration
> +            ? ((Java.NamedClassDeclaration) declaringClass).name
> +            : "UNNAMED"
> +        );
> +        this.unparseFunctionDeclaratorRest(cd);
> +        this.pw.print(' ');
> +        if (cd.optionalConstructorInvocation != null) {
> +            this.pw.println('{');
> +            this.pw.print(AutoIndentWriter.INDENT);
> +            this.unparseBlockStatement(cd.optionalConstructorInvocation);
> +            this.pw.println(';');
> +
> +            if (!cd.optionalStatements.isEmpty()) {
> +                this.pw.println();
> +                this.unparseStatements(cd.optionalStatements);
> +            }
> +            this.pw.print(AutoIndentWriter.UNINDENT + "}");
> +        } else
> +        if (cd.optionalStatements.isEmpty()) {
> +            this.pw.print("{}");
> +        } else
> +        {
> +            this.pw.println('{');
> +            this.pw.print(AutoIndentWriter.INDENT);
> +            this.unparseStatements(cd.optionalStatements);
> +            this.pw.print(AutoIndentWriter.UNINDENT + "}");
> +        }
> +    }
> +    public void visitMethodDeclarator(Java.MethodDeclarator md) {
> +        if (md.optionalStatements == null) {
> +            this.pw.print(';');
> +        } else
> +        if (md.optionalStatements.isEmpty()) {
> +            this.pw.print(" ");
> +        } else
> +        {
> +            this.pw.print(AutoIndentWriter.INDENT);
> +            this.unparseStatements(md.optionalStatements);
> +            this.pw.print(AutoIndentWriter.UNINDENT);
> +        }
> +    }
> +    public void visitFieldDeclaration(Java.FieldDeclaration fd) {
> +        this.unparseDocComment(fd);
> +        this.unparseModifiers(fd.modifiers);
> +        this.unparseType(fd.type);
> +        this.pw.print(' ');
> +        for (int i = 0; i < fd.variableDeclarators.length; ++i) {
> +            if (i > 0) this.pw.print(", ");
> +            this.unparseVariableDeclarator(fd.variableDeclarators[i]);
> +        }
> +        this.pw.print(';');
> +    }
> +    public void visitInitializer(Java.Initializer i) {
> +        if (i.statiC) this.pw.print("static ");
> +        this.unparseBlockStatement(i.block);
> +    }
> +    public void visitBlock(Java.Block b) {
> +        if (b.statements.isEmpty()) {
> +            this.pw.print("{}");
> +            return;
> +        }
> +        this.pw.println('{');
> +        this.pw.print(AutoIndentWriter.INDENT);
> +        this.unparseStatements(b.statements);
> +        this.pw.print(AutoIndentWriter.UNINDENT + "}");
> +    }
> +
> +    private void unparseStatements(List statements) {
> +        int state = -1;
> +        for (Iterator it = statements.iterator(); it.hasNext();) {
> +            Java.BlockStatement bs = (Java.BlockStatement) it.next();
> +            int x = (
> +                bs instanceof Java.Block                             ? 1 :
> +                bs instanceof Java.LocalClassDeclarationStatement    ? 2 :
> +                bs instanceof Java.LocalVariableDeclarationStatement ? 3 :
> +                bs instanceof Java.SynchronizedStatement             ? 4 :
> +                99
> +            );
> +            if (state != -1 && state != x)
> this.pw.println(AutoIndentWriter.CLEAR_TABULATORS);
> +            state = x;
> +
> +            this.unparseBlockStatement(bs);
> +            this.pw.println();
> +        }
> +    }
> +    public void visitBreakStatement(Java.BreakStatement bs) {
> +        this.pw.print("break");
> +        if (bs.optionalLabel != null) this.pw.print(' ' +
> bs.optionalLabel);
> +        this.pw.print(';');
> +    }
> +    public void visitContinueStatement(Java.ContinueStatement cs) {
> +        this.pw.print("continue");
> +        if (cs.optionalLabel != null) this.pw.print(' ' +
> cs.optionalLabel);
> +        this.pw.print(';');
> +    }
> +    public void visitDoStatement(Java.DoStatement ds) {
> +        this.pw.print("do ");
> +        this.unparseBlockStatement(ds.body);
> +        this.pw.print("while (");
> +        this.unparse(ds.condition);
> +        this.pw.print(");");
> +    }
> +    public void visitEmptyStatement(Java.EmptyStatement es) {
> +        this.pw.print(';');
> +    }
> +    public void visitExpressionStatement(Java.ExpressionStatement es) {
> +        this.unparse(es.rvalue);
> +        this.pw.print(';');
> +    }
> +    public void visitForStatement(Java.ForStatement fs) {
> +        this.pw.print("for (");
> +        if (fs.optionalInit != null) {
> +            this.unparseBlockStatement(fs.optionalInit);
> +        } else {
> +            this.pw.print(';');
> +        }
> +        if (fs.optionalCondition != null) {
> +            this.pw.print(' ');
> +            this.unparse(fs.optionalCondition);
> +        }
> +        this.pw.print(';');
> +        if (fs.optionalUpdate != null) {
> +            this.pw.print(' ');
> +            for (int i = 0; i < fs.optionalUpdate.length; ++i) {
> +                if (i > 0) this.pw.print(", ");
> +                this.unparse(fs.optionalUpdate[i]);
> +            }
> +        }
> +        this.pw.print(") ");
> +        this.unparseBlockStatement(fs.body);
> +    }
> +    public void visitIfStatement(Java.IfStatement is) {
> +        this.pw.print("if (");
> +        this.unparse(is.condition);
> +        this.pw.print(") ");
> +        this.unparseBlockStatement(is.thenStatement);
> +        if (is.optionalElseStatement != null) {
> +            this.pw.println(" else");
> +            this.unparseBlockStatement(is.optionalElseStatement);
> +        }
> +    }
> +    public void visitLabeledStatement(Java.LabeledStatement ls) {
> +        this.pw.println(ls.label + ':');
> +        this.unparseBlockStatement(ls.body);
> +    }
> +    public void
> visitLocalClassDeclarationStatement(Java.LocalClassDeclarationStatement
> lcds) {
> +        this.unparseTypeDeclaration(lcds.lcd);
> +    }
> +    public void
> visitLocalVariableDeclarationStatement(Java.LocalVariableDeclarationStatement
> lvds) {
> +        this.unparseModifiers(lvds.modifiers);
> +        this.unparseType(lvds.type);
> +        this.pw.print(' ');
> +        this.pw.print(AutoIndentWriter.TABULATOR);
> +        this.unparseVariableDeclarator(lvds.variableDeclarators[0]);
> +        for (int i = 1; i < lvds.variableDeclarators.length; ++i) {
> +            this.pw.print(", ");
> +            this.unparseVariableDeclarator(lvds.variableDeclarators[i]);
> +        }
> +        this.pw.print(';');
> +    }
> +    public void visitReturnStatement(Java.ReturnStatement rs) {
> +        this.pw.print("return");
> +        if (rs.optionalReturnValue != null) {
> +            this.pw.print(' ');
> +            this.unparse(rs.optionalReturnValue);
> +        }
> +        this.pw.print(';');
> +    }
> +    public void visitSwitchStatement(Java.SwitchStatement ss) {
> +        this.pw.print("switch (");
> +        this.unparse(ss.condition);
> +        this.pw.println(") {");
> +        for (Iterator it = ss.sbsgs.iterator(); it.hasNext();) {
> +            Java.SwitchStatement.SwitchBlockStatementGroup sbgs = (
> +                (Java.SwitchStatement.SwitchBlockStatementGroup) it.next()
> +            );
> +            this.pw.print(AutoIndentWriter.UNINDENT);
> +            try {
> +                for (Iterator it2 = sbgs.caseLabels.iterator();
> it2.hasNext();) {
> +                    Java.Rvalue rv = (Java.Rvalue) it2.next();
> +                    this.pw.print("case ");
> +                    this.unparse(rv);
> +                    this.pw.println(':');
> +                }
> +                if (sbgs.hasDefaultLabel) this.pw.println("default:");
> +            } finally {
> +                this.pw.print(AutoIndentWriter.INDENT);
> +            }
> +            for (Iterator it2 = sbgs.blockStatements.iterator();
> it2.hasNext();) {
> +                this.unparseBlockStatement((Java.BlockStatement)
> it2.next());
> +                this.pw.println();
> +            }
> +        }
> +        this.pw.print('}');
> +    }
> +    public void visitSynchronizedStatement(Java.SynchronizedStatement ss)
> {
> +        this.pw.print("synchronized (");
> +        this.unparse(ss.expression);
> +        this.pw.print(") ");
> +        this.unparseBlockStatement(ss.body);
> +    }
> +    public void visitThrowStatement(Java.ThrowStatement ts) {
> +        this.pw.print("throw ");
> +        this.unparse(ts.expression);
> +        this.pw.print(';');
> +    }
> +    public void visitTryStatement(Java.TryStatement ts) {
> +        this.pw.print("try ");
> +        this.unparseBlockStatement(ts.body);
> +        for (Iterator it = ts.catchClauses.iterator(); it.hasNext();) {
> +            Java.CatchClause cc = (Java.CatchClause) it.next();
> +            this.pw.print(" catch (");
> +            this.unparseFormalParameter(cc.caughtException);
> +            this.pw.print(") ");
> +            this.unparseBlockStatement(cc.body);
> +        }
> +        if (ts.optionalFinally != null) {
> +            this.pw.print(" finally ");
> +            this.unparseBlockStatement(ts.optionalFinally);
> +        }
> +    }
> +    public void visitWhileStatement(Java.WhileStatement ws) {
> +        this.pw.print("while (");
> +        this.unparse(ws.condition);
> +        this.pw.print(") ");
> +        this.unparseBlockStatement(ws.body);
> +    }
> +    public void unparseVariableDeclarator(Java.VariableDeclarator vd) {
> +        this.pw.print(vd.name);
> +        for (int i = 0; i < vd.brackets; ++i) this.pw.print("[]");
> +        if (vd.optionalInitializer != null) {
> +            this.pw.print(" = ");
> +            this.unparseArrayInitializerOrRvalue(vd.optionalInitializer);
> +        }
> +    }
> +    public void
> unparseFormalParameter(Java.FunctionDeclarator.FormalParameter fp) {
> +        if (fp.finaL) this.pw.print("final ");
> +        this.unparseType(fp.type);
> +        this.pw.print(" " + AutoIndentWriter.TABULATOR + fp.name);
> +    }
> +    public void visitMethodInvocation(Java.MethodInvocation mi) {
> +        if (mi.optionalTarget != null) {
> +            this.unparseLhs(mi.optionalTarget, ".");
> +            this.pw.print('.');
> +        }
> +        this.pw.print(mi.methodName);
> +        this.unparseFunctionInvocationArguments(mi.arguments);
> +    }
> +    public void
> visitAlternateConstructorInvocation(Java.AlternateConstructorInvocation
> aci) {
> +        this.pw.print("this");
> +        this.unparseFunctionInvocationArguments(aci.arguments);
> +    }
> +    public void
> visitSuperConstructorInvocation(Java.SuperConstructorInvocation sci) {
> +        if (sci.optionalQualification != null) {
> +            this.unparseLhs(sci.optionalQualification, ".");
> +            this.pw.print('.');
> +        }
> +        this.pw.print("super");
> +        this.unparseFunctionInvocationArguments(sci.arguments);
> +    }
> +    public void visitNewClassInstance(Java.NewClassInstance nci) {
> +        if (nci.optionalQualification != null) {
> +            this.unparseLhs(nci.optionalQualification, ".");
> +            this.pw.print('.');
> +        }
> +        this.pw.print("new " + nci.type.toString());
> +        this.unparseFunctionInvocationArguments(nci.arguments);
> +    }
> +    public void visitAssignment(Java.Assignment a) {
> +        this.unparseLhs(a.lhs, a.operator);
> +        this.pw.print(' ' + a.operator + ' ');
> +        this.unparseRhs(a.rhs, a.operator);
> +    }
> +    public void visitAmbiguousName(Java.AmbiguousName an) {
> this.pw.print(an.toString()); }
> +    public void visitArrayAccessExpression(Java.ArrayAccessExpression
> aae) {
> +        this.unparseLhs(aae.lhs, "[ ]");
> +        this.pw.print('[');
> +        this.unparse(aae.index);
> +        this.pw.print(']');
> +    }
> +    public void visitArrayLength(Java.ArrayLength al) {
> +        this.unparseLhs(al.lhs, ".");
> +        this.pw.print(".length");
> +    }
> +    public void visitArrayType(Java.ArrayType at) {
> +        this.unparseType(at.componentType);
> +        this.pw.print("[]");
> +    }
> +    public void visitBasicType(Java.BasicType bt) {
> +        this.pw.print(bt.toString());
> +    }
> +    public void visitBinaryOperation(Java.BinaryOperation bo) {
> +        this.unparseLhs(bo.lhs, bo.op);
> +        this.pw.print(' ' + bo.op + ' ');
> +        this.unparseRhs(bo.rhs, bo.op);
> +    }
> +    public void visitCast(Java.Cast c) {
> +        this.pw.print('(');
> +        this.unparseType(c.targetType);
> +        this.pw.print(") ");
> +        this.unparseRhs(c.value, "cast");
> +    }
> +    public void visitClassLiteral(Java.ClassLiteral cl) {
> +        this.unparseType(cl.type);
> +        this.pw.print(".class");
> +    }
> +    public void visitConditionalExpression(Java.ConditionalExpression ce)
> {
> +        this.unparseLhs(ce.lhs, "?:");
> +        this.pw.print(" ? ");
> +        this.unparseLhs(ce.mhs, "?:");
> +        this.pw.print(" : ");
> +        this.unparseRhs(ce.rhs, "?:");
> +    }
> +    public void visitCrement(Java.Crement c) {
> +        if (c.pre) {
> +            this.pw.print(c.operator);
> +            this.unparseUnaryOperation(c.operand, c.operator + "x");
> +        } else
> +        {
> +            this.unparseUnaryOperation(c.operand, "x" + c.operator);
> +            this.pw.print(c.operator);
> +        }
> +    }
> +    public void visitFieldAccess(Java.FieldAccess fa) {
> +        this.unparseLhs(fa.lhs, ".");
> +        this.pw.print('.' + fa.field.getName());
> +    }
> +    public void visitFieldAccessExpression(Java.FieldAccessExpression
> fae) {
> +        this.unparseLhs(fae.lhs, ".");
> +        this.pw.print('.' + fae.fieldName);
> +    }
> +    public void
> visitSuperclassFieldAccessExpression(Java.SuperclassFieldAccessExpression
> scfae) {
> +        if (scfae.optionalQualification != null) {
> +            this.unparseType(scfae.optionalQualification);
> +            this.pw.print(".super." + scfae.fieldName);
> +        } else
> +        {
> +            this.pw.print("super." + scfae.fieldName);
> +        }
> +    }
> +    public void visitInstanceof(Java.Instanceof io) {
> +        this.unparseLhs(io.lhs, "instanceof");
> +        this.pw.print(" instanceof ");
> +        this.unparseType(io.rhs);
> +    }
> +    public void visitLiteral(Java.Literal l) {
> this.pw.print(l.toString()); }
> +    public void visitLocalVariableAccess(Java.LocalVariableAccess lva) {
> this.pw.print(lva.toString()); }
> +    public void visitNewArray(Java.NewArray na) {
> +        this.pw.print("new ");
> +        this.unparseType(na.type);
> +        for (int i = 0; i < na.dimExprs.length; ++i) {
> +            this.pw.print('[');
> +            this.unparse(na.dimExprs[i]);
> +            this.pw.print(']');
> +        }
> +        for (int i = 0; i < na.dims; ++i) {
> +            this.pw.print("[]");
> +        }
> +    }
> +    public void visitNewInitializedArray(Java.NewInitializedArray nai) {
> +        this.pw.print("new ");
> +        this.unparseType(nai.arrayType);
> +        this.pw.print(" ");
> +        this.unparseArrayInitializerOrRvalue(nai.arrayInitializer);
> +    }
> +    public void visitPackage(Java.Package p) {
> this.pw.print(p.toString()); }
> +    public void visitParameterAccess(Java.ParameterAccess pa) {
> this.pw.print(pa.toString()); }
> +    public void visitQualifiedThisReference(Java.QualifiedThisReference
> qtr) {
> +        this.unparseType(qtr.qualification);
> +        this.pw.print(".this");
> +    }
> +    public void visitReferenceType(Java.ReferenceType rt) {
> this.pw.print(rt.toString()); }
> +    public void visitRvalueMemberType(Java.RvalueMemberType rmt) {
> this.pw.print(rmt.toString()); }
> +    public void visitSimpleType(Java.SimpleType st) {
> this.pw.print(st.toString()); }
> +    public void
> visitSuperclassMethodInvocation(Java.SuperclassMethodInvocation smi) {
> +        this.pw.print("super." + smi.methodName);
> +        this.unparseFunctionInvocationArguments(smi.arguments);
> +    }
> +    public void visitThisReference(Java.ThisReference tr) {
> +        this.pw.print("this");
> +    }
> +    public void visitUnaryOperation(Java.UnaryOperation uo) {
> +        this.pw.print(uo.operator);
> +        this.unparseUnaryOperation(uo.operand, uo.operator + "x");
> +    }
> +    public void visitParenthesizedExpression(Java.ParenthesizedExpression
> pe) {
> +        this.pw.print('(');
> +        this.unparse(pe.value);
> +        this.pw.print(')');
> +    }
> +
> +    // Helpers
> +
> +    private void unparseBlockStatement(Java.BlockStatement
> blockStatement) {
> +        blockStatement.accept(this);
> +    }
> +
> +    private void unparseTypeDeclaration(Java.TypeDeclaration
> typeDeclaration) {
> +        typeDeclaration.accept(this);
> +    }
> +
> +    private void unparseType(Java.Type type) {
> +        ((Java.Atom) type).accept(this);
> +    }
> +
> +    private void unparse(Java.Atom operand) {
> +        operand.accept(this);
> +    }
> +
> +    /**
> +     * Iff the <code>operand</code> is unnatural for the
> <code>unaryOperator</code>, enclose the
> +     * <code>operand</code> in parentheses. Example: "a+b" is an
> unnatural operand for unary "!x".
> +     *
> +     * @param unaryOperator ++x --x +x -x ~x !x x++ x--
> +     */
> +    private void unparseUnaryOperation(Java.Rvalue operand, String
> unaryOperator) {
> +        int cmp = ModifiedUnparseVisitor.comparePrecedence(unaryOperator,
> operand);
> +        this.unparse(operand, cmp < 0);
> +    }
> +
> +    /**
> +     * Iff the <code>lhs</code> is unnatural for the
> <code>binaryOperator</code>, enclose the
> +     * <code>lhs</code> in parentheses. Example: "a+b" is an unnatural
> lhs for operator "*".
> +     *
> +     * @param binaryOperator = +=... ?: || && | ^ & == != < > <= >=
> instanceof << >> >>> + - * / % cast
> +     */
> +
> +    private void unparseLhs(Java.Atom lhs, String binaryOperator) {
> +        int cmp =
> ModifiedUnparseVisitor.comparePrecedence(binaryOperator, lhs);
> +        this.unparse(lhs, cmp < 0 || (cmp == 0 &&
> ModifiedUnparseVisitor.isLeftAssociate(binaryOperator)));
> +    }
> +
> +
> +    /**
> +     * Iff the <code>rhs</code> is unnatural for the
> <code>binaryOperator</code>, enclose the
> +     * <code>rhs</code> in parentheses. Example: "a+b" is an unnatural
> rhs for operator "*".
> +     */
> +    private void unparseRhs(Java.Rvalue rhs, String binaryOperator) {
> +        int cmp =
> ModifiedUnparseVisitor.comparePrecedence(binaryOperator, rhs);
> +        this.unparse(rhs, cmp < 0 || (cmp == 0 &&
> ModifiedUnparseVisitor.isRightAssociate(binaryOperator)));
> +    }
> +
> +    private void unparse(Java.Atom operand, boolean natural) {
> +        if (!natural) this.pw.print("((( ");
> +        this.unparse(operand);
> +        if (!natural) this.pw.print(" )))");
> +    }
> +
> +    /**
> +     * Return true iff operator is right associative e.g. <code>a = b =
> c</code> evaluates as
> +     * <code>a = (b = c)</code>.
> +     *
> +     * @return Return true iff operator is right associative
> +     */
> +    private static boolean isRightAssociate(String op) {
> +        return
> ModifiedUnparseVisitor.RIGHT_ASSOCIATIVE_OPERATORS.contains(op);
> +    }
> +
> +    /**
> +     * Return true iff operator is left associative e.g. <code>a - b -
> c</code> evaluates as
> +     * <code>(a - b) - c</code>.
> +     *
> +     * @return Return true iff operator is left associative
> +     */
> +    private static boolean isLeftAssociate(String op) {
> +        return
> ModifiedUnparseVisitor.LEFT_ASSOCIATIVE_OPERATORS.contains(op);
> +    }
> +
> +    /**
> +     * Returns a value
> +     * <ul>
> +     *   <li>&lt; 0 iff the <code>operator</code> has lower precedence
> than the <code>operand</code>
> +     *   <li>==; 0 iff the <code>operator</code> has equal precedence
> than the <code>operand</code>
> +     *   <li>&gt; 0 iff the <code>operator</code> has higher precedence
> than the <code>operand</code>
> +     * </ul>
> +     */
> +    private static int comparePrecedence(String operator, Java.Atom
> operand) {
> +        if (operand instanceof Java.BinaryOperation) {
> +            return (
> +                ModifiedUnparseVisitor.getOperatorPrecedence(operator)
> +                -
> ModifiedUnparseVisitor.getOperatorPrecedence(((Java.BinaryOperation)
> operand).op)
> +            );
> +        } else
> +        if (operand instanceof Java.UnaryOperation) {
> +            return (
> +                ModifiedUnparseVisitor.getOperatorPrecedence(operator)
> +                -
> ModifiedUnparseVisitor.getOperatorPrecedence(((Java.UnaryOperation)
> operand).operator + "x")
> +            );
> +        } else
> +        if (operand instanceof Java.ConditionalExpression) {
> +            return ModifiedUnparseVisitor.getOperatorPrecedence(operator)
> - ModifiedUnparseVisitor.getOperatorPrecedence("?:");
> +        } else
> +        if (operand instanceof Java.Instanceof) {
> +            return ModifiedUnparseVisitor.getOperatorPrecedence(operator)
> - ModifiedUnparseVisitor.getOperatorPrecedence("instanceof");
> +        } else
> +        if (operand instanceof Java.Cast) {
> +            return ModifiedUnparseVisitor.getOperatorPrecedence(operator)
> - ModifiedUnparseVisitor.getOperatorPrecedence("cast");
> +        } else
> +        if (operand instanceof Java.MethodInvocation || operand
> instanceof Java.FieldAccess) {
> +            return ModifiedUnparseVisitor.getOperatorPrecedence(operator)
> - ModifiedUnparseVisitor.getOperatorPrecedence(".");
> +        } else
> +        if (operand instanceof Java.NewArray) {
> +            return ModifiedUnparseVisitor.getOperatorPrecedence(operator)
> - ModifiedUnparseVisitor.getOperatorPrecedence("new");
> +        } else
> +        if (operand instanceof Java.Crement) {
> +            Java.Crement c = (Java.Crement) operand;
> +            return (
> +                ModifiedUnparseVisitor.getOperatorPrecedence(operator)
> +                - ModifiedUnparseVisitor.getOperatorPrecedence(c.pre ?
> c.operator + "x" : "x" + c.operator)
> +            );
> +        } else
> +        {
> +            // All other rvalues (e.g. literal) have higher precedence
> than any operator.
> +            return -1;
> +        }
> +    }
> +    private static int getOperatorPrecedence(String operator) {
> +        return ((Integer)
> ModifiedUnparseVisitor.OPERATOR_PRECEDENCE.get(operator)).intValue();
> +    }
> +    private static final Set LEFT_ASSOCIATIVE_OPERATORS = new HashSet();
> +    private static final Set RIGHT_ASSOCIATIVE_OPERATORS = new HashSet();
> +    private static final Set UNARY_OPERATORS = new HashSet();
> +    private static final Map OPERATOR_PRECEDENCE = new HashMap();
> +    static {
> +        Object[] ops = {
> +            ModifiedUnparseVisitor.RIGHT_ASSOCIATIVE_OPERATORS, "=",
> "*=", "/=", "%=", "+=", "-=", "<<=", ">>=", ">>>=",
> +                                                        "&=", "^=", "|=",
> +            ModifiedUnparseVisitor.RIGHT_ASSOCIATIVE_OPERATORS, "?:",
> +            ModifiedUnparseVisitor.LEFT_ASSOCIATIVE_OPERATORS,  "||",
> +            ModifiedUnparseVisitor.LEFT_ASSOCIATIVE_OPERATORS,  "&&",
> +            ModifiedUnparseVisitor.LEFT_ASSOCIATIVE_OPERATORS,  "|",
> +            ModifiedUnparseVisitor.LEFT_ASSOCIATIVE_OPERATORS,  "^",
> +            ModifiedUnparseVisitor.LEFT_ASSOCIATIVE_OPERATORS,  "&",
> +            ModifiedUnparseVisitor.LEFT_ASSOCIATIVE_OPERATORS,  "==",
> "!=",
> +            ModifiedUnparseVisitor.LEFT_ASSOCIATIVE_OPERATORS,  "<", ">",
> "<=", ">=", "instanceof",
> +            ModifiedUnparseVisitor.LEFT_ASSOCIATIVE_OPERATORS,  "<<",
> ">>", ">>>",
> +            ModifiedUnparseVisitor.LEFT_ASSOCIATIVE_OPERATORS,  "+", "-",
> +            ModifiedUnparseVisitor.LEFT_ASSOCIATIVE_OPERATORS,  "*", "/",
> "%",
> +            ModifiedUnparseVisitor.RIGHT_ASSOCIATIVE_OPERATORS, "cast",
> +            ModifiedUnparseVisitor.UNARY_OPERATORS,             "++x",
> "--x", "+x", "-x", "~x", "!x",
> +            ModifiedUnparseVisitor.UNARY_OPERATORS,             "x++",
> "x--",
> +            ModifiedUnparseVisitor.LEFT_ASSOCIATIVE_OPERATORS,  "new",
> +            ModifiedUnparseVisitor.LEFT_ASSOCIATIVE_OPERATORS,  ".", "[
> ]",
> +        };
> +        int precedence = 0;
> +        LOOP1: for (int i = 0;;) {
> +            Set s = (Set) ops[i++];
> +            Integer pi = new Integer(++precedence);
> +            for (;;) {
> +                if (i == ops.length) break LOOP1;
> +                if (!(ops[i] instanceof String)) break;
> +                String op = (String) ops[i++];
> +                s.add(op);
> +                ModifiedUnparseVisitor.OPERATOR_PRECEDENCE.put(op, pi);
> +            }
> +        }
> +    }
> +
> +    private void unparseNamedClassDeclaration(Java.NamedClassDeclaration
> ncd) {
> +        this.unparseDocComment(ncd);
> +        this.unparseModifiers(ncd.getModifiers());
> +        this.pw.print("class " + ncd.name);
> +        if (ncd.optionalExtendedType != null) {
> +            this.pw.print(" extends ");
> +            this.unparseType(ncd.optionalExtendedType);
> +        }
> +        if (ncd.implementedTypes.length > 0) this.pw.print(" implements "
> + Java.join(ncd.implementedTypes, ", "));
> +        this.pw.println(" {");
> +        this.pw.print(AutoIndentWriter.INDENT);
> +        this.unparseClassDeclarationBody(ncd);
> +        this.pw.print(AutoIndentWriter.UNINDENT + "}");
> +    }
> +    private void
> unparseArrayInitializerOrRvalue(Java.ArrayInitializerOrRvalue aiorv) {
> +        if (aiorv instanceof Java.Rvalue) {
> +            this.unparse((Java.Rvalue) aiorv);
> +        } else
> +        if (aiorv instanceof Java.ArrayInitializer) {
> +            Java.ArrayInitializer ai = (Java.ArrayInitializer) aiorv;
> +            if (ai.values.length == 0) {
> +                this.pw.print("{}");
> +            } else
> +            {
> +                this.pw.print("{ ");
> +                this.unparseArrayInitializerOrRvalue(ai.values[0]);
> +                for (int i = 1; i < ai.values.length; ++i) {
> +                    this.pw.print(", ");
> +                    this.unparseArrayInitializerOrRvalue(ai.values[i]);
> +                }
> +                this.pw.print(" }");
> +            }
> +        } else
> +        {
> +            throw new JaninoRuntimeException(
> +                "Unexpected array initializer or rvalue class "
> +                + aiorv.getClass().getName()
> +            );
> +        }
> +    }
> +
> +    public void
> visitAnonymousClassDeclaration(Java.AnonymousClassDeclaration acd) {
> +        this.unparseType(acd.baseType);
> +        this.pw.println(" {");
> +        this.pw.print(AutoIndentWriter.INDENT);
> +        this.unparseClassDeclarationBody(acd);
> +        this.pw.print(AutoIndentWriter.UNINDENT + "}");
> +    }
> +    public void
> visitNewAnonymousClassInstance(Java.NewAnonymousClassInstance naci) {
> +        if (naci.optionalQualification != null) {
> +            this.unparseLhs(naci.optionalQualification, ".");
> +            this.pw.print('.');
> +        }
> +        this.pw.print("new " +
> naci.anonymousClassDeclaration.baseType.toString() + '(');
> +        for (int i = 0; i < naci.arguments.length; ++i) {
> +            if (i > 0) this.pw.print(", ");
> +            this.unparse(naci.arguments[i]);
> +        }
> +        this.pw.println(") {");
> +        this.pw.print(AutoIndentWriter.INDENT);
> +        this.unparseClassDeclarationBody(naci.anonymousClassDeclaration);
> +        this.pw.print(AutoIndentWriter.UNINDENT + "}");
> +    }
> +    // Multi-line!
> +    private void unparseClassDeclarationBody(Java.ClassDeclaration cd) {
> +        for (Iterator it = cd.constructors.iterator(); it.hasNext();) {
> +            this.pw.println();
> +            ((Java.ConstructorDeclarator) it.next()).accept(this);
> +            this.pw.println();
> +        }
> +        this.unparseAbstractTypeDeclarationBody(cd);
> +        for (Iterator it =
> cd.variableDeclaratorsAndInitializers.iterator(); it.hasNext();) {
> +            this.pw.println();
> +            ((Java.TypeBodyDeclaration) it.next()).accept(this);
> +            this.pw.println();
> +        }
> +    }
> +    private void unparseInterfaceDeclaration(Java.InterfaceDeclaration
> id) {
> +        this.unparseDocComment(id);
> +        this.unparseModifiers(id.getModifiers());
> +        //make sure we print "interface", even if it wasn't in the
> modifiers
> +        if ((id.getModifiers() & Mod.INTERFACE) == 0) {
> +            this.pw.print("interface ");
> +        }
> +        this.pw.print(id.name);
> +        if (id.extendedTypes.length > 0) this.pw.print(" extends " +
> Java.join(id.extendedTypes, ", "));
> +        this.pw.println(" {");
> +        this.pw.print(AutoIndentWriter.INDENT);
> +        this.unparseAbstractTypeDeclarationBody(id);
> +        for (Iterator it = id.constantDeclarations.iterator();
> it.hasNext();) {
> +            ((Java.TypeBodyDeclaration) it.next()).accept(this);
> +            this.pw.println();
> +        }
> +        this.pw.print(AutoIndentWriter.UNINDENT + "}");
> +    }
> +    // Multi-line!
> +    private void
> unparseAbstractTypeDeclarationBody(Java.AbstractTypeDeclaration atd) {
> +        for (Iterator it = atd.getMethodDeclarations().iterator();
> it.hasNext();) {
> +            this.pw.println();
> +            ((Java.MethodDeclarator) it.next()).accept(this);
> +            this.pw.println();
> +        }
> +        for (Iterator it = atd.getMemberTypeDeclarations().iterator();
> it.hasNext();) {
> +            this.pw.println();
> +            ((Java.TypeBodyDeclaration) it.next()).accept(this);
> +            this.pw.println();
> +        }
> +    }
> +    private void unparseFunctionDeclaratorRest(Java.FunctionDeclarator
> fd) {
> +        boolean big = fd.formalParameters.length >= 4;
> +        this.pw.print('(');
> +        if (big) { this.pw.println();
> this.pw.print(AutoIndentWriter.INDENT); }
> +        for (int i = 0; i < fd.formalParameters.length; ++i) {
> +            if (i > 0) {
> +                if (big) {
> +                    this.pw.println(',');
> +                } else
> +                {
> +                    this.pw.print(", ");
> +                }
> +            }
> +            this.unparseFormalParameter(fd.formalParameters[i]);
> +        }
> +        if (big) { this.pw.println();
> this.pw.print(AutoIndentWriter.UNINDENT); }
> +        this.pw.print(')');
> +        if (fd.thrownExceptions.length > 0) this.pw.print(" throws " +
> Java.join(fd.thrownExceptions, ", "));
> +    }
> +    private void unparseDocComment(Java.DocCommentable dc) {
> +        String optionalDocComment = dc.getDocComment();
> +        if (optionalDocComment != null) {
> +            this.pw.print("/**");
> +            BufferedReader br = new BufferedReader(new
> StringReader(optionalDocComment));
> +            for (;;) {
> +                String line;
> +                try {
> +                    line = br.readLine();
> +                } catch (IOException e) {
> +                    throw new JaninoRuntimeException();
> +                }
> +                if (line == null) break;
> +                this.pw.println(line);
> +                this.pw.print(" *");
> +            }
> +            this.pw.println("/");
> +        }
> +    }
> +    private void unparseModifiers(short modifiers) {
> +        if (modifiers != 0) {
> +            this.pw.print(Mod.shortToString(modifiers) + ' ');
> +        }
> +    }
> +    private void unparseFunctionInvocationArguments(Java.Rvalue[]
> arguments) {
> +        boolean big = arguments.length >= 5;
> +        this.pw.print('(');
> +        if (big) { this.pw.println();
> this.pw.print(AutoIndentWriter.INDENT); }
> +        for (int i = 0; i < arguments.length; ++i) {
> +            if (i > 0) {
> +                if (big) {
> +                    this.pw.println(',');
> +                } else
> +                {
> +                    this.pw.print(", ");
> +                }
> +            }
> +            this.unparse(arguments[i]);
> +        }
> +        if (big) { this.pw.println();
> this.pw.print(AutoIndentWriter.UNINDENT); }
> +        this.pw.print(')');
> +    }
> +
> +}
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/MathFunctions.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/MathFunctions.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/MathFunctions.java
> new file mode 100644
> index 0000000..702a0ad
> --- /dev/null
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/MathFunctions.java
> @@ -0,0 +1,48 @@
> +package org.apache.drill.exec.expr.fn.impl;
> +
> +import org.apache.drill.exec.expr.DrillFunc;
> +import org.apache.drill.exec.expr.annotations.FunctionTemplate;
> +import
> org.apache.drill.exec.expr.annotations.FunctionTemplate.FunctionScope;
> +import
> org.apache.drill.exec.expr.annotations.FunctionTemplate.NullHandling;
> +import org.apache.drill.exec.expr.annotations.Output;
> +import org.apache.drill.exec.expr.annotations.Param;
> +import
> org.apache.drill.exec.expr.holders.ValueHolderImplmenetations.IntHolder;
> +import
> org.apache.drill.exec.expr.holders.ValueHolderImplmenetations.LongHolder;
> +import org.apache.drill.exec.record.RecordBatch;
> +
> +public class MathFunctions{
> +  static final org.slf4j.Logger logger =
> org.slf4j.LoggerFactory.getLogger(MathFunctions.class);
> +
> +  private MathFunctions(){}
> +
> +  @FunctionTemplate(name = "add", scope = FunctionScope.SIMPLE, nulls =
> NullHandling.NULL_IF_NULL)
> +  public static class Add1 implements DrillFunc{
> +
> +    @Param IntHolder left;
> +    @Param IntHolder right;
> +    @Output IntHolder out;
> +
> +    public void setup(RecordBatch b){}
> +
> +    public void eval(){
> +      out.value = left.value + right.value;
> +    }
> +
> +  }
> +
> +  @FunctionTemplate(name = "add", scope = FunctionScope.SIMPLE, nulls =
> NullHandling.NULL_IF_NULL)
> +  public static class LongAdd1 implements DrillFunc{
> +
> +    @Param LongHolder left;
> +    @Param LongHolder right;
> +    @Output LongHolder out;
> +
> +    public void setup(RecordBatch b){}
> +
> +    public void eval(){
> +      out.value = left.value + right.value;
> +    }
> +
> +  }
> +
> +}
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/holders/ValueHolder.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/holders/ValueHolder.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/holders/ValueHolder.java
> new file mode 100644
> index 0000000..0ef2e04
> --- /dev/null
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/holders/ValueHolder.java
> @@ -0,0 +1,4 @@
> +package org.apache.drill.exec.expr.holders;
> +
> +public interface ValueHolder {
> +}
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/holders/ValueHolderImplmenetations.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/holders/ValueHolderImplmenetations.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/holders/ValueHolderImplmenetations.java
> new file mode 100644
> index 0000000..adf97fe
> --- /dev/null
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/holders/ValueHolderImplmenetations.java
> @@ -0,0 +1,44 @@
> +package org.apache.drill.exec.expr.holders;
> +
> +import org.apache.drill.common.types.TypeProtos.MajorType;
> +import org.apache.drill.common.types.TypeProtos.MinorType;
> +import org.apache.drill.common.types.Types;
> +
> +public class ValueHolderImplmenetations {
> +
> +  private ValueHolderImplmenetations(){}
> +
> +  public final static class BooleanHolder implements ValueHolder{
> +    public static final MajorType TYPE =
> Types.required(MinorType.BOOLEAN);
> +    public int value;
> +  }
> +
> +  public final static class NullableBooleanHolder implements ValueHolder {
> +    public static final MajorType TYPE =
> Types.optional(MinorType.BOOLEAN);
> +    public int value;
> +    public int isSet;
> +  }
> +
> +  public final static class IntHolder implements ValueHolder{
> +    public static final MajorType TYPE = Types.required(MinorType.INT);
> +    public int value;
> +  }
> +
> +  public final static class NullableIntHolder implements ValueHolder {
> +    public static final MajorType TYPE = Types.optional(MinorType.INT);
> +    public int value;
> +    public int isSet;
> +  }
> +
> +  public final static class LongHolder implements ValueHolder {
> +    public static final MajorType TYPE = Types.required(MinorType.BIGINT);
> +    public long value;
> +  }
> +
> +  public final static class NullableLongHolder implements ValueHolder {
> +    public static final MajorType TYPE = Types.optional(MinorType.BIGINT);
> +    public long value;
> +    public int isSet;
> +  }
> +
> +}
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/holders/VarBinaryHolder.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/holders/VarBinaryHolder.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/holders/VarBinaryHolder.java
> new file mode 100644
> index 0000000..554c7d3
> --- /dev/null
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/holders/VarBinaryHolder.java
> @@ -0,0 +1,12 @@
> +package org.apache.drill.exec.expr.holders;
> +
> +import io.netty.buffer.ByteBuf;
> +
> +public class VarBinaryHolder {
> +  static final org.slf4j.Logger logger =
> org.slf4j.LoggerFactory.getLogger(VarBinaryHolder.class);
> +
> +  public ByteBuf buffer;
> +  public int start;
> +  public int length;
> +
> +}
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/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 33707a0..1dd7de8 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
> @@ -17,7 +17,15 @@
>
> ******************************************************************************/
>  package org.apache.drill.exec.ops;
>
> +import java.io.IOException;
> +
>  import org.apache.drill.common.expression.LogicalExpression;
> +import org.apache.drill.exec.compile.ClassTransformer;
> +import org.apache.drill.exec.compile.QueryClassLoader;
> +import org.apache.drill.exec.compile.TemplateClassDefinition;
> +import org.apache.drill.exec.exception.ClassTransformationException;
> +import org.apache.drill.exec.expr.CodeGenerator;
> +import org.apache.drill.exec.expr.fn.FunctionImplementationRegistry;
>  import org.apache.drill.exec.memory.BufferAllocator;
>  import org.apache.drill.exec.metrics.SingleThreadNestedCounter;
>  import
> org.apache.drill.exec.physical.impl.FilteringRecordBatchTransformer;
> @@ -54,8 +62,13 @@ public class FragmentContext {
>    private final IncomingBuffers buffers;
>    private volatile Throwable failureCause;
>    private volatile boolean failed = false;
> +  private final FunctionImplementationRegistry funcRegistry;
> +  private final QueryClassLoader loader;
> +  private final ClassTransformer transformer;
>
> -  public FragmentContext(DrillbitContext dbContext, FragmentHandle
> handle, UserClientConnection connection, IncomingBuffers buffers) {
> +  public FragmentContext(DrillbitContext dbContext, FragmentHandle
> handle, UserClientConnection connection, IncomingBuffers buffers,
> FunctionImplementationRegistry funcRegistry) {
> +    this.loader = new QueryClassLoader(true);
> +    this.transformer = new ClassTransformer();
>      this.fragmentTime =
> dbContext.getMetrics().timer(METRIC_TIMER_FRAGMENT_TIME);
>      this.batchesCompleted = new SingleThreadNestedCounter(dbContext,
> METRIC_BATCHES_COMPLETED);
>      this.recordsCompleted = new SingleThreadNestedCounter(dbContext,
> METRIC_RECORDS_COMPLETED);
> @@ -64,6 +77,7 @@ public class FragmentContext {
>      this.connection = connection;
>      this.handle = handle;
>      this.buffers = buffers;
> +    this.funcRegistry = funcRegistry;
>    }
>
>    public void fail(Throwable cause) {
> @@ -89,6 +103,10 @@ public class FragmentContext {
>      return context.getAllocator();
>    }
>
> +  public <T> T getImplementationClass(TemplateClassDefinition<T, Void>
> templateDefinition, CodeGenerator cg) throws ClassTransformationException,
> IOException{
> +    return transformer.getImplementationClass(this.loader,
> templateDefinition, cg.generate(), null);
> +  }
> +
>    public FilteringRecordBatchTransformer
> getFilteringExpression(LogicalExpression expr){
>      return null;
>    }
> @@ -119,5 +137,11 @@ public class FragmentContext {
>      return failed;
>    }
>
> +  public FunctionImplementationRegistry getFunctionRegistry(){
> +    return funcRegistry;
> +  }
>
> +  public QueryClassLoader getClassLoader(){
> +    return loader;
> +  }
>  }
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/opt/BasicOptimizer.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/opt/BasicOptimizer.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/opt/BasicOptimizer.java
> index 98ef05f..dcaf823 100644
> ---
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/opt/BasicOptimizer.java
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/opt/BasicOptimizer.java
> @@ -1,28 +1,28 @@
>  package org.apache.drill.exec.opt;
>
> -import com.fasterxml.jackson.core.type.TypeReference;
> -import com.fasterxml.jackson.databind.ObjectMapper;
> -import org.apache.drill.common.JSONOptions;
> +import java.io.IOException;
> +import java.util.ArrayList;
> +import java.util.Collection;
> +import java.util.List;
> +
>  import org.apache.drill.common.PlanProperties;
>  import org.apache.drill.common.config.DrillConfig;
>  import org.apache.drill.common.logical.LogicalPlan;
> -import org.apache.drill.common.logical.data.*;
> +import org.apache.drill.common.logical.data.Project;
> +import org.apache.drill.common.logical.data.Scan;
> +import org.apache.drill.common.logical.data.SinkOperator;
> +import org.apache.drill.common.logical.data.Store;
>  import
> org.apache.drill.common.logical.data.visitors.AbstractLogicalVisitor;
> -import org.apache.drill.common.logical.data.visitors.LogicalVisitor;
> +import org.apache.drill.common.types.TypeProtos.DataMode;
> +import org.apache.drill.common.types.TypeProtos.MinorType;
>  import org.apache.drill.exec.exception.OptimizerException;
>  import org.apache.drill.exec.ops.QueryContext;
>  import org.apache.drill.exec.physical.PhysicalPlan;
>  import org.apache.drill.exec.physical.base.PhysicalOperator;
>  import org.apache.drill.exec.physical.config.MockScanPOP;
> -import org.apache.drill.exec.physical.config.MockStorePOP;
>  import org.apache.drill.exec.physical.config.Screen;
> -import org.apache.drill.exec.proto.SchemaDefProtos;
> -import org.apache.drill.exec.server.DrillbitContext;
>
> -import java.io.IOException;
> -import java.util.ArrayList;
> -import java.util.Collection;
> -import java.util.List;
> +import com.fasterxml.jackson.core.type.TypeReference;
>
>  /**
>   * Created with IntelliJ IDEA.
> @@ -99,8 +99,8 @@ public class BasicOptimizer extends Optimizer{
>                  }
>                  else{
>                      myObjects = new ArrayList<>();
> -                    MockScanPOP.MockColumn[] cols = { new
> MockScanPOP.MockColumn("blah", SchemaDefProtos.MinorType.INT,
> SchemaDefProtos.DataMode.REQUIRED,4,4,4),
> -                            new MockScanPOP.MockColumn("blah_2",
> SchemaDefProtos.MinorType.INT, SchemaDefProtos.DataMode.REQUIRED,4,4,4)};
> +                    MockScanPOP.MockColumn[] cols = { new
> MockScanPOP.MockColumn("blah", MinorType.INT, DataMode.REQUIRED,4,4,4),
> +                            new MockScanPOP.MockColumn("blah_2",
> MinorType.INT, DataMode.REQUIRED,4,4,4)};
>                      myObjects.add(new MockScanPOP.MockScanEntry(50,
> cols));
>                  }
>              } catch (IOException e) {
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/RecordField.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/RecordField.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/RecordField.java
> index db3390a..4e9eb4c 100644
> ---
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/RecordField.java
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/RecordField.java
> @@ -17,7 +17,7 @@
>
> ******************************************************************************/
>  package org.apache.drill.exec.physical;
>
> -import org.apache.drill.common.expression.types.DataType;
> +import org.apache.drill.common.types.TypeProtos.MajorType;
>
>  import com.fasterxml.jackson.annotation.JsonCreator;
>  import com.fasterxml.jackson.annotation.JsonProperty;
> @@ -26,17 +26,17 @@ public class RecordField {
>    static final org.slf4j.Logger logger =
> org.slf4j.LoggerFactory.getLogger(RecordField.class);
>
>
> -  private DataType type;
> +  private MajorType type;
>    private ValueMode mode;
>
>    @JsonCreator
> -  public RecordField(@JsonProperty("type") DataType type,
> @JsonProperty("mode") ValueMode mode) {
> +  public RecordField(@JsonProperty("type") MajorType type,
> @JsonProperty("mode") ValueMode mode) {
>      super();
>      this.type = type;
>      this.mode = mode;
>    }
>
> -  public DataType getType() {
> +  public MajorType getType() {
>      return type;
>    }
>
>
>

[42/53] [abbrv] Types transition

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/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
new file mode 100644
index 0000000..bc44490
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/ExpressionTreeMaterializer.java
@@ -0,0 +1,128 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+
+package org.apache.drill.exec.expr;
+
+import java.util.List;
+
+import org.apache.drill.common.expression.ErrorCollector;
+import org.apache.drill.common.expression.ExpressionValidator;
+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;
+import org.apache.drill.common.expression.visitors.SimpleExprVisitor;
+import org.apache.drill.exec.record.NullExpression;
+import org.apache.drill.exec.record.RecordBatch;
+import org.apache.drill.exec.record.RecordBatch.TypedFieldId;
+
+import com.google.common.collect.Lists;
+
+public class ExpressionTreeMaterializer {
+
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ExpressionTreeMaterializer.class);
+  
+  
+  
+  private ExpressionTreeMaterializer() {
+  };
+
+  public static LogicalExpression materialize(LogicalExpression expr, RecordBatch batch, ErrorCollector errorCollector) {
+    return expr.accept(new MaterializeVisitor(batch, errorCollector), null);
+  }
+
+  private static class MaterializeVisitor extends SimpleExprVisitor<LogicalExpression> {
+    private final ErrorCollector errorCollector;
+    private final RecordBatch batch;
+    private ExpressionValidator validator = new ExpressionValidator();
+
+    public MaterializeVisitor(RecordBatch batch, ErrorCollector errorCollector) {
+      this.batch = batch;
+      this.errorCollector = errorCollector;
+    }
+
+    private LogicalExpression validateNewExpr(LogicalExpression newExpr) {
+      newExpr.accept(validator, errorCollector);
+      return newExpr;
+    }
+
+    @Override
+    public LogicalExpression visitUnknown(LogicalExpression e, Void value) throws RuntimeException {
+      throw new UnsupportedOperationException(String.format("Expression tree materializer does not currently support materializing nodes of type %s.", e.getClass().getCanonicalName()));
+    }
+
+    @Override
+    public LogicalExpression visitFunctionCall(FunctionCall call) {
+      List<LogicalExpression> args = Lists.newArrayList();
+      for (int i = 0; i < call.args.size(); ++i) {
+        LogicalExpression newExpr = call.args.get(i).accept(this, null);
+        args.add(newExpr);
+      }
+
+      return validateNewExpr(new FunctionCall(call.getDefinition(), args, call.getPosition()));
+    }
+
+    @Override
+    public LogicalExpression visitIfExpression(IfExpression ifExpr) {
+      List<IfExpression.IfCondition> conditions = Lists.newArrayList(ifExpr.iterator());
+      LogicalExpression newElseExpr = ifExpr.elseExpression.accept(this, null);
+
+      for (int i = 0; i < conditions.size(); ++i) {
+        IfExpression.IfCondition condition = conditions.get(i);
+
+        LogicalExpression newCondition = condition.condition.accept(this, null);
+        LogicalExpression newExpr = condition.expression.accept(this, null);
+        conditions.set(i, new IfExpression.IfCondition(newCondition, newExpr));
+      }
+
+      return validateNewExpr(IfExpression.newBuilder().setElse(newElseExpr).addConditions(conditions).build());
+    }
+
+    @Override
+    public LogicalExpression visitSchemaPath(SchemaPath path) {
+      logger.debug("Visiting schema path {}", path);
+      TypedFieldId tfId = batch.getValueVector(path);
+      if (tfId == null) {
+        return NullExpression.INSTANCE;
+      } else {
+        return new ValueVectorReadExpression(tfId);
+      }
+    }
+
+    @Override
+    public LogicalExpression visitLongConstant(ValueExpressions.LongExpression intExpr) {
+      return intExpr;
+    }
+
+    @Override
+    public LogicalExpression visitDoubleConstant(ValueExpressions.DoubleExpression dExpr) {
+      return dExpr;
+    }
+
+    @Override
+    public LogicalExpression visitBooleanConstant(ValueExpressions.BooleanExpression e) {
+      return e;
+    }
+
+    @Override
+    public LogicalExpression visitQuotedStringConstant(ValueExpressions.QuotedString e) {
+      return e;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/SingleClassStringWriter.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/SingleClassStringWriter.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/SingleClassStringWriter.java
new file mode 100644
index 0000000..813a32f
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/SingleClassStringWriter.java
@@ -0,0 +1,40 @@
+package org.apache.drill.exec.expr;
+
+import java.io.IOException;
+import java.io.OutputStream;
+import java.io.StringWriter;
+import java.io.Writer;
+
+import com.google.common.base.Preconditions;
+import com.sun.codemodel.CodeWriter;
+import com.sun.codemodel.JPackage;
+
+public class SingleClassStringWriter extends CodeWriter{
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(SingleClassStringWriter.class);
+
+  private boolean used;
+  private StringWriter writer = new StringWriter();
+  
+  @Override
+  public OutputStream openBinary(JPackage pkg, String fileName) throws IOException {
+    throw new UnsupportedOperationException();
+  }
+
+  
+  @Override
+  public Writer openSource(JPackage pkg, String fileName) throws IOException {
+    Preconditions.checkArgument(!used, "The SingleClassStringWriter can only output once src file.");
+    used = true;
+    return writer;
+  }
+
+  @Override
+  public void close() throws IOException {
+  }
+
+  public StringBuffer getCode(){
+    return writer.getBuffer();
+  }
+  
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/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
new file mode 100644
index 0000000..c8d906d
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/ValueVectorReadExpression.java
@@ -0,0 +1,41 @@
+package org.apache.drill.exec.expr;
+
+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.RecordBatch.TypedFieldId;
+import org.apache.drill.exec.record.vector.ValueVector;
+
+public class ValueVectorReadExpression implements LogicalExpression{
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ValueVectorReadExpression.class);
+
+  private final MajorType type;
+  private final int fieldId;
+  
+  public ValueVectorReadExpression(TypedFieldId tfId) {
+    this.type = tfId.getType();
+    this.fieldId = tfId.getFieldId();
+  }
+
+  @Override
+  public MajorType getMajorType() {
+    return type;
+  }
+
+  @Override
+  public <T, V, E extends Exception> T accept(ExprVisitor<T, V, E> visitor, V value) throws E {
+    return visitor.visitUnknown(this, value);
+  }
+
+  public int getFieldId() {
+    return fieldId;
+  }
+
+  @Override
+  public ExpressionPosition getPosition() {
+    return ExpressionPosition.UNKNOWN;
+  }
+  
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/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
new file mode 100644
index 0000000..96d9d05
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/ValueVectorWriteExpression.java
@@ -0,0 +1,45 @@
+package org.apache.drill.exec.expr;
+
+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.common.types.Types;
+import org.apache.drill.exec.record.vector.ValueVector;
+
+public class ValueVectorWriteExpression implements LogicalExpression {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ValueVectorWriteExpression.class);
+
+  private final int fieldId;
+  private final LogicalExpression child;
+  
+  public ValueVectorWriteExpression(int fieldId, LogicalExpression child){
+    this.fieldId = fieldId;
+    this.child = child;
+  }
+  
+  public int getFieldId() {
+    return fieldId;
+  }
+
+  @Override
+  public MajorType getMajorType() {
+    return Types.NULL;
+  }
+
+  @Override
+  public <T, V, E extends Exception> T accept(ExprVisitor<T, V, E> visitor, V value) throws E {
+    return visitor.visitUnknown(this, value);
+  }
+
+  @Override
+  public ExpressionPosition getPosition() {
+    return ExpressionPosition.UNKNOWN;
+  }
+
+  public LogicalExpression getChild() {
+    return child;
+  }
+  
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/annotations/FunctionTemplate.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/annotations/FunctionTemplate.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/annotations/FunctionTemplate.java
new file mode 100644
index 0000000..9e32750
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/annotations/FunctionTemplate.java
@@ -0,0 +1,25 @@
+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;
+
+
+@Retention(RetentionPolicy.RUNTIME)
+@Target({ElementType.TYPE})
+public @interface FunctionTemplate {
+  
+  String name();
+  FunctionScope scope();
+  NullHandling nulls() default NullHandling.INTERNAL;
+  boolean isBinaryCommutative() default false;
+  
+  public static enum NullHandling {
+    INTERNAL, NULL_IF_NULL;
+  }
+  
+  public static enum FunctionScope{
+    SIMPLE, AGGREGATE, RUNNING;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/annotations/Output.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/annotations/Output.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/annotations/Output.java
new file mode 100644
index 0000000..2d77dfc
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/annotations/Output.java
@@ -0,0 +1,18 @@
+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;
+
+/**
+ * Describes the field will provide output from the given function.
+ */
+@Retention(RetentionPolicy.RUNTIME)
+@Target({ElementType.FIELD})
+public @interface Output {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(Output.class);
+
+  
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/annotations/Param.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/annotations/Param.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/annotations/Param.java
new file mode 100644
index 0000000..9ba7611
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/annotations/Param.java
@@ -0,0 +1,18 @@
+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.FIELD})
+public @interface Param {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(Param.class);
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/annotations/WorkSpace.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/annotations/WorkSpace.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/annotations/WorkSpace.java
new file mode 100644
index 0000000..cc106a7
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/annotations/WorkSpace.java
@@ -0,0 +1,14 @@
+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;
+
+/**
+ * Describes the field will provide output from the given function.
+ */
+@Retention(RetentionPolicy.RUNTIME)
+@Target({ElementType.FIELD})
+public @interface WorkSpace {
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/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
new file mode 100644
index 0000000..7276e7d
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/FunctionConverter.java
@@ -0,0 +1,186 @@
+package org.apache.drill.exec.expr.fn;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.StringReader;
+import java.lang.reflect.Field;
+import java.net.URL;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.drill.common.types.TypeProtos.MajorType;
+import org.apache.drill.exec.expr.DrillFunc;
+import org.apache.drill.exec.expr.annotations.FunctionTemplate;
+import org.apache.drill.exec.expr.annotations.Output;
+import org.apache.drill.exec.expr.annotations.Param;
+import org.apache.drill.exec.expr.annotations.WorkSpace;
+import org.apache.drill.exec.expr.fn.FunctionHolder.ValueReference;
+import org.apache.drill.exec.expr.holders.ValueHolder;
+import org.codehaus.commons.compiler.CompileException;
+import org.codehaus.janino.Java;
+import org.codehaus.janino.Java.CompilationUnit;
+import org.codehaus.janino.Parser;
+import org.codehaus.janino.Scanner;
+import org.mortbay.util.IO;
+
+import com.beust.jcommander.internal.Lists;
+import com.google.common.io.InputSupplier;
+import com.google.common.io.Resources;
+
+/**
+ * Converts FunctionCalls to Java Expressions.
+ */
+public class FunctionConverter {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(FunctionConverter.class);
+  
+  
+  public <T extends DrillFunc> FunctionHolder getHolder(Class<T> clazz){
+    FunctionTemplate template = clazz.getAnnotation(FunctionTemplate.class);
+    if(template == null){
+      return failure("Class does not declare FunctionTemplate annotation.", clazz);
+    }
+    
+    // start by getting field information.
+    List<ValueReference> params = Lists.newArrayList();
+    List<String> workspaceFields = Lists.newArrayList();
+    ValueReference outputField = null;
+    
+    
+    for(Field field : clazz.getDeclaredFields()){
+
+      Param param = field.getAnnotation(Param.class);
+      Output output = field.getAnnotation(Output.class);
+      WorkSpace workspace = field.getAnnotation(WorkSpace.class);
+      
+      int i =0;
+      if(param != null) i++;
+      if(output != null) i++;
+      if(workspace != null) i++;
+      if(i == 0){
+        return failure("The field must be either a @Param, @Output or @WorkSpace field.", clazz, field);
+      }else if(i > 1){
+        return failure("The field must be only one of @Param, @Output or @WorkSpace.  It currently has more than one of these annotations.", clazz, field);
+      }
+
+      
+      
+      if(param != null || output != null){
+        
+        // check that param and output are value holders.
+        if(!ValueHolder.class.isAssignableFrom(field.getType())){
+          return failure(String.format("The field doesn't holds value of type %s which does not implement the ValueHolder interface.  All fields of type @Param or @Output must extend this interface..", field.getType()), clazz, field);
+        }
+        
+        // get the type field from the value holder.
+        MajorType type = null;
+        try{
+          type = getStaticFieldValue("TYPE", field.getType(), MajorType.class);
+        }catch(Exception e){
+          return failure("Failure while trying to access the ValueHolder's TYPE static variable.  All ValueHolders must contain a static TYPE variable that defines their MajorType.", e, clazz, field.getName());
+        }
+        
+        
+        ValueReference p = new ValueReference(type, field.getName());
+        if(param != null){
+          params.add(p);
+        }else{ 
+          if(outputField != null){
+            return failure("You've declared more than one @Output field.  You must declare one and only @Output field per Function class.", clazz, field);
+          }else{
+            outputField = p; 
+            
+          }
+           
+        }
+        
+      }else{
+        // workspace work.
+        workspaceFields.add(field.getName());
+      }
+      
+    }
+    
+    
+    if(!workspaceFields.isEmpty()) return failure("This function declares one or more workspace fields.  However, those have not yet been implemented.", clazz);
+    if(outputField == null)  return failure("This function declares zero output fields.  A function must declare one output field.", clazz);
+    
+    // get function body.     
+   
+    CompilationUnit cu;
+    try {
+      cu = getClassBody(clazz);
+    } catch (CompileException | IOException e) {
+      return failure("Failure while getting class body.", e, clazz);
+    }
+    
+    Map<String, String> methods = MethodGrabbingVisitor.getMethods(cu, clazz);
+
+    // return holder
+    ValueReference[] ps = params.toArray(new ValueReference[params.size()]);
+    FunctionHolder fh = new FunctionHolder(template.scope(), template.nulls(), template.isBinaryCommutative(), template.name(), ps, outputField, methods);
+    return fh;
+  }
+  
+  
+  
+  private Java.CompilationUnit getClassBody(Class<?> c) throws CompileException, IOException{
+    String path = c.getName();
+    path = path.replaceFirst("\\$.*", "");
+    path = path.replace(".", File.separator);
+    path = "/" + path + ".java";
+    URL u = Resources.getResource(FunctionConverter.class, path);
+    InputSupplier<InputStream> supplier = Resources.newInputStreamSupplier(u);
+    try(InputStream is = supplier.getInput()){
+      if(is == null){
+        throw new IOException(String.format("Failure trying to located source code for Class %s, tried to read on classpath location %s", c.getName(), path));
+      }
+      String body = IO.toString(is);
+      
+      //TODO: Hack to remove annotations so Janino doesn't choke.  Need to reconsider this problem...
+      body = body.replaceAll("@(?:Output|Param|FunctionTemplate\\([^\\\\]*?\\))", "");
+      return new Parser(new Scanner(null, new StringReader(body))).parseCompilationUnit();
+    }
+    
+  }
+  
+  @SuppressWarnings("unchecked")
+  private <T> T getStaticFieldValue(String fieldName, Class<?> valueType, Class<T> c) throws NoSuchFieldException, SecurityException, IllegalArgumentException, IllegalAccessException{
+      Field f = valueType.getDeclaredField(fieldName);
+      Object val = f.get(null);
+      return (T) val;
+  }
+  
+  private static FunctionHolder failure(String message, Throwable t, Class<?> clazz, String fieldName){
+    logger.warn("Failure loading function class {}, field {}. " + message, clazz.getName(), fieldName, t);
+    return null;
+  }  
+  
+  private FunctionHolder failure(String message, Class<?> clazz, String fieldName){
+    logger.warn("Failure loading function class {}, field {}. " + message, clazz.getName(), fieldName);
+    return null;
+  }
+
+  private FunctionHolder failure(String message, Class<?> clazz){
+    logger.warn("Failure loading function class {}. " + message, clazz.getName());
+    return null;
+  }
+
+  private FunctionHolder failure(String message, Throwable t, Class<?> clazz){
+    logger.warn("Failure loading function class {}. " + message, t, clazz.getName());
+    return null;
+  }
+  
+  private FunctionHolder failure(String message, Class<?> clazz, Field field){
+    return failure(message, clazz, field.getName());
+  }
+  
+  public static void main(String[] args) throws Exception{
+    
+    URL u = Resources.getResource(FunctionConverter.class, "/org/apache/drill/exec/expr/fn/impl/MathFunctions.java");
+    InputStream is = Resources.newInputStreamSupplier(u).getInput();
+    String s = IO.toString(is);
+    System.out.println(s);
+  }
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/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
new file mode 100644
index 0000000..460169f
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/FunctionHolder.java
@@ -0,0 +1,158 @@
+package org.apache.drill.exec.expr.fn;
+
+import java.util.Arrays;
+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;
+import org.apache.drill.exec.expr.CodeGenerator.HoldingContainer;
+import org.apache.drill.exec.expr.annotations.FunctionTemplate;
+import org.apache.drill.exec.expr.annotations.FunctionTemplate.FunctionScope;
+import org.apache.drill.exec.expr.annotations.FunctionTemplate.NullHandling;
+
+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 {
+
+  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 ValueReference[] parameters;
+  private ValueReference returnValue;
+  
+  public FunctionHolder(FunctionScope scope, NullHandling nullHandling, boolean isBinaryCommutative, String functionName, ValueReference[] parameters, ValueReference returnValue, Map<String, String> methods) {
+    super();
+    this.scope = scope;
+    this.nullHandling = nullHandling;
+    this.isBinaryCommutative = isBinaryCommutative;
+    this.functionName = functionName;
+    this.setupBody = methods.get("setup");
+    this.addBody = methods.get("add");
+    this.evalBody = methods.get("eval");
+    this.parameters = parameters;
+    this.returnValue = returnValue;
+  }
+
+  public 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);
+    
+    
+    
+    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, false);
+        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(sub);
+    
+    // locally name external blocks.
+    
+    // internal out value.
+    JVar internalOutput = sub.decl(JMod.FINAL, g.getHolderType(returnValue.type), returnValue.name, JExpr._new(g.getHolderType(returnValue.type)));
+    
+    for(int i =0; i < inputVariables.length; i++){
+      
+      ValueReference parameter = parameters[i];
+      HoldingContainer inputVariable = inputVariables[i];
+      sub.decl(JMod.FINAL, inputVariable.getHolder().type(), parameter.name, inputVariable.getHolder());  
+    }
+    
+    // add function body.
+    sub.directStatement(evalBody);
+    
+    sub.assign(out.getHolder(), internalOutput);
+
+    g.getBlock().directStatement(String.format("//---- end of eval portion of %s function. ----//\n", functionName));
+    return out;
+  }
+  
+  public boolean matches(FunctionCall call){
+    if(!softCompare(call.getMajorType(), returnValue.type)) return false;
+    if(call.args.size() != parameters.length) return false;
+    for(int i =0; i < parameters.length; i++){
+      ValueReference param = parameters[i];
+      LogicalExpression arg = call.args.get(i);
+      if(!softCompare(param.type, arg.getMajorType())) return false;
+    }
+    
+    return true;
+  }
+  
+  private boolean softCompare(MajorType a, MajorType b){
+    return Types.softEquals(a, b, nullHandling == NullHandling.NULL_IF_NULL);
+  }
+  
+  public String getFunctionName() {
+    return functionName;
+  }
+
+  public static class ValueReference{
+    MajorType type;
+    String name;
+    public ValueReference(MajorType type, String name) {
+      super();
+      this.type = type;
+      this.name = name;
+    }
+    @Override
+    public String toString() {
+      return "ValueReference [type=" + type + ", name=" + name + "]";
+    }
+    
+    
+  }
+
+  @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/ce0da88d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/FunctionImplementationRegistry.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/FunctionImplementationRegistry.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/FunctionImplementationRegistry.java
new file mode 100644
index 0000000..5130f2b
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/FunctionImplementationRegistry.java
@@ -0,0 +1,43 @@
+package org.apache.drill.exec.expr.fn;
+
+import java.util.Set;
+
+import org.apache.drill.common.config.DrillConfig;
+import org.apache.drill.common.expression.FunctionCall;
+import org.apache.drill.common.util.PathScanner;
+import org.apache.drill.exec.ExecConstants;
+import org.apache.drill.exec.expr.DrillFunc;
+
+import com.google.common.collect.ArrayListMultimap;
+
+public class FunctionImplementationRegistry {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(FunctionImplementationRegistry.class);
+  
+  private ArrayListMultimap<String, FunctionHolder> methods = ArrayListMultimap.create();
+  
+  public FunctionImplementationRegistry(DrillConfig config){
+    FunctionConverter converter = new FunctionConverter();
+    Set<Class<? extends DrillFunc>> providerClasses = PathScanner.scanForImplementations(DrillFunc.class, config.getStringList(ExecConstants.FUNCTION_PACKAGES));
+    for (Class<? extends DrillFunc> clazz : providerClasses) {
+      FunctionHolder holder = converter.getHolder(clazz);
+      if(holder != null){
+        methods.put(holder.getFunctionName(), holder);
+        logger.debug("Registering function {}", holder);
+      }else{
+        logger.debug("Unable to initialize function for class {}", clazz.getName());
+      }
+    }
+  }
+  
+  public FunctionHolder getFunction(FunctionCall call){
+    for(FunctionHolder h : methods.get(call.getDefinition().getName())){
+      if(h.matches(call)){
+        return h;
+      }
+    }
+    throw new UnsupportedOperationException(String.format("Unable to find matching function implementation for call %s.", call));
+  }
+
+  
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/MethodGrabbingVisitor.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/MethodGrabbingVisitor.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/MethodGrabbingVisitor.java
new file mode 100644
index 0000000..22b9046
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/MethodGrabbingVisitor.java
@@ -0,0 +1,52 @@
+package org.apache.drill.exec.expr.fn;
+
+import java.io.StringWriter;
+import java.util.Map;
+
+import org.codehaus.janino.Java;
+import org.codehaus.janino.Java.ClassDeclaration;
+import org.codehaus.janino.Java.MethodDeclarator;
+import org.codehaus.janino.util.Traverser;
+
+import com.beust.jcommander.internal.Maps;
+
+public class MethodGrabbingVisitor{
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(MethodGrabbingVisitor.class);
+  
+  private Class<?> c;
+  private Map<String, String> methods = Maps.newHashMap();
+  private ClassFinder classFinder = new ClassFinder();
+  private boolean captureMethods = false;
+  
+  private MethodGrabbingVisitor(Class<?> c) {
+    super();
+    this.c = c;
+  }
+  
+  public class ClassFinder extends Traverser{
+
+    @Override
+    public void traverseClassDeclaration(ClassDeclaration cd) {
+      captureMethods = c.getName().equals(cd.getClassName());
+      super.traverseClassDeclaration(cd);
+    }
+
+    @Override
+    public void traverseMethodDeclarator(MethodDeclarator md) {
+      if(captureMethods){
+        StringWriter writer = new StringWriter();
+        ModifiedUnparseVisitor v = new ModifiedUnparseVisitor(writer);
+        md.accept(v);
+        methods.put(md.name, writer.getBuffer().toString());  
+      }
+    } 
+  }
+
+  
+  public static Map<String, String> getMethods(Java.CompilationUnit cu, Class<?> c){
+    MethodGrabbingVisitor visitor = new MethodGrabbingVisitor(c);
+    cu.getPackageMemberTypeDeclarations()[0].accept(visitor.classFinder.comprehensiveVisitor());
+    return visitor.methods;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/ModifiedUnparseVisitor.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/ModifiedUnparseVisitor.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/ModifiedUnparseVisitor.java
new file mode 100644
index 0000000..72b2008
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/ModifiedUnparseVisitor.java
@@ -0,0 +1,891 @@
+package org.apache.drill.exec.expr.fn;
+
+
+/*
+ * 
+ * Modified so that we can avoid printing some things.
+ * 
+ * Janino - An embedded Java[TM] compiler
+ *
+ * Copyright (c) 2001-2010, Arno Unkrig
+ * All rights reserved.
+ *
+ * Redistribution and use in source and binary forms, with or without modification, are permitted provided that the
+ * following conditions are met:
+ *
+ *    1. Redistributions of source code must retain the above copyright notice, this list of conditions and the
+ *       following disclaimer.
+ *    2. Redistributions in binary form must reproduce the above copyright notice, this list of conditions and the
+ *       following disclaimer in the documentation and/or other materials provided with the distribution.
+ *    3. The name of the author may not be used to endorse or promote products derived from this software without
+ *       specific prior written permission.
+ *
+ * THIS SOFTWARE IS PROVIDED BY THE AUTHOR ``AS IS'' AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED
+ * TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL
+ * THE AUTHOR BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING,
+ * BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS
+ * INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+ * (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE
+ * POSSIBILITY OF SUCH DAMAGE.
+ */
+
+
+import java.io.BufferedReader;
+import java.io.BufferedWriter;
+import java.io.FileReader;
+import java.io.IOException;
+import java.io.OutputStreamWriter;
+import java.io.PrintWriter;
+import java.io.StringReader;
+import java.io.Writer;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.codehaus.janino.JaninoRuntimeException;
+import org.codehaus.janino.Java;
+import org.codehaus.janino.Mod;
+import org.codehaus.janino.Parser;
+import org.codehaus.janino.Scanner;
+import org.codehaus.janino.Visitor.ComprehensiveVisitor;
+import org.codehaus.janino.util.AutoIndentWriter;
+
+/**
+ * A visitor that unparses (un-compiles) an AST to a {@link Writer}. See
+ * {@link #main(String[])} for a usage example.
+ */
+public class ModifiedUnparseVisitor implements ComprehensiveVisitor {
+    protected final AutoIndentWriter aiw;
+    protected final PrintWriter      pw;
+
+    /**
+     * Testing of parsing/unparsing.
+     * <p>
+     * Reads compilation units from the files named on the command line
+     * and unparses them to {@link System#out}.
+     */
+    public static void main(String[] args) throws Exception {
+        Writer w = new BufferedWriter(new OutputStreamWriter(System.out));
+        for (int i = 0; i < args.length; ++i) {
+            String fileName = args[i];
+
+            // Parse each compilation unit.
+            FileReader r = new FileReader(fileName);
+            Java.CompilationUnit cu;
+            try {
+                cu = new Parser(new Scanner(fileName, r)).parseCompilationUnit();
+            } finally {
+                r.close();
+            }
+
+            // Unparse each compilation unit.
+            ModifiedUnparseVisitor.unparse(cu, w);
+        }
+        w.flush();
+    }
+
+    /**
+     * Unparse the given {@link Java.CompilationUnit} to the given {@link Writer}.
+     */
+    public static void unparse(Java.CompilationUnit cu, Writer w) {
+        ModifiedUnparseVisitor uv = new ModifiedUnparseVisitor(w);
+        uv.unparseCompilationUnit(cu);
+        uv.close();
+    }
+
+    public ModifiedUnparseVisitor(Writer w) {
+        this.aiw = new AutoIndentWriter(w);
+        this.pw = new PrintWriter(this.aiw, true);
+    }
+
+    /**
+     * Flushes all generated code and closes the {@link Writer} that was passed
+     * to {@link #ModifiedUnparseVisitor(Writer)}.
+     */
+    public void close() {
+        this.pw.close();
+    }
+
+    public void unparseCompilationUnit(Java.CompilationUnit cu) {
+        if (cu.optionalPackageDeclaration != null) {
+            this.pw.println();
+            this.pw.println("package " + cu.optionalPackageDeclaration.packageName + ';');
+        }
+        if (!cu.importDeclarations.isEmpty()) {
+            this.pw.println();
+            for (Iterator it = cu.importDeclarations.iterator(); it.hasNext();) {
+                ((Java.CompilationUnit.ImportDeclaration) it.next()).accept(this);
+            }
+        }
+        for (Iterator it = cu.packageMemberTypeDeclarations.iterator(); it.hasNext();) {
+            this.pw.println();
+            this.unparseTypeDeclaration((Java.PackageMemberTypeDeclaration) it.next());
+            this.pw.println();
+        }
+    }
+
+    public void visitSingleTypeImportDeclaration(Java.CompilationUnit.SingleTypeImportDeclaration stid) {
+        this.pw.println("import " + Java.join(stid.identifiers, ".") + ';');
+    }
+    public void visitTypeImportOnDemandDeclaration(Java.CompilationUnit.TypeImportOnDemandDeclaration tiodd) {
+        this.pw.println("import " + Java.join(tiodd.identifiers, ".") + ".*;");
+    }
+    public void visitSingleStaticImportDeclaration(Java.CompilationUnit.SingleStaticImportDeclaration ssid) {
+        this.pw.println("import static " + Java.join(ssid.identifiers, ".") + ';');
+    }
+    public void visitStaticImportOnDemandDeclaration(Java.CompilationUnit.StaticImportOnDemandDeclaration siodd) {
+        this.pw.println("import static " + Java.join(siodd.identifiers, ".") + ".*;");
+    }
+
+    public void visitLocalClassDeclaration(Java.LocalClassDeclaration lcd) {
+        this.unparseNamedClassDeclaration(lcd);
+    }
+    public void visitMemberClassDeclaration(Java.MemberClassDeclaration mcd) {
+        this.unparseNamedClassDeclaration(mcd);
+    }
+    public void visitMemberInterfaceDeclaration(Java.MemberInterfaceDeclaration mid) {
+        this.unparseInterfaceDeclaration(mid);
+    }
+    public void visitPackageMemberClassDeclaration(Java.PackageMemberClassDeclaration pmcd) {
+        this.unparseNamedClassDeclaration(pmcd);
+    }
+    public void visitPackageMemberInterfaceDeclaration(Java.PackageMemberInterfaceDeclaration pmid) {
+        this.unparseInterfaceDeclaration(pmid);
+    }
+    public void visitConstructorDeclarator(Java.ConstructorDeclarator cd) {
+        this.unparseDocComment(cd);
+        this.unparseModifiers(cd.modifiers);
+        Java.ClassDeclaration declaringClass = cd.getDeclaringClass();
+        this.pw.print(
+            declaringClass instanceof Java.NamedClassDeclaration
+            ? ((Java.NamedClassDeclaration) declaringClass).name
+            : "UNNAMED"
+        );
+        this.unparseFunctionDeclaratorRest(cd);
+        this.pw.print(' ');
+        if (cd.optionalConstructorInvocation != null) {
+            this.pw.println('{');
+            this.pw.print(AutoIndentWriter.INDENT);
+            this.unparseBlockStatement(cd.optionalConstructorInvocation);
+            this.pw.println(';');
+
+            if (!cd.optionalStatements.isEmpty()) {
+                this.pw.println();
+                this.unparseStatements(cd.optionalStatements);
+            }
+            this.pw.print(AutoIndentWriter.UNINDENT + "}");
+        } else
+        if (cd.optionalStatements.isEmpty()) {
+            this.pw.print("{}");
+        } else
+        {
+            this.pw.println('{');
+            this.pw.print(AutoIndentWriter.INDENT);
+            this.unparseStatements(cd.optionalStatements);
+            this.pw.print(AutoIndentWriter.UNINDENT + "}");
+        }
+    }
+    public void visitMethodDeclarator(Java.MethodDeclarator md) {
+        if (md.optionalStatements == null) {
+            this.pw.print(';');
+        } else
+        if (md.optionalStatements.isEmpty()) {
+            this.pw.print(" ");
+        } else
+        {
+            this.pw.print(AutoIndentWriter.INDENT);
+            this.unparseStatements(md.optionalStatements);
+            this.pw.print(AutoIndentWriter.UNINDENT);
+        }
+    }
+    public void visitFieldDeclaration(Java.FieldDeclaration fd) {
+        this.unparseDocComment(fd);
+        this.unparseModifiers(fd.modifiers);
+        this.unparseType(fd.type);
+        this.pw.print(' ');
+        for (int i = 0; i < fd.variableDeclarators.length; ++i) {
+            if (i > 0) this.pw.print(", ");
+            this.unparseVariableDeclarator(fd.variableDeclarators[i]);
+        }
+        this.pw.print(';');
+    }
+    public void visitInitializer(Java.Initializer i) {
+        if (i.statiC) this.pw.print("static ");
+        this.unparseBlockStatement(i.block);
+    }
+    public void visitBlock(Java.Block b) {
+        if (b.statements.isEmpty()) {
+            this.pw.print("{}");
+            return;
+        }
+        this.pw.println('{');
+        this.pw.print(AutoIndentWriter.INDENT);
+        this.unparseStatements(b.statements);
+        this.pw.print(AutoIndentWriter.UNINDENT + "}");
+    }
+
+    private void unparseStatements(List statements) {
+        int state = -1;
+        for (Iterator it = statements.iterator(); it.hasNext();) {
+            Java.BlockStatement bs = (Java.BlockStatement) it.next();
+            int x = (
+                bs instanceof Java.Block                             ? 1 :
+                bs instanceof Java.LocalClassDeclarationStatement    ? 2 :
+                bs instanceof Java.LocalVariableDeclarationStatement ? 3 :
+                bs instanceof Java.SynchronizedStatement             ? 4 :
+                99
+            );
+            if (state != -1 && state != x) this.pw.println(AutoIndentWriter.CLEAR_TABULATORS);
+            state = x;
+
+            this.unparseBlockStatement(bs);
+            this.pw.println();
+        }
+    }
+    public void visitBreakStatement(Java.BreakStatement bs) {
+        this.pw.print("break");
+        if (bs.optionalLabel != null) this.pw.print(' ' + bs.optionalLabel);
+        this.pw.print(';');
+    }
+    public void visitContinueStatement(Java.ContinueStatement cs) {
+        this.pw.print("continue");
+        if (cs.optionalLabel != null) this.pw.print(' ' + cs.optionalLabel);
+        this.pw.print(';');
+    }
+    public void visitDoStatement(Java.DoStatement ds) {
+        this.pw.print("do ");
+        this.unparseBlockStatement(ds.body);
+        this.pw.print("while (");
+        this.unparse(ds.condition);
+        this.pw.print(");");
+    }
+    public void visitEmptyStatement(Java.EmptyStatement es) {
+        this.pw.print(';');
+    }
+    public void visitExpressionStatement(Java.ExpressionStatement es) {
+        this.unparse(es.rvalue);
+        this.pw.print(';');
+    }
+    public void visitForStatement(Java.ForStatement fs) {
+        this.pw.print("for (");
+        if (fs.optionalInit != null) {
+            this.unparseBlockStatement(fs.optionalInit);
+        } else {
+            this.pw.print(';');
+        }
+        if (fs.optionalCondition != null) {
+            this.pw.print(' ');
+            this.unparse(fs.optionalCondition);
+        }
+        this.pw.print(';');
+        if (fs.optionalUpdate != null) {
+            this.pw.print(' ');
+            for (int i = 0; i < fs.optionalUpdate.length; ++i) {
+                if (i > 0) this.pw.print(", ");
+                this.unparse(fs.optionalUpdate[i]);
+            }
+        }
+        this.pw.print(") ");
+        this.unparseBlockStatement(fs.body);
+    }
+    public void visitIfStatement(Java.IfStatement is) {
+        this.pw.print("if (");
+        this.unparse(is.condition);
+        this.pw.print(") ");
+        this.unparseBlockStatement(is.thenStatement);
+        if (is.optionalElseStatement != null) {
+            this.pw.println(" else");
+            this.unparseBlockStatement(is.optionalElseStatement);
+        }
+    }
+    public void visitLabeledStatement(Java.LabeledStatement ls) {
+        this.pw.println(ls.label + ':');
+        this.unparseBlockStatement(ls.body);
+    }
+    public void visitLocalClassDeclarationStatement(Java.LocalClassDeclarationStatement lcds) {
+        this.unparseTypeDeclaration(lcds.lcd);
+    }
+    public void visitLocalVariableDeclarationStatement(Java.LocalVariableDeclarationStatement lvds) {
+        this.unparseModifiers(lvds.modifiers);
+        this.unparseType(lvds.type);
+        this.pw.print(' ');
+        this.pw.print(AutoIndentWriter.TABULATOR);
+        this.unparseVariableDeclarator(lvds.variableDeclarators[0]);
+        for (int i = 1; i < lvds.variableDeclarators.length; ++i) {
+            this.pw.print(", ");
+            this.unparseVariableDeclarator(lvds.variableDeclarators[i]);
+        }
+        this.pw.print(';');
+    }
+    public void visitReturnStatement(Java.ReturnStatement rs) {
+        this.pw.print("return");
+        if (rs.optionalReturnValue != null) {
+            this.pw.print(' ');
+            this.unparse(rs.optionalReturnValue);
+        }
+        this.pw.print(';');
+    }
+    public void visitSwitchStatement(Java.SwitchStatement ss) {
+        this.pw.print("switch (");
+        this.unparse(ss.condition);
+        this.pw.println(") {");
+        for (Iterator it = ss.sbsgs.iterator(); it.hasNext();) {
+            Java.SwitchStatement.SwitchBlockStatementGroup sbgs = (
+                (Java.SwitchStatement.SwitchBlockStatementGroup) it.next()
+            );
+            this.pw.print(AutoIndentWriter.UNINDENT);
+            try {
+                for (Iterator it2 = sbgs.caseLabels.iterator(); it2.hasNext();) {
+                    Java.Rvalue rv = (Java.Rvalue) it2.next();
+                    this.pw.print("case ");
+                    this.unparse(rv);
+                    this.pw.println(':');
+                }
+                if (sbgs.hasDefaultLabel) this.pw.println("default:");
+            } finally {
+                this.pw.print(AutoIndentWriter.INDENT);
+            }
+            for (Iterator it2 = sbgs.blockStatements.iterator(); it2.hasNext();) {
+                this.unparseBlockStatement((Java.BlockStatement) it2.next());
+                this.pw.println();
+            }
+        }
+        this.pw.print('}');
+    }
+    public void visitSynchronizedStatement(Java.SynchronizedStatement ss) {
+        this.pw.print("synchronized (");
+        this.unparse(ss.expression);
+        this.pw.print(") ");
+        this.unparseBlockStatement(ss.body);
+    }
+    public void visitThrowStatement(Java.ThrowStatement ts) {
+        this.pw.print("throw ");
+        this.unparse(ts.expression);
+        this.pw.print(';');
+    }
+    public void visitTryStatement(Java.TryStatement ts) {
+        this.pw.print("try ");
+        this.unparseBlockStatement(ts.body);
+        for (Iterator it = ts.catchClauses.iterator(); it.hasNext();) {
+            Java.CatchClause cc = (Java.CatchClause) it.next();
+            this.pw.print(" catch (");
+            this.unparseFormalParameter(cc.caughtException);
+            this.pw.print(") ");
+            this.unparseBlockStatement(cc.body);
+        }
+        if (ts.optionalFinally != null) {
+            this.pw.print(" finally ");
+            this.unparseBlockStatement(ts.optionalFinally);
+        }
+    }
+    public void visitWhileStatement(Java.WhileStatement ws) {
+        this.pw.print("while (");
+        this.unparse(ws.condition);
+        this.pw.print(") ");
+        this.unparseBlockStatement(ws.body);
+    }
+    public void unparseVariableDeclarator(Java.VariableDeclarator vd) {
+        this.pw.print(vd.name);
+        for (int i = 0; i < vd.brackets; ++i) this.pw.print("[]");
+        if (vd.optionalInitializer != null) {
+            this.pw.print(" = ");
+            this.unparseArrayInitializerOrRvalue(vd.optionalInitializer);
+        }
+    }
+    public void unparseFormalParameter(Java.FunctionDeclarator.FormalParameter fp) {
+        if (fp.finaL) this.pw.print("final ");
+        this.unparseType(fp.type);
+        this.pw.print(" " + AutoIndentWriter.TABULATOR + fp.name);
+    }
+    public void visitMethodInvocation(Java.MethodInvocation mi) {
+        if (mi.optionalTarget != null) {
+            this.unparseLhs(mi.optionalTarget, ".");
+            this.pw.print('.');
+        }
+        this.pw.print(mi.methodName);
+        this.unparseFunctionInvocationArguments(mi.arguments);
+    }
+    public void visitAlternateConstructorInvocation(Java.AlternateConstructorInvocation aci) {
+        this.pw.print("this");
+        this.unparseFunctionInvocationArguments(aci.arguments);
+    }
+    public void visitSuperConstructorInvocation(Java.SuperConstructorInvocation sci) {
+        if (sci.optionalQualification != null) {
+            this.unparseLhs(sci.optionalQualification, ".");
+            this.pw.print('.');
+        }
+        this.pw.print("super");
+        this.unparseFunctionInvocationArguments(sci.arguments);
+    }
+    public void visitNewClassInstance(Java.NewClassInstance nci) {
+        if (nci.optionalQualification != null) {
+            this.unparseLhs(nci.optionalQualification, ".");
+            this.pw.print('.');
+        }
+        this.pw.print("new " + nci.type.toString());
+        this.unparseFunctionInvocationArguments(nci.arguments);
+    }
+    public void visitAssignment(Java.Assignment a) {
+        this.unparseLhs(a.lhs, a.operator);
+        this.pw.print(' ' + a.operator + ' ');
+        this.unparseRhs(a.rhs, a.operator);
+    }
+    public void visitAmbiguousName(Java.AmbiguousName an) { this.pw.print(an.toString()); }
+    public void visitArrayAccessExpression(Java.ArrayAccessExpression aae) {
+        this.unparseLhs(aae.lhs, "[ ]");
+        this.pw.print('[');
+        this.unparse(aae.index);
+        this.pw.print(']');
+    }
+    public void visitArrayLength(Java.ArrayLength al) {
+        this.unparseLhs(al.lhs, ".");
+        this.pw.print(".length");
+    }
+    public void visitArrayType(Java.ArrayType at) {
+        this.unparseType(at.componentType);
+        this.pw.print("[]");
+    }
+    public void visitBasicType(Java.BasicType bt) {
+        this.pw.print(bt.toString());
+    }
+    public void visitBinaryOperation(Java.BinaryOperation bo) {
+        this.unparseLhs(bo.lhs, bo.op);
+        this.pw.print(' ' + bo.op + ' ');
+        this.unparseRhs(bo.rhs, bo.op);
+    }
+    public void visitCast(Java.Cast c) {
+        this.pw.print('(');
+        this.unparseType(c.targetType);
+        this.pw.print(") ");
+        this.unparseRhs(c.value, "cast");
+    }
+    public void visitClassLiteral(Java.ClassLiteral cl) {
+        this.unparseType(cl.type);
+        this.pw.print(".class");
+    }
+    public void visitConditionalExpression(Java.ConditionalExpression ce) {
+        this.unparseLhs(ce.lhs, "?:");
+        this.pw.print(" ? ");
+        this.unparseLhs(ce.mhs, "?:");
+        this.pw.print(" : ");
+        this.unparseRhs(ce.rhs, "?:");
+    }
+    public void visitCrement(Java.Crement c) {
+        if (c.pre) {
+            this.pw.print(c.operator);
+            this.unparseUnaryOperation(c.operand, c.operator + "x");
+        } else
+        {
+            this.unparseUnaryOperation(c.operand, "x" + c.operator);
+            this.pw.print(c.operator);
+        }
+    }
+    public void visitFieldAccess(Java.FieldAccess fa) {
+        this.unparseLhs(fa.lhs, ".");
+        this.pw.print('.' + fa.field.getName());
+    }
+    public void visitFieldAccessExpression(Java.FieldAccessExpression fae) {
+        this.unparseLhs(fae.lhs, ".");
+        this.pw.print('.' + fae.fieldName);
+    }
+    public void visitSuperclassFieldAccessExpression(Java.SuperclassFieldAccessExpression scfae) {
+        if (scfae.optionalQualification != null) {
+            this.unparseType(scfae.optionalQualification);
+            this.pw.print(".super." + scfae.fieldName);
+        } else
+        {
+            this.pw.print("super." + scfae.fieldName);
+        }
+    }
+    public void visitInstanceof(Java.Instanceof io) {
+        this.unparseLhs(io.lhs, "instanceof");
+        this.pw.print(" instanceof ");
+        this.unparseType(io.rhs);
+    }
+    public void visitLiteral(Java.Literal l) { this.pw.print(l.toString()); }
+    public void visitLocalVariableAccess(Java.LocalVariableAccess lva) { this.pw.print(lva.toString()); }
+    public void visitNewArray(Java.NewArray na) {
+        this.pw.print("new ");
+        this.unparseType(na.type);
+        for (int i = 0; i < na.dimExprs.length; ++i) {
+            this.pw.print('[');
+            this.unparse(na.dimExprs[i]);
+            this.pw.print(']');
+        }
+        for (int i = 0; i < na.dims; ++i) {
+            this.pw.print("[]");
+        }
+    }
+    public void visitNewInitializedArray(Java.NewInitializedArray nai) {
+        this.pw.print("new ");
+        this.unparseType(nai.arrayType);
+        this.pw.print(" ");
+        this.unparseArrayInitializerOrRvalue(nai.arrayInitializer);
+    }
+    public void visitPackage(Java.Package p) { this.pw.print(p.toString()); }
+    public void visitParameterAccess(Java.ParameterAccess pa) { this.pw.print(pa.toString()); }
+    public void visitQualifiedThisReference(Java.QualifiedThisReference qtr) {
+        this.unparseType(qtr.qualification);
+        this.pw.print(".this");
+    }
+    public void visitReferenceType(Java.ReferenceType rt) { this.pw.print(rt.toString()); }
+    public void visitRvalueMemberType(Java.RvalueMemberType rmt) { this.pw.print(rmt.toString()); }
+    public void visitSimpleType(Java.SimpleType st) { this.pw.print(st.toString()); }
+    public void visitSuperclassMethodInvocation(Java.SuperclassMethodInvocation smi) {
+        this.pw.print("super." + smi.methodName);
+        this.unparseFunctionInvocationArguments(smi.arguments);
+    }
+    public void visitThisReference(Java.ThisReference tr) {
+        this.pw.print("this");
+    }
+    public void visitUnaryOperation(Java.UnaryOperation uo) {
+        this.pw.print(uo.operator);
+        this.unparseUnaryOperation(uo.operand, uo.operator + "x");
+    }
+    public void visitParenthesizedExpression(Java.ParenthesizedExpression pe) {
+        this.pw.print('(');
+        this.unparse(pe.value);
+        this.pw.print(')');
+    }
+
+    // Helpers
+
+    private void unparseBlockStatement(Java.BlockStatement blockStatement) {
+        blockStatement.accept(this);
+    }
+
+    private void unparseTypeDeclaration(Java.TypeDeclaration typeDeclaration) {
+        typeDeclaration.accept(this);
+    }
+
+    private void unparseType(Java.Type type) {
+        ((Java.Atom) type).accept(this);
+    }
+
+    private void unparse(Java.Atom operand) {
+        operand.accept(this);
+    }
+
+    /**
+     * Iff the <code>operand</code> is unnatural for the <code>unaryOperator</code>, enclose the
+     * <code>operand</code> in parentheses. Example: "a+b" is an unnatural operand for unary "!x".
+     *
+     * @param unaryOperator ++x --x +x -x ~x !x x++ x--
+     */
+    private void unparseUnaryOperation(Java.Rvalue operand, String unaryOperator) {
+        int cmp = ModifiedUnparseVisitor.comparePrecedence(unaryOperator, operand);
+        this.unparse(operand, cmp < 0);
+    }
+
+    /**
+     * Iff the <code>lhs</code> is unnatural for the <code>binaryOperator</code>, enclose the
+     * <code>lhs</code> in parentheses. Example: "a+b" is an unnatural lhs for operator "*".
+     *
+     * @param binaryOperator = +=... ?: || && | ^ & == != < > <= >= instanceof << >> >>> + - * / % cast
+     */
+
+    private void unparseLhs(Java.Atom lhs, String binaryOperator) {
+        int cmp = ModifiedUnparseVisitor.comparePrecedence(binaryOperator, lhs);
+        this.unparse(lhs, cmp < 0 || (cmp == 0 && ModifiedUnparseVisitor.isLeftAssociate(binaryOperator)));
+    }
+
+
+    /**
+     * Iff the <code>rhs</code> is unnatural for the <code>binaryOperator</code>, enclose the
+     * <code>rhs</code> in parentheses. Example: "a+b" is an unnatural rhs for operator "*".
+     */
+    private void unparseRhs(Java.Rvalue rhs, String binaryOperator) {
+        int cmp = ModifiedUnparseVisitor.comparePrecedence(binaryOperator, rhs);
+        this.unparse(rhs, cmp < 0 || (cmp == 0 && ModifiedUnparseVisitor.isRightAssociate(binaryOperator)));
+    }
+
+    private void unparse(Java.Atom operand, boolean natural) {
+        if (!natural) this.pw.print("((( ");
+        this.unparse(operand);
+        if (!natural) this.pw.print(" )))");
+    }
+
+    /**
+     * Return true iff operator is right associative e.g. <code>a = b = c</code> evaluates as
+     * <code>a = (b = c)</code>.
+     *
+     * @return Return true iff operator is right associative
+     */
+    private static boolean isRightAssociate(String op) {
+        return ModifiedUnparseVisitor.RIGHT_ASSOCIATIVE_OPERATORS.contains(op);
+    }
+
+    /**
+     * Return true iff operator is left associative e.g. <code>a - b - c</code> evaluates as
+     * <code>(a - b) - c</code>.
+     *
+     * @return Return true iff operator is left associative
+     */
+    private static boolean isLeftAssociate(String op) {
+        return ModifiedUnparseVisitor.LEFT_ASSOCIATIVE_OPERATORS.contains(op);
+    }
+
+    /**
+     * Returns a value
+     * <ul>
+     *   <li>&lt; 0 iff the <code>operator</code> has lower precedence than the <code>operand</code>
+     *   <li>==; 0 iff the <code>operator</code> has equal precedence than the <code>operand</code>
+     *   <li>&gt; 0 iff the <code>operator</code> has higher precedence than the <code>operand</code>
+     * </ul>
+     */
+    private static int comparePrecedence(String operator, Java.Atom operand) {
+        if (operand instanceof Java.BinaryOperation) {
+            return (
+                ModifiedUnparseVisitor.getOperatorPrecedence(operator)
+                - ModifiedUnparseVisitor.getOperatorPrecedence(((Java.BinaryOperation) operand).op)
+            );
+        } else
+        if (operand instanceof Java.UnaryOperation) {
+            return (
+                ModifiedUnparseVisitor.getOperatorPrecedence(operator)
+                - ModifiedUnparseVisitor.getOperatorPrecedence(((Java.UnaryOperation) operand).operator + "x")
+            );
+        } else
+        if (operand instanceof Java.ConditionalExpression) {
+            return ModifiedUnparseVisitor.getOperatorPrecedence(operator) - ModifiedUnparseVisitor.getOperatorPrecedence("?:");
+        } else
+        if (operand instanceof Java.Instanceof) {
+            return ModifiedUnparseVisitor.getOperatorPrecedence(operator) - ModifiedUnparseVisitor.getOperatorPrecedence("instanceof");
+        } else
+        if (operand instanceof Java.Cast) {
+            return ModifiedUnparseVisitor.getOperatorPrecedence(operator) - ModifiedUnparseVisitor.getOperatorPrecedence("cast");
+        } else
+        if (operand instanceof Java.MethodInvocation || operand instanceof Java.FieldAccess) {
+            return ModifiedUnparseVisitor.getOperatorPrecedence(operator) - ModifiedUnparseVisitor.getOperatorPrecedence(".");
+        } else
+        if (operand instanceof Java.NewArray) {
+            return ModifiedUnparseVisitor.getOperatorPrecedence(operator) - ModifiedUnparseVisitor.getOperatorPrecedence("new");
+        } else
+        if (operand instanceof Java.Crement) {
+            Java.Crement c = (Java.Crement) operand;
+            return (
+                ModifiedUnparseVisitor.getOperatorPrecedence(operator)
+                - ModifiedUnparseVisitor.getOperatorPrecedence(c.pre ? c.operator + "x" : "x" + c.operator)
+            );
+        } else
+        {
+            // All other rvalues (e.g. literal) have higher precedence than any operator.
+            return -1;
+        }
+    }
+    private static int getOperatorPrecedence(String operator) {
+        return ((Integer) ModifiedUnparseVisitor.OPERATOR_PRECEDENCE.get(operator)).intValue();
+    }
+    private static final Set LEFT_ASSOCIATIVE_OPERATORS = new HashSet();
+    private static final Set RIGHT_ASSOCIATIVE_OPERATORS = new HashSet();
+    private static final Set UNARY_OPERATORS = new HashSet();
+    private static final Map OPERATOR_PRECEDENCE = new HashMap();
+    static {
+        Object[] ops = {
+            ModifiedUnparseVisitor.RIGHT_ASSOCIATIVE_OPERATORS, "=", "*=", "/=", "%=", "+=", "-=", "<<=", ">>=", ">>>=",
+                                                        "&=", "^=", "|=",
+            ModifiedUnparseVisitor.RIGHT_ASSOCIATIVE_OPERATORS, "?:",
+            ModifiedUnparseVisitor.LEFT_ASSOCIATIVE_OPERATORS,  "||",
+            ModifiedUnparseVisitor.LEFT_ASSOCIATIVE_OPERATORS,  "&&",
+            ModifiedUnparseVisitor.LEFT_ASSOCIATIVE_OPERATORS,  "|",
+            ModifiedUnparseVisitor.LEFT_ASSOCIATIVE_OPERATORS,  "^",
+            ModifiedUnparseVisitor.LEFT_ASSOCIATIVE_OPERATORS,  "&",
+            ModifiedUnparseVisitor.LEFT_ASSOCIATIVE_OPERATORS,  "==", "!=",
+            ModifiedUnparseVisitor.LEFT_ASSOCIATIVE_OPERATORS,  "<", ">", "<=", ">=", "instanceof",
+            ModifiedUnparseVisitor.LEFT_ASSOCIATIVE_OPERATORS,  "<<", ">>", ">>>",
+            ModifiedUnparseVisitor.LEFT_ASSOCIATIVE_OPERATORS,  "+", "-",
+            ModifiedUnparseVisitor.LEFT_ASSOCIATIVE_OPERATORS,  "*", "/", "%",
+            ModifiedUnparseVisitor.RIGHT_ASSOCIATIVE_OPERATORS, "cast",
+            ModifiedUnparseVisitor.UNARY_OPERATORS,             "++x", "--x", "+x", "-x", "~x", "!x",
+            ModifiedUnparseVisitor.UNARY_OPERATORS,             "x++", "x--",
+            ModifiedUnparseVisitor.LEFT_ASSOCIATIVE_OPERATORS,  "new",
+            ModifiedUnparseVisitor.LEFT_ASSOCIATIVE_OPERATORS,  ".", "[ ]",
+        };
+        int precedence = 0;
+        LOOP1: for (int i = 0;;) {
+            Set s = (Set) ops[i++];
+            Integer pi = new Integer(++precedence);
+            for (;;) {
+                if (i == ops.length) break LOOP1;
+                if (!(ops[i] instanceof String)) break;
+                String op = (String) ops[i++];
+                s.add(op);
+                ModifiedUnparseVisitor.OPERATOR_PRECEDENCE.put(op, pi);
+            }
+        }
+    }
+
+    private void unparseNamedClassDeclaration(Java.NamedClassDeclaration ncd) {
+        this.unparseDocComment(ncd);
+        this.unparseModifiers(ncd.getModifiers());
+        this.pw.print("class " + ncd.name);
+        if (ncd.optionalExtendedType != null) {
+            this.pw.print(" extends ");
+            this.unparseType(ncd.optionalExtendedType);
+        }
+        if (ncd.implementedTypes.length > 0) this.pw.print(" implements " + Java.join(ncd.implementedTypes, ", "));
+        this.pw.println(" {");
+        this.pw.print(AutoIndentWriter.INDENT);
+        this.unparseClassDeclarationBody(ncd);
+        this.pw.print(AutoIndentWriter.UNINDENT + "}");
+    }
+    private void unparseArrayInitializerOrRvalue(Java.ArrayInitializerOrRvalue aiorv) {
+        if (aiorv instanceof Java.Rvalue) {
+            this.unparse((Java.Rvalue) aiorv);
+        } else
+        if (aiorv instanceof Java.ArrayInitializer) {
+            Java.ArrayInitializer ai = (Java.ArrayInitializer) aiorv;
+            if (ai.values.length == 0) {
+                this.pw.print("{}");
+            } else
+            {
+                this.pw.print("{ ");
+                this.unparseArrayInitializerOrRvalue(ai.values[0]);
+                for (int i = 1; i < ai.values.length; ++i) {
+                    this.pw.print(", ");
+                    this.unparseArrayInitializerOrRvalue(ai.values[i]);
+                }
+                this.pw.print(" }");
+            }
+        } else
+        {
+            throw new JaninoRuntimeException(
+                "Unexpected array initializer or rvalue class "
+                + aiorv.getClass().getName()
+            );
+        }
+    }
+
+    public void visitAnonymousClassDeclaration(Java.AnonymousClassDeclaration acd) {
+        this.unparseType(acd.baseType);
+        this.pw.println(" {");
+        this.pw.print(AutoIndentWriter.INDENT);
+        this.unparseClassDeclarationBody(acd);
+        this.pw.print(AutoIndentWriter.UNINDENT + "}");
+    }
+    public void visitNewAnonymousClassInstance(Java.NewAnonymousClassInstance naci) {
+        if (naci.optionalQualification != null) {
+            this.unparseLhs(naci.optionalQualification, ".");
+            this.pw.print('.');
+        }
+        this.pw.print("new " + naci.anonymousClassDeclaration.baseType.toString() + '(');
+        for (int i = 0; i < naci.arguments.length; ++i) {
+            if (i > 0) this.pw.print(", ");
+            this.unparse(naci.arguments[i]);
+        }
+        this.pw.println(") {");
+        this.pw.print(AutoIndentWriter.INDENT);
+        this.unparseClassDeclarationBody(naci.anonymousClassDeclaration);
+        this.pw.print(AutoIndentWriter.UNINDENT + "}");
+    }
+    // Multi-line!
+    private void unparseClassDeclarationBody(Java.ClassDeclaration cd) {
+        for (Iterator it = cd.constructors.iterator(); it.hasNext();) {
+            this.pw.println();
+            ((Java.ConstructorDeclarator) it.next()).accept(this);
+            this.pw.println();
+        }
+        this.unparseAbstractTypeDeclarationBody(cd);
+        for (Iterator it = cd.variableDeclaratorsAndInitializers.iterator(); it.hasNext();) {
+            this.pw.println();
+            ((Java.TypeBodyDeclaration) it.next()).accept(this);
+            this.pw.println();
+        }
+    }
+    private void unparseInterfaceDeclaration(Java.InterfaceDeclaration id) {
+        this.unparseDocComment(id);
+        this.unparseModifiers(id.getModifiers());
+        //make sure we print "interface", even if it wasn't in the modifiers
+        if ((id.getModifiers() & Mod.INTERFACE) == 0) {
+            this.pw.print("interface ");
+        }
+        this.pw.print(id.name);
+        if (id.extendedTypes.length > 0) this.pw.print(" extends " + Java.join(id.extendedTypes, ", "));
+        this.pw.println(" {");
+        this.pw.print(AutoIndentWriter.INDENT);
+        this.unparseAbstractTypeDeclarationBody(id);
+        for (Iterator it = id.constantDeclarations.iterator(); it.hasNext();) {
+            ((Java.TypeBodyDeclaration) it.next()).accept(this);
+            this.pw.println();
+        }
+        this.pw.print(AutoIndentWriter.UNINDENT + "}");
+    }
+    // Multi-line!
+    private void unparseAbstractTypeDeclarationBody(Java.AbstractTypeDeclaration atd) {
+        for (Iterator it = atd.getMethodDeclarations().iterator(); it.hasNext();) {
+            this.pw.println();
+            ((Java.MethodDeclarator) it.next()).accept(this);
+            this.pw.println();
+        }
+        for (Iterator it = atd.getMemberTypeDeclarations().iterator(); it.hasNext();) {
+            this.pw.println();
+            ((Java.TypeBodyDeclaration) it.next()).accept(this);
+            this.pw.println();
+        }
+    }
+    private void unparseFunctionDeclaratorRest(Java.FunctionDeclarator fd) {
+        boolean big = fd.formalParameters.length >= 4;
+        this.pw.print('(');
+        if (big) { this.pw.println(); this.pw.print(AutoIndentWriter.INDENT); }
+        for (int i = 0; i < fd.formalParameters.length; ++i) {
+            if (i > 0) {
+                if (big) {
+                    this.pw.println(',');
+                } else
+                {
+                    this.pw.print(", ");
+                }
+            }
+            this.unparseFormalParameter(fd.formalParameters[i]);
+        }
+        if (big) { this.pw.println(); this.pw.print(AutoIndentWriter.UNINDENT); }
+        this.pw.print(')');
+        if (fd.thrownExceptions.length > 0) this.pw.print(" throws " + Java.join(fd.thrownExceptions, ", "));
+    }
+    private void unparseDocComment(Java.DocCommentable dc) {
+        String optionalDocComment = dc.getDocComment();
+        if (optionalDocComment != null) {
+            this.pw.print("/**");
+            BufferedReader br = new BufferedReader(new StringReader(optionalDocComment));
+            for (;;) {
+                String line;
+                try {
+                    line = br.readLine();
+                } catch (IOException e) {
+                    throw new JaninoRuntimeException();
+                }
+                if (line == null) break;
+                this.pw.println(line);
+                this.pw.print(" *");
+            }
+            this.pw.println("/");
+        }
+    }
+    private void unparseModifiers(short modifiers) {
+        if (modifiers != 0) {
+            this.pw.print(Mod.shortToString(modifiers) + ' ');
+        }
+    }
+    private void unparseFunctionInvocationArguments(Java.Rvalue[] arguments) {
+        boolean big = arguments.length >= 5;
+        this.pw.print('(');
+        if (big) { this.pw.println(); this.pw.print(AutoIndentWriter.INDENT); }
+        for (int i = 0; i < arguments.length; ++i) {
+            if (i > 0) {
+                if (big) {
+                    this.pw.println(',');
+                } else
+                {
+                    this.pw.print(", ");
+                }
+            }
+            this.unparse(arguments[i]);
+        }
+        if (big) { this.pw.println(); this.pw.print(AutoIndentWriter.UNINDENT); }
+        this.pw.print(')');
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/MathFunctions.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/MathFunctions.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/MathFunctions.java
new file mode 100644
index 0000000..702a0ad
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/MathFunctions.java
@@ -0,0 +1,48 @@
+package org.apache.drill.exec.expr.fn.impl;
+
+import org.apache.drill.exec.expr.DrillFunc;
+import org.apache.drill.exec.expr.annotations.FunctionTemplate;
+import org.apache.drill.exec.expr.annotations.FunctionTemplate.FunctionScope;
+import org.apache.drill.exec.expr.annotations.FunctionTemplate.NullHandling;
+import org.apache.drill.exec.expr.annotations.Output;
+import org.apache.drill.exec.expr.annotations.Param;
+import org.apache.drill.exec.expr.holders.ValueHolderImplmenetations.IntHolder;
+import org.apache.drill.exec.expr.holders.ValueHolderImplmenetations.LongHolder;
+import org.apache.drill.exec.record.RecordBatch;
+
+public class MathFunctions{
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(MathFunctions.class);
+  
+  private MathFunctions(){}
+  
+  @FunctionTemplate(name = "add", scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
+  public static class Add1 implements DrillFunc{
+    
+    @Param IntHolder left;
+    @Param IntHolder right;
+    @Output IntHolder out;
+
+    public void setup(RecordBatch b){}
+    
+    public void eval(){
+      out.value = left.value + right.value;
+    }
+
+  }
+  
+  @FunctionTemplate(name = "add", scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
+  public static class LongAdd1 implements DrillFunc{
+    
+    @Param LongHolder left;
+    @Param LongHolder right;
+    @Output LongHolder out;
+
+    public void setup(RecordBatch b){}
+    
+    public void eval(){
+      out.value = left.value + right.value;
+    }
+
+  }
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/holders/ValueHolder.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/holders/ValueHolder.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/holders/ValueHolder.java
new file mode 100644
index 0000000..0ef2e04
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/holders/ValueHolder.java
@@ -0,0 +1,4 @@
+package org.apache.drill.exec.expr.holders;
+
+public interface ValueHolder {
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/holders/ValueHolderImplmenetations.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/holders/ValueHolderImplmenetations.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/holders/ValueHolderImplmenetations.java
new file mode 100644
index 0000000..adf97fe
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/holders/ValueHolderImplmenetations.java
@@ -0,0 +1,44 @@
+package org.apache.drill.exec.expr.holders;
+
+import org.apache.drill.common.types.TypeProtos.MajorType;
+import org.apache.drill.common.types.TypeProtos.MinorType;
+import org.apache.drill.common.types.Types;
+
+public class ValueHolderImplmenetations {
+  
+  private ValueHolderImplmenetations(){}
+  
+  public final static class BooleanHolder implements ValueHolder{
+    public static final MajorType TYPE = Types.required(MinorType.BOOLEAN); 
+    public int value;
+  }
+  
+  public final static class NullableBooleanHolder implements ValueHolder {
+    public static final MajorType TYPE = Types.optional(MinorType.BOOLEAN);
+    public int value;
+    public int isSet;
+  }
+  
+  public final static class IntHolder implements ValueHolder{
+    public static final MajorType TYPE = Types.required(MinorType.INT); 
+    public int value;
+  }
+  
+  public final static class NullableIntHolder implements ValueHolder {
+    public static final MajorType TYPE = Types.optional(MinorType.INT);
+    public int value;
+    public int isSet;
+  }
+
+  public final static class LongHolder implements ValueHolder {
+    public static final MajorType TYPE = Types.required(MinorType.BIGINT);
+    public long value;
+  }
+  
+  public final static class NullableLongHolder implements ValueHolder {
+    public static final MajorType TYPE = Types.optional(MinorType.BIGINT);
+    public long value;
+    public int isSet;
+  }
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/holders/VarBinaryHolder.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/holders/VarBinaryHolder.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/holders/VarBinaryHolder.java
new file mode 100644
index 0000000..554c7d3
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/holders/VarBinaryHolder.java
@@ -0,0 +1,12 @@
+package org.apache.drill.exec.expr.holders;
+
+import io.netty.buffer.ByteBuf;
+
+public class VarBinaryHolder {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(VarBinaryHolder.class);
+  
+  public ByteBuf buffer;
+  public int start;
+  public int length;
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/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 33707a0..1dd7de8 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
@@ -17,7 +17,15 @@
  ******************************************************************************/
 package org.apache.drill.exec.ops;
 
+import java.io.IOException;
+
 import org.apache.drill.common.expression.LogicalExpression;
+import org.apache.drill.exec.compile.ClassTransformer;
+import org.apache.drill.exec.compile.QueryClassLoader;
+import org.apache.drill.exec.compile.TemplateClassDefinition;
+import org.apache.drill.exec.exception.ClassTransformationException;
+import org.apache.drill.exec.expr.CodeGenerator;
+import org.apache.drill.exec.expr.fn.FunctionImplementationRegistry;
 import org.apache.drill.exec.memory.BufferAllocator;
 import org.apache.drill.exec.metrics.SingleThreadNestedCounter;
 import org.apache.drill.exec.physical.impl.FilteringRecordBatchTransformer;
@@ -54,8 +62,13 @@ public class FragmentContext {
   private final IncomingBuffers buffers;
   private volatile Throwable failureCause;
   private volatile boolean failed = false;
+  private final FunctionImplementationRegistry funcRegistry;
+  private final QueryClassLoader loader;
+  private final ClassTransformer transformer;
   
-  public FragmentContext(DrillbitContext dbContext, FragmentHandle handle, UserClientConnection connection, IncomingBuffers buffers) {
+  public FragmentContext(DrillbitContext dbContext, FragmentHandle handle, UserClientConnection connection, IncomingBuffers buffers, FunctionImplementationRegistry funcRegistry) {
+    this.loader = new QueryClassLoader(true);
+    this.transformer = new ClassTransformer();
     this.fragmentTime = dbContext.getMetrics().timer(METRIC_TIMER_FRAGMENT_TIME);
     this.batchesCompleted = new SingleThreadNestedCounter(dbContext, METRIC_BATCHES_COMPLETED);
     this.recordsCompleted = new SingleThreadNestedCounter(dbContext, METRIC_RECORDS_COMPLETED);
@@ -64,6 +77,7 @@ public class FragmentContext {
     this.connection = connection;
     this.handle = handle;
     this.buffers = buffers;
+    this.funcRegistry = funcRegistry;
   }
 
   public void fail(Throwable cause) {
@@ -89,6 +103,10 @@ public class FragmentContext {
     return context.getAllocator();
   }
 
+  public <T> T getImplementationClass(TemplateClassDefinition<T, Void> templateDefinition, CodeGenerator cg) throws ClassTransformationException, IOException{
+    return transformer.getImplementationClass(this.loader, templateDefinition, cg.generate(), null);
+  }
+  
   public FilteringRecordBatchTransformer getFilteringExpression(LogicalExpression expr){
     return null;
   }
@@ -119,5 +137,11 @@ public class FragmentContext {
     return failed;
   }
   
+  public FunctionImplementationRegistry getFunctionRegistry(){
+    return funcRegistry;
+  }
   
+  public QueryClassLoader getClassLoader(){
+    return loader;
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/opt/BasicOptimizer.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/opt/BasicOptimizer.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/opt/BasicOptimizer.java
index 98ef05f..dcaf823 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/opt/BasicOptimizer.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/opt/BasicOptimizer.java
@@ -1,28 +1,28 @@
 package org.apache.drill.exec.opt;
 
-import com.fasterxml.jackson.core.type.TypeReference;
-import com.fasterxml.jackson.databind.ObjectMapper;
-import org.apache.drill.common.JSONOptions;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+
 import org.apache.drill.common.PlanProperties;
 import org.apache.drill.common.config.DrillConfig;
 import org.apache.drill.common.logical.LogicalPlan;
-import org.apache.drill.common.logical.data.*;
+import org.apache.drill.common.logical.data.Project;
+import org.apache.drill.common.logical.data.Scan;
+import org.apache.drill.common.logical.data.SinkOperator;
+import org.apache.drill.common.logical.data.Store;
 import org.apache.drill.common.logical.data.visitors.AbstractLogicalVisitor;
-import org.apache.drill.common.logical.data.visitors.LogicalVisitor;
+import org.apache.drill.common.types.TypeProtos.DataMode;
+import org.apache.drill.common.types.TypeProtos.MinorType;
 import org.apache.drill.exec.exception.OptimizerException;
 import org.apache.drill.exec.ops.QueryContext;
 import org.apache.drill.exec.physical.PhysicalPlan;
 import org.apache.drill.exec.physical.base.PhysicalOperator;
 import org.apache.drill.exec.physical.config.MockScanPOP;
-import org.apache.drill.exec.physical.config.MockStorePOP;
 import org.apache.drill.exec.physical.config.Screen;
-import org.apache.drill.exec.proto.SchemaDefProtos;
-import org.apache.drill.exec.server.DrillbitContext;
 
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.List;
+import com.fasterxml.jackson.core.type.TypeReference;
 
 /**
  * Created with IntelliJ IDEA.
@@ -99,8 +99,8 @@ public class BasicOptimizer extends Optimizer{
                 }
                 else{
                     myObjects = new ArrayList<>();
-                    MockScanPOP.MockColumn[] cols = { new MockScanPOP.MockColumn("blah", SchemaDefProtos.MinorType.INT, SchemaDefProtos.DataMode.REQUIRED,4,4,4),
-                            new MockScanPOP.MockColumn("blah_2", SchemaDefProtos.MinorType.INT, SchemaDefProtos.DataMode.REQUIRED,4,4,4)};
+                    MockScanPOP.MockColumn[] cols = { new MockScanPOP.MockColumn("blah", MinorType.INT, DataMode.REQUIRED,4,4,4),
+                            new MockScanPOP.MockColumn("blah_2", MinorType.INT, DataMode.REQUIRED,4,4,4)};
                     myObjects.add(new MockScanPOP.MockScanEntry(50, cols));
                 }
             } catch (IOException e) {

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/RecordField.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/RecordField.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/RecordField.java
index db3390a..4e9eb4c 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/RecordField.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/RecordField.java
@@ -17,7 +17,7 @@
  ******************************************************************************/
 package org.apache.drill.exec.physical;
 
-import org.apache.drill.common.expression.types.DataType;
+import org.apache.drill.common.types.TypeProtos.MajorType;
 
 import com.fasterxml.jackson.annotation.JsonCreator;
 import com.fasterxml.jackson.annotation.JsonProperty;
@@ -26,17 +26,17 @@ public class RecordField {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(RecordField.class);
 
   
-  private DataType type;
+  private MajorType type;
   private ValueMode mode;
   
   @JsonCreator
-  public RecordField(@JsonProperty("type") DataType type, @JsonProperty("mode") ValueMode mode) {
+  public RecordField(@JsonProperty("type") MajorType type, @JsonProperty("mode") ValueMode mode) {
     super();
     this.type = type;
     this.mode = mode;
   }
 
-  public DataType getType() {
+  public MajorType getType() {
     return type;
   }
 


[29/53] [abbrv] git commit: Added Visitor for logical plan graph and accompanying accept methods in each of the LogicalOperator classes. Created basic optimizer for logical to physical plan coversion.

Posted by ja...@apache.org.
Added Visitor for logical plan graph and accompanying accept methods in each of the LogicalOperator classes. Created basic optimizer for logical to physical plan coversion.

Signed-off-by: Jacques Nadeau <ja...@apache.org>


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

Branch: refs/heads/master
Commit: 1be6bd489827aae45155e45bc23ee7179d140897
Parents: 01278ae
Author: Jason Altekruse <al...@gmial.com>
Authored: Thu Jun 13 16:39:51 2013 -0700
Committer: Jacques Nadeau <ja...@apache.org>
Committed: Wed Jun 19 11:11:03 2013 -0700

----------------------------------------------------------------------
 .gitignore                                      |   1 +
 .../org/apache/drill/common/JSONOptions.java    |  17 ++-
 .../apache/drill/common/graph/GraphAlgos.java   |  20 ++-
 .../drill/common/logical/LogicalPlan.java       |   8 +-
 .../logical/data/CollapsingAggregate.java       |  14 ++
 .../drill/common/logical/data/Filter.java       |  14 ++
 .../drill/common/logical/data/Flatten.java      |  14 ++
 .../apache/drill/common/logical/data/Join.java  |  16 ++-
 .../apache/drill/common/logical/data/Limit.java |  18 ++-
 .../common/logical/data/LogicalOperator.java    |  21 +--
 .../logical/data/LogicalOperatorBase.java       |  25 ++--
 .../apache/drill/common/logical/data/Order.java |  17 ++-
 .../drill/common/logical/data/Project.java      |  15 +++
 .../common/logical/data/RunningAggregate.java   |  14 ++
 .../apache/drill/common/logical/data/Scan.java  |  19 ++-
 .../drill/common/logical/data/Segment.java      |  17 ++-
 .../drill/common/logical/data/Sequence.java     |  16 ++-
 .../apache/drill/common/logical/data/Store.java |  18 ++-
 .../drill/common/logical/data/Transform.java    |  15 +++
 .../apache/drill/common/logical/data/Union.java |  25 +++-
 .../drill/common/logical/data/WindowFrame.java  |  17 ++-
 .../data/visitors/AbstractLogicalVisitor.java   |  94 ++++++++++++++
 .../logical/data/visitors/LogicalVisitor.java   |  35 +++++
 .../java/org/apache/drill/storage/Simple.java   |   2 +-
 .../common/src/test/resources/jdbc_plan.json    |  33 +++++
 .../apache/drill/exec/client/DrillClient.java   |   2 +-
 .../exec/exception/OptimizerException.java      |  14 ++
 .../apache/drill/exec/opt/BasicOptimizer.java   | 127 +++++++++++++++++++
 .../drill/exec/physical/base/AbstractStore.java |   2 +-
 .../drill/exec/physical/config/Screen.java      |   2 +-
 .../apache/drill/exec/work/foreman/Foreman.java |   5 +-
 .../exec/client/DrillClientSystemTest.java      |   2 +
 .../drill/exec/opt/BasicOptimizerTest.java      |  24 ++++
 .../impl/TestDistributedFragmentRun.java        |  18 +++
 .../physical/impl/TestSimpleFragmentRun.java    |  92 +++++++-------
 .../src/test/resources/scan_screen_logical.json |  46 +++++++
 .../apache/drill/exec/ref/RecordPointer.java    |  18 +++
 37 files changed, 752 insertions(+), 105 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/1be6bd48/.gitignore
----------------------------------------------------------------------
diff --git a/.gitignore b/.gitignore
index 628a4e3..6bda406 100644
--- a/.gitignore
+++ b/.gitignore
@@ -5,4 +5,5 @@
 .idea/
 *.iml
 target/
+*.DS_Store
 *~

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/1be6bd48/sandbox/prototype/common/src/main/java/org/apache/drill/common/JSONOptions.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/JSONOptions.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/JSONOptions.java
index ad4926b..e309c60 100644
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/JSONOptions.java
+++ b/sandbox/prototype/common/src/main/java/org/apache/drill/common/JSONOptions.java
@@ -18,7 +18,12 @@
 package org.apache.drill.common;
 
 import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
 
+import com.fasterxml.jackson.core.*;
+import com.fasterxml.jackson.core.type.TypeReference;
+import com.fasterxml.jackson.databind.ObjectMapper;
 import org.apache.drill.common.JSONOptions.De;
 import org.apache.drill.common.JSONOptions.Se;
 import org.apache.drill.common.config.DrillConfig;
@@ -26,12 +31,6 @@ import org.apache.drill.common.exceptions.LogicalPlanParsingException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import com.fasterxml.jackson.core.JsonGenerationException;
-import com.fasterxml.jackson.core.JsonGenerator;
-import com.fasterxml.jackson.core.JsonLocation;
-import com.fasterxml.jackson.core.JsonParser;
-import com.fasterxml.jackson.core.JsonProcessingException;
-import com.fasterxml.jackson.core.TreeNode;
 import com.fasterxml.jackson.databind.DeserializationContext;
 import com.fasterxml.jackson.databind.JsonNode;
 import com.fasterxml.jackson.databind.SerializerProvider;
@@ -62,6 +61,12 @@ public class JSONOptions {
       throw new LogicalPlanParsingException(String.format("Failure while trying to convert late bound json options to type of %s. Reference was originally located at line %d, column %d.", c.getCanonicalName(), location.getLineNr(), location.getColumnNr()), e);
     }
   }
+
+  public <T> T getListWith(DrillConfig config, TypeReference<T> t) throws IOException {
+      ObjectMapper mapper = config.getMapper();
+      return mapper.treeAsTokens(root).readValueAs(t);
+     // return mapper.treeToValue(root,  mapper.getTypeFactory().constructCollectionType(List.class, c));
+  }
   
   public JsonNode path(String name){
     return root.path(name);

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/1be6bd48/sandbox/prototype/common/src/main/java/org/apache/drill/common/graph/GraphAlgos.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/graph/GraphAlgos.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/graph/GraphAlgos.java
index b83f1bd..692fd1b 100644
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/graph/GraphAlgos.java
+++ b/sandbox/prototype/common/src/main/java/org/apache/drill/common/graph/GraphAlgos.java
@@ -32,10 +32,15 @@ public class GraphAlgos {
     final List<AdjacencyList<V>.Node> sorted = new LinkedList<AdjacencyList<V>.Node>();
     final AdjacencyList<V> rGraph;
 
-    private TopoSorter(AdjacencyList<V> graph) {
+    private TopoSorter(AdjacencyList<V> graph, boolean reverse) {
       graph.clearVisited();
 
-      this.rGraph = graph.getReversedList();
+      if (reverse){
+        this.rGraph = graph.getReversedList();
+      }
+      else{
+        this.rGraph = graph;
+      }
       Collection<AdjacencyList<V>.Node> sourceNodes = rGraph.getInternalRootNodes();
 
       for (AdjacencyList<V>.Node n : sourceNodes) {
@@ -67,15 +72,20 @@ public class GraphAlgos {
      *          List of nodes that
      * @return
      */
-    static <V extends GraphValue<V>> List<AdjacencyList<V>.Node> sortInternal(AdjacencyList<V> graph) {
-      TopoSorter<V> ts = new TopoSorter<V>(graph);
+    static <V extends GraphValue<V>> List<AdjacencyList<V>.Node> sortInternal(AdjacencyList<V> graph, boolean reverse) {
+      TopoSorter<V> ts = new TopoSorter<V>(graph, reverse);
       return ts.sorted;
     }
 
     public static <V extends GraphValue<V>> List<V> sort(Graph<V, ?, ?> graph) {
       AdjacencyList<V> l = graph.getAdjList();
-      return l.convert(sortInternal(l));
+      return l.convert(sortInternal(l, true));
     }
+
+      public static <V extends GraphValue<V>> List<V> sortLogical(Graph<V, ?, ?> graph) {
+          AdjacencyList<V> l = graph.getAdjList();
+          return l.convert(sortInternal(l, false));
+      }
   }
 
   static <V extends GraphValue<V>> List<List<AdjacencyList<V>.Node>> checkDirected(AdjacencyList<V> graph) {

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/1be6bd48/sandbox/prototype/common/src/main/java/org/apache/drill/common/logical/LogicalPlan.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/logical/LogicalPlan.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/logical/LogicalPlan.java
index a2f2499..cf15f48 100644
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/logical/LogicalPlan.java
+++ b/sandbox/prototype/common/src/main/java/org/apache/drill/common/logical/LogicalPlan.java
@@ -47,7 +47,7 @@ public class LogicalPlan {
 
   private final PlanProperties properties;
   private final Map<String, StorageEngineConfig> storageEngineMap;
-  private final Graph<LogicalOperator, SourceOperator, SinkOperator> graph;
+  private final Graph<LogicalOperator, SinkOperator, SourceOperator> graph;
 
   @JsonCreator
   public LogicalPlan(@JsonProperty("head") PlanProperties head,
@@ -55,12 +55,12 @@ public class LogicalPlan {
       @JsonProperty("query") List<LogicalOperator> operators) {
     this.storageEngineMap = storageEngineMap;
     this.properties = head;
-    this.graph = Graph.newGraph(operators, SourceOperator.class, SinkOperator.class);
+    this.graph = Graph.newGraph(operators, SinkOperator.class, SourceOperator.class);
   }
 
   @JsonProperty("query")
   public List<LogicalOperator> getSortedOperators() {
-    return GraphAlgos.TopoSorter.sort(graph);
+    return GraphAlgos.TopoSorter.sortLogical(graph);
   }
 
   public StorageEngineConfig getStorageEngine(String name) {
@@ -68,7 +68,7 @@ public class LogicalPlan {
   }
 
   @JsonIgnore
-  public Graph<LogicalOperator, SourceOperator, SinkOperator> getGraph() {
+  public Graph<LogicalOperator, SinkOperator, SourceOperator> getGraph() {
     return graph;
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/1be6bd48/sandbox/prototype/common/src/main/java/org/apache/drill/common/logical/data/CollapsingAggregate.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/logical/data/CollapsingAggregate.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/logical/data/CollapsingAggregate.java
index 6c10444..49015ec 100644
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/logical/data/CollapsingAggregate.java
+++ b/sandbox/prototype/common/src/main/java/org/apache/drill/common/logical/data/CollapsingAggregate.java
@@ -17,11 +17,16 @@
  ******************************************************************************/
 package org.apache.drill.common.logical.data;
 
+import com.google.common.collect.Iterators;
 import org.apache.drill.common.expression.FieldReference;
 
 import com.fasterxml.jackson.annotation.JsonCreator;
 import com.fasterxml.jackson.annotation.JsonProperty;
 import com.fasterxml.jackson.annotation.JsonTypeName;
+import org.apache.drill.common.logical.data.visitors.LogicalVisitor;
+
+import java.util.Iterator;
+import java.util.List;
 
 @JsonTypeName("collapsingaggregate")
 public class CollapsingAggregate extends SingleInputOperator{
@@ -56,4 +61,13 @@ public class CollapsingAggregate extends SingleInputOperator{
     return aggregations;
   }
 
+    @Override
+    public <T, X, E extends Throwable> T accept(LogicalVisitor<T, X, E> logicalVisitor, X value) throws E {
+        return logicalVisitor.visitCollapsingAggregate(this, value);
+    }
+
+    @Override
+    public Iterator<LogicalOperator> iterator() {
+        return Iterators.singletonIterator(getInput());
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/1be6bd48/sandbox/prototype/common/src/main/java/org/apache/drill/common/logical/data/Filter.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/logical/data/Filter.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/logical/data/Filter.java
index 21422d2..e23f9ff 100644
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/logical/data/Filter.java
+++ b/sandbox/prototype/common/src/main/java/org/apache/drill/common/logical/data/Filter.java
@@ -17,11 +17,15 @@
  ******************************************************************************/
 package org.apache.drill.common.logical.data;
 
+import com.google.common.collect.Iterators;
 import org.apache.drill.common.expression.LogicalExpression;
 
 import com.fasterxml.jackson.annotation.JsonCreator;
 import com.fasterxml.jackson.annotation.JsonProperty;
 import com.fasterxml.jackson.annotation.JsonTypeName;
+import org.apache.drill.common.logical.data.visitors.LogicalVisitor;
+
+import java.util.Iterator;
 
 @JsonTypeName("filter")
 public class Filter extends SingleInputOperator{
@@ -36,6 +40,16 @@ public class Filter extends SingleInputOperator{
     return expr;
   }
 
+    @Override
+    public <T, X, E extends Throwable> T accept(LogicalVisitor<T, X, E> logicalVisitor, X value) throws E {
+        return logicalVisitor.visitFilter(this, value);
+    }
+
+    @Override
+    public Iterator<LogicalOperator> iterator() {
+        return Iterators.singletonIterator(getInput());
+    }
+
   
 	
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/1be6bd48/sandbox/prototype/common/src/main/java/org/apache/drill/common/logical/data/Flatten.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/logical/data/Flatten.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/logical/data/Flatten.java
index 9c8b060..114ba0a 100644
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/logical/data/Flatten.java
+++ b/sandbox/prototype/common/src/main/java/org/apache/drill/common/logical/data/Flatten.java
@@ -17,12 +17,16 @@
  ******************************************************************************/
 package org.apache.drill.common.logical.data;
 
+import com.google.common.collect.Iterators;
 import org.apache.drill.common.expression.FieldReference;
 import org.apache.drill.common.expression.LogicalExpression;
 
 import com.fasterxml.jackson.annotation.JsonCreator;
 import com.fasterxml.jackson.annotation.JsonProperty;
 import com.fasterxml.jackson.annotation.JsonTypeName;
+import org.apache.drill.common.logical.data.visitors.LogicalVisitor;
+
+import java.util.Iterator;
 
 @JsonTypeName("flatten")
 public class Flatten extends SingleInputOperator{
@@ -48,6 +52,16 @@ public class Flatten extends SingleInputOperator{
   public boolean isDrop() {
     return drop;
   }
+
+    @Override
+    public <T, X, E extends Throwable> T accept(LogicalVisitor<T, X, E> logicalVisitor, X value) throws E {
+        return logicalVisitor.visitFlatten(this, value);
+    }
+
+    @Override
+    public Iterator<LogicalOperator> iterator() {
+        return Iterators.singletonIterator(getInput());
+    }
   
   
 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/1be6bd48/sandbox/prototype/common/src/main/java/org/apache/drill/common/logical/data/Join.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/logical/data/Join.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/logical/data/Join.java
index 9742cd7..865c78e 100644
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/logical/data/Join.java
+++ b/sandbox/prototype/common/src/main/java/org/apache/drill/common/logical/data/Join.java
@@ -17,12 +17,16 @@
  ******************************************************************************/
 package org.apache.drill.common.logical.data;
 
+import com.google.common.collect.Iterators;
 import org.apache.drill.common.exceptions.ExpressionParsingException;
 
 import com.fasterxml.jackson.annotation.JsonCreator;
 import com.fasterxml.jackson.annotation.JsonIgnore;
 import com.fasterxml.jackson.annotation.JsonProperty;
 import com.fasterxml.jackson.annotation.JsonTypeName;
+import org.apache.drill.common.logical.data.visitors.LogicalVisitor;
+
+import java.util.Iterator;
 
 @JsonTypeName("join")
 public class Join extends LogicalOperatorBase {
@@ -48,9 +52,9 @@ public class Join extends LogicalOperatorBase {
     this.conditions = conditions;
     this.left = left;
     this.right = right;
-    this.type = JoinType.resolve(type);
     left.registerAsSubscriber(this);
     right.registerAsSubscriber(this);
+    this.type = JoinType.resolve(type);
 
   }
 
@@ -74,4 +78,14 @@ public class Join extends LogicalOperatorBase {
   public String getType(){
     return type.name();
   }
+
+    @Override
+    public <T, X, E extends Throwable> T accept(LogicalVisitor<T, X, E> logicalVisitor, X value) throws E {
+        return logicalVisitor.visitJoin(this, value);
+    }
+
+    @Override
+    public Iterator<LogicalOperator> iterator() {
+        return Iterators.forArray(getLeft(), getRight());
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/1be6bd48/sandbox/prototype/common/src/main/java/org/apache/drill/common/logical/data/Limit.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/logical/data/Limit.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/logical/data/Limit.java
index d4162f1..1774790 100644
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/logical/data/Limit.java
+++ b/sandbox/prototype/common/src/main/java/org/apache/drill/common/logical/data/Limit.java
@@ -20,6 +20,10 @@ package org.apache.drill.common.logical.data;
 import com.fasterxml.jackson.annotation.JsonCreator;
 import com.fasterxml.jackson.annotation.JsonProperty;
 import com.fasterxml.jackson.annotation.JsonTypeName;
+import com.google.common.collect.Iterators;
+import org.apache.drill.common.logical.data.visitors.LogicalVisitor;
+
+import java.util.Iterator;
 
 @JsonTypeName("limit")
 public class Limit extends SingleInputOperator{
@@ -41,6 +45,16 @@ public class Limit extends SingleInputOperator{
   public int getLast() {
     return last;
   }
-  
-  
+
+    @Override
+    public <T, X, E extends Throwable> T accept(LogicalVisitor<T, X, E> logicalVisitor, X value) throws E {
+        return logicalVisitor.visitLimit(this, value);
+    }
+
+    @Override
+    public Iterator<LogicalOperator> iterator() {
+        return Iterators.singletonIterator(getInput());
+    }
+
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/1be6bd48/sandbox/prototype/common/src/main/java/org/apache/drill/common/logical/data/LogicalOperator.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/logical/data/LogicalOperator.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/logical/data/LogicalOperator.java
index a5301bb..3bed4d2 100644
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/logical/data/LogicalOperator.java
+++ b/sandbox/prototype/common/src/main/java/org/apache/drill/common/logical/data/LogicalOperator.java
@@ -20,22 +20,27 @@ package org.apache.drill.common.logical.data;
 import java.util.Collection;
 import java.util.List;
 
+import com.fasterxml.jackson.annotation.*;
 import org.apache.drill.common.graph.GraphValue;
 import org.apache.drill.common.logical.ValidationError;
 
-import com.fasterxml.jackson.annotation.JsonIdentityInfo;
-import com.fasterxml.jackson.annotation.JsonPropertyOrder;
-import com.fasterxml.jackson.annotation.JsonTypeInfo;
-import com.fasterxml.jackson.annotation.ObjectIdGenerators;
+import org.apache.drill.common.logical.data.visitors.LogicalVisitor;
 
 @JsonPropertyOrder({"@id", "memo", "input"}) // op will always be first since it is wrapped.
 @JsonIdentityInfo(generator=ObjectIdGenerators.IntSequenceGenerator.class, property="@id")
 @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, include = JsonTypeInfo.As.PROPERTY, property="op")
 public interface LogicalOperator extends GraphValue<LogicalOperator>{
-	
-	public void registerAsSubscriber(LogicalOperator operator);
+
 	public void setupAndValidate(List<LogicalOperator> operators, Collection<ValidationError> errors);
-	
-	
+
+    /**
+     * Provides capability to build a set of output based on traversing a query graph tree.
+     *
+     * @param logicalVisitor
+     * @return
+     */
+    public <T, X, E extends Throwable> T accept(LogicalVisitor<T, X, E> logicalVisitor, X value) throws E;
+
+    public void registerAsSubscriber(LogicalOperator operator);
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/1be6bd48/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 5802056..23d3c6e 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
@@ -34,8 +34,8 @@ import com.fasterxml.jackson.annotation.JsonProperty;
 
 public abstract class LogicalOperatorBase implements LogicalOperator{
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(LogicalOperatorBase.class);
-  
-	private List<LogicalOperator> children = new ArrayList<LogicalOperator>();
+
+    private List<LogicalOperator> children = new ArrayList<LogicalOperator>();
 	private String memo;
 	
 	public final int hashCode(){
@@ -48,25 +48,20 @@ public abstract class LogicalOperatorBase implements LogicalOperator{
 	}
 
   @Override
-  public void registerAsSubscriber(LogicalOperator operator) {
-    if(operator == null) throw new IllegalArgumentException("You attempted to register a null operators.");
-    children.add(operator);
-  }
-
-  @Override
   public void accept(GraphVisitor<LogicalOperator> visitor) {
     if(visitor.enter(this)){
-      for(LogicalOperator o : children){
-        o.accept(visitor);
-      }
+        for(LogicalOperator o : children){
+            o.accept(visitor);
+        }
     }
     visitor.leave(this);
   }
 
-  @Override
-  public Iterator<LogicalOperator> iterator() {
-    return children.iterator();
-  }
+    @Override
+    public void registerAsSubscriber(LogicalOperator operator) {
+        if(operator == null) throw new IllegalArgumentException("You attempted to register a null operators.");
+        children.add(operator);
+    }
 
   @Override
   public String toString() {

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/1be6bd48/sandbox/prototype/common/src/main/java/org/apache/drill/common/logical/data/Order.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/logical/data/Order.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/logical/data/Order.java
index 36ebbd3..a9355de 100644
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/logical/data/Order.java
+++ b/sandbox/prototype/common/src/main/java/org/apache/drill/common/logical/data/Order.java
@@ -17,6 +17,7 @@
  ******************************************************************************/
 package org.apache.drill.common.logical.data;
 
+import com.google.common.collect.Iterators;
 import org.apache.drill.common.expression.FieldReference;
 import org.apache.drill.common.expression.LogicalExpression;
 
@@ -24,6 +25,9 @@ import com.fasterxml.jackson.annotation.JsonCreator;
 import com.fasterxml.jackson.annotation.JsonIgnore;
 import com.fasterxml.jackson.annotation.JsonProperty;
 import com.fasterxml.jackson.annotation.JsonTypeName;
+import org.apache.drill.common.logical.data.visitors.LogicalVisitor;
+
+import java.util.Iterator;
 
 @JsonTypeName("order")
 public class Order extends SingleInputOperator {
@@ -45,7 +49,18 @@ public class Order extends SingleInputOperator {
     return within;
   }
 
-  public static class Ordering {
+    @Override
+    public <T, X, E extends Throwable> T accept(LogicalVisitor<T, X, E> logicalVisitor, X value) throws E {
+        return logicalVisitor.visitOrder(this, value);
+    }
+
+    @Override
+    public Iterator<LogicalOperator> iterator() {
+        return Iterators.singletonIterator(getInput());
+    }
+
+
+    public static class Ordering {
 
     private final Direction direction;
     private final LogicalExpression expr;

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/1be6bd48/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 84c8104..aca114f 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
@@ -17,12 +17,16 @@
  ******************************************************************************/
 package org.apache.drill.common.logical.data;
 
+import com.google.common.collect.Iterators;
 import org.apache.drill.common.exceptions.ExpressionParsingException;
 import org.apache.drill.common.expression.PathSegment;
 
 import com.fasterxml.jackson.annotation.JsonCreator;
 import com.fasterxml.jackson.annotation.JsonProperty;
 import com.fasterxml.jackson.annotation.JsonTypeName;
+import org.apache.drill.common.logical.data.visitors.LogicalVisitor;
+
+import java.util.Iterator;
 
 @JsonTypeName("project")
 public class Project extends SingleInputOperator {
@@ -48,4 +52,15 @@ public class Project extends SingleInputOperator {
     return selections;
   }
 
+    @Override
+    public <T, X, E extends Throwable> T accept(LogicalVisitor<T, X, E> logicalVisitor, X value) throws E {
+        return logicalVisitor.visitProject(this, value);
+    }
+
+    @Override
+    public Iterator<LogicalOperator> iterator() {
+        return Iterators.singletonIterator(getInput());
+    }
+
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/1be6bd48/sandbox/prototype/common/src/main/java/org/apache/drill/common/logical/data/RunningAggregate.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/logical/data/RunningAggregate.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/logical/data/RunningAggregate.java
index 01df9a0..a649e9d 100644
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/logical/data/RunningAggregate.java
+++ b/sandbox/prototype/common/src/main/java/org/apache/drill/common/logical/data/RunningAggregate.java
@@ -17,11 +17,15 @@
  ******************************************************************************/
 package org.apache.drill.common.logical.data;
 
+import com.google.common.collect.Iterators;
 import org.apache.drill.common.expression.FieldReference;
 
 import com.fasterxml.jackson.annotation.JsonCreator;
 import com.fasterxml.jackson.annotation.JsonProperty;
 import com.fasterxml.jackson.annotation.JsonTypeName;
+import org.apache.drill.common.logical.data.visitors.LogicalVisitor;
+
+import java.util.Iterator;
 
 @JsonTypeName("runningaggregate")
 public class RunningAggregate extends SingleInputOperator{
@@ -44,6 +48,16 @@ public class RunningAggregate extends SingleInputOperator{
     return aggregations;
   }
 
+    @Override
+    public <T, X, E extends Throwable> T accept(LogicalVisitor<T, X, E> logicalVisitor, X value) throws E {
+        return logicalVisitor.visitRunningAggregate(this, value);
+    }
+
+    @Override
+    public Iterator<LogicalOperator> iterator() {
+        return Iterators.singletonIterator(getInput());
+    }
+
 
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/1be6bd48/sandbox/prototype/common/src/main/java/org/apache/drill/common/logical/data/Scan.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/logical/data/Scan.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/logical/data/Scan.java
index 7933e40..14f44b4 100644
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/logical/data/Scan.java
+++ b/sandbox/prototype/common/src/main/java/org/apache/drill/common/logical/data/Scan.java
@@ -17,12 +17,16 @@
  ******************************************************************************/
 package org.apache.drill.common.logical.data;
 
+import com.google.common.collect.Iterators;
 import org.apache.drill.common.JSONOptions;
 import org.apache.drill.common.expression.FieldReference;
 
 import com.fasterxml.jackson.annotation.JsonCreator;
 import com.fasterxml.jackson.annotation.JsonProperty;
 import com.fasterxml.jackson.annotation.JsonTypeName;
+import org.apache.drill.common.logical.data.visitors.LogicalVisitor;
+
+import java.util.Iterator;
 
 @JsonTypeName("scan")
 public class Scan extends SourceOperator{
@@ -51,7 +55,18 @@ public class Scan extends SourceOperator{
   public FieldReference getOutputReference() {
     return outputReference;
   }
-	
-  
+
+    @Override
+    public <T, X, E extends Throwable> T accept(LogicalVisitor<T, X, E> logicalVisitor, X value) throws E {
+        return logicalVisitor.visitScan(this, value);
+    }
+
+    @Override
+    public Iterator<LogicalOperator> iterator() {
+        return Iterators.emptyIterator();
+    }
+
+
+
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/1be6bd48/sandbox/prototype/common/src/main/java/org/apache/drill/common/logical/data/Segment.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/logical/data/Segment.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/logical/data/Segment.java
index 5b10e07..8177dc3 100644
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/logical/data/Segment.java
+++ b/sandbox/prototype/common/src/main/java/org/apache/drill/common/logical/data/Segment.java
@@ -17,12 +17,16 @@
  ******************************************************************************/
 package org.apache.drill.common.logical.data;
 
+import com.google.common.collect.Iterators;
 import org.apache.drill.common.expression.FieldReference;
 import org.apache.drill.common.expression.LogicalExpression;
 
 import com.fasterxml.jackson.annotation.JsonCreator;
 import com.fasterxml.jackson.annotation.JsonProperty;
 import com.fasterxml.jackson.annotation.JsonTypeName;
+import org.apache.drill.common.logical.data.visitors.LogicalVisitor;
+
+import java.util.Iterator;
 
 @JsonTypeName("segment")
 public class Segment extends SingleInputOperator{
@@ -43,5 +47,16 @@ public class Segment extends SingleInputOperator{
   public FieldReference getName() {
     return name;
   }
-  
+
+    @Override
+    public <T, X, E extends Throwable> T accept(LogicalVisitor<T, X, E> logicalVisitor, X value) throws E {
+        return logicalVisitor.visitSegment(this, value);
+    }
+
+    @Override
+    public Iterator<LogicalOperator> iterator() {
+        return Iterators.singletonIterator(getInput());
+    }
+
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/1be6bd48/sandbox/prototype/common/src/main/java/org/apache/drill/common/logical/data/Sequence.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/logical/data/Sequence.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/logical/data/Sequence.java
index 12c7c08..86337da 100644
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/logical/data/Sequence.java
+++ b/sandbox/prototype/common/src/main/java/org/apache/drill/common/logical/data/Sequence.java
@@ -18,8 +18,11 @@
 package org.apache.drill.common.logical.data;
 
 import java.io.IOException;
+import java.util.Iterator;
 
+import com.google.common.collect.Iterators;
 import org.apache.drill.common.logical.data.Sequence.De;
+import org.apache.drill.common.logical.data.visitors.LogicalVisitor;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -56,7 +59,18 @@ public class Sequence extends LogicalOperatorBase {
   @JsonProperty("do")
   public LogicalOperator[] stream;
 
-  public static class De extends StdDeserializer<LogicalOperator> {
+    @Override
+    public <T, X, E extends Throwable> T accept(LogicalVisitor<T, X, E> logicalVisitor, X value) throws E {
+        return logicalVisitor.visitSequence(this, value);
+    }
+
+    @Override
+    public Iterator<LogicalOperator> iterator() {
+        return Iterators.singletonIterator(stream[stream.length - 1]);
+    }
+
+
+    public static class De extends StdDeserializer<LogicalOperator> {
 
     protected De() {
       super(Sequence.class);

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/1be6bd48/sandbox/prototype/common/src/main/java/org/apache/drill/common/logical/data/Store.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/logical/data/Store.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/logical/data/Store.java
index 0569b8a..78b5d41 100644
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/logical/data/Store.java
+++ b/sandbox/prototype/common/src/main/java/org/apache/drill/common/logical/data/Store.java
@@ -17,12 +17,16 @@
  ******************************************************************************/
 package org.apache.drill.common.logical.data;
 
+import com.google.common.collect.Iterators;
 import org.apache.drill.common.JSONOptions;
 import org.apache.drill.common.defs.PartitionDef;
 
 import com.fasterxml.jackson.annotation.JsonCreator;
 import com.fasterxml.jackson.annotation.JsonProperty;
 import com.fasterxml.jackson.annotation.JsonTypeName;
+import org.apache.drill.common.logical.data.visitors.LogicalVisitor;
+
+import java.util.Iterator;
 
 @JsonTypeName("store")
 public class Store extends SinkOperator{
@@ -52,6 +56,16 @@ public class Store extends SinkOperator{
     return partition;
   }
 
-  
-  
+    @Override
+    public <T, X, E extends Throwable> T accept(LogicalVisitor<T, X, E> logicalVisitor, X value) throws E {
+        return logicalVisitor.visitStore(this, value);
+    }
+
+    @Override
+    public Iterator<LogicalOperator> iterator() {
+        return Iterators.singletonIterator(getInput());
+    }
+
+
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/1be6bd48/sandbox/prototype/common/src/main/java/org/apache/drill/common/logical/data/Transform.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/logical/data/Transform.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/logical/data/Transform.java
index 7396d6b..3cd69b6 100644
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/logical/data/Transform.java
+++ b/sandbox/prototype/common/src/main/java/org/apache/drill/common/logical/data/Transform.java
@@ -20,6 +20,10 @@ package org.apache.drill.common.logical.data;
 import com.fasterxml.jackson.annotation.JsonCreator;
 import com.fasterxml.jackson.annotation.JsonProperty;
 import com.fasterxml.jackson.annotation.JsonTypeName;
+import com.google.common.collect.Iterators;
+import org.apache.drill.common.logical.data.visitors.LogicalVisitor;
+
+import java.util.Iterator;
 
 @JsonTypeName("transform")
 public class Transform extends SingleInputOperator{
@@ -36,4 +40,15 @@ public class Transform extends SingleInputOperator{
     return transforms;
   }
 
+    @Override
+    public <T, X, E extends Throwable> T accept(LogicalVisitor<T, X, E> logicalVisitor, X value) throws E {
+        return logicalVisitor.visitTransform(this, value);
+    }
+
+    @Override
+    public Iterator<LogicalOperator> iterator() {
+        return Iterators.singletonIterator(getInput());
+    }
+
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/1be6bd48/sandbox/prototype/common/src/main/java/org/apache/drill/common/logical/data/Union.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/logical/data/Union.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/logical/data/Union.java
index 487401b..3cb2c54 100644
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/logical/data/Union.java
+++ b/sandbox/prototype/common/src/main/java/org/apache/drill/common/logical/data/Union.java
@@ -20,6 +20,10 @@ package org.apache.drill.common.logical.data;
 import com.fasterxml.jackson.annotation.JsonCreator;
 import com.fasterxml.jackson.annotation.JsonProperty;
 import com.fasterxml.jackson.annotation.JsonTypeName;
+import com.google.common.collect.Iterators;
+import org.apache.drill.common.logical.data.visitors.LogicalVisitor;
+
+import java.util.Iterator;
 
 @JsonTypeName("union")
 public class Union extends LogicalOperatorBase {
@@ -34,9 +38,9 @@ public class Union extends LogicalOperatorBase {
   @JsonCreator
   public Union(@JsonProperty("inputs") LogicalOperator[] inputs, @JsonProperty("distinct") Boolean distinct){
     this.inputs = inputs;
-    for (LogicalOperator o : inputs) {
-      o.registerAsSubscriber(this);
-    }
+      for (LogicalOperator o : inputs) {
+          o.registerAsSubscriber(this);
+      }
     this.distinct = distinct == null ? false : distinct;
   }
 
@@ -48,6 +52,17 @@ public class Union extends LogicalOperatorBase {
     return distinct;
   }
 
-  
-  
+    @Override
+    public <T, X, E extends Throwable> T accept(LogicalVisitor<T, X, E> logicalVisitor, X value) throws E {
+        return logicalVisitor.visitUnion(this, value);
+    }
+
+    @Override
+    public Iterator<LogicalOperator> iterator() {
+        return Iterators.forArray(inputs);
+    }
+
+
+
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/1be6bd48/sandbox/prototype/common/src/main/java/org/apache/drill/common/logical/data/WindowFrame.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/logical/data/WindowFrame.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/logical/data/WindowFrame.java
index ec14663..4cd7967 100644
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/logical/data/WindowFrame.java
+++ b/sandbox/prototype/common/src/main/java/org/apache/drill/common/logical/data/WindowFrame.java
@@ -17,11 +17,15 @@
  ******************************************************************************/
 package org.apache.drill.common.logical.data;
 
+import com.google.common.collect.Iterators;
 import org.apache.drill.common.expression.FieldReference;
 
 import com.fasterxml.jackson.annotation.JsonCreator;
 import com.fasterxml.jackson.annotation.JsonProperty;
 import com.fasterxml.jackson.annotation.JsonTypeName;
+import org.apache.drill.common.logical.data.visitors.LogicalVisitor;
+
+import java.util.Iterator;
 
 @JsonTypeName("windowframe")
 public class WindowFrame extends SingleInputOperator{
@@ -60,7 +64,18 @@ public class WindowFrame extends SingleInputOperator{
     return end;
   }
 
-  public static class FrameRef{
+    @Override
+    public <T, X, E extends Throwable> T accept(LogicalVisitor<T, X, E> logicalVisitor, X value) throws E {
+        return logicalVisitor.visitWindowFrame(this, value);
+    }
+
+    @Override
+    public Iterator<LogicalOperator> iterator() {
+        return Iterators.singletonIterator(getInput());
+    }
+
+
+    public static class FrameRef{
     private final FieldReference segment;
     private final FieldReference position;
     

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/1be6bd48/sandbox/prototype/common/src/main/java/org/apache/drill/common/logical/data/visitors/AbstractLogicalVisitor.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/logical/data/visitors/AbstractLogicalVisitor.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/logical/data/visitors/AbstractLogicalVisitor.java
new file mode 100644
index 0000000..a293d0b
--- /dev/null
+++ b/sandbox/prototype/common/src/main/java/org/apache/drill/common/logical/data/visitors/AbstractLogicalVisitor.java
@@ -0,0 +1,94 @@
+package org.apache.drill.common.logical.data.visitors;
+
+import org.apache.drill.common.logical.data.*;
+
+/**
+ * Created with IntelliJ IDEA.
+ * User: jaltekruse
+ * Date: 6/10/13
+ * Time: 1:55 PM
+ * To change this template use File | Settings | File Templates.
+ */
+public abstract class AbstractLogicalVisitor<T, X, E extends Throwable> implements LogicalVisitor<T, X, E> {
+
+    public T visitOp(LogicalOperator op, X value) throws E{
+        throw new UnsupportedOperationException(String.format(
+                "The LogicalVisitor of type %s does not currently support visiting the PhysicalOperator type %s.", this
+                .getClass().getCanonicalName(), op.getClass().getCanonicalName()));
+    }
+
+    @Override
+    public T visitScan(Scan scan, X value) throws E {
+        return visitOp(scan, value);
+    }
+
+    @Override
+    public T visitStore(Store store, X value) throws E {
+        return visitOp(store, value);
+    }
+
+    @Override
+    public T visitFilter(Filter filter, X value) throws E {
+        return visitOp(filter, value);
+    }
+
+    @Override
+    public T visitFlatten(Flatten flatten, X value) throws E {
+        return visitOp(flatten, value);
+    }
+
+    @Override
+    public T visitProject(Project project, X value) throws E {
+        return visitOp(project, value);
+    }
+
+    @Override
+    public T visitOrder(Order order, X value) throws E {
+        return visitOp(order, value);
+    }
+
+    @Override
+    public T visitJoin(Join join, X value) throws E {
+        return visitOp(join, value);
+    }
+
+    @Override
+    public T visitLimit(Limit limit, X value) throws E {
+        return visitOp(limit, value);
+    }
+
+    @Override
+    public T visitRunningAggregate(RunningAggregate runningAggregate, X value) throws E {
+        return visitOp(runningAggregate, value);
+    }
+
+    @Override
+    public T visitSegment(Segment segment, X value) throws E {
+        return visitOp(segment, value);
+    }
+
+    @Override
+    public T visitSequence(Sequence sequence, X value) throws E {
+        return visitOp(sequence, value);
+    }
+
+    @Override
+    public T visitTransform(Transform transform, X value) throws E {
+        return visitOp(transform, value);
+    }
+
+    @Override
+    public T visitUnion(Union union, X value) throws E {
+        return visitOp(union, value);
+    }
+
+    @Override
+    public T visitCollapsingAggregate(CollapsingAggregate collapsingAggregate, X value) throws E {
+        return visitOp(collapsingAggregate, value);
+    }
+
+    @Override
+    public T visitWindowFrame(WindowFrame windowFrame, X value) throws E {
+        return visitOp(windowFrame, value);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/1be6bd48/sandbox/prototype/common/src/main/java/org/apache/drill/common/logical/data/visitors/LogicalVisitor.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/logical/data/visitors/LogicalVisitor.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/logical/data/visitors/LogicalVisitor.java
new file mode 100644
index 0000000..1f08913
--- /dev/null
+++ b/sandbox/prototype/common/src/main/java/org/apache/drill/common/logical/data/visitors/LogicalVisitor.java
@@ -0,0 +1,35 @@
+package org.apache.drill.common.logical.data.visitors;
+
+
+import org.apache.drill.common.graph.GraphVisitor;
+import org.apache.drill.common.logical.data.*;
+
+/**
+ * Visitor class designed to traversal of a operator tree.  Basis for a number of operator manipulations including fragmentation and materialization.
+ * @param <RETURN> The class associated with the return of each visit method.
+ * @param <EXTRA> The class object associated with additional data required for a particular operator modification.
+ * @param <EXCEP> An optional exception class that can be thrown when a portion of a modification or traversal fails.  Must extend Throwable.  In the case where the visitor does not throw any caught exception, this can be set as RuntimeException.
+ */
+public interface LogicalVisitor<RETURN, EXTRA, EXCEP extends Throwable> {
+    static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(LogicalVisitor.class);
+
+
+    //public RETURN visitExchange(Exchange exchange, EXTRA value) throws EXCEP;
+    public RETURN visitScan(Scan scan, EXTRA value) throws EXCEP;
+    public RETURN visitStore(Store store, EXTRA value) throws EXCEP;
+    public RETURN visitCollapsingAggregate(CollapsingAggregate collapsingAggregate, EXTRA value) throws EXCEP;
+
+    public RETURN visitFilter(Filter filter, EXTRA value) throws EXCEP;
+    public RETURN visitFlatten(Flatten flatten, EXTRA value) throws EXCEP;
+
+    public RETURN visitProject(Project project, EXTRA value) throws EXCEP;
+    public RETURN visitOrder(Order order, EXTRA value) throws EXCEP;
+    public RETURN visitJoin(Join join, EXTRA value) throws EXCEP;
+    public RETURN visitLimit(Limit limit, EXTRA value) throws EXCEP;
+    public RETURN visitRunningAggregate(RunningAggregate runningAggregate, EXTRA value) throws EXCEP;
+    public RETURN visitSegment(Segment segment, EXTRA value) throws EXCEP;
+    public RETURN visitSequence(Sequence sequence, EXTRA value) throws EXCEP;
+    public RETURN visitTransform(Transform transform, EXTRA value) throws EXCEP;
+    public RETURN visitUnion(Union union, EXTRA value) throws EXCEP;
+    public RETURN visitWindowFrame(WindowFrame windowFrame, EXTRA value) throws EXCEP;
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/1be6bd48/sandbox/prototype/common/src/test/java/org/apache/drill/storage/Simple.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/test/java/org/apache/drill/storage/Simple.java b/sandbox/prototype/common/src/test/java/org/apache/drill/storage/Simple.java
index a821b59..4748cc1 100644
--- a/sandbox/prototype/common/src/test/java/org/apache/drill/storage/Simple.java
+++ b/sandbox/prototype/common/src/test/java/org/apache/drill/storage/Simple.java
@@ -26,7 +26,7 @@ public class Simple {
   
   public static void main(String[] args) throws Exception {
     DrillConfig c = DrillConfig.create();
-    LogicalPlan plan = LogicalPlan.parse(c, FileUtils.getResourceAsString("/simple_plan.json"));
+    LogicalPlan plan = LogicalPlan.parse(c, FileUtils.getResourceAsString("/jdbc_plan.json"));
     System.out.println(plan.toJsonString(c));
   }
   

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/1be6bd48/sandbox/prototype/common/src/test/resources/jdbc_plan.json
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/test/resources/jdbc_plan.json b/sandbox/prototype/common/src/test/resources/jdbc_plan.json
new file mode 100644
index 0000000..2d3b9d2
--- /dev/null
+++ b/sandbox/prototype/common/src/test/resources/jdbc_plan.json
@@ -0,0 +1,33 @@
+{
+    "head":{"type":"APACHE_DRILL_LOGICAL",
+        "version":"1",
+        "generator":{"type":"manual","info":"na"}
+    },
+    "storage":{
+        "donuts-json":{"type":"classpath"},
+        "queue":{"type":"queue"}
+    },
+    "query":[
+        {"op":"sequence",
+          "do":[
+                {
+                    "op":"scan",
+                    "memo":"initial_scan",
+                    "ref":"_MAP",
+                    "storageengine":"donuts-json"
+                    ,"selection":{"path":"/donuts.json","type":"JSON"}
+                },
+                {
+                    "op":"filter",
+                    "expr":"(3 < 4)"
+                },
+                {
+                    "op":"store",
+                    "storageengine":"queue",
+                    "memo":"output sink",
+                    "target":{"number":0}
+                }
+            ]
+        }
+    ]
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/1be6bd48/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/client/DrillClient.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/client/DrillClient.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/client/DrillClient.java
index c35e834..5d80ee4 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/client/DrillClient.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/client/DrillClient.java
@@ -90,7 +90,7 @@ public class DrillClient implements Closeable{
     Collection<DrillbitEndpoint> endpoints = clusterCoordinator.getAvailableEndpoints();
     checkState(!endpoints.isEmpty(), "No DrillbitEndpoint can be found");
     // just use the first endpoint for now
-    DrillbitEndpoint endpoint = get(endpoints, 0);
+    DrillbitEndpoint endpoint = endpoints.iterator().next();
     ByteBufAllocator bb = new PooledByteBufAllocator(true);
     this.client = new UserClient(bb, new NioEventLoopGroup(1, new NamedThreadFactory("Client-")));
     try {

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/1be6bd48/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/exception/OptimizerException.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/exception/OptimizerException.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/exception/OptimizerException.java
new file mode 100644
index 0000000..c57ce4a
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/exception/OptimizerException.java
@@ -0,0 +1,14 @@
+package org.apache.drill.exec.exception;
+
+/**
+ * Created with IntelliJ IDEA.
+ * User: jaltekruse
+ * Date: 6/11/13
+ * Time: 5:37 PM
+ * To change this template use File | Settings | File Templates.
+ */
+public class OptimizerException extends Exception{
+    public OptimizerException(String s) {
+        super(s);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/1be6bd48/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/opt/BasicOptimizer.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/opt/BasicOptimizer.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/opt/BasicOptimizer.java
new file mode 100644
index 0000000..98ef05f
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/opt/BasicOptimizer.java
@@ -0,0 +1,127 @@
+package org.apache.drill.exec.opt;
+
+import com.fasterxml.jackson.core.type.TypeReference;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import org.apache.drill.common.JSONOptions;
+import org.apache.drill.common.PlanProperties;
+import org.apache.drill.common.config.DrillConfig;
+import org.apache.drill.common.logical.LogicalPlan;
+import org.apache.drill.common.logical.data.*;
+import org.apache.drill.common.logical.data.visitors.AbstractLogicalVisitor;
+import org.apache.drill.common.logical.data.visitors.LogicalVisitor;
+import org.apache.drill.exec.exception.OptimizerException;
+import org.apache.drill.exec.ops.QueryContext;
+import org.apache.drill.exec.physical.PhysicalPlan;
+import org.apache.drill.exec.physical.base.PhysicalOperator;
+import org.apache.drill.exec.physical.config.MockScanPOP;
+import org.apache.drill.exec.physical.config.MockStorePOP;
+import org.apache.drill.exec.physical.config.Screen;
+import org.apache.drill.exec.proto.SchemaDefProtos;
+import org.apache.drill.exec.server.DrillbitContext;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+
+/**
+ * Created with IntelliJ IDEA.
+ * User: jaltekruse
+ * Date: 6/11/13
+ * Time: 5:32 PM
+ * To change this template use File | Settings | File Templates.
+ */
+public class BasicOptimizer extends Optimizer{
+
+    private DrillConfig config;
+    private QueryContext context;
+
+    public BasicOptimizer(DrillConfig config, QueryContext context){
+        this.config = config;
+        this.context = context;
+    }
+
+    @Override
+    public void init(DrillConfig config) {
+
+    }
+
+    @Override
+    public PhysicalPlan optimize(OptimizationContext context, LogicalPlan plan) {
+        Object obj = new Object();
+        Collection<SinkOperator> roots = plan.getGraph().getRoots();
+        List<PhysicalOperator> physOps = new ArrayList<PhysicalOperator>(roots.size());
+        LogicalConverter converter = new LogicalConverter();
+        for ( SinkOperator op : roots){
+            try {
+                PhysicalOperator pop  = op.accept(converter, obj);
+                System.out.println(pop);
+                physOps.add(pop);
+            } catch (OptimizerException e) {
+                e.printStackTrace();  //To change body of catch statement use File | Settings | File Templates.
+            } catch (Throwable throwable) {
+                throwable.printStackTrace();  //To change body of catch statement use File | Settings | File Templates.
+            }
+        }
+
+        PlanProperties props = new PlanProperties();
+        props.type = PlanProperties.PlanType.APACHE_DRILL_PHYSICAL;
+        props.version = plan.getProperties().version;
+        props.generator = plan.getProperties().generator;
+        return new PhysicalPlan(props, physOps);
+    }
+
+    @Override
+    public void close() {
+
+    }
+
+    public static class BasicOptimizationContext implements OptimizationContext {
+
+        @Override
+        public int getPriority() {
+            return 1;
+        }
+    }
+
+
+    private class LogicalConverter extends AbstractLogicalVisitor<PhysicalOperator, Object, OptimizerException> {
+
+        @Override
+        public MockScanPOP visitScan(Scan scan, Object obj) throws OptimizerException {
+            List<MockScanPOP.MockScanEntry> myObjects;
+
+            try {
+                if ( scan.getStorageEngine().equals("local-logs")){
+                    myObjects = scan.getSelection().getListWith(config,
+                            new TypeReference<ArrayList<MockScanPOP.MockScanEntry>>() {
+                    });
+                }
+                else{
+                    myObjects = new ArrayList<>();
+                    MockScanPOP.MockColumn[] cols = { new MockScanPOP.MockColumn("blah", SchemaDefProtos.MinorType.INT, SchemaDefProtos.DataMode.REQUIRED,4,4,4),
+                            new MockScanPOP.MockColumn("blah_2", SchemaDefProtos.MinorType.INT, SchemaDefProtos.DataMode.REQUIRED,4,4,4)};
+                    myObjects.add(new MockScanPOP.MockScanEntry(50, cols));
+                }
+            } catch (IOException e) {
+                e.printStackTrace();
+                throw new OptimizerException("Error reading selection attribute of Scan node in Logical to Physical plan conversion.");
+            }
+
+            return new MockScanPOP("http://apache.org", myObjects);
+        }
+
+        @Override
+        public Screen visitStore(Store store, Object obj) throws OptimizerException {
+            if ( ! store.iterator().hasNext()){
+                throw new OptimizerException("Store node in logical plan does not have a child.");
+            }
+            return new Screen(store.iterator().next().accept(this, obj), context.getCurrentEndpoint());
+        }
+
+        @Override
+        public PhysicalOperator visitProject(Project project, Object obj) throws OptimizerException {
+            return project.getInput().accept(this, obj);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/1be6bd48/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractStore.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractStore.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractStore.java
index a833a4e..2f25d99 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractStore.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractStore.java
@@ -20,7 +20,7 @@ package org.apache.drill.exec.physical.base;
 
 
 
-public abstract class AbstractStore extends AbstractSingle implements Store{
+public abstract class AbstractStore extends AbstractSingle implements Store, Root{
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(AbstractStore.class);
 
   public AbstractStore(PhysicalOperator child) {

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/1be6bd48/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/Screen.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/Screen.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/Screen.java
index 688c6b5..b51f7a3 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/Screen.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/Screen.java
@@ -38,7 +38,7 @@ import com.fasterxml.jackson.annotation.JsonTypeName;
 import com.google.common.base.Preconditions;
 
 @JsonTypeName("screen")
-public class Screen extends AbstractStore implements Root{
+public class Screen extends AbstractStore {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(Screen.class);
 
   private final DrillbitEndpoint endpoint;

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/1be6bd48/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 f86c4fb..ba103ed 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
@@ -21,10 +21,12 @@ import java.io.Closeable;
 import java.io.IOException;
 import java.util.List;
 
+import org.apache.drill.common.config.DrillConfig;
 import org.apache.drill.common.exceptions.ExecutionSetupException;
 import org.apache.drill.common.logical.LogicalPlan;
 import org.apache.drill.exec.exception.FragmentSetupException;
 import org.apache.drill.exec.ops.QueryContext;
+import org.apache.drill.exec.opt.BasicOptimizer;
 import org.apache.drill.exec.physical.PhysicalPlan;
 import org.apache.drill.exec.physical.base.PhysicalOperator;
 import org.apache.drill.exec.physical.impl.materialize.QueryWritableBatch;
@@ -33,7 +35,6 @@ import org.apache.drill.exec.planner.fragment.MakeFragmentsVisitor;
 import org.apache.drill.exec.planner.fragment.PlanningSet;
 import org.apache.drill.exec.planner.fragment.SimpleParallelizer;
 import org.apache.drill.exec.planner.fragment.StatsCollector;
-import org.apache.drill.exec.proto.ExecProtos.FragmentHandle;
 import org.apache.drill.exec.proto.ExecProtos.PlanFragment;
 import org.apache.drill.exec.proto.GeneralRPCProtos.Ack;
 import org.apache.drill.exec.proto.UserBitShared.DrillPBError;
@@ -225,7 +226,7 @@ public class Foreman implements Runnable, Closeable, Comparable<Object>{
   }
 
   private PhysicalPlan convert(LogicalPlan plan) {
-    throw new UnsupportedOperationException();
+    return new BasicOptimizer(DrillConfig.create(), context).optimize(new BasicOptimizer.BasicOptimizationContext(), plan);
   }
 
   public QueryResult getResult(UserClientConnection connection, RequestResults req) {

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/1be6bd48/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/client/DrillClientSystemTest.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/client/DrillClientSystemTest.java b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/client/DrillClientSystemTest.java
index dc463e3..bcd13e9 100644
--- a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/client/DrillClientSystemTest.java
+++ b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/client/DrillClientSystemTest.java
@@ -21,8 +21,10 @@ public class DrillClientSystemTest extends DrillSystemTestBase {
 
   @BeforeClass
   public static void setUp() throws Exception {
+
     DrillSystemTestBase.setUp();
     plan = Resources.toString(Resources.getResource("simple_plan.json"), Charsets.UTF_8);
+
   }
 
   @After

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/1be6bd48/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/opt/BasicOptimizerTest.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/opt/BasicOptimizerTest.java b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/opt/BasicOptimizerTest.java
new file mode 100644
index 0000000..30602e6
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/opt/BasicOptimizerTest.java
@@ -0,0 +1,24 @@
+package org.apache.drill.exec.opt;
+
+import org.apache.drill.common.config.DrillConfig;
+import org.apache.drill.common.logical.LogicalPlan;
+import org.apache.drill.common.util.FileUtils;
+import org.junit.Test;
+
+/**
+ * Created with IntelliJ IDEA.
+ * User: jaltekruse
+ * Date: 6/12/13
+ * Time: 12:09 PM
+ * To change this template use File | Settings | File Templates.
+ */
+public class BasicOptimizerTest {
+
+    @Test
+    public void parseSimplePlan() throws Exception{
+        DrillConfig c = DrillConfig.create();
+        LogicalPlan plan = LogicalPlan.parse(c, FileUtils.getResourceAsString("/scan_screen_logical.json"));
+        System.out.println(plan.unparse(c));
+        //System.out.println( new BasicOptimizer(DrillConfig.create()).convert(plan).unparse(c.getMapper().writer()));
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/1be6bd48/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestDistributedFragmentRun.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestDistributedFragmentRun.java b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestDistributedFragmentRun.java
index 9bcd84f..1513c99 100644
--- a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestDistributedFragmentRun.java
+++ b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestDistributedFragmentRun.java
@@ -75,6 +75,24 @@ public class TestDistributedFragmentRun extends PopUnitTestBase{
 
   }
 
+    @Test
+    public void oneBitOneExchangeTwoEntryRunLogical() throws Exception{
+        RemoteServiceSet serviceSet = RemoteServiceSet.getLocalServiceSet();
+
+        try(Drillbit bit1 = new Drillbit(CONFIG, serviceSet); DrillClient client = new DrillClient(CONFIG, serviceSet.getCoordinator());){
+            bit1.run();
+            client.connect();
+            List<QueryResultBatch> results = client.runQuery(QueryType.LOGICAL, Files.toString(FileUtils.getResourceAsFile("/scan_screen_logical.json"), Charsets.UTF_8));
+            int count = 0;
+            for(QueryResultBatch b : results){
+                count += b.getHeader().getRowCount();
+            }
+            assertEquals(100, count);
+        }
+
+
+    }
+
   @Test
     public void twoBitOneExchangeTwoEntryRun() throws Exception{
       RemoteServiceSet serviceSet = RemoteServiceSet.getLocalServiceSet();

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/1be6bd48/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 9f9d502..594a3a2 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
@@ -42,59 +42,61 @@ public class TestSimpleFragmentRun extends PopUnitTestBase {
   @Test
   public void runNoExchangeFragment() throws Exception {
     try(RemoteServiceSet serviceSet = RemoteServiceSet.getLocalServiceSet(); 
-        Drillbit bit = new Drillbit(CONFIG, serviceSet); 
-        DrillClient client = new DrillClient(CONFIG, serviceSet.getCoordinator());){
+            Drillbit bit = new Drillbit(CONFIG, serviceSet);
+            DrillClient client = new DrillClient(CONFIG, serviceSet.getCoordinator());){
     
-    // run query.
-    bit.run();
-    client.connect();
-    List<QueryResultBatch> results = client.runQuery(QueryType.PHYSICAL, Files.toString(FileUtils.getResourceAsFile("/physical_test2.json"), Charsets.UTF_8));
-    
-    // look at records
-    RecordBatchLoader batchLoader = new RecordBatchLoader(bit.getContext().getAllocator());
-    int recordCount = 0;
-    for (QueryResultBatch batch : results) {
-      if(!batch.hasData()) continue;
-      boolean schemaChanged = batchLoader.load(batch.getHeader().getDef(), batch.getData());
-      boolean firstColumn = true;
+        // run query.
+        bit.run();
+        client.connect();
+        List<QueryResultBatch> results = client.runQuery(QueryType.PHYSICAL, Files.toString(FileUtils.getResourceAsFile("/physical_test2.json"), Charsets.UTF_8));
+
+        // look at records
+        RecordBatchLoader batchLoader = new RecordBatchLoader(bit.getContext().getAllocator());
+        int recordCount = 0;
+        for (QueryResultBatch batch : results) {
+          if(!batch.hasData()) continue;
+          boolean schemaChanged = batchLoader.load(batch.getHeader().getDef(), batch.getData());
+          boolean firstColumn = true;
 
-      // print headers.
-      if (schemaChanged) {
-        System.out.println("\n\n========NEW SCHEMA=========\n\n");
-        for (IntObjectCursor<ValueVector<?>> v : batchLoader) {
+          // print headers.
+          if (schemaChanged) {
+            System.out.println("\n\n========NEW SCHEMA=========\n\n");
+            for (IntObjectCursor<ValueVector<?>> v : batchLoader) {
 
-          if (firstColumn) {
-            firstColumn = false;
-          } else {
-            System.out.print("\t");
+              if (firstColumn) {
+                firstColumn = false;
+              } else {
+                System.out.print("\t");
+              }
+              System.out.print(v.value.getField().getName());
+              System.out.print("[");
+              System.out.print(v.value.getField().getType().getMinorType());
+              System.out.print("]");
+            }
+            System.out.println();
           }
-          System.out.print(v.value.getField().getName());
-          System.out.print("[");
-          System.out.print(v.value.getField().getType().getMinorType());
-          System.out.print("]");
-        }
-        System.out.println();
-      }
 
 
-      for (int i = 0; i < batchLoader.getRecordCount(); i++) {
-        boolean first = true;
-        recordCount++;
-        for (IntObjectCursor<ValueVector<?>> v : batchLoader) {
-          if (first) {
-            first = false;
-          } else {
-            System.out.print("\t");
+          for (int i = 0; i < batchLoader.getRecordCount(); i++) {
+            boolean first = true;
+            recordCount++;
+            for (IntObjectCursor<ValueVector<?>> v : batchLoader) {
+              if (first) {
+                first = false;
+              } else {
+                System.out.print("\t");
+              }
+              System.out.print(v.value.getObject(i));
+            }
+            if(!first) System.out.println();
           }
-          System.out.print(v.value.getObject(i));
-        }
-        if(!first) System.out.println();
-      }
 
-    }
-    logger.debug("Received results {}", results);
-    assertEquals(recordCount, 200);
-    }
+        }
+        logger.debug("Received results {}", results);
+        assertEquals(recordCount, 200);
+        }
   }
 
+
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/1be6bd48/sandbox/prototype/exec/java-exec/src/test/resources/scan_screen_logical.json
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/resources/scan_screen_logical.json b/sandbox/prototype/exec/java-exec/src/test/resources/scan_screen_logical.json
new file mode 100644
index 0000000..3e5ebd7
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/test/resources/scan_screen_logical.json
@@ -0,0 +1,46 @@
+{
+  head:{
+    type:"APACHE_DRILL_LOGICAL",
+    version:"1",
+    generator:{
+      type:"manual",
+      info:"na"
+    }
+  },
+  query:[
+    {
+      @id:"1",
+      op:"scan",
+      memo:"initial_scan",
+      storageengine:"local-logs",
+      selection: [
+       {
+            records : 100,
+            types : [
+
+                    {
+                        name : "superhero_name",
+                        type : "VARCHAR1",
+                        mode : "REQUIRED",
+                        width : 100,
+                        precision : 1,
+                        scale : 2
+
+                    }
+             ]
+         }
+      ]
+    },
+    {
+      @id:"2",
+      input: 1,
+      op: "store",
+      memo: "output sink",
+      target: {
+        file: "console:///stdout"
+      }
+      
+    }
+    
+  ]
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/1be6bd48/sandbox/prototype/exec/ref/src/main/java/org/apache/drill/exec/ref/RecordPointer.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/ref/src/main/java/org/apache/drill/exec/ref/RecordPointer.java b/sandbox/prototype/exec/ref/src/main/java/org/apache/drill/exec/ref/RecordPointer.java
index b7c523b..56283b2 100644
--- a/sandbox/prototype/exec/ref/src/main/java/org/apache/drill/exec/ref/RecordPointer.java
+++ b/sandbox/prototype/exec/ref/src/main/java/org/apache/drill/exec/ref/RecordPointer.java
@@ -1,3 +1,21 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+
 package org.apache.drill.exec.ref;
 
 import java.io.IOException;


[10/53] [abbrv] Update typing system. Update RPC system. Add Fragmenting Implementation. Working single node. Distributed failing due to threading issues.

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/BitConnection.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/BitConnection.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/BitConnection.java
new file mode 100644
index 0000000..73980f9
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/BitConnection.java
@@ -0,0 +1,168 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.rpc.bit;
+
+import io.netty.channel.Channel;
+import io.netty.channel.ChannelFuture;
+import io.netty.util.concurrent.GenericFutureListener;
+
+import java.util.UUID;
+import java.util.concurrent.ConcurrentMap;
+
+import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
+import org.apache.drill.exec.proto.ExecProtos.FragmentHandle;
+import org.apache.drill.exec.proto.ExecProtos.FragmentStatus;
+import org.apache.drill.exec.proto.ExecProtos.PlanFragment;
+import org.apache.drill.exec.proto.ExecProtos.RpcType;
+import org.apache.drill.exec.proto.GeneralRPCProtos.Ack;
+import org.apache.drill.exec.record.FragmentWritableBatch;
+import org.apache.drill.exec.rpc.DrillRpcFuture;
+import org.apache.drill.exec.rpc.RemoteConnection;
+import org.apache.drill.exec.rpc.RpcBus;
+
+import com.google.common.base.Preconditions;
+import com.google.common.io.Closeables;
+
+public class BitConnection extends RemoteConnection{
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(BitConnection.class); 
+  
+  private final RpcBus<RpcType, BitConnection> bus;
+  private final ConcurrentMap<DrillbitEndpoint, BitConnection> registry;
+  private final ListenerPool listeners;
+
+  private final AvailabilityListener listener;
+  private volatile DrillbitEndpoint endpoint;
+  private volatile boolean active = false;
+  private final UUID id;
+  
+  public BitConnection(AvailabilityListener listener, Channel channel, RpcBus<RpcType, BitConnection> bus, ConcurrentMap<DrillbitEndpoint, BitConnection> registry, ListenerPool listeners){
+    super(channel);
+    this.bus = bus;
+    this.registry = registry;
+    // we use a local listener pool unless a global one is provided.
+    this.listeners = listeners != null ? listeners : new ListenerPool(2);
+    this.listener = listener;
+    this.id = UUID.randomUUID();
+  }
+
+  protected DrillbitEndpoint getEndpoint() {
+    return endpoint;
+  }
+
+  public ListenerPool getListenerPool(){
+    return listeners;
+  }
+  
+  protected void setEndpoint(DrillbitEndpoint endpoint) {
+    Preconditions.checkNotNull(endpoint);
+    Preconditions.checkArgument(this.endpoint == null);
+    
+    this.endpoint = endpoint;
+    BitServer.logger.debug("Adding new endpoint to available BitServer connections.  Endpoint: {}.", endpoint);
+    synchronized(this){
+      BitConnection c = registry.putIfAbsent(endpoint, this);
+      
+      if(c != null){ // the registry already has a connection like this
+        
+        // give the awaiting future an alternative connection.
+        if(listener != null){
+          listener.isAvailable(c);
+        }
+        
+        // shut this down if this is a client as it won't be available in the registry.
+        // otherwise we'll leave as, possibly allowing to bit coms to use different tunnels to talk to each other.  This shouldn't cause a problem.
+        logger.debug("Shutting down connection to {} since the registry already has an active connection that endpoint.", endpoint);
+        shutdownIfClient();
+        
+      }
+      active = true;
+      if(listener != null) listener.isAvailable(this);
+    }
+  }
+
+  public DrillRpcFuture<Ack> sendRecordBatch(FragmentContext context, FragmentWritableBatch batch){
+    return bus.send(this, RpcType.REQ_RECORD_BATCH, batch.getHeader(), Ack.class, batch.getBuffers());
+  }
+  
+  public DrillRpcFuture<Ack> sendFragment(PlanFragment fragment){
+    return bus.send(this, RpcType.REQ_INIATILIZE_FRAGMENT, fragment, Ack.class);
+  }
+  
+  public DrillRpcFuture<Ack> cancelFragment(FragmentHandle handle){
+    return bus.send(this,  RpcType.REQ_CANCEL_FRAGMENT, handle, Ack.class);
+  }
+  
+  public DrillRpcFuture<Ack> sendFragmentStatus(FragmentStatus status){
+    return bus.send(this,  RpcType.REQ_FRAGMENT_STATUS, status, Ack.class);
+  }
+
+  public void disable(){
+    active = false;
+  }
+  
+  public boolean isActive(){
+    return active;
+  }
+  
+  @Override
+  public int hashCode() {
+    final int prime = 31;
+    int result = 1;
+    result = prime * result + ((id == null) ? 0 : id.hashCode());
+    return result;
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (this == obj) return true;
+    if (obj == null) return false;
+    if (getClass() != obj.getClass()) return false;
+    BitConnection other = (BitConnection) obj;
+    if (id == null) {
+      if (other.id != null) return false;
+    } else if (!id.equals(other.id)) return false;
+    return true;
+  }
+
+  public GenericFutureListener<ChannelFuture> getCloseHandler(GenericFutureListener<ChannelFuture> parent){
+    return new CloseHandler(this, parent);
+  }
+  
+  private class CloseHandler implements GenericFutureListener<ChannelFuture>{
+    private BitConnection connection;
+    private GenericFutureListener<ChannelFuture> parent;
+    
+    public CloseHandler(BitConnection connection, GenericFutureListener<ChannelFuture> parent) {
+      super();
+      this.connection = connection;
+      this.parent = parent;
+    }
+
+    @Override
+    public void operationComplete(ChannelFuture future) throws Exception {
+      if(connection.getEndpoint() != null) registry.remove(connection.getEndpoint(), connection);
+      parent.operationComplete(future);
+    }
+    
+  }
+  
+  public void shutdownIfClient(){
+    if(bus.isClient()) Closeables.closeQuietly(bus);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/BitConnectionManager.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/BitConnectionManager.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/BitConnectionManager.java
new file mode 100644
index 0000000..0160d24
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/BitConnectionManager.java
@@ -0,0 +1,80 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.rpc.bit;
+
+import java.util.concurrent.atomic.AtomicReference;
+
+import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
+import org.apache.drill.exec.rpc.RpcException;
+
+import com.google.common.util.concurrent.CheckedFuture;
+
+public class BitConnectionManager {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(BitConnectionManager.class);
+  
+  private final int maxAttempts;
+  private final BitComImpl com;
+  private final DrillbitEndpoint endpoint;
+  private final AtomicReference<BitConnection> connection;
+  private final AtomicReference<CheckedFuture<BitConnection, RpcException>> future;
+
+  BitConnectionManager(DrillbitEndpoint endpoint, BitComImpl com, BitConnection connection, CheckedFuture<BitConnection, RpcException> future, int maxAttempts) {
+    assert endpoint != null && endpoint.getAddress() != null && endpoint.getBitPort() > 0;
+    this.com = com;
+    this.connection =  new AtomicReference<BitConnection>(connection);
+    this.future = new AtomicReference<CheckedFuture<BitConnection, RpcException>>(future);
+    this.endpoint = endpoint;
+    this.maxAttempts = maxAttempts;
+  }
+  
+  BitConnection getConnection(int attempt) throws RpcException{
+    BitConnection con = connection.get();
+    
+    if(con != null){
+      if(con.isActive()) return con;
+      connection.compareAndSet(con, null);
+    }
+    
+    CheckedFuture<BitConnection, RpcException> fut = future.get();
+
+    if(fut != null){
+      try{
+        return fut.checkedGet();
+      }catch(RpcException ex){
+        future.compareAndSet(fut, null);
+        if(attempt < maxAttempts){
+          return getConnection(attempt + 1);
+        }else{
+          throw ex;
+        }
+      }
+    }
+    
+    // no checked future, let's make one.
+    fut = com.getConnectionAsync(endpoint);
+    future.compareAndSet(null, fut);
+    return getConnection(attempt);
+    
+  }
+
+  public DrillbitEndpoint getEndpoint() {
+    return endpoint;
+  }
+  
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/BitRpcConfig.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/BitRpcConfig.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/BitRpcConfig.java
new file mode 100644
index 0000000..32fd4f9
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/BitRpcConfig.java
@@ -0,0 +1,46 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.rpc.bit;
+
+
+import org.apache.drill.exec.proto.ExecProtos.BitHandshake;
+import org.apache.drill.exec.proto.ExecProtos.FragmentHandle;
+import org.apache.drill.exec.proto.ExecProtos.FragmentRecordBatch;
+import org.apache.drill.exec.proto.ExecProtos.FragmentStatus;
+import org.apache.drill.exec.proto.ExecProtos.PlanFragment;
+import org.apache.drill.exec.proto.ExecProtos.RpcType;
+import org.apache.drill.exec.proto.GeneralRPCProtos.Ack;
+import org.apache.drill.exec.rpc.Acks;
+import org.apache.drill.exec.rpc.Response;
+import org.apache.drill.exec.rpc.RpcConfig;
+
+public class BitRpcConfig {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(BitRpcConfig.class);
+  
+  public static RpcConfig MAPPING = RpcConfig.newBuilder("BIT-RPC-MAPPING") //
+      .add(RpcType.HANDSHAKE, BitHandshake.class, RpcType.HANDSHAKE, BitHandshake.class)
+      .add(RpcType.REQ_INIATILIZE_FRAGMENT, PlanFragment.class, RpcType.ACK, Ack.class)
+      .add(RpcType.REQ_CANCEL_FRAGMENT, FragmentHandle.class, RpcType.ACK, Ack.class)
+      .add(RpcType.REQ_FRAGMENT_STATUS, FragmentStatus.class, RpcType.ACK, Ack.class)
+      .add(RpcType.REQ_RECORD_BATCH, FragmentRecordBatch.class, RpcType.ACK, Ack.class)
+      .build();
+  
+  public static int RPC_VERSION = 2;
+  
+  public static final Response OK = new Response(RpcType.ACK, Acks.OK);
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/BitServer.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/BitServer.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/BitServer.java
index e17b25c..88ac6cc 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/BitServer.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/BitServer.java
@@ -18,47 +18,76 @@
 package org.apache.drill.exec.rpc.bit;
 
 import io.netty.buffer.ByteBuf;
-import io.netty.buffer.ByteBufAllocator;
+import io.netty.channel.Channel;
 import io.netty.channel.ChannelFuture;
-import io.netty.channel.EventLoopGroup;
-import io.netty.channel.socket.SocketChannel;
 import io.netty.util.concurrent.GenericFutureListener;
 
+import java.util.concurrent.ConcurrentMap;
+
+import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
+import org.apache.drill.exec.proto.ExecProtos.BitHandshake;
+import org.apache.drill.exec.proto.ExecProtos.BitStatus;
+import org.apache.drill.exec.proto.ExecProtos.FragmentHandle;
+import org.apache.drill.exec.proto.ExecProtos.FragmentStatus;
 import org.apache.drill.exec.proto.ExecProtos.RpcType;
+import org.apache.drill.exec.proto.GeneralRPCProtos.Ack;
 import org.apache.drill.exec.rpc.BasicServer;
 import org.apache.drill.exec.rpc.Response;
 import org.apache.drill.exec.rpc.RpcException;
-import org.apache.drill.exec.server.DrillbitContext;
+import org.apache.drill.exec.server.BootStrapContext;
+import org.apache.drill.exec.work.batch.BitComHandler;
 
 import com.google.protobuf.MessageLite;
 
-public class BitServer extends BasicServer<RpcType>{
+public class BitServer extends BasicServer<RpcType, BitConnection>{
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(BitServer.class);
   
-  private final DrillbitContext context;
   private final BitComHandler handler;
+  private final ConcurrentMap<DrillbitEndpoint, BitConnection> registry;
+  private final ListenerPool listeners;
   
-  public BitServer(BitComHandler handler, ByteBufAllocator alloc, EventLoopGroup eventLoopGroup, DrillbitContext context) {
-    super(alloc, eventLoopGroup);
-    this.context = context;
+  public BitServer(BitComHandler handler, BootStrapContext context, ConcurrentMap<DrillbitEndpoint, BitConnection> registry, ListenerPool listeners) {
+    super(BitRpcConfig.MAPPING, context.getAllocator().getUnderlyingAllocator(), context.getBitLoopGroup());
     this.handler = handler;
+    this.registry = registry;
+    this.listeners = listeners;
   }
   
   @Override
-  protected MessageLite getResponseDefaultInstance(int rpcType) throws RpcException {
-    return handler.getResponseDefaultInstance(rpcType);
+  public MessageLite getResponseDefaultInstance(int rpcType) throws RpcException {
+    return BitComDefaultInstanceHandler.getResponseDefaultInstance(rpcType);
   }
 
   @Override
-  protected Response handle(SocketChannel ch, int rpcType, ByteBuf pBody, ByteBuf dBody) throws RpcException {
-    return handler.handle(context, rpcType, pBody, dBody);
+  protected Response handle(BitConnection connection, int rpcType, ByteBuf pBody, ByteBuf dBody) throws RpcException {
+    return handler.handle(connection, rpcType, pBody, dBody);
   }
 
   @Override
-  protected GenericFutureListener<ChannelFuture> getCloseHandler(SocketChannel ch) {
-    
-    return super.getCloseHandler(ch);
+  protected GenericFutureListener<ChannelFuture> getCloseHandler(BitConnection connection) {
+    return connection.getCloseHandler(super.getCloseHandler(connection));
+  }
+
+  @Override
+  public BitConnection initRemoteConnection(Channel channel) {
+    return new BitConnection(null, channel, this, registry, listeners);
+  }
+  
+  
+  @Override
+  protected ServerHandshakeHandler<BitHandshake> getHandshakeHandler() {
+    return new ServerHandshakeHandler<BitHandshake>(RpcType.HANDSHAKE, BitHandshake.PARSER){
+      
+      @Override
+      public MessageLite getHandshakeResponse(BitHandshake inbound) throws Exception {
+        logger.debug("Handling handshake from other bit. {}", inbound);
+        if(inbound.getRpcVersion() != BitRpcConfig.RPC_VERSION) throw new RpcException(String.format("Invalid rpc version.  Expected %d, actual %d.", inbound.getRpcVersion(), BitRpcConfig.RPC_VERSION));
+        return BitHandshake.newBuilder().setRpcVersion(BitRpcConfig.RPC_VERSION).build();
+      }
+
+    };
   }
 
+
   
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/BitTunnel.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/BitTunnel.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/BitTunnel.java
index 02991ad..652fa52 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/BitTunnel.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/BitTunnel.java
@@ -17,47 +17,222 @@
  ******************************************************************************/
 package org.apache.drill.exec.rpc.bit;
 
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Executor;
+
 import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
 import org.apache.drill.exec.proto.ExecProtos.FragmentHandle;
 import org.apache.drill.exec.proto.ExecProtos.FragmentStatus;
 import org.apache.drill.exec.proto.ExecProtos.PlanFragment;
 import org.apache.drill.exec.proto.GeneralRPCProtos.Ack;
-import org.apache.drill.exec.record.RecordBatch;
+import org.apache.drill.exec.record.FragmentWritableBatch;
+import org.apache.drill.exec.record.WritableBatch;
 import org.apache.drill.exec.rpc.DrillRpcFuture;
-import org.apache.drill.exec.rpc.RpcBus;
+import org.apache.drill.exec.rpc.RpcException;
+import org.apache.drill.exec.rpc.RpcOutcomeListener;
 
-import com.google.common.io.Closeables;
+import com.google.common.util.concurrent.AbstractCheckedFuture;
+import com.google.common.util.concurrent.CheckedFuture;
+import com.google.common.util.concurrent.MoreExecutors;
+import com.google.common.util.concurrent.SettableFuture;
 
 /**
- * Interface provided for communication between two bits.  Provided by both a server and a client implementation.
+ * Interface provided for communication between two bits. Underlying connection may be server or client based. Resilient
+ * to connection loss. Right now, this has to jump through some hoops and bridge futures between the connection creation
+ * and action. A better approach should be done.
  */
 public class BitTunnel {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(BitTunnel.class);
 
-  final RpcBus<?> bus;
+  private static final int MAX_ATTEMPTS = 3;
 
-  public BitTunnel(RpcBus<?> bus){
-    this.bus = bus;
-  }
+  private final BitConnectionManager manager;
+  private final Executor exec;
   
-  public DrillRpcFuture<Ack> sendRecordBatch(FragmentContext context, RecordBatch batch){
-    return null;
+
+  public BitTunnel(Executor exec, DrillbitEndpoint endpoint, BitComImpl com, BitConnection connection) {
+    this.manager = new BitConnectionManager(endpoint, com, connection, null, MAX_ATTEMPTS);
+    this.exec = exec;
   }
-  
-  public DrillRpcFuture<FragmentHandle> sendFragment(FragmentContext context, PlanFragment fragment){
-    return null;
+
+  public BitTunnel(Executor exec, DrillbitEndpoint endpoint, BitComImpl com,
+      CheckedFuture<BitConnection, RpcException> future) {
+    this.manager = new BitConnectionManager(endpoint, com, (BitConnection) null, future, MAX_ATTEMPTS);
+    this.exec = exec;
   }
   
-  public DrillRpcFuture<Ack> cancelFragment(FragmentContext context, FragmentHandle handle){
-    return null;
+  public DrillbitEndpoint getEndpoint(){
+    return manager.getEndpoint();
   }
-  
-  public DrillRpcFuture<FragmentStatus> getFragmentStatus(FragmentContext context, FragmentHandle handle){
-    return null;
+
+  private <T> DrillRpcFuture<T> submit(BitCommand<T> command) {
+    exec.execute(command);
+    return command;
+  }
+
+  public DrillRpcFuture<Ack> sendRecordBatch(FragmentContext context, FragmentWritableBatch batch) {
+    return submit(new SendBatch(batch, context));
+  }
+
+  public DrillRpcFuture<Ack> sendFragment(PlanFragment fragment) {
+    return submit(new SendFragment(fragment));
+  }
+
+  public DrillRpcFuture<Ack> cancelFragment(FragmentHandle handle) {
+    return submit(new CancelFragment(handle));
   }
+
+  public DrillRpcFuture<Ack> sendFragmentStatus(FragmentStatus status){
+    return submit(new SendFragmentStatus(status));
+  }
+
+  public class SendBatch extends BitCommand<Ack> {
+    final FragmentWritableBatch batch;
+    final FragmentContext context;
+
+    public SendBatch(FragmentWritableBatch batch, FragmentContext context) {
+      super();
+      this.batch = batch;
+      this.context = context;
+    }
+
+    @Override
+    public CheckedFuture<Ack, RpcException> doRpcCall(BitConnection connection) {
+      logger.debug("Sending record batch. {}", batch);
+      return connection.sendRecordBatch(context, batch);
+    }
+
+  }
+
+  public class SendFragmentStatus extends BitCommand<Ack> {
+    final FragmentStatus status;
+
+    public SendFragmentStatus(FragmentStatus status) {
+      super();
+      this.status = status;
+    }
+
+    @Override
+    public CheckedFuture<Ack, RpcException> doRpcCall(BitConnection connection) {
+      return connection.sendFragmentStatus(status);
+    }
+  }
+
+  public class CancelFragment extends BitCommand<Ack> {
+    final FragmentHandle handle;
+
+    public CancelFragment(FragmentHandle handle) {
+      super();
+      this.handle = handle;
+    }
+
+    @Override
+    public CheckedFuture<Ack, RpcException> doRpcCall(BitConnection connection) {
+      return connection.cancelFragment(handle);
+    }
+
+  }
+
+  public class SendFragment extends BitCommand<Ack> {
+    final PlanFragment fragment;
+
+    public SendFragment(PlanFragment fragment) {
+      super();
+      this.fragment = fragment;
+    }
+
+    @Override
+    public CheckedFuture<Ack, RpcException> doRpcCall(BitConnection connection) {
+      return connection.sendFragment(fragment);
+    }
+
+  }
+
+
   
-  public void shutdownIfClient(){
-    if(bus.isClient()) Closeables.closeQuietly(bus);
+
+  private abstract class BitCommand<T> extends AbstractCheckedFuture<T, RpcException> implements Runnable, DrillRpcFuture<T> {
+
+    public void addLightListener(RpcOutcomeListener<T> outcomeListener){
+      this.addListener(new RpcOutcomeListenerWrapper(outcomeListener), MoreExecutors.sameThreadExecutor());
+    }
+
+    public BitCommand() {
+      super(SettableFuture.<T> create());
+    }
+
+    public abstract CheckedFuture<T, RpcException> doRpcCall(BitConnection connection);
+
+    public final void run() {
+      
+      try {
+        
+        BitConnection connection = manager.getConnection(0);
+        assert connection != null : "The connection manager should never return a null connection.  Worse case, it should throw an exception.";
+        CheckedFuture<T, RpcException> rpc = doRpcCall(connection);
+        rpc.addListener(new FutureBridge<T>((SettableFuture<T>) delegate(), rpc), MoreExecutors.sameThreadExecutor());
+      } catch (RpcException ex) {
+        ((SettableFuture<T>) delegate()).setException(ex);
+      }
+
+    }
+
+    @Override
+    protected RpcException mapException(Exception e) {
+      Throwable t = e;
+      if (e instanceof ExecutionException) {
+        t = e.getCause();
+      }
+      if (t instanceof RpcException) return (RpcException) t;
+      return new RpcException(t);
+    }
+
+    public class RpcOutcomeListenerWrapper implements Runnable{
+      final RpcOutcomeListener<T> inner;
+      
+      public RpcOutcomeListenerWrapper(RpcOutcomeListener<T> inner) {
+        this.inner = inner;
+      }
+
+      @Override
+      public void run() {
+        try{
+          inner.success(BitCommand.this.checkedGet());
+        }catch(RpcException e){
+          inner.failed(e);
+        }
+      }
+    }
+
+    @Override
+    public String toString() {
+      return "BitCommand ["+this.getClass().getSimpleName()+"]";
+    }
+    
+    
+    
+  }
+
+  private class FutureBridge<T> implements Runnable {
+    final SettableFuture<T> out;
+    final CheckedFuture<T, RpcException> in;
+
+    public FutureBridge(SettableFuture<T> out, CheckedFuture<T, RpcException> in) {
+      super();
+      this.out = out;
+      this.in = in;
+    }
+
+    @Override
+    public void run() {
+      try {
+        out.set(in.checkedGet());
+      } catch (RpcException ex) {
+        out.setException(ex);
+      }
+    }
+
   }
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/ListenerPool.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/ListenerPool.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/ListenerPool.java
new file mode 100644
index 0000000..8f299d2
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/ListenerPool.java
@@ -0,0 +1,56 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.rpc.bit;
+
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+
+import org.apache.drill.exec.proto.ExecProtos.FragmentHandle;
+import org.apache.drill.exec.proto.ExecProtos.FragmentStatus;
+import org.apache.drill.exec.rpc.RpcException;
+import org.apache.drill.exec.work.foreman.FragmentStatusListener;
+import org.apache.drill.exec.work.fragment.RemoteFragmentHandler;
+
+public class ListenerPool {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ListenerPool.class);
+  
+  private final ConcurrentMap<FragmentHandle, FragmentStatusListener> listeners;
+  
+  public ListenerPool(int par){
+    listeners = new ConcurrentHashMap<FragmentHandle, FragmentStatusListener>(16, 0.75f, par);
+  }
+  
+  public void removeFragmentStatusListener(FragmentHandle handle) throws RpcException{
+    listeners.remove(handle);
+  }
+  
+  public void addFragmentStatusListener(FragmentHandle handle, FragmentStatusListener listener) throws RpcException{
+    FragmentStatusListener old = listeners.putIfAbsent(handle, listener);
+    if(old != null) throw new RpcException("Failure.  The provided handle already exists in the listener pool.  You need to remove one listener before adding another.");
+  }
+  
+  public void status(FragmentStatus status){
+    FragmentStatusListener l = listeners.get(status.getHandle());
+    if(l == null){
+      logger.info("A fragment message arrived but there was no registered listener for that message.");
+      return;
+    }else{
+      l.statusUpdate(status);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/QueryResultBatch.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/QueryResultBatch.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/QueryResultBatch.java
new file mode 100644
index 0000000..3df88b7
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/QueryResultBatch.java
@@ -0,0 +1,49 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.rpc.user;
+
+import io.netty.buffer.ByteBuf;
+
+import org.apache.drill.exec.proto.UserProtos.QueryResult;
+
+public class QueryResultBatch {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(QueryResultBatch.class);
+  
+  private final QueryResult header;
+  private final ByteBuf data;
+  
+  public QueryResultBatch(QueryResult header, ByteBuf data) {
+//    logger.debug("New Result Batch with header {} and data {}", header, data);
+    this.header = header;
+    this.data = data;
+  }
+
+  public QueryResult getHeader() {
+    return header;
+  }
+
+  public ByteBuf getData() {
+    return data;
+  }
+  
+  
+  public boolean hasData(){
+    return data != null;
+  }
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserClient.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserClient.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserClient.java
index 0088522..5d2e799 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserClient.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserClient.java
@@ -20,57 +20,194 @@ package org.apache.drill.exec.rpc.user;
 import io.netty.buffer.ByteBuf;
 import io.netty.buffer.ByteBufAllocator;
 import io.netty.channel.EventLoopGroup;
-import io.netty.channel.socket.SocketChannel;
 
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.Future;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+
+import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
 import org.apache.drill.exec.proto.GeneralRPCProtos.Ack;
+import org.apache.drill.exec.proto.UserBitShared.QueryId;
 import org.apache.drill.exec.proto.UserProtos.BitToUserHandshake;
-import org.apache.drill.exec.proto.UserProtos.QueryHandle;
 import org.apache.drill.exec.proto.UserProtos.QueryResult;
 import org.apache.drill.exec.proto.UserProtos.RpcType;
 import org.apache.drill.exec.proto.UserProtos.RunQuery;
-import org.apache.drill.exec.rpc.BasicClient;
-import org.apache.drill.exec.rpc.DrillRpcFuture;
+import org.apache.drill.exec.proto.UserProtos.UserToBitHandshake;
+import org.apache.drill.exec.rpc.BasicClientWithConnection;
 import org.apache.drill.exec.rpc.Response;
 import org.apache.drill.exec.rpc.RpcException;
+import org.apache.drill.exec.rpc.RpcOutcomeListener;
 
+import com.google.common.collect.Maps;
+import com.google.common.collect.Queues;
 import com.google.protobuf.MessageLite;
 
-public class UserClient extends BasicClient<RpcType> {
+public class UserClient extends BasicClientWithConnection<RpcType> {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(UserClient.class);
-  
+
+  private ConcurrentMap<QueryId, UserResultsListener> resultsListener = Maps.newConcurrentMap();
+
   public UserClient(ByteBufAllocator alloc, EventLoopGroup eventLoopGroup) {
-    super(alloc, eventLoopGroup);
+    super(UserRpcConfig.MAPPING, alloc, eventLoopGroup);
   }
 
+  public Future<Void> submitQuery(RunQuery query, UserResultsListener resultsListener) throws RpcException {
+    this.send(RpcType.RUN_QUERY, query, QueryId.class).addLightListener(new SubmissionListener(resultsListener));
+    return resultsListener.getFuture();
+  }
 
-  public DrillRpcFuture<QueryHandle> submitQuery(RunQuery query) throws RpcException {
-    return this.send(RpcType.RUN_QUERY, query, QueryHandle.class, null);
+  public BitToUserHandshake connect(DrillbitEndpoint endpoint) throws RpcException, InterruptedException{
+    return this.connectAsClient(RpcType.HANDSHAKE, UserToBitHandshake.newBuilder().setRpcVersion(UserRpcConfig.RPC_VERSION).build(), endpoint.getAddress(), endpoint.getUserPort(), BitToUserHandshake.class);
   }
   
-  public DrillRpcFuture<QueryHandle> submitQuery(RunQuery query, ByteBuf data) throws RpcException {
-    return this.send(RpcType.RUN_QUERY, query, QueryHandle.class, data);
+  private class BufferingListener extends UserResultsListener {
+
+    private ConcurrentLinkedQueue<QueryResultBatch> results = Queues.newConcurrentLinkedQueue();
+    private ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
+    private volatile UserResultsListener output;
+
+    public boolean transferTo(UserResultsListener l) {
+      lock.writeLock().lock();
+      output = l;
+      boolean last = false;
+      for (QueryResultBatch r : results) {
+        l.resultArrived(r);
+        last = r.getHeader().getIsLastChunk();
+      }
+      if (future.isDone()) {
+        l.set();
+      }
+      return last;
+    }
+
+    @Override
+    public void resultArrived(QueryResultBatch result) {
+      logger.debug("Result arrvied.");
+      lock.readLock().lock();
+      try {
+        if (output == null) {
+          this.results.add(result);
+        } else {
+          output.resultArrived(result);
+        }
+
+      } finally {
+        lock.readLock().unlock();
+      }
+
+    }
+
+    @Override
+    public void submissionFailed(RpcException ex) {
+      throw new UnsupportedOperationException("You cannot report failed submissions to a buffering listener.");
+    }
+
+  }
+
+  private class SubmissionListener extends RpcOutcomeListener<QueryId> {
+    private UserResultsListener listener;
+
+    public SubmissionListener(UserResultsListener listener) {
+      super();
+      this.listener = listener;
+    }
+
+    @Override
+    public void failed(RpcException ex) {
+      listener.submissionFailed(ex);
+    }
+
+    @Override
+    public void success(QueryId queryId) {
+      logger.debug("Received QueryId {} succesfully.  Adding listener {}", queryId, listener);
+      UserResultsListener oldListener = resultsListener.putIfAbsent(queryId, listener);
+
+      // we need to deal with the situation where we already received results by the time we got the query id back. In
+      // that case, we'll need to transfer the buffering listener over, grabbing a lock against reception of additional
+      // results during the transition
+      if (oldListener != null) {
+        logger.debug("Unable to place user results listener, buffering listener was already in place.");
+        if (oldListener instanceof BufferingListener) {
+          resultsListener.remove(oldListener);
+          boolean all = ((BufferingListener) oldListener).transferTo(this.listener);
+          // simply remove the buffering listener if we already have the last response.
+          if (all) {
+            resultsListener.remove(oldListener);
+          } else {
+            boolean replaced = resultsListener.replace(queryId, oldListener, listener);
+            if (!replaced) throw new IllegalStateException();
+          }
+        } else {
+          throw new IllegalStateException("Trying to replace a non-buffering User Results listener.");
+        }
+      }
+
+    }
+
   }
 
   @Override
   protected MessageLite getResponseDefaultInstance(int rpcType) throws RpcException {
-    switch(rpcType){
+    switch (rpcType) {
     case RpcType.ACK_VALUE:
       return Ack.getDefaultInstance();
     case RpcType.HANDSHAKE_VALUE:
       return BitToUserHandshake.getDefaultInstance();
     case RpcType.QUERY_HANDLE_VALUE:
-      return QueryHandle.getDefaultInstance();
+      return QueryId.getDefaultInstance();
     case RpcType.QUERY_RESULT_VALUE:
       return QueryResult.getDefaultInstance();
     }
     throw new RpcException(String.format("Unable to deal with RpcType of %d", rpcType));
   }
 
+  protected Response handle(int rpcType, ByteBuf pBody, ByteBuf dBody) throws RpcException {
+    switch (rpcType) {
+    case RpcType.QUERY_RESULT_VALUE:
+      final QueryResult result = get(pBody, QueryResult.PARSER);
+      final QueryResultBatch batch = new QueryResultBatch(result, dBody);
+      UserResultsListener l = resultsListener.get(result.getQueryId());
+//      logger.debug("For QueryId [{}], retrieved result listener {}", result.getQueryId(), l);
+      if (l != null) {
+//        logger.debug("Results listener available, using existing.");
+        l.resultArrived(batch);
+        if (result.getIsLastChunk()) {
+          resultsListener.remove(result.getQueryId(), l);
+          l.set();
+        }
+      } else {
+        logger.debug("Results listener not available, creating a buffering listener.");
+        // manage race condition where we start getting results before we receive the queryid back.
+        BufferingListener bl = new BufferingListener();
+        l = resultsListener.putIfAbsent(result.getQueryId(), bl);
+        if (l != null) {
+          l.resultArrived(batch);
+        } else {
+          bl.resultArrived(batch);
+        }
+      }
+
+      return new Response(RpcType.ACK, Ack.getDefaultInstance());
+    default:
+      throw new RpcException(String.format("Unknown Rpc Type %d. ", rpcType));
+    }
+
+  }
 
   @Override
-  protected Response handle(SocketChannel ch, int rpcType, ByteBuf pBody, ByteBuf dBody) throws RpcException {
-    logger.debug("Received a server > client message of type " + rpcType);
-    return new Response(RpcType.ACK, Ack.getDefaultInstance(), null);
+  protected ClientHandshakeHandler<BitToUserHandshake> getHandshakeHandler() {
+    return new ClientHandshakeHandler<BitToUserHandshake>(RpcType.HANDSHAKE, BitToUserHandshake.class, BitToUserHandshake.PARSER) {
+
+      @Override
+      protected void validateHandshake(BitToUserHandshake inbound) throws Exception {
+        logger.debug("Handling handshake from bit to user. {}", inbound);
+        if (inbound.getRpcVersion() != UserRpcConfig.RPC_VERSION)
+          throw new RpcException(String.format("Invalid rpc version.  Expected %d, actual %d.",
+              inbound.getRpcVersion(), UserRpcConfig.RPC_VERSION));
+      }
+
+    };
   }
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserResultsListener.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserResultsListener.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserResultsListener.java
new file mode 100644
index 0000000..3ce14f0
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserResultsListener.java
@@ -0,0 +1,41 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.rpc.user;
+
+import java.util.concurrent.Future;
+
+import org.apache.drill.exec.proto.UserProtos.QueryResult;
+import org.apache.drill.exec.rpc.RpcException;
+
+import com.google.common.util.concurrent.SettableFuture;
+
+public abstract class UserResultsListener {
+  SettableFuture<Void> future = SettableFuture.create();
+  
+  final void set(){
+    future.set(null);
+  }
+  
+  Future<Void> getFuture(){
+    return future;
+  }
+
+  public abstract void submissionFailed(RpcException ex); 
+  public abstract void resultArrived(QueryResultBatch result);
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserRpcConfig.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserRpcConfig.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserRpcConfig.java
new file mode 100644
index 0000000..893e432
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserRpcConfig.java
@@ -0,0 +1,39 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.rpc.user;
+
+import org.apache.drill.exec.proto.GeneralRPCProtos.Ack;
+import org.apache.drill.exec.proto.UserBitShared.QueryId;
+import org.apache.drill.exec.proto.UserProtos.BitToUserHandshake;
+import org.apache.drill.exec.proto.UserProtos.QueryResult;
+import org.apache.drill.exec.proto.UserProtos.RpcType;
+import org.apache.drill.exec.proto.UserProtos.RunQuery;
+import org.apache.drill.exec.proto.UserProtos.UserToBitHandshake;
+import org.apache.drill.exec.rpc.RpcConfig;
+
+public class UserRpcConfig {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(UserRpcConfig.class);
+  
+  public static RpcConfig MAPPING = RpcConfig.newBuilder("USER-RPC-MAPPING") //
+      .add(RpcType.HANDSHAKE, UserToBitHandshake.class, RpcType.HANDSHAKE, BitToUserHandshake.class) //user to bit.
+      .add(RpcType.RUN_QUERY, RunQuery.class, RpcType.QUERY_HANDLE, QueryId.class) //user to bit
+      .add(RpcType.QUERY_RESULT, QueryResult.class, RpcType.ACK, Ack.class) //bit to user
+      .build();
+  
+  public static int RPC_VERSION = 1;
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserServer.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserServer.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserServer.java
index cccaa55..406afc4 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserServer.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserServer.java
@@ -19,31 +19,36 @@ package org.apache.drill.exec.rpc.user;
 
 import io.netty.buffer.ByteBuf;
 import io.netty.buffer.ByteBufAllocator;
+import io.netty.buffer.ByteBufInputStream;
+import io.netty.channel.Channel;
 import io.netty.channel.EventLoopGroup;
-import io.netty.channel.socket.SocketChannel;
 
+import org.apache.drill.exec.physical.impl.materialize.QueryWritableBatch;
 import org.apache.drill.exec.proto.GeneralRPCProtos.Ack;
 import org.apache.drill.exec.proto.UserProtos.BitToUserHandshake;
-import org.apache.drill.exec.proto.UserProtos.QueryHandle;
 import org.apache.drill.exec.proto.UserProtos.QueryResult;
+import org.apache.drill.exec.proto.UserProtos.RequestResults;
 import org.apache.drill.exec.proto.UserProtos.RpcType;
 import org.apache.drill.exec.proto.UserProtos.RunQuery;
+import org.apache.drill.exec.proto.UserProtos.UserToBitHandshake;
 import org.apache.drill.exec.rpc.BasicServer;
 import org.apache.drill.exec.rpc.DrillRpcFuture;
+import org.apache.drill.exec.rpc.RemoteConnection;
 import org.apache.drill.exec.rpc.Response;
 import org.apache.drill.exec.rpc.RpcException;
-import org.apache.drill.exec.server.DrillbitContext;
+import org.apache.drill.exec.work.user.UserWorker;
 
+import com.google.protobuf.InvalidProtocolBufferException;
 import com.google.protobuf.MessageLite;
 
-public class UserServer extends BasicServer<RpcType> {
+public class UserServer extends BasicServer<RpcType, UserServer.UserClientConnection> {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(UserServer.class);
-  
-  final DrillbitContext context;
-  
-  public UserServer(ByteBufAllocator alloc, EventLoopGroup eventLoopGroup, DrillbitContext context) {
-    super(alloc, eventLoopGroup);
-    this.context = context;
+
+  final UserWorker worker;
+
+  public UserServer(ByteBufAllocator alloc, EventLoopGroup eventLoopGroup, UserWorker worker) {
+    super(UserRpcConfig.MAPPING, alloc, eventLoopGroup);
+    this.worker = worker;
   }
 
   @Override
@@ -55,36 +60,70 @@ public class UserServer extends BasicServer<RpcType> {
     default:
       throw new UnsupportedOperationException();
     }
-
   }
 
-  public DrillRpcFuture<QueryResult> sendResult(RunQuery query, ByteBuf data) throws RpcException {
-    return this.send(RpcType.QUERY_RESULT, query, QueryResult.class, data);
-  }
-  
-  
   @Override
-  protected Response handle(SocketChannel channel, int rpcType, ByteBuf pBody, ByteBuf dBody) throws RpcException {
+  protected Response handle(UserClientConnection connection, int rpcType, ByteBuf pBody, ByteBuf dBody)
+      throws RpcException {
     switch (rpcType) {
 
     case RpcType.HANDSHAKE_VALUE:
-//      logger.debug("Received handshake, responding in kind.");
-      return new Response(RpcType.HANDSHAKE, BitToUserHandshake.getDefaultInstance(), null);
-      
+      // logger.debug("Received handshake, responding in kind.");
+      return new Response(RpcType.HANDSHAKE, BitToUserHandshake.getDefaultInstance());
+
     case RpcType.RUN_QUERY_VALUE:
-//      logger.debug("Received query to run.  Returning query handle.");
-      return new Response(RpcType.QUERY_HANDLE, QueryHandle.newBuilder().setQueryId(1).build(), null);
-      
+      // logger.debug("Received query to run.  Returning query handle.");
+      try {
+        RunQuery query = RunQuery.PARSER.parseFrom(new ByteBufInputStream(pBody));
+        return new Response(RpcType.QUERY_HANDLE, worker.submitWork(connection, query));
+      } catch (InvalidProtocolBufferException e) {
+        throw new RpcException("Failure while decoding RunQuery body.", e);
+      }
+
     case RpcType.REQUEST_RESULTS_VALUE:
-//      logger.debug("Received results requests.  Returning empty query result.");
-      return new Response(RpcType.QUERY_RESULT, QueryResult.getDefaultInstance(), null);
-      
+      // logger.debug("Received results requests.  Returning empty query result.");
+      try {
+        RequestResults req = RequestResults.PARSER.parseFrom(new ByteBufInputStream(pBody));
+        return new Response(RpcType.QUERY_RESULT, worker.getResult(connection, req));
+      } catch (InvalidProtocolBufferException e) {
+        throw new RpcException("Failure while decoding RequestResults body.", e);
+      }
+
     default:
       throw new UnsupportedOperationException();
     }
 
   }
+
+  public class UserClientConnection extends RemoteConnection {
+    public UserClientConnection(Channel channel) {
+      super(channel);
+    }
+
+    public DrillRpcFuture<Ack> sendResult(QueryWritableBatch result){
+      return send(this, RpcType.QUERY_RESULT, result.getHeader(), Ack.class, result.getBuffers());
+    }
+
+  }
+
+  @Override
+  public UserClientConnection initRemoteConnection(Channel channel) {
+    return new UserClientConnection(channel);
+  }
   
-  
+  @Override
+  protected ServerHandshakeHandler<UserToBitHandshake> getHandshakeHandler() {
+    return new ServerHandshakeHandler<UserToBitHandshake>(RpcType.HANDSHAKE, UserToBitHandshake.PARSER){
+
+      @Override
+      public MessageLite getHandshakeResponse(UserToBitHandshake inbound) throws Exception {
+        logger.debug("Handling handshake from user to bit. {}", inbound);
+        if(inbound.getRpcVersion() != UserRpcConfig.RPC_VERSION) throw new RpcException(String.format("Invalid rpc version.  Expected %d, actual %d.", inbound.getRpcVersion(), UserRpcConfig.RPC_VERSION));
+        return BitToUserHandshake.newBuilder().setRpcVersion(UserRpcConfig.RPC_VERSION).build();
+      }
 
+    };
+  }
+
+  
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/server/BootStrapContext.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/server/BootStrapContext.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/server/BootStrapContext.java
new file mode 100644
index 0000000..3c4d9af
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/server/BootStrapContext.java
@@ -0,0 +1,68 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.server;
+
+import io.netty.channel.nio.NioEventLoopGroup;
+
+import java.io.Closeable;
+
+import org.apache.drill.common.config.DrillConfig;
+import org.apache.drill.exec.ExecConstants;
+import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.rpc.NamedThreadFactory;
+
+import com.yammer.metrics.MetricRegistry;
+
+public class BootStrapContext implements Closeable{
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(BootStrapContext.class);
+  
+  private final DrillConfig config;
+  private final NioEventLoopGroup loop;
+  private final MetricRegistry metrics;
+  private final BufferAllocator allocator;
+  
+  public BootStrapContext(DrillConfig config) {
+    super();
+    this.config = config;
+    this.loop = new NioEventLoopGroup(4, new NamedThreadFactory("BitServer-"));
+    this.metrics = new MetricRegistry(config.getString(ExecConstants.METRICS_CONTEXT_NAME));
+    this.allocator = BufferAllocator.getAllocator(config);
+  }
+
+  public DrillConfig getConfig() {
+    return config;
+  }
+
+  public NioEventLoopGroup getBitLoopGroup() {
+    return loop;
+  }
+
+  public MetricRegistry getMetrics() {
+    return metrics;
+  }
+
+  public BufferAllocator getAllocator() {
+    return allocator;
+  }
+
+  public void close(){
+    loop.shutdown();
+    allocator.close();
+  }
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/server/Drillbit.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/server/Drillbit.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/server/Drillbit.java
index c33afce..7d745e1 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/server/Drillbit.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/server/Drillbit.java
@@ -17,10 +17,9 @@
  ******************************************************************************/
 package org.apache.drill.exec.server;
 
-import java.net.InetAddress;
+import java.io.Closeable;
 
 import org.apache.drill.common.config.DrillConfig;
-import org.apache.drill.common.proto.CoordinationProtos.DrillbitEndpoint;
 import org.apache.drill.exec.ExecConstants;
 import org.apache.drill.exec.cache.DistributedCache;
 import org.apache.drill.exec.cache.HazelCache;
@@ -28,15 +27,16 @@ import org.apache.drill.exec.coord.ClusterCoordinator;
 import org.apache.drill.exec.coord.ClusterCoordinator.RegistrationHandle;
 import org.apache.drill.exec.coord.ZKClusterCoordinator;
 import org.apache.drill.exec.exception.DrillbitStartupException;
-import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
 import org.apache.drill.exec.service.ServiceEngine;
+import org.apache.drill.exec.work.WorkManager;
 
 import com.google.common.io.Closeables;
 
 /**
  * Starts, tracks and stops all the required services for a Drillbit daemon to work.
  */
-public class Drillbit {
+public class Drillbit implements Closeable{
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(Drillbit.class);
 
   public static Drillbit start(StartupOptions options) throws DrillbitStartupException {
@@ -47,7 +47,7 @@ public class Drillbit {
     Drillbit bit;
     try {
       logger.debug("Setting up Drillbit.");
-      bit = new Drillbit(config);
+      bit = new Drillbit(config, null);
     } catch (Exception ex) {
       throw new DrillbitStartupException("Failure while initializing values in Drillbit.", ex);
     }
@@ -65,35 +65,37 @@ public class Drillbit {
     start(options);
   }
 
-  private final DrillbitContext context;
-  final BufferAllocator pool;
   final ClusterCoordinator coord;
   final ServiceEngine engine;
   final DistributedCache cache;
-  final DrillConfig config;
-  private RegistrationHandle handle;
-
-  public Drillbit(DrillConfig config) throws Exception {
-    final DrillbitContext context = new DrillbitContext(config, this);
-    Runtime.getRuntime().addShutdownHook(new ShutdownThread(config));
-    this.context = context;
-    this.pool = BufferAllocator.getAllocator(context);
-    this.coord = new ZKClusterCoordinator(config);
-    this.engine = new ServiceEngine(context);
-    this.cache = new HazelCache(context.getConfig());
-    this.config = config;
+  final WorkManager manager;
+  final BootStrapContext context;
+  
+  private volatile RegistrationHandle handle;
+
+  public Drillbit(DrillConfig config, RemoteServiceSet serviceSet) throws Exception {
+    if(serviceSet != null){
+      this.context = new BootStrapContext(config);
+      this.manager = new WorkManager(context);
+      this.coord = serviceSet.getCoordinator();
+      this.engine = new ServiceEngine(manager.getBitComWorker(), manager.getUserWorker(), context);
+      this.cache = serviceSet.getCache();
+    }else{
+      Runtime.getRuntime().addShutdownHook(new ShutdownThread(config));
+      this.context = new BootStrapContext(config);
+      this.manager = new WorkManager(context);
+      this.coord = new ZKClusterCoordinator(config);
+      this.engine = new ServiceEngine(manager.getBitComWorker(), manager.getUserWorker(), context);
+      this.cache = new HazelCache(config);
+    }
   }
 
   public void run() throws Exception {
-    coord.start();
-    engine.start();
-    DrillbitEndpoint md = DrillbitEndpoint.newBuilder()
-      .setAddress(InetAddress.getLocalHost().getHostAddress())
-      .setBitPort(engine.getBitPort())
-      .setUserPort(engine.getUserPort())
-      .build();
+    coord.start(10000);
+    DrillbitEndpoint md = engine.start();
+    cache.run();
+    manager.start(md, cache, engine.getBitCom(), coord);
     handle = coord.register(md);
-    cache.run(md);
   }
 
   public void close() {
@@ -107,7 +109,8 @@ public class Drillbit {
 
     Closeables.closeQuietly(engine);
     Closeables.closeQuietly(coord);
-    Closeables.closeQuietly(pool);
+    Closeables.closeQuietly(manager);
+    Closeables.closeQuietly(context);
     logger.info("Shutdown completed.");
   }
 
@@ -123,5 +126,11 @@ public class Drillbit {
     }
 
   }
+  public ClusterCoordinator getCoordinator(){
+    return coord;
+  }
 
+  public DrillbitContext getContext(){
+    return this.manager.getContext();
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/server/DrillbitContext.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/server/DrillbitContext.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/server/DrillbitContext.java
index b08b070..d5aaab2 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/server/DrillbitContext.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/server/DrillbitContext.java
@@ -23,42 +23,60 @@ import java.util.Collection;
 
 import org.apache.drill.common.config.DrillConfig;
 import org.apache.drill.common.logical.StorageEngineConfig;
-import org.apache.drill.common.proto.CoordinationProtos.DrillbitEndpoint;
 import org.apache.drill.exec.ExecConstants;
 import org.apache.drill.exec.cache.DistributedCache;
+import org.apache.drill.exec.coord.ClusterCoordinator;
 import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.planner.PhysicalPlanReader;
+import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
 import org.apache.drill.exec.rpc.NamedThreadFactory;
 import org.apache.drill.exec.rpc.bit.BitCom;
 import org.apache.drill.exec.store.StorageEngine;
 
+import com.google.common.base.Preconditions;
 import com.yammer.metrics.MetricRegistry;
 
 public class DrillbitContext {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillbitContext.class);
+
+  private BootStrapContext context;
+
+  private PhysicalPlanReader reader;
+  private final ClusterCoordinator coord;
+  private final BitCom com;
+  private final DistributedCache cache;
+  private final DrillbitEndpoint endpoint;
   
-  private final DrillConfig config;
-  private final Drillbit underlyingBit;
-  private final NioEventLoopGroup loop;
-  private final MetricRegistry metrics;
-  
-  public DrillbitContext(DrillConfig config, Drillbit underlyingBit) {
+  public DrillbitContext(DrillbitEndpoint endpoint, BootStrapContext context, ClusterCoordinator coord, BitCom com, DistributedCache cache) {
     super();
-    this.config = config;
-    this.underlyingBit = underlyingBit;
-    this.loop = new NioEventLoopGroup(1, new NamedThreadFactory("BitServer-"));
-    this.metrics = new MetricRegistry(config.getString(ExecConstants.METRICS_CONTEXT_NAME));
+    Preconditions.checkNotNull(endpoint);
+    Preconditions.checkNotNull(context);
+    Preconditions.checkNotNull(context);
+    Preconditions.checkNotNull(com);
+    Preconditions.checkNotNull(com);
+    
+    this.context = context;
+    this.coord = coord;
+    this.com = com;
+    this.cache = cache;
+    this.endpoint = endpoint;
+    this.reader = new PhysicalPlanReader(context.getConfig(), context.getConfig().getMapper(), endpoint);
+  }
+  
+  public DrillbitEndpoint getEndpoint(){
+    return endpoint;
   }
   
   public DrillConfig getConfig() {
-    return config;
+    return context.getConfig();
   }
   
   public Collection<DrillbitEndpoint> getBits(){
-    return underlyingBit.coord.getAvailableEndpoints();
+    return coord.getAvailableEndpoints();
   }
 
   public BufferAllocator getAllocator(){
-    return underlyingBit.pool;
+    return context.getAllocator();
   }
   
   public StorageEngine getStorageEngine(StorageEngineConfig config){
@@ -66,19 +84,23 @@ public class DrillbitContext {
   }
   
   public NioEventLoopGroup getBitLoopGroup(){
-    return loop;
+    return context.getBitLoopGroup();
   }
   
   public BitCom getBitCom(){
-    return underlyingBit.engine.getBitCom();
+    return com;
   }
   
   public MetricRegistry getMetrics(){
-    return metrics;
+    return context.getMetrics();
   }
   
   public DistributedCache getCache(){
-    return underlyingBit.cache;
+    return cache;
+  }
+  
+  public PhysicalPlanReader getPlanReader(){
+    return reader;
   }
   
   

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/server/RemoteServiceSet.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/server/RemoteServiceSet.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/server/RemoteServiceSet.java
new file mode 100644
index 0000000..0337a68
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/server/RemoteServiceSet.java
@@ -0,0 +1,59 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.server;
+
+import java.io.Closeable;
+import java.io.IOException;
+
+import org.apache.drill.exec.cache.DistributedCache;
+import org.apache.drill.exec.cache.LocalCache;
+import org.apache.drill.exec.coord.ClusterCoordinator;
+import org.apache.drill.exec.coord.LocalClusterCoordinator;
+
+public class RemoteServiceSet implements Closeable{
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(RemoteServiceSet.class);
+  
+  private final DistributedCache cache;
+  private final ClusterCoordinator coordinator;
+  
+  public RemoteServiceSet(DistributedCache cache, ClusterCoordinator coordinator) {
+    super();
+    this.cache = cache;
+    this.coordinator = coordinator;
+  }
+
+  public DistributedCache getCache() {
+    return cache;
+  }
+
+  public ClusterCoordinator getCoordinator() {
+    return coordinator;
+  }
+  
+  
+  @Override
+  public void close() throws IOException {
+    cache.close();
+    coordinator.close();
+  }
+
+  public static RemoteServiceSet getLocalServiceSet(){
+    return new RemoteServiceSet(new LocalCache(), new LocalClusterCoordinator());
+  }
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/service/ServiceEngine.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/service/ServiceEngine.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/service/ServiceEngine.java
index 5d83bdb..d6d3b9c 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/service/ServiceEngine.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/service/ServiceEngine.java
@@ -17,49 +17,48 @@
  ******************************************************************************/
 package org.apache.drill.exec.service;
 
-import io.netty.buffer.ByteBufAllocator;
 import io.netty.channel.nio.NioEventLoopGroup;
 
 import java.io.Closeable;
 import java.io.IOException;
+import java.net.InetAddress;
+import java.net.UnknownHostException;
 
+import org.apache.drill.common.config.DrillConfig;
 import org.apache.drill.exec.ExecConstants;
 import org.apache.drill.exec.exception.DrillbitStartupException;
+import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
 import org.apache.drill.exec.rpc.NamedThreadFactory;
 import org.apache.drill.exec.rpc.bit.BitCom;
 import org.apache.drill.exec.rpc.bit.BitComImpl;
 import org.apache.drill.exec.rpc.user.UserServer;
-import org.apache.drill.exec.server.DrillbitContext;
+import org.apache.drill.exec.server.BootStrapContext;
+import org.apache.drill.exec.work.batch.BitComHandler;
+import org.apache.drill.exec.work.user.UserWorker;
 
 import com.google.common.io.Closeables;
 
 public class ServiceEngine implements Closeable{
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ServiceEngine.class);
   
-  UserServer userServer;
-  BitComImpl bitCom;
-  int userPort;
-  int bitPort;
-  DrillbitContext context;
+  private final UserServer userServer;
+  private final BitCom bitCom;
+  private final DrillConfig config;
   
-  public ServiceEngine(DrillbitContext context){
-    this.context = context;
-    ByteBufAllocator allocator = context.getAllocator().getUnderlyingAllocator();
-    this.userServer = new UserServer(allocator, new NioEventLoopGroup(1, new NamedThreadFactory("UserServer-")), context);
-    this.bitCom = new BitComImpl(context);
+  public ServiceEngine(BitComHandler bitComWorker, UserWorker userWorker, BootStrapContext context){
+    this.userServer = new UserServer(context.getAllocator().getUnderlyingAllocator(), new NioEventLoopGroup(1, new NamedThreadFactory("UserServer-")), userWorker);
+    this.bitCom = new BitComImpl(context, bitComWorker);
+    this.config = context.getConfig();
   }
   
-  public void start() throws DrillbitStartupException, InterruptedException{
-    userPort = userServer.bind(context.getConfig().getInt(ExecConstants.INITIAL_USER_PORT));
-    bitPort = bitCom.start();
-  }
-  
-  public int getBitPort(){
-    return bitPort;
-  }
-  
-  public int getUserPort(){
-    return userPort;
+  public DrillbitEndpoint start() throws DrillbitStartupException, InterruptedException, UnknownHostException{
+    int userPort = userServer.bind(config.getInt(ExecConstants.INITIAL_USER_PORT));
+    int bitPort = bitCom.start();
+    return DrillbitEndpoint.newBuilder()
+        .setAddress(InetAddress.getLocalHost().getHostAddress())
+        .setBitPort(bitPort)
+        .setUserPort(userPort)
+        .build();
   }
 
   public BitCom getBitCom(){

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/AbstractStorageEngine.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/AbstractStorageEngine.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/AbstractStorageEngine.java
index d89b431..80704fa 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/AbstractStorageEngine.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/AbstractStorageEngine.java
@@ -23,8 +23,8 @@ import java.util.Collections;
 import java.util.List;
 
 import org.apache.drill.common.logical.data.Scan;
-import org.apache.drill.common.proto.CoordinationProtos.DrillbitEndpoint;
 import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
 
 import com.google.common.collect.ListMultimap;
 import com.google.common.collect.Multimap;

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/RecordReader.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/RecordReader.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/RecordReader.java
index 9fc4165..67c84ed 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/RecordReader.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/RecordReader.java
@@ -17,9 +17,8 @@
  ******************************************************************************/
 package org.apache.drill.exec.store;
 
-import org.apache.drill.exec.exception.ExecutionSetupException;
-import org.apache.drill.exec.ops.OutputMutator;
-import org.apache.drill.exec.record.BatchSchema;
+import org.apache.drill.common.exceptions.ExecutionSetupException;
+import org.apache.drill.exec.physical.impl.OutputMutator;
 
 public interface RecordReader {
 
@@ -35,7 +34,7 @@ public interface RecordReader {
    *          mutating the set of schema values for that particular record.
    * @throws ExecutionSetupException
    */
-  public abstract void setup(BatchSchema expectedSchema, OutputMutator output) throws ExecutionSetupException;
+  public abstract void setup(OutputMutator output) throws ExecutionSetupException;
 
   /**
    * Increment record reader forward, writing into the provided output batch.  

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/StorageEngine.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/StorageEngine.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/StorageEngine.java
index 67ea5b6..4884b7a 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/StorageEngine.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/StorageEngine.java
@@ -22,8 +22,8 @@ import java.util.Collection;
 import java.util.List;
 
 import org.apache.drill.common.logical.data.Scan;
-import org.apache.drill.common.proto.CoordinationProtos.DrillbitEndpoint;
 import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
 
 import com.google.common.collect.ListMultimap;
 import com.google.common.collect.Multimap;

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/util/AtomicState.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/util/AtomicState.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/util/AtomicState.java
new file mode 100644
index 0000000..d2e8b8f
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/util/AtomicState.java
@@ -0,0 +1,58 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.util;
+
+import java.util.concurrent.atomic.AtomicInteger;
+
+import org.apache.drill.exec.proto.ExecProtos.FragmentStatus.FragmentState;
+
+import com.google.protobuf.Internal.EnumLite;
+
+/**
+ * Simple wrapper class around AtomicInteger which allows management of a State value extending EnumLite.
+ * @param <T> The type of EnumLite to use for state.
+ */
+public abstract class AtomicState<T extends EnumLite> {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(AtomicState.class);
+  
+  private final AtomicInteger state = new AtomicInteger();
+  
+  /**
+   * Constructor that defines initial T state.
+   * @param initial
+   */
+  public AtomicState(T initial){
+    state.set(initial.getNumber());
+  }
+  
+  protected abstract T getStateFromNumber(int i);
+  
+  /**
+   * Does an atomic conditional update from one state to another.  
+   * @param oldState The expected current state.
+   * @param newState The desired new state.
+   * @return Whether or not the update was successful.
+   */
+  public boolean updateState(T oldState, T newState){
+    return state.compareAndSet(oldState.getNumber(), newState.getNumber());
+  }
+  
+  public T getState(){
+    return getStateFromNumber(state.get());
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/AbstractFragmentRunnerListener.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/AbstractFragmentRunnerListener.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/AbstractFragmentRunnerListener.java
new file mode 100644
index 0000000..0e8edd5
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/AbstractFragmentRunnerListener.java
@@ -0,0 +1,109 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.work;
+
+import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.proto.ExecProtos.FragmentHandle;
+import org.apache.drill.exec.proto.ExecProtos.FragmentStatus;
+import org.apache.drill.exec.proto.ExecProtos.FragmentStatus.FragmentState;
+import org.apache.drill.exec.work.foreman.ErrorHelper;
+
+public class AbstractFragmentRunnerListener implements FragmentRunnerListener{
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(AbstractFragmentRunnerListener.class);
+  
+  private FragmentContext context;
+  private volatile long startNanos;
+  
+  public AbstractFragmentRunnerListener(FragmentContext context) {
+    super();
+    this.context = context;
+  }
+  
+  private  FragmentStatus.Builder getBuilder(FragmentState state){
+    FragmentStatus.Builder status = FragmentStatus.newBuilder();
+    context.addMetricsToStatus(status);
+    status.setState(state);
+    status.setRunningTime(System.nanoTime() - startNanos);
+    status.setHandle(context.getHandle());
+    status.setMemoryUse(context.getAllocator().getAllocatedMemory());
+    return status;
+  }
+  
+  @Override
+  public void stateChanged(FragmentHandle handle, FragmentState newState) {
+    FragmentStatus.Builder status = getBuilder(newState);
+
+    switch(newState){
+    case AWAITING_ALLOCATION:
+      awaitingAllocation(handle, status);
+      break;
+    case CANCELLED:
+      cancelled(handle, status);
+      break;
+    case FAILED:
+      // no op since fail should have also been called.
+      break;
+    case FINISHED:
+      finished(handle, status);
+      break;
+    case RUNNING:
+      this.startNanos = System.nanoTime();
+      running(handle, status);
+      break;
+    case SENDING:
+      // no op.
+      break;
+    default:
+      break;
+    
+    }
+  }
+  
+  protected void awaitingAllocation(FragmentHandle handle, FragmentStatus.Builder statusBuilder){
+    statusChange(handle, statusBuilder.build());
+  }
+  
+  protected void running(FragmentHandle handle, FragmentStatus.Builder statusBuilder){
+    statusChange(handle, statusBuilder.build());
+  }
+
+  protected void cancelled(FragmentHandle handle, FragmentStatus.Builder statusBuilder){
+    statusChange(handle, statusBuilder.build());
+  }
+
+  protected void finished(FragmentHandle handle, FragmentStatus.Builder statusBuilder){
+    statusChange(handle, statusBuilder.build());
+  }
+  
+  protected void statusChange(FragmentHandle handle, FragmentStatus status){
+    
+  }
+  
+  @Override
+  public final void fail(FragmentHandle handle, String message, Throwable excep) {
+    FragmentStatus.Builder status = getBuilder(FragmentState.FAILED);
+    status.setError(ErrorHelper.logAndConvertError(context.getIdentity(), message, excep, logger));
+    fail(handle, status);
+  }
+
+  protected void fail(FragmentHandle handle, FragmentStatus.Builder statusBuilder){
+    statusChange(handle, statusBuilder.build());
+  }
+
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/CancelableQuery.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/CancelableQuery.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/CancelableQuery.java
new file mode 100644
index 0000000..3c7ef04
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/CancelableQuery.java
@@ -0,0 +1,22 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.work;
+
+public interface CancelableQuery {
+  public void cancel();
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/EndpointListener.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/EndpointListener.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/EndpointListener.java
new file mode 100644
index 0000000..f6a9786
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/EndpointListener.java
@@ -0,0 +1,45 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.work;
+
+import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
+import org.apache.drill.exec.rpc.RpcException;
+import org.apache.drill.exec.rpc.RpcOutcomeListener;
+
+public abstract class EndpointListener<RET, V> extends RpcOutcomeListener<RET>{
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(EndpointListener.class);
+
+  protected final DrillbitEndpoint endpoint;
+  protected final V value;
+  
+  public EndpointListener(DrillbitEndpoint endpoint, V value) {
+    super();
+    this.endpoint = endpoint;
+    this.value = value;
+  }
+
+  protected DrillbitEndpoint getEndpoint() {
+    return endpoint;
+  }
+
+  protected V getValue() {
+    return value;
+  }
+
+  
+}


Re: [50/53] [abbrv] git commit: Pom version cleaning and updates.

Posted by Ted Dunning <te...@gmail.com>.
On Fri, Jul 19, 2013 at 6:58 PM, <ja...@apache.org> wrote:

> Pom version cleaning and updates.
>
>
> Project: http://git-wip-us.apache.org/repos/asf/incubator-drill/repo
> Commit:
> http://git-wip-us.apache.org/repos/asf/incubator-drill/commit/57de7ed6
> Tree: http://git-wip-us.apache.org/repos/asf/incubator-drill/tree/57de7ed6
> Diff: http://git-wip-us.apache.org/repos/asf/incubator-drill/diff/57de7ed6
>
> Branch: refs/heads/master
> Commit: 57de7ed6b693b25b4c9a0c0580fa751be0535802
> Parents: 3d41be4
> Author: Jacques Nadeau <ja...@apache.org>
> Authored: Tue Jul 16 11:00:14 2013 -0700
> Committer: Jacques Nadeau <ja...@apache.org>
> Committed: Fri Jul 19 14:53:31 2013 -0700
>
> ----------------------------------------------------------------------
>  sandbox/prototype/pom.xml | 37 +++++++++++++++++++++++++++++++------
>  1 file changed, 31 insertions(+), 6 deletions(-)
> ----------------------------------------------------------------------
>
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/57de7ed6/sandbox/prototype/pom.xml
> ----------------------------------------------------------------------
> diff --git a/sandbox/prototype/pom.xml b/sandbox/prototype/pom.xml
> index b7b9a40..2e5dd02 100644
> --- a/sandbox/prototype/pom.xml
> +++ b/sandbox/prototype/pom.xml
> @@ -20,7 +20,7 @@
>
>  <target.gen.source.path>${project.basedir}/target/generated-sources</target.gen.source.path>
>      <proto.cas.path>${project.basedir}/src/main/protobuf/</proto.cas.path>
>      <dep.junit.version>4.11</dep.junit.version>
> -    <dep.slf4j.version>1.7.2</dep.slf4j.version>
> +    <dep.slf4j.version>1.7.5</dep.slf4j.version>
>    </properties>
>
>
> @@ -67,7 +67,6 @@
>        <plugin>
>          <groupId>org.apache.maven.plugins</groupId>
>          <artifactId>maven-resources-plugin</artifactId>
> -        <version>2.6</version>
>          <configuration>
>            <encoding>UTF-8</encoding>
>          </configuration>
> @@ -75,7 +74,6 @@
>        <plugin>
>          <groupId>org.apache.maven.plugins</groupId>
>          <artifactId>maven-compiler-plugin</artifactId>
> -        <version>3.0</version>
>          <configuration>
>            <source>1.7</source>
>            <target>1.7</target>
> @@ -83,7 +81,6 @@
>        </plugin>
>        <plugin>
>          <artifactId>maven-enforcer-plugin</artifactId>
> -        <version>1.2</version>
>          <executions>
>            <execution>
>              <id>no_commons_logging</id>
> @@ -104,8 +101,23 @@
>        </plugin>
>      </plugins>
>      <pluginManagement>
> +
>        <plugins>
>          <plugin>
> +          <groupId>org.apache.maven.plugins</groupId>
> +          <artifactId>maven-resources-plugin</artifactId>
> +          <version>2.6</version>
> +        </plugin>
> +        <plugin>
> +          <groupId>org.apache.maven.plugins</groupId>
> +          <artifactId>maven-compiler-plugin</artifactId>
> +          <version>3.0</version>
> +        </plugin>
> +        <plugin>
> +          <artifactId>maven-enforcer-plugin</artifactId>
> +          <version>1.2</version>
> +        </plugin>
> +        <plugin>
>            <artifactId>maven-surefire-plugin</artifactId>
>            <version>2.15</version>
>          </plugin>
> @@ -144,6 +156,19 @@
>                      <ignore></ignore>
>                    </action>
>                  </pluginExecution>
> +                <pluginExecution>
> +                  <pluginExecutionFilter>
> +                    <groupId>com.googlecode.fmpp-maven-plugin</groupId>
> +                    <artifactId>fmpp-maven-plugin</artifactId>
> +                    <versionRange>[1.0,)</versionRange>
> +                    <goals>
> +                      <goal>run</goal>
> +                    </goals>
> +                  </pluginExecutionFilter>
> +                  <action>
> +                    <ignore></ignore>
> +                  </action>
> +                </pluginExecution>
>                </pluginExecutions>
>              </lifecycleMappingMetadata>
>            </configuration>
> @@ -187,7 +212,7 @@
>      <dependency>
>        <groupId>com.googlecode.jmockit</groupId>
>        <artifactId>jmockit</artifactId>
> -      <version>1.2</version>
> +      <version>1.3</version>
>        <scope>test</scope>
>      </dependency>
>      <dependency>
> @@ -199,7 +224,7 @@
>      <dependency>
>        <groupId>ch.qos.logback</groupId>
>        <artifactId>logback-classic</artifactId>
> -      <version>1.0.7</version>
> +      <version>1.0.13</version>
>        <scope>test</scope>
>      </dependency>
>      <dependency>
>
>

[50/53] [abbrv] git commit: Pom version cleaning and updates.

Posted by ja...@apache.org.
Pom version cleaning and updates.


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

Branch: refs/heads/master
Commit: 57de7ed6b693b25b4c9a0c0580fa751be0535802
Parents: 3d41be4
Author: Jacques Nadeau <ja...@apache.org>
Authored: Tue Jul 16 11:00:14 2013 -0700
Committer: Jacques Nadeau <ja...@apache.org>
Committed: Fri Jul 19 14:53:31 2013 -0700

----------------------------------------------------------------------
 sandbox/prototype/pom.xml | 37 +++++++++++++++++++++++++++++++------
 1 file changed, 31 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/57de7ed6/sandbox/prototype/pom.xml
----------------------------------------------------------------------
diff --git a/sandbox/prototype/pom.xml b/sandbox/prototype/pom.xml
index b7b9a40..2e5dd02 100644
--- a/sandbox/prototype/pom.xml
+++ b/sandbox/prototype/pom.xml
@@ -20,7 +20,7 @@
     <target.gen.source.path>${project.basedir}/target/generated-sources</target.gen.source.path>
     <proto.cas.path>${project.basedir}/src/main/protobuf/</proto.cas.path>
     <dep.junit.version>4.11</dep.junit.version>
-    <dep.slf4j.version>1.7.2</dep.slf4j.version>
+    <dep.slf4j.version>1.7.5</dep.slf4j.version>
   </properties>
 
 
@@ -67,7 +67,6 @@
       <plugin>
         <groupId>org.apache.maven.plugins</groupId>
         <artifactId>maven-resources-plugin</artifactId>
-        <version>2.6</version>
         <configuration>
           <encoding>UTF-8</encoding>
         </configuration>
@@ -75,7 +74,6 @@
       <plugin>
         <groupId>org.apache.maven.plugins</groupId>
         <artifactId>maven-compiler-plugin</artifactId>
-        <version>3.0</version>
         <configuration>
           <source>1.7</source>
           <target>1.7</target>
@@ -83,7 +81,6 @@
       </plugin>
       <plugin>
         <artifactId>maven-enforcer-plugin</artifactId>
-        <version>1.2</version>
         <executions>
           <execution>
             <id>no_commons_logging</id>
@@ -104,8 +101,23 @@
       </plugin>
     </plugins>
     <pluginManagement>
+
       <plugins>
         <plugin>
+          <groupId>org.apache.maven.plugins</groupId>
+          <artifactId>maven-resources-plugin</artifactId>
+          <version>2.6</version>
+        </plugin>
+        <plugin>
+          <groupId>org.apache.maven.plugins</groupId>
+          <artifactId>maven-compiler-plugin</artifactId>
+          <version>3.0</version>
+        </plugin>
+        <plugin>
+          <artifactId>maven-enforcer-plugin</artifactId>
+          <version>1.2</version>
+        </plugin>
+        <plugin>
           <artifactId>maven-surefire-plugin</artifactId>
           <version>2.15</version>
         </plugin>
@@ -144,6 +156,19 @@
                     <ignore></ignore>
                   </action>
                 </pluginExecution>
+                <pluginExecution>
+                  <pluginExecutionFilter>
+                    <groupId>com.googlecode.fmpp-maven-plugin</groupId>
+                    <artifactId>fmpp-maven-plugin</artifactId>
+                    <versionRange>[1.0,)</versionRange>
+                    <goals>
+                      <goal>run</goal>
+                    </goals>
+                  </pluginExecutionFilter>
+                  <action>
+                    <ignore></ignore>
+                  </action>
+                </pluginExecution>
               </pluginExecutions>
             </lifecycleMappingMetadata>
           </configuration>
@@ -187,7 +212,7 @@
     <dependency>
       <groupId>com.googlecode.jmockit</groupId>
       <artifactId>jmockit</artifactId>
-      <version>1.2</version>
+      <version>1.3</version>
       <scope>test</scope>
     </dependency>
     <dependency>
@@ -199,7 +224,7 @@
     <dependency>
       <groupId>ch.qos.logback</groupId>
       <artifactId>logback-classic</artifactId>
-      <version>1.0.7</version>
+      <version>1.0.13</version>
       <scope>test</scope>
     </dependency>
     <dependency>


[05/53] [abbrv] WIP fragmentation, physical plan, byte compiling, some vector work

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/AbstractPhysicalVisitor.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/AbstractPhysicalVisitor.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/AbstractPhysicalVisitor.java
new file mode 100644
index 0000000..49f7bda
--- /dev/null
+++ b/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/AbstractPhysicalVisitor.java
@@ -0,0 +1,80 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.common.physical.pop.base;
+
+import org.apache.drill.common.physical.pop.Filter;
+import org.apache.drill.common.physical.pop.PartitionToRandomExchange;
+import org.apache.drill.common.physical.pop.Project;
+import org.apache.drill.common.physical.pop.Sort;
+
+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);
+
+  @Override
+  public T visitExchange(Exchange exchange, X value) throws E{
+    return visitUnknown(exchange, value);
+  }
+
+  @Override
+  public T visitFilter(Filter filter, X value) throws E{
+    return visitUnknown(filter, value);
+  }
+
+  @Override
+  public T visitProject(Project project, X value) throws E{
+    return visitUnknown(project, value);
+  }
+
+  @Override
+  public T visitSort(Sort sort, X value) throws E{
+    return visitUnknown(sort, value);
+  }
+
+  @Override
+  public T visitSender(Sender sender, X value) throws E {
+    return visitUnknown(sender, value);
+  }
+
+  @Override
+  public T visitReceiver(Receiver receiver, X value) throws E {
+    return visitUnknown(receiver, value);
+  }
+
+  @Override
+  public T visitScan(Scan<?> scan, X value) throws E{
+    return visitUnknown(scan, value);
+  }
+
+  @Override
+  public T visitStore(Store store, X value) throws E{
+    return visitUnknown(store, value);
+  }
+
+  @Override
+  public T visitPartitionToRandomExchange(PartitionToRandomExchange partitionToRandom, X value) throws E{
+    return visitExchange(partitionToRandom, value);
+  }
+
+  @Override
+  public T visitUnknown(PhysicalOperator op, X value) throws E{
+    throw new UnsupportedOperationException(String.format(
+        "The PhysicalVisitor of type %s does not currently support visiting the PhysicalOperator type %s.", this
+            .getClass().getCanonicalName(), op.getClass().getCanonicalName()));
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/AbstractReceiver.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/AbstractReceiver.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/AbstractReceiver.java
new file mode 100644
index 0000000..fd9d93c
--- /dev/null
+++ b/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/AbstractReceiver.java
@@ -0,0 +1,32 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.common.physical.pop.base;
+
+import java.util.Iterator;
+
+import com.google.common.collect.Iterators;
+
+public abstract class AbstractReceiver extends AbstractBase implements Receiver{
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(AbstractReceiver.class);
+
+  @Override
+  public Iterator<PhysicalOperator> iterator() {
+    return Iterators.emptyIterator();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/AbstractScan.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/AbstractScan.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/AbstractScan.java
new file mode 100644
index 0000000..3727139
--- /dev/null
+++ b/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/AbstractScan.java
@@ -0,0 +1,62 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.common.physical.pop.base;
+
+import java.util.Iterator;
+import java.util.List;
+
+import org.apache.drill.common.physical.ReadEntry;
+import org.apache.drill.common.proto.CoordinationProtos.DrillbitEndpoint;
+
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.google.common.collect.Iterators;
+
+public abstract class AbstractScan<R extends ReadEntry> extends AbstractBase implements Scan<R>{
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(AbstractScan.class);
+  
+  private List<R> readEntries;
+  
+  public AbstractScan(List<R> readEntries) {
+    this.readEntries = readEntries;
+  }
+
+  @Override
+  @JsonProperty("entries")
+  public List<R> getReadEntries() {
+    return readEntries;
+  }
+  
+  @Override
+  public Iterator<PhysicalOperator> iterator() {
+    return Iterators.emptyIterator();
+  }
+
+  @Override
+  public boolean isExecutable() {
+    return true;
+  }
+
+  @Override
+  public <T, X, E extends Throwable> T accept(PhysicalVisitor<T, X, E> physicalVisitor, X value) throws E{
+    return physicalVisitor.visitScan(this, value);
+  }
+  
+  
+  
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/AbstractSender.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/AbstractSender.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/AbstractSender.java
new file mode 100644
index 0000000..8b0608a
--- /dev/null
+++ b/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/AbstractSender.java
@@ -0,0 +1,29 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.common.physical.pop.base;
+
+
+
+public abstract class AbstractSender extends AbstractSingle implements Sender {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(AbstractSender.class);
+
+  public AbstractSender(PhysicalOperator child) {
+    super(child);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/AbstractSingle.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/AbstractSingle.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/AbstractSingle.java
new file mode 100644
index 0000000..49358df
--- /dev/null
+++ b/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/AbstractSingle.java
@@ -0,0 +1,48 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.common.physical.pop.base;
+
+import java.util.Iterator;
+
+
+import com.google.common.collect.Iterators;
+
+/**
+ * Describes an operator that expects a single child operator as its input.
+ * @param <T> The type of Exec model supported.
+ */
+public abstract class AbstractSingle extends AbstractBase{
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(AbstractSingle.class);
+  
+  private final PhysicalOperator child;
+
+  public AbstractSingle(PhysicalOperator child) {
+    super();
+    this.child = child;
+  }
+
+  @Override
+  public Iterator<PhysicalOperator> iterator() {
+    return Iterators.singletonIterator(child);
+  }
+
+  public PhysicalOperator getChild(){
+    return child;
+  }
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/AbstractStore.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/AbstractStore.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/AbstractStore.java
new file mode 100644
index 0000000..58edf03
--- /dev/null
+++ b/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/AbstractStore.java
@@ -0,0 +1,42 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.common.physical.pop.base;
+
+import java.util.List;
+
+import org.apache.drill.common.physical.EndpointAffinity;
+import org.apache.drill.common.proto.CoordinationProtos.DrillbitEndpoint;
+
+
+
+public abstract class AbstractStore extends AbstractSingle implements Store{
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(AbstractStore.class);
+
+  public AbstractStore(PhysicalOperator child) {
+    super(child);
+  }
+
+  @Override
+  public <T, X, E extends Throwable> T accept(PhysicalVisitor<T, X, E> physicalVisitor, X value) throws E{
+    return physicalVisitor.visitStore(this, value);
+  }
+
+
+  
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/Exchange.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/Exchange.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/Exchange.java
new file mode 100644
index 0000000..d779eb8
--- /dev/null
+++ b/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/Exchange.java
@@ -0,0 +1,69 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.common.physical.pop.base;
+
+import java.util.List;
+
+import org.apache.drill.common.physical.OperatorCost;
+import org.apache.drill.common.proto.CoordinationProtos.DrillbitEndpoint;
+
+import com.fasterxml.jackson.annotation.JsonIgnore;
+import com.fasterxml.jackson.annotation.JsonProperty;
+
+public interface Exchange extends PhysicalOperator{
+
+  @JsonIgnore
+  public abstract OperatorCost getAggregateSendCost();
+
+  @JsonIgnore
+  public abstract OperatorCost getAggregateReceiveCost();
+
+  @JsonProperty("cost")
+  public abstract ExchangeCost getExchangeCost();
+
+  /**
+   * Inform this Exchange node about its sender locations.
+   * @param senderLocations
+   */
+  public abstract void setupSenders(List<DrillbitEndpoint> senderLocations);
+
+  /**
+   * Inform this Exchange node about its receiver locations.
+   * @param receiverLocations
+   */
+  public abstract void setupReceivers(List<DrillbitEndpoint> receiverLocations);
+
+  /**
+   * Get the Sender associated with the given minorFragmentId.  
+   * @param minorFragmentId The minor fragment id, must be in the range [0, fragment.width).
+   * @param child The feeding node for the requested sender.
+   * @return The materialized sender for the given arguments.
+   */
+  public abstract Sender getSender(int minorFragmentId, PhysicalOperator child);
+
+  /**
+   * Get the Receiver associated with the given minorFragmentId.
+   * @param minorFragmentId The minor fragment id, must be in the range [0, fragment.width).
+   * @return The materialized recevier for the given arguments.
+   */
+  public abstract Receiver getReceiver(int minorFragmentId);
+
+  public abstract int getMaxSendWidth();
+
+  public PhysicalOperator getChild();
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/ExchangeCost.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/ExchangeCost.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/ExchangeCost.java
new file mode 100644
index 0000000..f17203e
--- /dev/null
+++ b/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/ExchangeCost.java
@@ -0,0 +1,55 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.common.physical.pop.base;
+
+import org.apache.drill.common.physical.OperatorCost;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonIgnore;
+import com.fasterxml.jackson.annotation.JsonProperty;
+
+public class ExchangeCost {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ExchangeCost.class);
+  
+  private final OperatorCost send;
+  private final OperatorCost receive;
+  private final OperatorCost combined;
+  
+  @JsonCreator
+  public ExchangeCost(@JsonProperty("send") OperatorCost send, @JsonProperty("receive") OperatorCost receive) {
+    this.send = send;
+    this.receive = receive;
+    this.combined =  OperatorCost.combine(send,  receive);
+  }
+
+  @JsonIgnore
+  public OperatorCost getCombinedCost(){
+    return combined;
+  }
+
+  public OperatorCost getSend() {
+    return send;
+  }
+
+  public OperatorCost getReceive() {
+    return receive;
+  }
+  
+  
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/FragmentLeaf.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/FragmentLeaf.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/FragmentLeaf.java
new file mode 100644
index 0000000..4557df4
--- /dev/null
+++ b/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/FragmentLeaf.java
@@ -0,0 +1,25 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.common.physical.pop.base;
+
+
+/**
+ * A POP which relies on no other nodes within the current fragment.
+ */
+public interface FragmentLeaf extends PhysicalOperator{
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/FragmentRoot.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/FragmentRoot.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/FragmentRoot.java
new file mode 100644
index 0000000..8d87d56
--- /dev/null
+++ b/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/FragmentRoot.java
@@ -0,0 +1,25 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.common.physical.pop.base;
+
+
+/**
+ * Describes the root operation within a particular Fragment. This includes things like Sinks, and Sender nodes. 
+ */
+public interface FragmentRoot extends PhysicalOperator{
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/HasAffinity.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/HasAffinity.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/HasAffinity.java
new file mode 100644
index 0000000..feb32ec
--- /dev/null
+++ b/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/HasAffinity.java
@@ -0,0 +1,26 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.common.physical.pop.base;
+
+import java.util.List;
+
+import org.apache.drill.common.physical.EndpointAffinity;
+
+public interface HasAffinity extends PhysicalOperator{
+  public List<EndpointAffinity> getOperatorAffinity();
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/Leaf.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/Leaf.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/Leaf.java
new file mode 100644
index 0000000..28efb94
--- /dev/null
+++ b/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/Leaf.java
@@ -0,0 +1,21 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.common.physical.pop.base;
+
+public interface Leaf extends PhysicalOperator{
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/PhysicalOperator.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/PhysicalOperator.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/PhysicalOperator.java
new file mode 100644
index 0000000..d8d1b64
--- /dev/null
+++ b/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/PhysicalOperator.java
@@ -0,0 +1,59 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.common.physical.pop.base;
+
+import org.apache.drill.common.graph.GraphValue;
+import org.apache.drill.common.physical.OperatorCost;
+
+import com.fasterxml.jackson.annotation.JsonIdentityInfo;
+import com.fasterxml.jackson.annotation.JsonIgnore;
+import com.fasterxml.jackson.annotation.JsonInclude;
+import com.fasterxml.jackson.annotation.JsonInclude.Include;
+import com.fasterxml.jackson.annotation.JsonPropertyOrder;
+import com.fasterxml.jackson.annotation.JsonTypeInfo;
+import com.fasterxml.jackson.annotation.ObjectIdGenerators;
+
+@JsonInclude(Include.NON_NULL)
+@JsonPropertyOrder({ "@id" })
+@JsonIdentityInfo(generator = ObjectIdGenerators.IntSequenceGenerator.class, property = "@id")
+@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, include = JsonTypeInfo.As.PROPERTY, property = "pop")
+public interface PhysicalOperator extends GraphValue<PhysicalOperator> {
+
+  public OperatorCost getCost();
+
+  /**
+   * Describes whether or not a particular physical operator can actually be executed. Most physical operators can be
+   * executed. However, Exchange nodes cannot be executed. In order to be executed, they must be converted into their
+   * Exec sub components.
+   * 
+   * @return
+   */
+  @JsonIgnore
+  public boolean isExecutable();
+  
+  /**
+   * Provides capability to build a set of output based on traversing a query graph tree.
+   * @param physicalVisitor
+   * @return
+   */
+  public <T, X, E extends Throwable> T accept(PhysicalVisitor<T, X, E> physicalVisitor, X value) throws E;
+
+
+  
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/PhysicalOperatorUtil.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/PhysicalOperatorUtil.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/PhysicalOperatorUtil.java
new file mode 100644
index 0000000..fb1fdcd
--- /dev/null
+++ b/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/PhysicalOperatorUtil.java
@@ -0,0 +1,34 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.common.physical.pop.base;
+
+import org.apache.drill.common.config.CommonConstants;
+import org.apache.drill.common.config.DrillConfig;
+import org.apache.drill.common.util.PathScanner;
+
+public class PhysicalOperatorUtil {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(PhysicalOperatorUtil.class);
+  
+  private PhysicalOperatorUtil(){}
+  
+  public synchronized static Class<?>[] getSubTypes(DrillConfig config){
+    Class<?>[] ops = PathScanner.scanForImplementationsArr(PhysicalOperator.class, config.getStringList(CommonConstants.PHYSICAL_OPERATOR_SCAN_PACKAGES));
+    logger.debug("Adding Physical Operator sub types: {}", ((Object) ops) );
+    return ops;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/PhysicalVisitor.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/PhysicalVisitor.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/PhysicalVisitor.java
new file mode 100644
index 0000000..2ecc6ce
--- /dev/null
+++ b/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/PhysicalVisitor.java
@@ -0,0 +1,43 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.common.physical.pop.base;
+
+import org.apache.drill.common.physical.pop.Filter;
+import org.apache.drill.common.physical.pop.PartitionToRandomExchange;
+import org.apache.drill.common.physical.pop.Project;
+import org.apache.drill.common.physical.pop.Sort;
+
+public interface PhysicalVisitor<RETURN, EXTRA, EXCEP extends Throwable> {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(PhysicalVisitor.class);
+  
+  
+  public RETURN visitExchange(Exchange exchange, EXTRA value) throws EXCEP;
+  public RETURN visitScan(Scan<?> scan, EXTRA value) throws EXCEP;
+  public RETURN visitStore(Store store, EXTRA value) throws EXCEP;
+
+  public RETURN visitFilter(Filter filter, 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;
+  public RETURN visitReceiver(Receiver receiver, EXTRA value) throws EXCEP;
+  
+  public RETURN visitUnknown(PhysicalOperator op, EXTRA value) throws EXCEP;
+  
+  public RETURN visitPartitionToRandomExchange(PartitionToRandomExchange partitionToRandom, EXTRA value) throws EXCEP; 
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/Receiver.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/Receiver.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/Receiver.java
new file mode 100644
index 0000000..db8f71f
--- /dev/null
+++ b/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/Receiver.java
@@ -0,0 +1,38 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.common.physical.pop.base;
+
+import java.util.List;
+
+import org.apache.drill.common.proto.CoordinationProtos.DrillbitEndpoint;
+
+public interface Receiver extends FragmentLeaf {
+  public abstract List<DrillbitEndpoint> getProvidingEndpoints();
+
+  /**
+   * Whether or not this receive supports out of order exchange. This provides a hint for the scheduling node on whether
+   * the receiver can start work if only a subset of all sending endpoints are currently providing data. A random
+   * receiver would supports this form of operation. A NWAY receiver would not.
+   * 
+   * @return True if this receiver supports working on a streaming/out of order input.
+   */
+  public abstract boolean supportsOutOfOrderExchange();
+  
+  
+  public int getSenderCount();
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/Root.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/Root.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/Root.java
new file mode 100644
index 0000000..c4f9982
--- /dev/null
+++ b/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/Root.java
@@ -0,0 +1,24 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.common.physical.pop.base;
+
+/**
+ * Marker interface describe the root of a query plan.
+ */
+public interface Root extends PhysicalOperator{
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/Scan.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/Scan.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/Scan.java
new file mode 100644
index 0000000..c7b45a8
--- /dev/null
+++ b/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/Scan.java
@@ -0,0 +1,36 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.common.physical.pop.base;
+
+import java.util.List;
+
+import org.apache.drill.common.physical.ReadEntry;
+import org.apache.drill.common.proto.CoordinationProtos.DrillbitEndpoint;
+
+import com.fasterxml.jackson.annotation.JsonProperty;
+
+public interface Scan<R extends ReadEntry> extends Leaf, HasAffinity{
+
+  @JsonProperty("entries")
+  public abstract List<R> getReadEntries();
+
+  public abstract void applyAssignments(List<DrillbitEndpoint> endpoints);
+
+  public abstract Scan<?> getSpecificScan(int minorFragmentId);
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/Sender.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/Sender.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/Sender.java
new file mode 100644
index 0000000..1859657
--- /dev/null
+++ b/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/Sender.java
@@ -0,0 +1,29 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.common.physical.pop.base;
+
+import java.util.List;
+
+import org.apache.drill.common.proto.CoordinationProtos.DrillbitEndpoint;
+
+
+
+
+public interface Sender extends FragmentRoot{
+  public abstract List<DrillbitEndpoint> getDestinations();
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/Store.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/Store.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/Store.java
new file mode 100644
index 0000000..eec4a6c
--- /dev/null
+++ b/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/Store.java
@@ -0,0 +1,30 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.common.physical.pop.base;
+
+import java.util.List;
+
+import org.apache.drill.common.proto.CoordinationProtos.DrillbitEndpoint;
+
+public interface Store extends Root, HasAffinity{
+
+  public abstract void applyAssignments(List<DrillbitEndpoint> endpoints);
+  public abstract Store getSpecificStore(PhysicalOperator child, int minorFragmentId);
+  public abstract int getMaxWidth();
+  public abstract PhysicalOperator getChild();
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/props/OrderProp.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/props/OrderProp.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/props/OrderProp.java
deleted file mode 100644
index 5e618e7..0000000
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/props/OrderProp.java
+++ /dev/null
@@ -1,45 +0,0 @@
-/*******************************************************************************
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- ******************************************************************************/
-package org.apache.drill.common.physical.props;
-
-import org.apache.drill.common.logical.defs.OrderDef;
-
-import com.fasterxml.jackson.annotation.JsonCreator;
-import com.fasterxml.jackson.annotation.JsonProperty;
-import com.fasterxml.jackson.annotation.JsonTypeName;
-
-@JsonTypeName("ordered")
-public class OrderProp implements PhysicalProp{
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(OrderProp.class);
-  
-  private final OrderDef[] orderings;
-
-  @JsonCreator
-  public OrderProp(@JsonProperty("fields") OrderDef[] orderings) {
-    super();
-    this.orderings = orderings;
-  }
-  
-  @JsonProperty("fields")
-  public OrderDef[] getOrderings(){
-    return orderings;
-  }
-  
-  
-  
-}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/props/PartitionProp.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/props/PartitionProp.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/props/PartitionProp.java
deleted file mode 100644
index d855b73..0000000
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/props/PartitionProp.java
+++ /dev/null
@@ -1,36 +0,0 @@
-/*******************************************************************************
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- ******************************************************************************/
-package org.apache.drill.common.physical.props;
-
-import org.apache.drill.common.expression.LogicalExpression;
-import org.apache.drill.common.logical.defs.PartitionDef;
-
-import com.fasterxml.jackson.annotation.JsonCreator;
-import com.fasterxml.jackson.annotation.JsonProperty;
-import com.fasterxml.jackson.annotation.JsonTypeName;
-
-@JsonTypeName("partitioned")
-public class PartitionProp extends PartitionDef implements PhysicalProp{
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(PartitionProp.class);
-
-  @JsonCreator
-  public PartitionProp(@JsonProperty("partitionType") PartitionType partitionType, @JsonProperty("exprs") LogicalExpression[] expressions, @JsonProperty("starts") LogicalExpression[] starts) {
-    super(partitionType, expressions, starts);
-  }
-  
-}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/props/PhysicalProp.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/props/PhysicalProp.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/props/PhysicalProp.java
deleted file mode 100644
index 0776d66..0000000
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/props/PhysicalProp.java
+++ /dev/null
@@ -1,24 +0,0 @@
-/*******************************************************************************
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- ******************************************************************************/
-package org.apache.drill.common.physical.props;
-
-import com.fasterxml.jackson.annotation.JsonTypeInfo;
-
-@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, include = JsonTypeInfo.As.PROPERTY, property="trait")
-public interface PhysicalProp {
-}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/props/SegmentProp.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/props/SegmentProp.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/props/SegmentProp.java
deleted file mode 100644
index d76fe48..0000000
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/props/SegmentProp.java
+++ /dev/null
@@ -1,42 +0,0 @@
-/*******************************************************************************
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- ******************************************************************************/
-package org.apache.drill.common.physical.props;
-
-import org.apache.drill.common.expression.FieldReference;
-
-import com.fasterxml.jackson.annotation.JsonCreator;
-import com.fasterxml.jackson.annotation.JsonProperty;
-import com.fasterxml.jackson.annotation.JsonTypeName;
-
-@JsonTypeName("segmented")
-public class SegmentProp implements PhysicalProp{
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(SegmentProp.class);
-  
-  private FieldReference segments;
-
-  @JsonCreator
-  public SegmentProp(@JsonProperty("segments") FieldReference segments) {
-    super();
-    this.segments = segments;
-  }
-
-  public FieldReference getSegments() {
-    return segments;
-  }
-  
-}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/common/src/main/protobuf/Coordination.proto
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/protobuf/Coordination.proto b/sandbox/prototype/common/src/main/protobuf/Coordination.proto
new file mode 100644
index 0000000..f98d2c5
--- /dev/null
+++ b/sandbox/prototype/common/src/main/protobuf/Coordination.proto
@@ -0,0 +1,26 @@
+package exec;
+
+option java_package = "org.apache.drill.common.proto";
+option java_outer_classname = "CoordinationProtos";
+option optimize_for = LITE_RUNTIME;
+
+message DrillbitEndpoint{
+  optional string address = 1;
+  optional int32 user_port = 2;
+  optional int32 bit_port = 3;
+  optional Roles roles = 4;
+}
+
+message DrillServiceInstance{
+  optional string id = 1;
+  optional int64 registrationTimeUTC = 2;
+  optional DrillbitEndpoint endpoint = 3;
+}
+
+message Roles{
+	optional bool sql_query = 1 [default = true];
+	optional bool logical_plan = 2 [default = true];
+	optional bool physical_plan = 3 [default = true];
+	optional bool java_executor = 4 [default = true];
+	optional bool distributed_cache = 5 [default = true];
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/common/src/test/java/org/apache/drill/common/physical/MockScanPOP.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/test/java/org/apache/drill/common/physical/MockScanPOP.java b/sandbox/prototype/common/src/test/java/org/apache/drill/common/physical/MockScanPOP.java
index f77ac88..1b042c5 100644
--- a/sandbox/prototype/common/src/test/java/org/apache/drill/common/physical/MockScanPOP.java
+++ b/sandbox/prototype/common/src/test/java/org/apache/drill/common/physical/MockScanPOP.java
@@ -17,23 +17,26 @@
  ******************************************************************************/
 package org.apache.drill.common.physical;
 
+import java.util.Collections;
 import java.util.List;
 
-import org.apache.drill.common.physical.pop.ScanPOP;
+import org.apache.drill.common.physical.pop.base.AbstractScan;
+import org.apache.drill.common.physical.pop.base.Scan;
+import org.apache.drill.common.proto.CoordinationProtos.DrillbitEndpoint;
 
 import com.fasterxml.jackson.annotation.JsonCreator;
 import com.fasterxml.jackson.annotation.JsonProperty;
 import com.fasterxml.jackson.annotation.JsonTypeName;
 
 @JsonTypeName("mock-scan")
-public class MockScanPOP extends ScanPOP<MockScanPOP.MockScanEntry>{
+public class MockScanPOP extends AbstractScan<MockScanPOP.MockScanEntry>{
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(MockScanPOP.class);
   
   private final String url;
   
   @JsonCreator
-  public MockScanPOP(@JsonProperty("url") String url, @JsonProperty("entries") List<MockScanEntry> readEntries, @JsonProperty("output") FieldSet fields) {
-    super(readEntries, fields);
+  public MockScanPOP(@JsonProperty("url") String url, @JsonProperty("entries") List<MockScanEntry> readEntries) {
+    super(readEntries);
     this.url = url;
   }
   
@@ -44,4 +47,23 @@ public class MockScanPOP extends ScanPOP<MockScanPOP.MockScanEntry>{
   public static class MockScanEntry implements ReadEntry{
     public int id;
   }
+
+  @Override
+  public List<EndpointAffinity> getOperatorAffinity() {
+    return Collections.emptyList();
+  }
+
+  @Override
+  public void applyAssignments(List<DrillbitEndpoint> endpoints) {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public Scan<?> getSpecificScan(int minorFragmentId) {
+    return this;
+  }
+  
+
+  
+  
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/common/src/test/java/org/apache/drill/common/physical/MockStorePOP.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/test/java/org/apache/drill/common/physical/MockStorePOP.java b/sandbox/prototype/common/src/test/java/org/apache/drill/common/physical/MockStorePOP.java
index cf2df59..f48c539 100644
--- a/sandbox/prototype/common/src/test/java/org/apache/drill/common/physical/MockStorePOP.java
+++ b/sandbox/prototype/common/src/test/java/org/apache/drill/common/physical/MockStorePOP.java
@@ -17,40 +17,46 @@
  ******************************************************************************/
 package org.apache.drill.common.physical;
 
+import java.util.Collections;
 import java.util.List;
 
-import org.apache.drill.common.defs.PartitionDef;
-import org.apache.drill.common.physical.MockStorePOP.MockWriteEntry;
-import org.apache.drill.common.physical.pop.StorePOP;
+import org.apache.drill.common.physical.pop.base.AbstractStore;
+import org.apache.drill.common.physical.pop.base.PhysicalOperator;
+import org.apache.drill.common.physical.pop.base.Store;
+import org.apache.drill.common.proto.CoordinationProtos.DrillbitEndpoint;
 
 import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonIgnore;
 import com.fasterxml.jackson.annotation.JsonProperty;
 import com.fasterxml.jackson.annotation.JsonTypeName;
 
 @JsonTypeName("mock-store")
-public class MockStorePOP extends StorePOP<MockWriteEntry>{
+public class MockStorePOP extends AbstractStore {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(MockStorePOP.class);
 
-  private List<String> fieldNames;
-
-  
   @JsonCreator
-  public MockStorePOP(@JsonProperty("output") FieldSet fields, @JsonProperty("mode") StoreMode mode, @JsonProperty("entries") List<MockWriteEntry> entries, @JsonProperty("partition") PartitionDef partition, @JsonProperty("fieldNames") List<String> fieldNames) {
-    super(fields, mode, partition, entries);
-    this.fieldNames = fieldNames;
+  public MockStorePOP(@JsonProperty("child") PhysicalOperator child) {
+    super(child);
   }
 
-  
-  public List<String> getFieldNames() {
-    return fieldNames;
+  public int getMaxWidth() {
+    return 1;
   }
 
-  
-  public static class MockWriteEntry implements WriteEntry{
-    public String path;
-    public String key;
-    public String type;
+  @Override
+  public List<EndpointAffinity> getOperatorAffinity() {
+    return Collections.emptyList();
+  }
+
+  @Override
+  public void applyAssignments(List<DrillbitEndpoint> endpoints) {
+    throw new UnsupportedOperationException();
   }
-  
-}
 
+  @Override
+  public Store getSpecificStore(PhysicalOperator child, int minorFragmentId) {
+    return this;
+  }
+
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/common/src/test/java/org/apache/drill/common/physical/ParsePhysicalPlan.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/test/java/org/apache/drill/common/physical/ParsePhysicalPlan.java b/sandbox/prototype/common/src/test/java/org/apache/drill/common/physical/ParsePhysicalPlan.java
index 9656823..0ad1f76 100644
--- a/sandbox/prototype/common/src/test/java/org/apache/drill/common/physical/ParsePhysicalPlan.java
+++ b/sandbox/prototype/common/src/test/java/org/apache/drill/common/physical/ParsePhysicalPlan.java
@@ -21,6 +21,8 @@ import org.apache.drill.common.config.DrillConfig;
 import org.apache.drill.common.util.FileUtils;
 import org.junit.Test;
 
+import com.fasterxml.jackson.databind.ObjectReader;
+import com.fasterxml.jackson.databind.ObjectWriter;
 import com.google.common.base.Charsets;
 import com.google.common.io.Files;
 
@@ -31,7 +33,9 @@ public class ParsePhysicalPlan {
   @Test 
   public void parseSimplePlan() throws Exception{
     DrillConfig c = DrillConfig.create();
-    PhysicalPlan plan = PhysicalPlan.parse(c, Files.toString(FileUtils.getResourceAsFile("/dsort-physical.json"), Charsets.UTF_8));
-    System.out.println(plan.unparse(c));
+    ObjectReader r = c.getMapper().reader(PhysicalPlan.class);
+    ObjectWriter writer = c.getMapper().writer();
+    PhysicalPlan plan = PhysicalPlan.parse(r, Files.toString(FileUtils.getResourceAsFile("/physical_test1.json"), Charsets.UTF_8));
+    System.out.println(plan.unparse(writer));
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/common/src/test/java/org/apache/drill/common/physical/ParsePlan.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/test/java/org/apache/drill/common/physical/ParsePlan.java b/sandbox/prototype/common/src/test/java/org/apache/drill/common/physical/ParsePlan.java
deleted file mode 100644
index e3e43aa..0000000
--- a/sandbox/prototype/common/src/test/java/org/apache/drill/common/physical/ParsePlan.java
+++ /dev/null
@@ -1,36 +0,0 @@
-/*******************************************************************************
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- ******************************************************************************/
-package org.apache.drill.common.physical;
-
-import org.apache.drill.common.config.DrillConfig;
-import org.apache.drill.common.util.FileUtils;
-import org.junit.Test;
-
-import com.google.common.base.Charsets;
-import com.google.common.io.Files;
-
-public class ParsePlan {
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ParsePlan.class);
-  
-  
-  @Test public void parseSimplePlan() throws Exception{
-    DrillConfig c = DrillConfig.create();
-    PhysicalPlan plan = PhysicalPlan.parse(c, Files.toString(FileUtils.getResourceAsFile("/dsort_physical.json"), Charsets.UTF_8));
-    System.out.println(plan.unparse(c));
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/common/src/test/resources/drill-module.conf
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/test/resources/drill-module.conf b/sandbox/prototype/common/src/test/resources/drill-module.conf
index d304d7b..86e828a 100644
--- a/sandbox/prototype/common/src/test/resources/drill-module.conf
+++ b/sandbox/prototype/common/src/test/resources/drill-module.conf
@@ -1,2 +1,2 @@
 drill.logical.storage.packages += "org.apache.drill.storage"
-drill.physical.operator.packages += "org.apache.drill.common.physical"
\ No newline at end of file
+drill.physical.operator.packages += "org.apache.drill.common.physical.pop"
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/common/src/test/resources/dsort-physical.json
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/test/resources/dsort-physical.json b/sandbox/prototype/common/src/test/resources/dsort-physical.json
deleted file mode 100644
index 131265a..0000000
--- a/sandbox/prototype/common/src/test/resources/dsort-physical.json
+++ /dev/null
@@ -1,66 +0,0 @@
-{
-    head:{
-        type:"APACHE_DRILL_PHYSICAL",
-        version:"1",
-        generator:{
-            type:"manual"
-        }
-    },
-    graph:[
-        {
-            @id:1,
-            pop:"mock-scan",
-            url: "http://apache.org",
-            entries:[
-            	{id:1}
-            ],
-            output: [
-                {mode: "VECTOR", type: "INT32"}, //field 0
-                {mode: "VECTOR", type: "BYTES"}, //field 1
-                {mode: "VECTOR", type: "BYTES"} //field 2
-            ]
-        },
-        {
-            @id:2,
-            child: 1,
-            pop:"project",
-            fields: [0, 1],
-            exprs: [],
-            output: [
-                {mode: "VECTOR", type: "INT32"}, //field 0
-                {mode: "VECTOR", type: "BYTES"} //field 1
-            ]            
-        },   
-        {
-            @id:3,
-            child: 2,
-            pop:"sort",
-            field: 0,
-            output: [
-                {mode: "VECTOR", type: "INT32"}, //field 0
-                {mode: "VECTOR", type: "BYTES"} //field 1
-            ]            
-        },
-        {
-            @id:4,
-            child:3,
-            pop: "mock-store",
-            mode: "SYSTEM_CHOICE",
-            fieldNames: ["key", "value"], // maps to incoming fieldids
-            entries:[
-                {
-                    path:"/sort/sorted/${partition_number}.seq",
-                    key:"Text",
-                    type:"JAVA_SEQUENCE"
-                }
-            ],
-            output:[
-            // the output here is data about the result of the query.
-                {mode: "VECTOR", type:"INT32"},
-				{mode: "VECTOR", type:"INT64"},
-				{mode: "VECTOR", type:"INT64"},
-				{mode: "VECTOR", type:"INT64"}
-            ]
-        }
-    ]
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/common/src/test/resources/physical_test1.json
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/test/resources/physical_test1.json b/sandbox/prototype/common/src/test/resources/physical_test1.json
new file mode 100644
index 0000000..16bc87a
--- /dev/null
+++ b/sandbox/prototype/common/src/test/resources/physical_test1.json
@@ -0,0 +1,33 @@
+{
+    head:{
+        type:"APACHE_DRILL_PHYSICAL",
+        version:"1",
+        generator:{
+            type:"manual"
+        }
+    },
+    graph:[
+        {
+            @id:1,
+            pop:"mock-scan",
+            url: "http://apache.org",
+            entries:[
+            	{id:1}
+            ],
+            cost: { disk: 1.0, memory: 1.0, cpu: 1.0, network: 1.0}
+        },
+        {
+            @id:2,
+            child: 1,
+            pop:"filter",
+            expr: "b > 5",
+            cost: { disk: 1.0, memory: 1.0, cpu: 1.0, network: 1.0}
+        },
+        {
+            @id: 3,
+            child: 2,
+            pop: "mock-store",
+            cost: { disk: 1.0, memory: 1.0, cpu: 1.0, network: 1.0}
+        }
+    ]
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/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 663bab4..9766df7 100644
--- a/sandbox/prototype/exec/java-exec/pom.xml
+++ b/sandbox/prototype/exec/java-exec/pom.xml
@@ -18,14 +18,45 @@
 
 	<dependencies>
 		<dependency>
+			<groupId>asm</groupId>
+			<artifactId>asm-util</artifactId>
+			<version>3.3.1</version>
+		</dependency>
+		<dependency>
+			<groupId>asm</groupId>
+			<artifactId>asm-commons</artifactId>
+			<version>3.3.1</version>
+		</dependency>
+		<dependency>
+			<groupId>org.codehaus.janino</groupId>
+			<artifactId>commons-compiler-jdk</artifactId>
+			<version>2.6.1</version>
+		</dependency>
+		<dependency>
+			<groupId>net.hydromatic</groupId>
+			<artifactId>optiq</artifactId>
+			<version>0.3.2</version>
+		</dependency>
+		<dependency>
 			<groupId>com.twitter</groupId>
 			<artifactId>parquet-column</artifactId>
 			<version>1.0.0-SNAPSHOT</version>
 		</dependency>
 		<dependency>
+			<groupId>com.yammer.metrics</groupId>
+			<artifactId>metrics-core</artifactId>
+			<version>3.0.0-BETA1</version>
+		</dependency>
+		<dependency>
+			<groupId>org.apache.drill</groupId>
+			<artifactId>common</artifactId>
+			<version>1.0-SNAPSHOT</version>
+		</dependency>
+		<dependency>
 			<groupId>org.apache.drill</groupId>
 			<artifactId>common</artifactId>
 			<version>1.0-SNAPSHOT</version>
+			<classifier>test</classifier>
 		</dependency>
 		<dependency>
 			<groupId>com.beust</groupId>
@@ -89,8 +120,8 @@
 		<dependency>
 			<groupId>com.hazelcast</groupId>
 			<artifactId>hazelcast</artifactId>
-			<version>2.5</version>
-		</dependency>		
+			<version>2.5.1</version>
+		</dependency>
 	</dependencies>
 
 	<build>

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/BufferAllocator.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/BufferAllocator.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/BufferAllocator.java
deleted file mode 100644
index a398607..0000000
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/BufferAllocator.java
+++ /dev/null
@@ -1,52 +0,0 @@
-/*******************************************************************************
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- ******************************************************************************/
-package org.apache.drill.exec;
-
-import io.netty.buffer.ByteBuf;
-import io.netty.buffer.ByteBufAllocator;
-
-import java.io.Closeable;
-
-import org.apache.drill.exec.server.DrillbitContext;
-
-/**
- * Wrapper class to deal with byte buffer allocation. Ensures users only use designated methods.  Also allows inser 
- */
-public abstract class BufferAllocator implements Closeable{
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(BufferAllocator.class);
-  
-  /**
-   * Allocate a new or reused buffer of the provided size.  Note that the buffer may technically be larger than the requested size for rounding purposes.  However, the buffers capacity will be set to the configured size.
-   * @param size The size in bytes.
-   * @return A new ByteBuf.
-   */
-  public abstract ByteBuf buffer(int size);
-  
-  public abstract ByteBufAllocator getUnderlyingAllocator();
-  
-  /**
-   * Close and release all buffers generated from this buffer pool.
-   */
-  @Override
-  public abstract void close(); 
-  
-  public static BufferAllocator getAllocator(DrillbitContext context){
-    // TODO: support alternative allocators (including a debugging allocator that records all allocation locations for each buffer).
-    return new DirectBufferAllocator();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ByteReorder.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ByteReorder.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ByteReorder.java
new file mode 100644
index 0000000..82a8a85
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ByteReorder.java
@@ -0,0 +1,54 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec;
+
+import java.util.Arrays;
+
+import com.google.common.base.Charsets;
+import com.google.common.primitives.Bytes;
+import com.google.common.primitives.UnsignedBytes;
+
+public class ByteReorder {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ByteReorder.class);
+  
+  public static void main(String[] args){
+    String[] strings = {"hello", "goodbye", "my friend"};
+    byte[][] bytes = new byte[strings.length][];
+    for(int i =0; i < strings.length; i++){
+      bytes[i] = strings[i].getBytes(Charsets.UTF_8);
+    }
+    
+    for(int i =0; i < bytes.length; i++){
+      for(int v = 0; v < bytes[i].length; v++){
+        bytes[i][v] = (byte) ~bytes[i][v];
+      }
+    }
+    
+    Arrays.sort(bytes, UnsignedBytes.lexicographicalComparator());
+
+    for(int i =0; i < bytes.length; i++){
+      for(int v = 0; v < bytes[i].length; v++){
+        bytes[i][v] = (byte) ~bytes[i][v];
+      }
+    }
+
+    for(int i =0; i < bytes.length; i++){
+      System.out.println(new String(bytes[i]));
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/DirectBufferAllocator.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/DirectBufferAllocator.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/DirectBufferAllocator.java
deleted file mode 100644
index 8c81dd6..0000000
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/DirectBufferAllocator.java
+++ /dev/null
@@ -1,47 +0,0 @@
-/*******************************************************************************
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- ******************************************************************************/
-package org.apache.drill.exec;
-
-import io.netty.buffer.ByteBuf;
-import io.netty.buffer.ByteBufAllocator;
-import io.netty.buffer.PooledByteBufAllocator;
-
-public class DirectBufferAllocator extends BufferAllocator{
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DirectBufferAllocator.class);
-
-  private final PooledByteBufAllocator buffer = new PooledByteBufAllocator(true);
-  
-  @Override
-  public ByteBuf buffer(int size) {
-    return buffer.directBuffer(size);
-  }
-
-  
-  @Override
-  public ByteBufAllocator getUnderlyingAllocator() {
-    return buffer;
-  }
-
-
-  @Override
-  public void close() {
-    // TODO: collect all buffers and release them away using a weak hashmap so we don't impact pool work
-  }
-  
-  
-}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ExecConstants.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ExecConstants.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ExecConstants.java
index d81870b..0ccaa22 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ExecConstants.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ExecConstants.java
@@ -28,4 +28,5 @@ public interface ExecConstants {
   public static final String SERVICE_NAME = "drill.exec.cluster-id";
   public static final String INITIAL_BIT_PORT = "drill.exec.rpc.bit.port";
   public static final String INITIAL_USER_PORT = "drill.exec.rpc.user.port";
+  public static final String METRICS_CONTEXT_NAME = "drill.exec.metrics.context";
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/cache/DistributedCache.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/cache/DistributedCache.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/cache/DistributedCache.java
index 1684960..2928dbe 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/cache/DistributedCache.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/cache/DistributedCache.java
@@ -20,9 +20,10 @@ package org.apache.drill.exec.cache;
 import java.io.Closeable;
 import java.util.List;
 
+import org.apache.drill.common.proto.CoordinationProtos.DrillbitEndpoint;
 import org.apache.drill.exec.exception.DrillbitStartupException;
-import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
-import org.apache.drill.exec.proto.CoordinationProtos.WorkQueueStatus;
+import org.apache.drill.exec.proto.ExecProtos.PlanFragment;
+import org.apache.drill.exec.proto.ExecProtos.WorkQueueStatus;
 
 
 public interface DistributedCache extends Closeable{
@@ -35,4 +36,7 @@ public interface DistributedCache extends Closeable{
   
   public void updateLocalQueueLength(int length);
   public List<WorkQueueStatus> getQueueLengths(); 
+  
+  public PlanFragment getFragment(long fragmentId);
+  public void storeFragment(PlanFragment fragment);
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/cache/HazelCache.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/cache/HazelCache.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/cache/HazelCache.java
index b7477a3..943031d 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/cache/HazelCache.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/cache/HazelCache.java
@@ -23,9 +23,10 @@ import java.io.IOException;
 import java.util.List;
 
 import org.apache.drill.common.config.DrillConfig;
+import org.apache.drill.common.proto.CoordinationProtos.DrillbitEndpoint;
 import org.apache.drill.exec.ExecConstants;
-import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
-import org.apache.drill.exec.proto.CoordinationProtos.WorkQueueStatus;
+import org.apache.drill.exec.proto.ExecProtos.PlanFragment;
+import org.apache.drill.exec.proto.ExecProtos.WorkQueueStatus;
 
 import com.beust.jcommander.internal.Lists;
 import com.google.common.cache.Cache;
@@ -133,6 +134,16 @@ public class HazelCache implements DistributedCache {
   public void close() throws IOException {
     this.instance.getLifecycleService().shutdown();
   }
+
+  @Override
+  public PlanFragment getFragment(long fragmentId) {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void storeFragment(PlanFragment fragment) {
+    throw new UnsupportedOperationException();
+  }
   
 
   

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/client/DrillClient.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/client/DrillClient.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/client/DrillClient.java
index 74474a4..ee63213 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/client/DrillClient.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/client/DrillClient.java
@@ -17,27 +17,27 @@
  ******************************************************************************/
 package org.apache.drill.exec.client;
 
+import static com.google.common.base.Preconditions.checkState;
+import static com.google.common.collect.Iterables.get;
+import static org.apache.drill.exec.proto.UserProtos.QueryResultsMode.STREAM_FULL;
+import static org.apache.drill.exec.proto.UserProtos.RunQuery.newBuilder;
 import io.netty.buffer.ByteBufAllocator;
 import io.netty.buffer.PooledByteBufAllocator;
 import io.netty.channel.nio.NioEventLoopGroup;
+
+import java.io.IOException;
+import java.util.Collection;
+
 import org.apache.drill.common.config.DrillConfig;
+import org.apache.drill.common.proto.CoordinationProtos.DrillbitEndpoint;
 import org.apache.drill.exec.coord.ClusterCoordinator;
 import org.apache.drill.exec.coord.ZKClusterCoordinator;
+import org.apache.drill.exec.proto.UserProtos.QueryHandle;
 import org.apache.drill.exec.rpc.DrillRpcFuture;
 import org.apache.drill.exec.rpc.NamedThreadFactory;
 import org.apache.drill.exec.rpc.RpcException;
 import org.apache.drill.exec.rpc.user.UserClient;
 
-import java.io.IOException;
-import java.util.Collection;
-
-import static com.google.common.base.Preconditions.checkState;
-import static com.google.common.collect.Iterables.get;
-import static org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
-import static org.apache.drill.exec.proto.UserProtos.QueryHandle;
-import static org.apache.drill.exec.proto.UserProtos.QueryResultsMode.STREAM_FULL;
-import static org.apache.drill.exec.proto.UserProtos.RunQuery.newBuilder;
-
 /**
  * Thin wrapper around a UserClient that handles connect/close and transforms String into ByteBuf
  */


[33/53] [abbrv] Updated value vectors inheritance model. Moved Mutables to separate Mutator subclasses. Broke VVs into separate files rather than one large class.

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/36793bb2/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/JSONRecordReader.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/JSONRecordReader.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/JSONRecordReader.java
index f2f97b7..e637518 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/JSONRecordReader.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/JSONRecordReader.java
@@ -1,17 +1,15 @@
 package org.apache.drill.exec.store;
 
-import com.carrotsearch.hppc.IntObjectOpenHashMap;
-import com.carrotsearch.hppc.cursors.IntObjectCursor;
-import com.carrotsearch.hppc.cursors.ObjectCursor;
-import com.fasterxml.jackson.core.JsonFactory;
-import com.fasterxml.jackson.core.JsonParser;
-import com.fasterxml.jackson.core.JsonToken;
-import com.google.common.base.Charsets;
-import com.google.common.collect.Iterables;
-import com.google.common.collect.Lists;
-import com.google.common.io.Files;
-import com.google.common.io.InputSupplier;
-import com.google.common.io.Resources;
+import static com.fasterxml.jackson.core.JsonToken.END_ARRAY;
+import static com.fasterxml.jackson.core.JsonToken.END_OBJECT;
+import static com.fasterxml.jackson.core.JsonToken.FIELD_NAME;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.nio.charset.Charset;
+import java.util.List;
+
 import org.apache.drill.common.exceptions.DrillRuntimeException;
 import org.apache.drill.common.exceptions.ExecutionSetupException;
 import org.apache.drill.common.expression.SchemaPath;
@@ -21,17 +19,34 @@ import org.apache.drill.exec.ops.FragmentContext;
 import org.apache.drill.exec.physical.impl.OutputMutator;
 import org.apache.drill.exec.proto.SchemaDefProtos;
 import org.apache.drill.exec.record.MaterializedField;
-import org.apache.drill.exec.record.vector.*;
-import org.apache.drill.exec.schema.*;
+import org.apache.drill.exec.schema.DiffSchema;
+import org.apache.drill.exec.schema.Field;
+import org.apache.drill.exec.schema.IdGenerator;
+import org.apache.drill.exec.schema.ListSchema;
+import org.apache.drill.exec.schema.NamedField;
+import org.apache.drill.exec.schema.ObjectSchema;
+import org.apache.drill.exec.schema.OrderedField;
+import org.apache.drill.exec.schema.RecordSchema;
+import org.apache.drill.exec.schema.SchemaIdGenerator;
 import org.apache.drill.exec.schema.json.jackson.JacksonHelper;
+import org.apache.drill.exec.vector.NullableBitVector;
+import org.apache.drill.exec.vector.NullableFloat4Vector;
+import org.apache.drill.exec.vector.NullableIntVector;
+import org.apache.drill.exec.vector.NullableVarChar4Vector;
+import org.apache.drill.exec.vector.TypeHelper;
+import org.apache.drill.exec.vector.ValueVector;
 
-import java.io.File;
-import java.io.IOException;
-import java.io.InputStreamReader;
-import java.nio.charset.Charset;
-import java.util.List;
-
-import static com.fasterxml.jackson.core.JsonToken.*;
+import com.carrotsearch.hppc.IntObjectOpenHashMap;
+import com.carrotsearch.hppc.cursors.ObjectCursor;
+import com.fasterxml.jackson.core.JsonFactory;
+import com.fasterxml.jackson.core.JsonParser;
+import com.fasterxml.jackson.core.JsonToken;
+import com.google.common.base.Charsets;
+import com.google.common.collect.Iterables;
+import com.google.common.collect.Lists;
+import com.google.common.io.Files;
+import com.google.common.io.InputSupplier;
+import com.google.common.io.Resources;
 
 public class JSONRecordReader implements RecordReader {
     static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(JSONRecordReader.class);
@@ -341,42 +356,40 @@ public class JSONRecordReader implements RecordReader {
             switch (minorType) {
                 case INT: {
                     holder.incAndCheckLength(32);
-                    ValueVector.NullableInt int4 = (ValueVector.NullableInt) holder.getValueVector();
-                    if (val == null) {
-                      int4.setNull(index);
-                    } else {
-                      int4.set(index, (Integer) val);
+                    NullableIntVector int4 = (NullableIntVector) holder.getValueVector();
+                    NullableIntVector.Mutator m = int4.getMutator();
+                    if (val != null) {
+                      m.set(index, (Integer) val);
                     }
                     return holder.hasEnoughSpace(32);
                 }
                 case FLOAT4: {
                     holder.incAndCheckLength(32);
-                    ValueVector.NullableFloat4 float4 = (ValueVector.NullableFloat4) holder.getValueVector();
-                    if (val == null) {
-                      float4.setNull(index);
-                    } else {
-                      float4.set(index, (Float) val);
+                    NullableFloat4Vector float4 = (NullableFloat4Vector) holder.getValueVector();
+                    NullableFloat4Vector.Mutator m = float4.getMutator();
+                    if (val != null) {
+                      m.set(index, (Float) val);
                     }
                     return holder.hasEnoughSpace(32);
                 }
                 case VARCHAR4: {
                     if (val == null) {
-                        ((ValueVector.NullableVarChar4) holder.getValueVector()).setNull(index);
                         return (index + 1) * 4 <= holder.getLength();
                     } else {
                         byte[] bytes = ((String) val).getBytes(UTF_8);
                         int length = bytes.length;
                         holder.incAndCheckLength(length);
-                        ValueVector.NullableVarChar4 varLen4 = (ValueVector.NullableVarChar4) holder.getValueVector();
-                        varLen4.set(index, bytes);
+                        NullableVarChar4Vector varLen4 = (NullableVarChar4Vector) holder.getValueVector();
+                        NullableVarChar4Vector.Mutator m = varLen4.getMutator();
+                        m.set(index, bytes);
                         return holder.hasEnoughSpace(length);
                     }
                 }
                 case BOOLEAN: {
                     holder.incAndCheckLength(1);
-                    ValueVector.NullableBit bit = (ValueVector.NullableBit) holder.getValueVector();
+                    NullableBitVector bit = (NullableBitVector) holder.getValueVector();
                     if (val != null) {
-                        bit.set(index, (Boolean)val ? 1 : 0);
+                        bit.getMutator().set(index, (Boolean)val ? 1 : 0);
                     }
                     return holder.hasEnoughSpace(1);
                 }
@@ -409,7 +422,7 @@ public class JSONRecordReader implements RecordReader {
             SchemaDefProtos.MajorType type = field.getFieldType();
             int fieldId = field.getFieldId();
             MaterializedField f = MaterializedField.create(new SchemaPath(field.getFieldName()), fieldId, parentFieldId, type);
-            ValueVector.Base v = TypeHelper.getNewVector(f, allocator);
+            ValueVector v = TypeHelper.getNewVector(f, allocator);
             v.allocateNew(batchSize);
             VectorHolder holder = new VectorHolder(batchSize, v);
             valueVectorMap.put(fieldId, holder);

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/36793bb2/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/VectorHolder.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/VectorHolder.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/VectorHolder.java
index 4043913..fa0cbd5 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/VectorHolder.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/VectorHolder.java
@@ -18,22 +18,19 @@
 
 package org.apache.drill.exec.store;
 
-import org.apache.drill.exec.memory.BufferAllocator;
-import org.apache.drill.exec.record.MaterializedField;
-import org.apache.drill.exec.record.vector.TypeHelper;
-import org.apache.drill.exec.record.vector.ValueVector;
+import org.apache.drill.exec.vector.ValueVector;
 
 public class VectorHolder {
     private int length;
-    private ValueVector.Base vector;
+    private ValueVector vector;
     private int currentLength;
 
-    VectorHolder(int length, ValueVector.Base vector) {
+    VectorHolder(int length, ValueVector vector) {
         this.length = length;
         this.vector = vector;
     }
 
-    public ValueVector.Base getValueVector() {
+    public ValueVector getValueVector() {
         return vector;
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/36793bb2/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/BitVector.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/BitVector.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/BitVector.java
new file mode 100644
index 0000000..d18a29d
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/BitVector.java
@@ -0,0 +1,123 @@
+package org.apache.drill.exec.vector;
+
+import java.util.Random;
+
+import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.record.DeadBuf;
+import org.apache.drill.exec.record.MaterializedField;
+/**
+ * Bit implements a vector of bit-width values.  Elements in the vector are accessed
+ * by position from the logical start of the vector.
+ *   The width of each element is 1 bit.
+ *   The equivalent Java primitive is an int containing the value '0' or '1'.
+ *
+ * NB: this class is automatically generated from ValueVectorTypes.tdd using FreeMarker.
+ */
+public final class BitVector extends ValueVector {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(BitVector.class);
+
+  public BitVector(MaterializedField field, BufferAllocator allocator) {
+    super(field, allocator);
+  }
+
+  /**
+   * Get the byte holding the desired bit, then mask all other bits.  Iff the result is 0, the
+   * bit was not set.
+   *
+   * @param  index   position of the bit in the vector
+   * @return 1 if set, otherwise 0
+   */
+  public int get(int index) {
+    // logger.debug("BIT GET: index: {}, byte: {}, mask: {}, masked byte: {}",
+    //             index,
+    //             data.getByte((int)Math.floor(index/8)),
+    //             (int)Math.pow(2, (index % 8)),
+    //             data.getByte((int)Math.floor(index/8)) & (int)Math.pow(2, (index % 8)));
+    return ((data.getByte((int)Math.floor(index/8)) & (int)Math.pow(2, (index % 8))) == 0) ? 0 : 1;
+  }
+
+  @Override
+  public Object getObject(int index) {
+    return new Boolean(get(index) != 0);
+  }
+
+  /**
+   * Get the size requirement (in bytes) for the given number of values.
+   */
+  @Override
+  public int getSizeFromCount(int valueCount) {
+    return (int) Math.ceil(valueCount / 8);
+  }
+
+  @Override
+  public int getAllocatedSize() {
+    return totalBytes;
+  }
+
+  public Mutator getMutator() {
+    return new Mutator();
+  }
+
+  /**
+   * Allocate a new memory space for this vector.  Must be called prior to using the ValueVector.
+   *
+   * @param valueCount  The number of values which can be contained within this vector.
+   */
+  @Override
+  public void allocateNew(int valueCount) {
+    allocateNew(getSizeFromCount(valueCount), null, valueCount);
+    for (int i = 0; i < getSizeFromCount(valueCount); i++) {
+      data.setByte(i, 0);
+    }
+  }
+
+  
+  /**
+   * MutableBit implements a vector of bit-width values.  Elements in the vector are accessed
+   * by position from the logical start of the vector.  Values should be pushed onto the vector
+   * sequentially, but may be randomly accessed.
+   *
+   * NB: this class is automatically generated from ValueVectorTypes.tdd using FreeMarker.
+   */
+  public class Mutator implements ValueVector.Mutator{
+
+    private Mutator(){}
+    
+    /**
+     * Set the bit at the given index to the specified value.
+     *
+     * @param index   position of the bit to set
+     * @param value   value to set (either 1 or 0)
+     */
+    public void set(int index, int value) {
+      byte currentByte = data.getByte((int)Math.floor(index/8));
+      if (value != 0) {
+        // true
+        currentByte |= (byte) Math.pow(2, (index % 8));
+      }
+      else if ((currentByte & (byte) Math.pow(2, (index % 8))) == (byte) Math.pow(2, (index % 8))) {
+        // false, and bit was previously set
+        currentByte -= (byte) Math.pow(2, (index % 8));
+      }
+      data.setByte((int) Math.floor(index/8), currentByte);
+    }
+
+    
+    @Override
+    public void setRecordCount(int recordCount) {
+      BitVector.this.setRecordCount(recordCount);
+    }
+
+    @Override
+    public void randomizeData() {
+      if (data != DeadBuf.DEAD_BUFFER) {
+        Random r = new Random();
+        for (int i = 0; i < data.capacity() - 1; i++) {
+          byte[] bytes = new byte[1];
+          r.nextBytes(bytes);
+          data.setByte(i, bytes[0]);
+        }
+      }
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/36793bb2/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/ValueVector.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/ValueVector.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/ValueVector.java
new file mode 100644
index 0000000..718478e
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/ValueVector.java
@@ -0,0 +1,192 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.vector;
+
+import io.netty.buffer.ByteBuf;
+
+import java.io.Closeable;
+
+import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.proto.UserBitShared.FieldMetadata;
+import org.apache.drill.exec.record.DeadBuf;
+import org.apache.drill.exec.record.MaterializedField;
+
+/**
+ * ValueVectorTypes defines a set of template-generated classes which implement type-specific
+ * value vectors.  The template approach was chosen due to the lack of multiple inheritence.  It
+ * is also important that all related logic be as efficient as possible.
+ */
+public abstract class ValueVector implements Closeable {
+
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ValueVector.class);
+
+  protected final BufferAllocator allocator;
+  protected ByteBuf data = DeadBuf.DEAD_BUFFER;
+  protected MaterializedField field;
+  protected int recordCount;
+  protected int totalBytes;
+
+  ValueVector(MaterializedField field, BufferAllocator allocator) {
+    this.allocator = allocator;
+    this.field = field;
+  }
+
+  /**
+   * Get the explicitly specified size of the allocated buffer, if available.  Otherwise
+   * calculate the size based on width and record count.
+   */
+  public abstract int getAllocatedSize();
+
+  /**
+   * Get the size requirement (in bytes) for the given number of values.  Takes derived
+   * type specs into account.
+   */
+  public abstract int getSizeFromCount(int valueCount);
+
+  /**
+   * Get the Java Object representation of the element at the specified position
+   *
+   * @param index   Index of the value to get
+   */
+  public abstract Object getObject(int index);
+
+  
+  public abstract Mutator getMutator();
+  
+  /**
+   * Return the underlying buffers associated with this vector. Note that this doesn't impact the
+   * reference counts for this buffer so it only should be used for in-context access. Also note
+   * that this buffer changes regularly thus external classes shouldn't hold a reference to
+   * it (unless they change it).
+   *
+   * @return The underlying ByteBuf.
+   */
+  public ByteBuf[] getBuffers() {
+    return new ByteBuf[]{data};
+  }
+
+  /**
+   * Returns the maximum number of values contained within this vector.
+   * @return Vector size
+   */
+  public int capacity() {
+    return getRecordCount();
+  }
+
+  /**
+   * Release supporting resources.
+   */
+  @Override
+  public void close() {
+    clear();
+  }
+
+  /**
+   * Get information about how this field is materialized.
+   * @return
+   */
+  public MaterializedField getField() {
+    return field;
+  }
+
+  /**
+   * Get the number of records allocated for this value vector.
+   * @return number of allocated records
+   */
+  public int getRecordCount() {
+    return recordCount;
+  }
+
+  /**
+   * Get the metadata for this field.
+   * @return
+   */
+  public FieldMetadata getMetadata() {
+    int len = 0;
+    for(ByteBuf b : getBuffers()){
+      len += b.writerIndex();
+    }
+    return FieldMetadata.newBuilder()
+             .setDef(getField().getDef())
+             .setValueCount(getRecordCount())
+             .setBufferLength(len)
+             .build();
+  }
+
+  /**
+   * Allocate a new memory space for this vector.  Must be called prior to using the ValueVector.
+   *
+   * @param totalBytes   Optional desired size of the underlying buffer.  Specifying 0 will
+   *                     estimate the size based on valueCount.
+   * @param sourceBuffer Optional ByteBuf to use for storage (null will allocate automatically).
+   * @param valueCount   Number of values in the vector.
+   */
+  public void allocateNew(int totalBytes, ByteBuf sourceBuffer, int valueCount) {
+    clear();
+    this.recordCount = valueCount;
+    this.totalBytes = totalBytes > 0 ? totalBytes : getSizeFromCount(valueCount);
+    this.data = (sourceBuffer != null) ? sourceBuffer : allocator.buffer(this.totalBytes);
+    this.data.retain();
+    data.readerIndex(0);
+  }
+
+  /**
+   * Allocate a new memory space for this vector.  Must be called prior to using the ValueVector.
+   *
+   * @param valueCount
+   *          The number of elements which can be contained within this vector.
+   */
+  public void allocateNew(int valueCount) {
+    allocateNew(0, null, valueCount);
+  }
+
+  /**
+   * Release the underlying ByteBuf and reset the ValueVector
+   */
+  protected void clear() {
+    if (data != DeadBuf.DEAD_BUFFER) {
+      data.release();
+      data = DeadBuf.DEAD_BUFFER;
+      recordCount = 0;
+      totalBytes = 0;
+    }
+  }
+
+  //public abstract <T extends Mutator> T getMutator();
+  
+  /**
+   * Define the number of records that are in this value vector.
+   * @param recordCount Number of records active in this vector.
+   */
+  void setRecordCount(int recordCount) {
+    data.writerIndex(getSizeFromCount(recordCount));
+    this.recordCount = recordCount;
+  }
+
+  /**
+   * For testing only -- randomize the buffer contents
+   */
+  public void randomizeData() { }
+
+  
+  public static interface Mutator{
+    public void randomizeData();
+    public void setRecordCount(int recordCount);
+  }
+}
+

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/36793bb2/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 3edf283..3fe0622 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
@@ -26,10 +26,10 @@ import org.apache.drill.exec.client.DrillClient;
 import org.apache.drill.exec.pop.PopUnitTestBase;
 import org.apache.drill.exec.proto.UserProtos.QueryType;
 import org.apache.drill.exec.record.RecordBatchLoader;
-import org.apache.drill.exec.record.vector.ValueVector;
 import org.apache.drill.exec.rpc.user.QueryResultBatch;
 import org.apache.drill.exec.server.Drillbit;
 import org.apache.drill.exec.server.RemoteServiceSet;
+import org.apache.drill.exec.vector.ValueVector;
 import org.junit.Test;
 
 import com.carrotsearch.hppc.cursors.IntObjectCursor;
@@ -61,7 +61,7 @@ public class TestSimpleFragmentRun extends PopUnitTestBase {
       // print headers.
       if (schemaChanged) {
         System.out.println("\n\n========NEW SCHEMA=========\n\n");
-        for (IntObjectCursor<ValueVector.Base> v : batchLoader) {
+        for (IntObjectCursor<ValueVector> v : batchLoader) {
 
           if (firstColumn) {
             firstColumn = false;
@@ -80,7 +80,7 @@ public class TestSimpleFragmentRun extends PopUnitTestBase {
       for (int i = 0; i < batchLoader.getRecordCount(); i++) {
         boolean first = true;
         recordCount++;
-        for (IntObjectCursor<ValueVector.Base> v : batchLoader) {
+        for (IntObjectCursor<ValueVector> v : batchLoader) {
           if (first) {
             first = false;
           } else {

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/36793bb2/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/record/vector/TestValueVector.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/record/vector/TestValueVector.java b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/record/vector/TestValueVector.java
index 5924f7d..ae4f644 100644
--- a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/record/vector/TestValueVector.java
+++ b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/record/vector/TestValueVector.java
@@ -1,19 +1,21 @@
 package org.apache.drill.exec.record.vector;
 
-import io.netty.buffer.ByteBuf;
-import org.apache.drill.exec.memory.DirectBufferAllocator;
-import org.apache.drill.exec.proto.SchemaDefProtos;
-import org.apache.drill.exec.record.MaterializedField;
-
-import static org.junit.Assert.assertArrayEquals;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 
-import org.apache.hadoop.io.UTF8;
-import org.junit.Test;
-
 import java.nio.charset.Charset;
 
+import org.apache.drill.exec.memory.DirectBufferAllocator;
+import org.apache.drill.exec.proto.SchemaDefProtos;
+import org.apache.drill.exec.record.MaterializedField;
+import org.apache.drill.exec.vector.BitVector;
+import org.apache.drill.exec.vector.NullableFloat4Vector;
+import org.apache.drill.exec.vector.NullableUInt4Vector;
+import org.apache.drill.exec.vector.NullableVarChar2Vector;
+import org.apache.drill.exec.vector.TypeHelper;
+import org.apache.drill.exec.vector.UInt4Vector;
+import org.junit.Test;
+
 public class TestValueVector {
 
   DirectBufferAllocator allocator = new DirectBufferAllocator();
@@ -34,15 +36,16 @@ public class TestValueVector {
         MaterializedField field = MaterializedField.create(defBuilder.build());
 
     // Create a new value vector for 1024 integers
-    ValueVector.MutableUInt4 v = new ValueVector.MutableUInt4(field, allocator);
+    UInt4Vector v = new UInt4Vector(field, allocator);
+    UInt4Vector.Mutator m = v.getMutator();
     v.allocateNew(1024);
 
     // Put and set a few values
-    v.set(0, 100);
-    v.set(1, 101);
-    v.set(100, 102);
-    v.set(1022, 103);
-    v.set(1023, 104);
+    m.set(0, 100);
+    m.set(1, 101);
+    m.set(100, 102);
+    m.set(1022, 103);
+    m.set(1023, 104);
     assertEquals(100, v.get(0));
     assertEquals(101, v.get(1));
     assertEquals(102, v.get(100));
@@ -69,16 +72,17 @@ public class TestValueVector {
     MaterializedField field = MaterializedField.create(defBuilder.build());
 
     // Create a new value vector for 1024 integers
-    ValueVector.NullableVarChar2 v = new ValueVector.NullableVarChar2(field, allocator);
+    NullableVarChar2Vector v = new NullableVarChar2Vector(field, allocator);
+    NullableVarChar2Vector.Mutator m = v.getMutator();
     v.allocateNew(1024);
 
     // Create and set 3 sample strings
     String str1 = new String("AAAAA1");
     String str2 = new String("BBBBBBBBB2");
     String str3 = new String("CCCC3");
-    v.set(0, str1.getBytes(Charset.forName("UTF-8")));
-    v.set(1, str2.getBytes(Charset.forName("UTF-8")));
-    v.set(2, str3.getBytes(Charset.forName("UTF-8")));
+    m.set(0, str1.getBytes(Charset.forName("UTF-8")));
+    m.set(1, str2.getBytes(Charset.forName("UTF-8")));
+    m.set(2, str3.getBytes(Charset.forName("UTF-8")));
 
     // Check the sample strings
     assertEquals(str1, new String(v.get(0), Charset.forName("UTF-8")));
@@ -86,10 +90,16 @@ public class TestValueVector {
     assertEquals(str3, new String(v.get(2), Charset.forName("UTF-8")));
 
     // Ensure null value throws
+    boolean b = false;
     try {
       v.get(3);
-      assertFalse(false);
-    } catch(NullValueException e) { }
+    } catch(AssertionError e) { 
+      b = true;
+    }finally{
+      if(!b){
+        assert false;
+      }
+    }
 
   }
 
@@ -110,15 +120,16 @@ public class TestValueVector {
     MaterializedField field = MaterializedField.create(defBuilder.build());
 
     // Create a new value vector for 1024 integers
-    ValueVector.NullableUInt4 v = new ValueVector.NullableUInt4(field, allocator);
+    NullableUInt4Vector v = new NullableUInt4Vector(field, allocator);
+    NullableUInt4Vector.Mutator m = v.getMutator();
     v.allocateNew(1024);
 
     // Put and set a few values
-    v.set(0, 100);
-    v.set(1, 101);
-    v.set(100, 102);
-    v.set(1022, 103);
-    v.set(1023, 104);
+    m.set(0, 100);
+    m.set(1, 101);
+    m.set(100, 102);
+    m.set(1022, 103);
+    m.set(1023, 104);
     assertEquals(100, v.get(0));
     assertEquals(101, v.get(1));
     assertEquals(102, v.get(100));
@@ -126,22 +137,39 @@ public class TestValueVector {
     assertEquals(104, v.get(1023));
 
     // Ensure null values throw
-    try {
-      v.get(3);
-      assertFalse(false);
-    } catch(NullValueException e) { }
-
+    {
+      boolean b = false;
+      try {
+        v.get(3);
+      } catch(AssertionError e) { 
+        b = true;
+      }finally{
+        if(!b){
+          assert false;
+        }
+      }      
+    }
+
+    
     v.allocateNew(2048);
-    try {
-      v.get(0);
-      assertFalse(false);
-    } catch(NullValueException e) { }
-
-    v.set(0, 100);
-    v.set(1, 101);
-    v.set(100, 102);
-    v.set(1022, 103);
-    v.set(1023, 104);
+    {
+      boolean b = false;
+      try {
+        v.get(0);
+      } catch(AssertionError e) { 
+        b = true;
+      }finally{
+        if(!b){
+          assert false;
+        }
+      }   
+    }
+    
+    m.set(0, 100);
+    m.set(1, 101);
+    m.set(100, 102);
+    m.set(1022, 103);
+    m.set(1023, 104);
     assertEquals(100, v.get(0));
     assertEquals(101, v.get(1));
     assertEquals(102, v.get(100));
@@ -149,10 +177,19 @@ public class TestValueVector {
     assertEquals(104, v.get(1023));
 
     // Ensure null values throw
-    try {
-      v.get(3);
-      assertFalse(false);
-    } catch(NullValueException e) { }
+    
+    {
+      boolean b = false;
+      try {
+        v.get(3);
+      } catch(AssertionError e) { 
+        b = true;
+      }finally{
+        if(!b){
+          assert false;
+        }
+      }   
+    }
 
   }
 
@@ -172,16 +209,16 @@ public class TestValueVector {
     MaterializedField field = MaterializedField.create(defBuilder.build());
 
     // Create a new value vector for 1024 integers
-    ValueVector.NullableFloat4 v = (ValueVector.NullableFloat4) TypeHelper.getNewVector(field, allocator);
-
+    NullableFloat4Vector v = (NullableFloat4Vector) TypeHelper.getNewVector(field, allocator);
+    NullableFloat4Vector.Mutator m = v.getMutator();
     v.allocateNew(1024);
 
     // Put and set a few values
-    v.set(0, 100.1f);
-    v.set(1, 101.2f);
-    v.set(100, 102.3f);
-    v.set(1022, 103.4f);
-    v.set(1023, 104.5f);
+    m.set(0, 100.1f);
+    m.set(1, 101.2f);
+    m.set(100, 102.3f);
+    m.set(1022, 103.4f);
+    m.set(1023, 104.5f);
     assertEquals(100.1f, v.get(0), 0);
     assertEquals(101.2f, v.get(1), 0);
     assertEquals(102.3f, v.get(100), 0);
@@ -189,17 +226,32 @@ public class TestValueVector {
     assertEquals(104.5f, v.get(1023), 0);
 
     // Ensure null values throw
-    try {
-      v.get(3);
-      assertFalse(false);
-    } catch(NullValueException e) { }
-
+    {
+      boolean b = false;
+      try {
+        v.get(3);
+      } catch(AssertionError e) { 
+        b = true;
+      }finally{
+        if(!b){
+          assert false;
+        }
+      }   
+    }
+    
     v.allocateNew(2048);
-    try {
-      v.get(0);
-      assertFalse(false);
-    } catch(NullValueException e) { }
-
+    {
+      boolean b = false;
+      try {
+        v.get(0);
+      } catch(AssertionError e) { 
+        b = true;
+      }finally{
+        if(!b){
+          assert false;
+        }
+      }   
+    }
   }
 
   @Test
@@ -218,30 +270,31 @@ public class TestValueVector {
     MaterializedField field = MaterializedField.create(defBuilder.build());
 
     // Create a new value vector for 1024 integers
-    ValueVector.MutableBit v = new ValueVector.MutableBit(field, allocator);
+    BitVector v = new BitVector(field, allocator);
+    BitVector.Mutator m = v.getMutator();
     v.allocateNew(1024);
 
     // Put and set a few values
-    v.set(0, 1);
-    v.set(1, 0);
-    v.set(100, 0);
-    v.set(1022, 1);
+    m.set(0, 1);
+    m.set(1, 0);
+    m.set(100, 0);
+    m.set(1022, 1);
     assertEquals(1, v.get(0));
     assertEquals(0, v.get(1));
     assertEquals(0, v.get(100));
     assertEquals(1, v.get(1022));
 
     // test setting the same value twice
-    v.set(0, 1);
-    v.set(0, 1);
-    v.set(1, 0);
-    v.set(1, 0);
+    m.set(0, 1);
+    m.set(0, 1);
+    m.set(1, 0);
+    m.set(1, 0);
     assertEquals(1, v.get(0));
     assertEquals(0, v.get(1));
 
     // test toggling the values
-    v.set(0, 0);
-    v.set(1, 1);
+    m.set(0, 0);
+    m.set(1, 1);
     assertEquals(0, v.get(0));
     assertEquals(1, v.get(1));
 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/36793bb2/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/JSONRecordReaderTest.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/JSONRecordReaderTest.java b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/JSONRecordReaderTest.java
index cef40ff..117414c 100644
--- a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/JSONRecordReaderTest.java
+++ b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/JSONRecordReaderTest.java
@@ -1,8 +1,16 @@
 package org.apache.drill.exec.store;
 
-import com.beust.jcommander.internal.Lists;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import java.io.IOException;
+import java.nio.charset.Charset;
+import java.util.Arrays;
+import java.util.List;
+
 import mockit.Expectations;
 import mockit.Injectable;
+
 import org.apache.drill.common.exceptions.ExecutionSetupException;
 import org.apache.drill.exec.exception.SchemaChangeException;
 import org.apache.drill.exec.memory.DirectBufferAllocator;
@@ -10,17 +18,11 @@ import org.apache.drill.exec.ops.FragmentContext;
 import org.apache.drill.exec.physical.impl.OutputMutator;
 import org.apache.drill.exec.proto.SchemaDefProtos;
 import org.apache.drill.exec.proto.UserBitShared;
-import org.apache.drill.exec.record.vector.ValueVector;
+import org.apache.drill.exec.vector.ValueVector;
 import org.junit.Ignore;
 import org.junit.Test;
 
-import java.io.IOException;
-import java.nio.charset.Charset;
-import java.util.Arrays;
-import java.util.List;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
+import com.beust.jcommander.internal.Lists;
 
 public class JSONRecordReaderTest {
     private static final Charset UTF_8 = Charset.forName("UTF-8");
@@ -31,7 +33,7 @@ public class JSONRecordReaderTest {
 
     class MockOutputMutator implements OutputMutator {
         List<Integer> removedFields = Lists.newArrayList();
-        List<ValueVector.Base> addFields = Lists.newArrayList();
+        List<ValueVector> addFields = Lists.newArrayList();
 
         @Override
         public void removeField(int fieldId) throws SchemaChangeException {
@@ -39,7 +41,7 @@ public class JSONRecordReaderTest {
         }
 
         @Override
-        public void addField(int fieldId, ValueVector.Base vector) throws SchemaChangeException {
+        public void addField(int fieldId, ValueVector vector) throws SchemaChangeException {
             addFields.add(vector);
         }
 
@@ -51,16 +53,16 @@ public class JSONRecordReaderTest {
             return removedFields;
         }
 
-        List<ValueVector.Base> getAddFields() {
+        List<ValueVector> getAddFields() {
             return addFields;
         }
     }
 
-    private <T> void assertField(ValueVector.Base valueVector, int index, SchemaDefProtos.MinorType expectedMinorType, T value, String name) {
+    private <T> void assertField(ValueVector valueVector, int index, SchemaDefProtos.MinorType expectedMinorType, T value, String name) {
         assertField(valueVector, index, expectedMinorType, value, name, 0);
     }
 
-    private <T> void assertField(ValueVector.Base valueVector, int index, SchemaDefProtos.MinorType expectedMinorType, T value, String name, int parentFieldId) {
+    private <T> void assertField(ValueVector valueVector, int index, SchemaDefProtos.MinorType expectedMinorType, T value, String name, int parentFieldId) {
         UserBitShared.FieldMetadata metadata = valueVector.getMetadata();
         SchemaDefProtos.FieldDef def = metadata.getDef();
         assertEquals(expectedMinorType, def.getMajorType().getMinorType());
@@ -90,7 +92,7 @@ public class JSONRecordReaderTest {
         JSONRecordReader jr = new JSONRecordReader(context, getResource("scan_json_test_1.json"));
 
         MockOutputMutator mutator = new MockOutputMutator();
-        List<ValueVector.Base> addFields = mutator.getAddFields();
+        List<ValueVector> addFields = mutator.getAddFields();
         jr.setup(mutator);
         assertEquals(2, jr.next());
         assertEquals(3, addFields.size());
@@ -116,7 +118,7 @@ public class JSONRecordReaderTest {
 
         JSONRecordReader jr = new JSONRecordReader(context, getResource("scan_json_test_2.json"));
         MockOutputMutator mutator = new MockOutputMutator();
-        List<ValueVector.Base> addFields = mutator.getAddFields();
+        List<ValueVector> addFields = mutator.getAddFields();
 
         jr.setup(mutator);
         assertEquals(3, jr.next());
@@ -142,7 +144,7 @@ public class JSONRecordReaderTest {
         assertEquals(0, jr.next());
     }
 
-    @Test
+    @Test @Ignore
     public void testChangedSchemaInTwoBatches(@Injectable final FragmentContext context) throws IOException, ExecutionSetupException {
         new Expectations() {
             {
@@ -153,7 +155,7 @@ public class JSONRecordReaderTest {
 
         JSONRecordReader jr = new JSONRecordReader(context, getResource("scan_json_test_2.json"), 64); // batch only fits 1 int
         MockOutputMutator mutator = new MockOutputMutator();
-        List<ValueVector.Base> addFields = mutator.getAddFields();
+        List<ValueVector> addFields = mutator.getAddFields();
         List<Integer> removedFields = mutator.getRemovedFields();
 
         jr.setup(mutator);
@@ -201,7 +203,7 @@ public class JSONRecordReaderTest {
         JSONRecordReader jr = new JSONRecordReader(context, getResource("scan_json_test_3.json"));
 
         MockOutputMutator mutator = new MockOutputMutator();
-        List<ValueVector.Base> addFields = mutator.getAddFields();
+        List<ValueVector> addFields = mutator.getAddFields();
         jr.setup(mutator);
         assertEquals(2, jr.next());
         assertEquals(5, addFields.size());


[02/53] [abbrv] WIP fragmentation, physical plan, byte compiling, some vector work

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/BitVector.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/BitVector.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/BitVector.java
index c157b12..83ad599 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/BitVector.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/BitVector.java
@@ -19,7 +19,7 @@ package org.apache.drill.exec.record.vector;
 
 import org.apache.drill.common.expression.types.DataType;
 import org.apache.drill.common.physical.RecordField.ValueMode;
-import org.apache.drill.exec.BufferAllocator;
+import org.apache.drill.exec.memory.BufferAllocator;
 import org.apache.drill.exec.record.MaterializedField;
 
 /**
@@ -40,17 +40,25 @@ public class BitVector extends AbstractFixedValueVector<BitVector> {
     return field;
   }
   
-  /** Returns true or false for the specified bit index.
-   * The index should be less than the OpenBitSet size
-   */
-  public boolean get(int index) {
+//  /** Returns true or false for the specified bit index.
+//   * The index should be less than the OpenBitSet size
+//   */
+//  public boolean get(int index) {
+//    assert index >= 0 && index < this.valueCount;
+//    int i = index >> 3;               // div 8
+//    // signed shift will keep a negative index and force an
+//    // array-index-out-of-bounds-exception, removing the need for an explicit check.
+//    int bit = index & 0x3f;           // mod 64
+//    long bitmask = 1L << bit;
+//    return (data.getLong(i) & bitmask) != 0;
+//  }
+  
+  public int getBit(int index) {
+    
     assert index >= 0 && index < this.valueCount;
-    int i = index >> 3;               // div 8
-    // signed shift will keep a negative index and force an
-    // array-index-out-of-bounds-exception, removing the need for an explicit check.
-    int bit = index & 0x3f;           // mod 64
-    long bitmask = 1L << bit;
-    return (data.getLong(i) & bitmask) != 0;
+    int i = 8*(index >> 6); // div 8
+    int bit = index & 0x3f; // mod 64
+    return ((int) (data.getLong(i) >>> bit)) & 0x01;
   }
   
   /** Sets the bit at the specified index.
@@ -73,41 +81,42 @@ public class BitVector extends AbstractFixedValueVector<BitVector> {
    }
    
    
+   
    /** Clears a range of bits.  Clearing past the end does not change the size of the set.
    *
-   * @param startIndex lower index
-   * @param endIndex one-past the last bit to clear
+   * @param startBitIndex lower index
+   * @param lastBitIndex one-past the last bit to clear
    */
-  private void clear(int startIndex, int endIndex) {
-    if (endIndex <= startIndex) return;
+  private void clear2(int startBitIndex, int lastBitIndex) {
+    if (lastBitIndex <= startBitIndex) return;
 
-    int startWord = (startIndex>>3);
-    if (startWord >= this.longWords) return;
+    int firstWordStart = (startBitIndex>>3);
+    if (firstWordStart >= this.longWords) return;
 
     // since endIndex is one past the end, this is index of the last
     // word to be changed.
-    int endWord   = ((endIndex-1)>>3);
+    int lastWordStart   = ((lastBitIndex-1)>>3);
 
-    long startmask = -1L << startIndex;
-    long endmask = -1L >>> -endIndex;  // 64-(endIndex&0x3f) is the same as -endIndex due to wrap
+    long startmask = -1L << startBitIndex;
+    long endmask = -1L >>> -lastBitIndex;  // 64-(endIndex&0x3f) is the same as -endIndex due to wrap
 
     // invert masks since we are clearing
     startmask = ~startmask;
     endmask = ~endmask;
 
-    if (startWord == endWord) {
-      data.setLong(startWord,  data.getLong(startWord) & (startmask | endmask));
+    if (firstWordStart == lastWordStart) {
+      data.setLong(firstWordStart,  data.getLong(firstWordStart) & (startmask | endmask));
       return;
     }
+    data.setLong(firstWordStart,  data.getLong(firstWordStart) & startmask);
 
-    data.setLong(startWord,  data.getLong(startWord) & startmask);
-
-    int middle = Math.min(this.longWords, endWord);
-    for(int i =startWord+1; i < middle; i += 8){
+    int middle = Math.min(this.longWords, lastWordStart);
+    
+    for(int i =firstWordStart+8; i < middle; i += 8){
       data.setLong(i, 0L);
     }
-    if (endWord < this.longWords) {
-      data.setLong(endWord,  data.getLong(endWord) & endmask);
+    if (lastWordStart < this.longWords) {
+      data.setLong(lastWordStart,  data.getLong(lastWordStart) & endmask);
     }
   }
   
@@ -115,4 +124,43 @@ public class BitVector extends AbstractFixedValueVector<BitVector> {
     clear(0, valueCount);
   }
 
+  
+  public void clear(int startIndex, int endIndex) {
+    if (endIndex <= startIndex) return;
+
+    int startWord = (startIndex >> 6);
+    if (startWord >= longWords) return;
+
+    // since endIndex is one past the end, this is index of the last
+    // word to be changed.
+    int endWord = ((endIndex - 1) >> 6);
+
+    long startmask = -1L << startIndex;
+    long endmask = -1L >>> -endIndex; // 64-(endIndex&0x3f) is the same as -endIndex due to wrap
+
+    // invert masks since we are clearing
+    startmask = ~startmask;
+    endmask = ~endmask;
+    
+    int startWordPos = startWord * 8;
+    if (startWord == endWord) {
+      data.setLong(startWordPos, data.getLong(startWordPos) & (startmask | endmask));
+      return;
+    }
+
+    int endWordPos = endWord * 8;
+
+    data.setLong(startWordPos, data.getLong(startWordPos) & startmask);
+
+    int middle = Math.min(longWords, endWord)*8;
+    
+    
+    for(int i =startWordPos+8; i < middle; i += 8){
+      data.setLong(i, 0L);
+    }
+    
+    if (endWordPos < startWordPos) {
+      data.setLong(endWordPos, data.getLong(endWordPos) & endmask);
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/ByteVector.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/ByteVector.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/ByteVector.java
index 258f1cc..d8e1c80 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/ByteVector.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/ByteVector.java
@@ -19,7 +19,7 @@ package org.apache.drill.exec.record.vector;
 
 import org.apache.drill.common.expression.types.DataType;
 import org.apache.drill.common.physical.RecordField.ValueMode;
-import org.apache.drill.exec.BufferAllocator;
+import org.apache.drill.exec.memory.BufferAllocator;
 import org.apache.drill.exec.record.MaterializedField;
 
 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/Int16Vector.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/Int16Vector.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/Int16Vector.java
new file mode 100644
index 0000000..779b01b
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/Int16Vector.java
@@ -0,0 +1,52 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.record.vector;
+
+import org.apache.drill.common.expression.types.DataType;
+import org.apache.drill.common.physical.RecordField.ValueMode;
+import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.record.MaterializedField;
+
+public final class Int16Vector extends AbstractFixedValueVector<Int16Vector>{
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(Int16Vector.class);
+  
+  private final MaterializedField field;
+
+  public Int16Vector(int fieldId, BufferAllocator allocator) {
+    super(fieldId, allocator, 32);
+    this.field = new MaterializedField(fieldId, DataType.INT16, false, ValueMode.VECTOR, this.getClass());
+  }
+
+  @Override
+  public MaterializedField getField() {
+    return field;
+  }
+
+  public final void set(int index, short value){
+    index*=2;
+    data.setShort(index, value);
+  }
+  
+  public final short get(int index){
+    index*=2;
+    return data.getShort(index);
+  }
+  
+  
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/Int32Vector.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/Int32Vector.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/Int32Vector.java
index d8add04..d142367 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/Int32Vector.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/Int32Vector.java
@@ -19,10 +19,10 @@ package org.apache.drill.exec.record.vector;
 
 import org.apache.drill.common.expression.types.DataType;
 import org.apache.drill.common.physical.RecordField.ValueMode;
-import org.apache.drill.exec.BufferAllocator;
+import org.apache.drill.exec.memory.BufferAllocator;
 import org.apache.drill.exec.record.MaterializedField;
 
-public class Int32Vector extends AbstractFixedValueVector<Int32Vector>{
+public final class Int32Vector extends AbstractFixedValueVector<Int32Vector>{
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(Int32Vector.class);
   
   private final MaterializedField field;

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/NullableInt32Vector.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/NullableInt32Vector.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/NullableInt32Vector.java
new file mode 100644
index 0000000..372de13
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/NullableInt32Vector.java
@@ -0,0 +1,47 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.record.vector;
+
+import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.record.MaterializedField;
+
+public final class NullableInt32Vector extends NullableValueVector<NullableInt32Vector, Int32Vector>{
+
+  public NullableInt32Vector(int fieldId, BufferAllocator allocator) {
+    super(fieldId, allocator, NullableInt32Vector.class);
+  }
+
+
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(NullableInt32Vector.class);
+  
+  
+  public int get(int index){
+    return this.value.get(index);
+  }
+  
+  public void set(int index, int value){
+    this.value.set(index, value);
+  }
+
+
+  @Override
+  protected Int32Vector getNewValueVector(int fieldId, BufferAllocator allocator) {
+    return new Int32Vector(fieldId, allocator);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/NullableValueVector.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/NullableValueVector.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/NullableValueVector.java
index b9bad6e..8e714ed 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/NullableValueVector.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/NullableValueVector.java
@@ -19,7 +19,8 @@ package org.apache.drill.exec.record.vector;
 
 import io.netty.buffer.ByteBuf;
 
-import org.apache.drill.exec.BufferAllocator;
+import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.record.MaterializedField;
 
 /**
  * Abstract class supports null versions.
@@ -29,24 +30,33 @@ abstract class NullableValueVector<T extends NullableValueVector<T, E>, E extend
 
   protected BitVector bits;
   protected E value;
+  private final MaterializedField field;
 
-  public NullableValueVector(int fieldId, BufferAllocator allocator) {
+  public NullableValueVector(int fieldId, BufferAllocator allocator, Class<T> valueClass) {
     super(fieldId, allocator);
     bits = new BitVector(fieldId, allocator);
     value = getNewValueVector(fieldId, allocator);
+    this.field = value.getField().getNullableVersion(valueClass);
   }
   
   protected abstract E getNewValueVector(int fieldId, BufferAllocator allocator);
 
+  public int isNull(int index){
+    return bits.getBit(index);
+  }
+  
   @Override
   protected int getAllocationSize(int valueCount) {
     return bits.getAllocationSize(valueCount) + value.getAllocationSize(valueCount);
   }
   
-  
+  @Override
+  public MaterializedField getField() {
+    return field;
+  }
+
   @Override
   protected void childResetAllocation(int valueCount, ByteBuf buf) {
-    super.resetAllocation(valueCount, buf);
     int firstSize = bits.getAllocationSize(valueCount);
     value.resetAllocation(valueCount, buf.slice(firstSize, value.getAllocationSize(valueCount)));
     bits.resetAllocation(valueCount, buf.slice(0, firstSize));

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/SelectionVector.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/SelectionVector.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/SelectionVector.java
new file mode 100644
index 0000000..e9faa93
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/SelectionVector.java
@@ -0,0 +1,31 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.record.vector;
+
+import io.netty.buffer.ByteBufAllocator;
+
+import org.apache.drill.exec.memory.BufferAllocator;
+
+public class SelectionVector extends UInt16Vector{
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(SelectionVector.class);
+
+  public SelectionVector(int fieldId, BufferAllocator allocator) {
+    super(fieldId, allocator);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/UInt16Vector.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/UInt16Vector.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/UInt16Vector.java
new file mode 100644
index 0000000..87c306b
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/UInt16Vector.java
@@ -0,0 +1,51 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.record.vector;
+
+import org.apache.drill.common.expression.types.DataType;
+import org.apache.drill.common.physical.RecordField.ValueMode;
+import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.record.MaterializedField;
+
+public class UInt16Vector extends AbstractFixedValueVector<Int32Vector>{
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(UInt16Vector.class);
+  
+  private final MaterializedField field;
+
+  public UInt16Vector(int fieldId, BufferAllocator allocator) {
+    super(fieldId, allocator, 16);
+    this.field = new MaterializedField(fieldId, DataType.UINT16, false, ValueMode.VECTOR, this.getClass());
+  }
+
+  @Override
+  public MaterializedField getField() {
+    return field;
+  }
+
+  public final void set(int index, char value){
+    index*=2;
+    data.setChar(index, value);
+  }
+  
+  public final char get(int index){
+    index*=2;
+    return data.getChar(index);
+  }
+  
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/ValueVector.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/ValueVector.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/ValueVector.java
index 97a9b3b..76b0e90 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/ValueVector.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/ValueVector.java
@@ -66,6 +66,7 @@ public interface ValueVector<T extends ValueVector<T>> extends Closeable {
    */
   public abstract int size();
 
+
   /**
    * Release supporting resources.
    */

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/VariableVector.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/VariableVector.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/VariableVector.java
index bae45dc..dd84c94 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/VariableVector.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/VariableVector.java
@@ -19,7 +19,7 @@ package org.apache.drill.exec.record.vector;
 
 import io.netty.buffer.ByteBuf;
 
-import org.apache.drill.exec.BufferAllocator;
+import org.apache.drill.exec.memory.BufferAllocator;
 import org.apache.drill.exec.record.DeadBuf;
 
 /** 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/BitCom.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/BitCom.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/BitCom.java
index 5c1bf21..2349899 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/BitCom.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/BitCom.java
@@ -22,9 +22,10 @@ import io.netty.channel.socket.SocketChannel;
 import io.netty.util.concurrent.GenericFutureListener;
 
 import java.io.Closeable;
+import java.util.Collection;
 
+import org.apache.drill.common.proto.CoordinationProtos.DrillbitEndpoint;
 import org.apache.drill.exec.ops.FragmentContext;
-import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
 import org.apache.drill.exec.proto.ExecProtos.FragmentHandle;
 import org.apache.drill.exec.proto.ExecProtos.FragmentStatus;
 import org.apache.drill.exec.proto.ExecProtos.PlanFragment;
@@ -41,13 +42,21 @@ import org.apache.drill.exec.rpc.RpcBus;
 public interface BitCom extends Closeable{
 
   /**
-   * Send a record batch to another node.  
+   * Routes the output of a RecordBatch to another node.  The record batch
    * @param node The node id to send the record batch to.
    * @param batch The record batch to send.
-   * @return A Future<Ack> object that can be used to determine the outcome of sending.
+   * @return A SendProgress object which can be used to monitor the sending of the batch.
    */
   public abstract DrillRpcFuture<Ack> sendRecordBatch(FragmentContext context, DrillbitEndpoint node, RecordBatch batch);
 
+  
+  /**
+   * Requests an iterator to access an incoming record batch.  
+   * @param fragmentId
+   * @return
+   */
+  public RecordBatch getReceivingRecordBatchHandle(int majorFragmentId, int minorFragmentId);
+  
   /**
    * Send a query PlanFragment to another bit.   
    * @param context
@@ -56,6 +65,9 @@ public interface BitCom extends Closeable{
    * @return
    */
   public abstract DrillRpcFuture<FragmentHandle> sendFragment(FragmentContext context, DrillbitEndpoint node, PlanFragment fragment);
+
+  public abstract void startQuery(Collection<DrillbitEndpoint> firstNodes, long queryId);
+    
   
   public abstract DrillRpcFuture<Ack> cancelFragment(FragmentContext context, DrillbitEndpoint node, FragmentHandle handle);
   
@@ -65,5 +77,13 @@ public interface BitCom extends Closeable{
   public interface TunnelListener extends GenericFutureListener<ChannelFuture> {
     public void connectionEstablished(SocketChannel channel, DrillbitEndpoint endpoint, RpcBus<?> bus);
   }
+  
+  public interface SendManager{
+    /**
+     * Sender responsible for regularly checking this value to see whether it should continue to send or yield the process
+     * @return
+     */
+    public boolean canContinue();
+  }
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/BitComHandler.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/BitComHandler.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/BitComHandler.java
index 94e3eff..b2c5cbb 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/BitComHandler.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/BitComHandler.java
@@ -21,7 +21,7 @@ import io.netty.buffer.ByteBuf;
 import io.netty.channel.ChannelFuture;
 import io.netty.channel.socket.SocketChannel;
 
-import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
+import org.apache.drill.common.proto.CoordinationProtos.DrillbitEndpoint;
 import org.apache.drill.exec.proto.ExecProtos.BitBatchChunk;
 import org.apache.drill.exec.proto.ExecProtos.BitHandshake;
 import org.apache.drill.exec.proto.ExecProtos.BitStatus;

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/BitComImpl.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/BitComImpl.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/BitComImpl.java
index dd00e04..aada154 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/BitComImpl.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/BitComImpl.java
@@ -21,12 +21,13 @@ import io.netty.channel.Channel;
 import io.netty.channel.socket.SocketChannel;
 import io.netty.util.concurrent.Future;
 
+import java.util.Collection;
 import java.util.Map;
 
+import org.apache.drill.common.proto.CoordinationProtos.DrillbitEndpoint;
 import org.apache.drill.exec.ExecConstants;
 import org.apache.drill.exec.exception.DrillbitStartupException;
 import org.apache.drill.exec.ops.FragmentContext;
-import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
 import org.apache.drill.exec.proto.ExecProtos.FragmentHandle;
 import org.apache.drill.exec.proto.ExecProtos.FragmentStatus;
 import org.apache.drill.exec.proto.ExecProtos.PlanFragment;
@@ -81,10 +82,7 @@ public class BitComImpl implements BitCom {
 //    return null;
   }
 
-  @Override
-  public DrillRpcFuture<Ack> sendRecordBatch(FragmentContext context, DrillbitEndpoint node, RecordBatch batch) {
-    return null;
-  }
+  
 
   @Override
   public DrillRpcFuture<FragmentHandle> sendFragment(FragmentContext context, DrillbitEndpoint node,
@@ -129,7 +127,6 @@ public class BitComImpl implements BitCom {
         tunnels.put(endpoint, new BitTunnel(bus));
       }
     }
-
   }
 
   public void close() {
@@ -139,4 +136,19 @@ public class BitComImpl implements BitCom {
     }
   }
 
+
+  @Override
+  public DrillRpcFuture<Ack> sendRecordBatch(FragmentContext context, DrillbitEndpoint node, RecordBatch batch) {
+    return null;
+  }
+
+  @Override
+  public RecordBatch getReceivingRecordBatchHandle(int majorFragmentId, int minorFragmentId) {
+    return null;
+  }
+
+  @Override
+  public void startQuery(Collection<DrillbitEndpoint> firstNodes, long queryId) {
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/SendProgress.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/SendProgress.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/SendProgress.java
new file mode 100644
index 0000000..8ede9e6
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/SendProgress.java
@@ -0,0 +1,22 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.rpc.bit;
+
+public class SendProgress {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(SendProgress.class);
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserClient.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserClient.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserClient.java
index cd6e15d..0088522 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserClient.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserClient.java
@@ -43,6 +43,10 @@ public class UserClient extends BasicClient<RpcType> {
   }
 
 
+  public DrillRpcFuture<QueryHandle> submitQuery(RunQuery query) throws RpcException {
+    return this.send(RpcType.RUN_QUERY, query, QueryHandle.class, null);
+  }
+  
   public DrillRpcFuture<QueryHandle> submitQuery(RunQuery query, ByteBuf data) throws RpcException {
     return this.send(RpcType.RUN_QUERY, query, QueryHandle.class, data);
   }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserServer.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserServer.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserServer.java
index fe70c85..cccaa55 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserServer.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserServer.java
@@ -66,7 +66,7 @@ public class UserServer extends BasicServer<RpcType> {
   @Override
   protected Response handle(SocketChannel channel, int rpcType, ByteBuf pBody, ByteBuf dBody) throws RpcException {
     switch (rpcType) {
-    
+
     case RpcType.HANDSHAKE_VALUE:
 //      logger.debug("Received handshake, responding in kind.");
       return new Response(RpcType.HANDSHAKE, BitToUserHandshake.getDefaultInstance(), null);

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/server/Drillbit.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/server/Drillbit.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/server/Drillbit.java
index 2961fae..c33afce 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/server/Drillbit.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/server/Drillbit.java
@@ -17,9 +17,10 @@
  ******************************************************************************/
 package org.apache.drill.exec.server;
 
-import com.google.common.io.Closeables;
+import java.net.InetAddress;
+
 import org.apache.drill.common.config.DrillConfig;
-import org.apache.drill.exec.BufferAllocator;
+import org.apache.drill.common.proto.CoordinationProtos.DrillbitEndpoint;
 import org.apache.drill.exec.ExecConstants;
 import org.apache.drill.exec.cache.DistributedCache;
 import org.apache.drill.exec.cache.HazelCache;
@@ -27,10 +28,10 @@ import org.apache.drill.exec.coord.ClusterCoordinator;
 import org.apache.drill.exec.coord.ClusterCoordinator.RegistrationHandle;
 import org.apache.drill.exec.coord.ZKClusterCoordinator;
 import org.apache.drill.exec.exception.DrillbitStartupException;
-import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
+import org.apache.drill.exec.memory.BufferAllocator;
 import org.apache.drill.exec.service.ServiceEngine;
 
-import java.net.InetAddress;
+import com.google.common.io.Closeables;
 
 /**
  * Starts, tracks and stops all the required services for a Drillbit daemon to work.
@@ -69,6 +70,7 @@ public class Drillbit {
   final ClusterCoordinator coord;
   final ServiceEngine engine;
   final DistributedCache cache;
+  final DrillConfig config;
   private RegistrationHandle handle;
 
   public Drillbit(DrillConfig config) throws Exception {
@@ -79,6 +81,7 @@ public class Drillbit {
     this.coord = new ZKClusterCoordinator(config);
     this.engine = new ServiceEngine(context);
     this.cache = new HazelCache(context.getConfig());
+    this.config = config;
   }
 
   public void run() throws Exception {

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/server/DrillbitContext.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/server/DrillbitContext.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/server/DrillbitContext.java
index e3a24d2..b08b070 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/server/DrillbitContext.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/server/DrillbitContext.java
@@ -20,13 +20,18 @@ package org.apache.drill.exec.server;
 import io.netty.channel.nio.NioEventLoopGroup;
 
 import java.util.Collection;
-import java.util.List;
 
 import org.apache.drill.common.config.DrillConfig;
-import org.apache.drill.exec.BufferAllocator;
-import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
+import org.apache.drill.common.logical.StorageEngineConfig;
+import org.apache.drill.common.proto.CoordinationProtos.DrillbitEndpoint;
+import org.apache.drill.exec.ExecConstants;
+import org.apache.drill.exec.cache.DistributedCache;
+import org.apache.drill.exec.memory.BufferAllocator;
 import org.apache.drill.exec.rpc.NamedThreadFactory;
 import org.apache.drill.exec.rpc.bit.BitCom;
+import org.apache.drill.exec.store.StorageEngine;
+
+import com.yammer.metrics.MetricRegistry;
 
 public class DrillbitContext {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillbitContext.class);
@@ -34,12 +39,14 @@ public class DrillbitContext {
   private final DrillConfig config;
   private final Drillbit underlyingBit;
   private final NioEventLoopGroup loop;
-
+  private final MetricRegistry metrics;
+  
   public DrillbitContext(DrillConfig config, Drillbit underlyingBit) {
     super();
     this.config = config;
     this.underlyingBit = underlyingBit;
     this.loop = new NioEventLoopGroup(1, new NamedThreadFactory("BitServer-"));
+    this.metrics = new MetricRegistry(config.getString(ExecConstants.METRICS_CONTEXT_NAME));
   }
   
   public DrillConfig getConfig() {
@@ -54,6 +61,9 @@ public class DrillbitContext {
     return underlyingBit.pool;
   }
   
+  public StorageEngine getStorageEngine(StorageEngineConfig config){
+    throw new UnsupportedOperationException();
+  }
   
   public NioEventLoopGroup getBitLoopGroup(){
     return loop;
@@ -63,4 +73,13 @@ public class DrillbitContext {
     return underlyingBit.engine.getBitCom();
   }
   
+  public MetricRegistry getMetrics(){
+    return metrics;
+  }
+  
+  public DistributedCache getCache(){
+    return underlyingBit.cache;
+  }
+  
+  
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/AbstractStorageEngine.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/AbstractStorageEngine.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/AbstractStorageEngine.java
new file mode 100644
index 0000000..d89b431
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/AbstractStorageEngine.java
@@ -0,0 +1,83 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.store;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.drill.common.logical.data.Scan;
+import org.apache.drill.common.proto.CoordinationProtos.DrillbitEndpoint;
+import org.apache.drill.exec.ops.FragmentContext;
+
+import com.google.common.collect.ListMultimap;
+import com.google.common.collect.Multimap;
+
+public class AbstractStorageEngine implements StorageEngine{
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(AbstractStorageEngine.class);
+
+  @Override
+  public boolean supportsRead() {
+    return false;
+  }
+
+  @Override
+  public boolean supportsWrite() {
+    return false;
+  }
+
+  @Override
+  public List<QueryOptimizerRule> getOptimizerRules() {
+    return Collections.emptyList();
+  }
+
+  @Override
+  public Collection<ReadEntry> getReadEntries(Scan scan) throws IOException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public ListMultimap<ReadEntry, DrillbitEndpoint> getReadLocations(Collection<ReadEntry> entries) {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public RecordReader getReader(FragmentContext context, ReadEntry readEntry) throws IOException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public RecordRecorder getWriter(FragmentContext context, WriteEntry writeEntry) throws IOException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public Multimap<DrillbitEndpoint, ReadEntry> getEntryAssignments(List<DrillbitEndpoint> assignments,
+      Collection<ReadEntry> entries) {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public Multimap<DrillbitEndpoint, WriteEntry> getWriteAssignments(List<DrillbitEndpoint> assignments,
+      Collection<ReadEntry> entries) {
+    throw new UnsupportedOperationException();
+  }
+  
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/StorageEngine.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/StorageEngine.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/StorageEngine.java
index 83749c7..67ea5b6 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/StorageEngine.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/StorageEngine.java
@@ -22,10 +22,11 @@ import java.util.Collection;
 import java.util.List;
 
 import org.apache.drill.common.logical.data.Scan;
+import org.apache.drill.common.proto.CoordinationProtos.DrillbitEndpoint;
 import org.apache.drill.exec.ops.FragmentContext;
-import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
 
 import com.google.common.collect.ListMultimap;
+import com.google.common.collect.Multimap;
 
 public interface StorageEngine {
   public boolean supportsRead();
@@ -58,7 +59,22 @@ public interface StorageEngine {
   public ListMultimap<ReadEntry, DrillbitEndpoint> getReadLocations(Collection<ReadEntry> entries);
 
   /**
+   * Apply read entry assignments based on the list of actually assigned Endpoints. A storage engine is allowed to
+   * update or modify the read entries based on the nature of the assignments. For example, if two blocks are initially
+   * considered separate read entries but then the storage engine realizes that the assignments for those two reads are
+   * on the same system, the storage engine may decide to collapse those entries into a single read entry that covers
+   * both original read entries.
+   * 
+   * @param assignments
+   * @param entries
+   * @return
+   */
+  public Multimap<DrillbitEndpoint, ReadEntry> getEntryAssignments(List<DrillbitEndpoint> assignments,
+      Collection<ReadEntry> entries);
+
+  /**
    * Get a particular reader for a fragment context.
+   * 
    * @param context
    * @param readEntry
    * @return
@@ -67,6 +83,19 @@ public interface StorageEngine {
   public RecordReader getReader(FragmentContext context, ReadEntry readEntry) throws IOException;
 
   /**
+   * Apply write entry assignments based on the list of actually assigned endpoints. A storage engine is allowed to
+   * rewrite the WriteEntries if desired based on the nature of the assignments. For example, a storage engine could
+   * hold off actually determining the specific level of partitioning required until it finds out exactly the number of
+   * nodes associated with the operation.
+   * 
+   * @param assignments
+   * @param entries
+   * @return
+   */
+  public Multimap<DrillbitEndpoint, WriteEntry> getWriteAssignments(List<DrillbitEndpoint> assignments,
+      Collection<ReadEntry> entries);
+
+  /**
    * 
    * @param context
    * @param writeEntry
@@ -75,7 +104,6 @@ public interface StorageEngine {
    */
   public RecordRecorder getWriter(FragmentContext context, WriteEntry writeEntry) throws IOException;
 
-  
   public interface ReadEntry {
     public Cost getCostEstimate();
   }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/StorageEngineRegistry.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/StorageEngineRegistry.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/StorageEngineRegistry.java
index b2e31e9..eef878e 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/StorageEngineRegistry.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/StorageEngineRegistry.java
@@ -51,14 +51,14 @@ public class StorageEngineRegistry {
       for(Constructor<?> c : engine.getConstructors()){
         Class<?>[] params = c.getParameterTypes();
         if(params.length != 2 || params[1] == DrillbitContext.class || !StorageEngineConfig.class.isAssignableFrom(params[0])){
-          logger.debug("Skipping ReferenceStorageEngine constructor {} for engine class {} since it doesn't implement a [constructor(StorageEngineConfig, DrillbitContext)]", c, engine);
+          logger.debug("Skipping StorageEngine constructor {} for engine class {} since it doesn't implement a [constructor(StorageEngineConfig, DrillbitContext)]", c, engine);
           continue;
         }
         availableEngines.put(params[0], (Constructor<? extends StorageEngine>) c);
         i++;
       }
       if(i == 0){
-        logger.debug("Skipping registration of ReferenceStorageEngine {} as it doesn't have a constructor with the parameters of (StorangeEngineConfig, Config)", engine.getCanonicalName());
+        logger.debug("Skipping registration of StorageEngine {} as it doesn't have a constructor with the parameters of (StorangeEngineConfig, Config)", engine.getCanonicalName());
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/exec/java-exec/src/main/protobuf/Coordination.proto
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/protobuf/Coordination.proto b/sandbox/prototype/exec/java-exec/src/main/protobuf/Coordination.proto
deleted file mode 100644
index 43c408d..0000000
--- a/sandbox/prototype/exec/java-exec/src/main/protobuf/Coordination.proto
+++ /dev/null
@@ -1,32 +0,0 @@
-package exec;
-
-option java_package = "org.apache.drill.exec.proto";
-option java_outer_classname = "CoordinationProtos";
-option optimize_for = LITE_RUNTIME;
-
-message DrillbitEndpoint{
-  optional string address = 1;
-  optional int32 user_port = 2;
-  optional int32 bit_port = 3;
-  optional Roles roles = 4;
-}
-
-message DrillServiceInstance{
-  optional string id = 1;
-  optional int64 registrationTimeUTC = 2;
-  optional DrillbitEndpoint endpoint = 3;
-}
-
-message WorkQueueStatus{
-	optional DrillbitEndpoint endpoint = 1;
-	optional int32 queue_length = 2;
-	optional int64 report_time = 3;
-}
-
-message Roles{
-	optional bool sql_query = 1 [default = true];
-	optional bool logical_plan = 2 [default = true];
-	optional bool physical_plan = 3 [default = true];
-	optional bool java_executor = 4 [default = true];
-	optional bool distributed_cache = 5 [default = true];
-}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/exec/java-exec/src/main/protobuf/ExecutionProtos.proto
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/protobuf/ExecutionProtos.proto b/sandbox/prototype/exec/java-exec/src/main/protobuf/ExecutionProtos.proto
index cd8bda2..77a7ee1 100644
--- a/sandbox/prototype/exec/java-exec/src/main/protobuf/ExecutionProtos.proto
+++ b/sandbox/prototype/exec/java-exec/src/main/protobuf/ExecutionProtos.proto
@@ -29,37 +29,63 @@ enum RpcType {
 }
 
 
-message BitColumnData {
+message BitHandshake{
+	optional DrillbitEndpoint endpoint = 1;
+}
 
-    enum ColumnEncoding {
-      PROTOBUF = 0;
-    }
-    
-	message BitColumn {
-		optional int32 field = 1;
-		optional int32 length = 2;
-		optional ColumnEncoding mode = 3;
-	}	
+message BitBatchChunk {}
+
+message BitStatus {
+	repeated ActiveFragment fragment = 1;
+}
+
+message ActiveFragment {
+	optional FragmentStatus status = 1;
+	optional int64 fragment_id = 2;
+	optional int64 query_id = 3; 
+}
+
+message FragmentStatus {
 	
-	optional SchemaDef schema = 1;
-	optional int32 record_count = 2;
-	optional int32 total_size = 3;
-	repeated BitColumn column = 4;
+	enum FragmentState {
+	  AWAITING_ALLOCATION = 0;
+	  RUNNING = 1;
+	  FINISHED = 2;
+	  CANCELLED = 3;
+	  FAILED = 4;
+	}
 	
+	optional int64 memory_use = 1;
+	optional int64 batches_completed = 2;
+	optional int64 records_completed = 3;
+	optional int32 estimated_completion_percentage = 4;
+	optional FragmentState state = 5;
+	optional int64 data_processed = 6;
 }
 
-
-message BitHandshake{
-	optional DrillbitEndpoint endpoint = 1;
+message RecordBatchHeader {
 }
 
-message BitBatchChunk {}
-message BitStatus {}
-message FragmentStatus {}
-message RecordBatchHeader {}
-message PlanFragment {}
+message PlanFragment {
+	optional int64 query_id = 1;
+	optional int32 major_fragment_id = 2;
+	optional int32 minor_fragment_id = 3;
+	optional float network_cost = 4;
+	optional float cpu_cost = 5;
+	optional float disk_cost = 6;
+	optional float memory_cost = 7;
+	optional string fragment_json = 8;
+	optional bool self_driven = 9;
+	optional DrillbitEndpoint assignment = 10;
+}
 
 message FragmentHandle {
-	optional int64 fragment_id = 1;
+	optional int32 major_fragment_id = 1;
+	optional int32 minor_fragment_id = 1;
 }
 
+message WorkQueueStatus{
+	optional DrillbitEndpoint endpoint = 1;
+	optional int32 queue_length = 2;
+	optional int64 report_time = 3;
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/exec/java-exec/src/main/protobuf/SchemaDef.proto
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/protobuf/SchemaDef.proto b/sandbox/prototype/exec/java-exec/src/main/protobuf/SchemaDef.proto
index 44e2df9..6e983d4 100644
--- a/sandbox/prototype/exec/java-exec/src/main/protobuf/SchemaDef.proto
+++ b/sandbox/prototype/exec/java-exec/src/main/protobuf/SchemaDef.proto
@@ -27,11 +27,18 @@ message SchemaDef {
   repeated FieldDef field = 1;
 }
 
+enum ValueMode {
+	VALUE_VECTOR = 0;
+	RLE = 1;
+	DICT = 2;
+}
+
 message FieldDef {
   optional string name = 1;
-  optional DataMode mode = 2;
+  optional DataMode data_mode = 2;
+  optional ValueMode value_mode = 3;
   
   // If DataMode == 0-2, type should be populated and fields should be empty.  Otherwise, type should empty and fields should be defined. 
-  optional DataType type = 3;
-  repeated FieldDef fields = 4;
+  optional DataType type = 4;
+  repeated FieldDef fields = 5;
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/exec/java-exec/src/test/java/BBOutputStream.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/java/BBOutputStream.java b/sandbox/prototype/exec/java-exec/src/test/java/BBOutputStream.java
deleted file mode 100644
index 7f03dfa..0000000
--- a/sandbox/prototype/exec/java-exec/src/test/java/BBOutputStream.java
+++ /dev/null
@@ -1,38 +0,0 @@
-import java.io.IOException;
-import java.io.OutputStream;
-import java.nio.ByteBuffer;
-
-/*******************************************************************************
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- ******************************************************************************/
-
-public class BBOutputStream extends OutputStream{
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(BBOutputStream.class);
-
-  private ByteBuffer buf;
-  
-  public BBOutputStream(ByteBuffer buf) {
-    this.buf = buf;
-  }
-
-  @Override
-  public void write(int b) throws IOException {
-    buf.put((byte) b);
-  }
-  
-  
-}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/exec/java-exec/src/test/java/CompressingBytesColumn.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/java/CompressingBytesColumn.java b/sandbox/prototype/exec/java-exec/src/test/java/CompressingBytesColumn.java
deleted file mode 100644
index 4fb67ed..0000000
--- a/sandbox/prototype/exec/java-exec/src/test/java/CompressingBytesColumn.java
+++ /dev/null
@@ -1,46 +0,0 @@
-import java.io.OutputStream;
-import java.nio.ByteBuffer;
-
-/*******************************************************************************
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- ******************************************************************************/
-
-public class CompressingBytesColumn {
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(CompressingBytesColumn.class);
-
-  int mb = 1024*1024;
-  
-  ByteBuffer values = ByteBuffer.allocateDirect(20*mb);
-  ByteBuffer fromCompressBuffer = ByteBuffer.allocateDirect(mb);
-  ByteBuffer toCompressBuffer = ByteBuffer.allocateDirect(mb);
-
-  
-  public CompressingBytesColumn(){
-  }
-  
-  public void add(byte[] bytes, int start, int length){
-    
-  }
-  
-  public void add(ByteBuffer buffer){
-    
-  }
-  public void write(OutputStream stream){
-    
-  }
-  
-}

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

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/exec/java-exec/src/test/java/GenerateExternalSortData.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/java/GenerateExternalSortData.java b/sandbox/prototype/exec/java-exec/src/test/java/GenerateExternalSortData.java
deleted file mode 100644
index dca7d27..0000000
--- a/sandbox/prototype/exec/java-exec/src/test/java/GenerateExternalSortData.java
+++ /dev/null
@@ -1,124 +0,0 @@
-import java.io.BufferedReader;
-import java.nio.ByteBuffer;
-import java.nio.IntBuffer;
-import java.nio.file.FileSystems;
-import java.nio.file.Files;
-
-import org.apache.hadoop.conf.Configuration;
-import org.xerial.snappy.Snappy;
-
-import com.google.common.base.Charsets;
-import com.google.protobuf.CodedOutputStream;
-
-/*******************************************************************************
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- ******************************************************************************/
-
-public class GenerateExternalSortData {
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(GenerateExternalSortData.class);
-  
-  /** Convert sequence file in to compressed columnar format. 
-   * 
-   * @param args
-   * @throws Exception
-   */
-  public static void main(String[] args) throws Exception{
-    int mb = 1024*1024;
-    final int blockSize = 1024;
-    ByteBuffer keys = ByteBuffer.allocateDirect(2*mb);
-    ByteBuffer values = ByteBuffer.allocateDirect(20*mb);
-    ByteBuffer fromCompressBuffer = ByteBuffer.allocateDirect(mb);
-    ByteBuffer toCompressBuffer = ByteBuffer.allocateDirect(mb);
-    
-    ByteBuffer valueLengthB = ByteBuffer.allocateDirect(blockSize*4);
-    IntBuffer valueLengths = valueLengthB.asIntBuffer();
-    //Opaque value stored as len,data.
-    
-    //
-    //Snappy.compress(uncompressed, compressed);
-    String file = "/opt/data/tera1gb/part-00000";
-    Configuration config = new Configuration();
-    //SequenceFile.Reader sf = new SequenceFile.Reader(FileSystem.getLocal(config), new Path(file), config);
-    
-    BufferedReader reader = Files.newBufferedReader(FileSystems.getDefault().getPath(file), Charsets.UTF_8);
-    
-    CodedOutputStream cos = CodedOutputStream.newInstance(new BBOutputStream(values));
-    
-    long originalBytes = 0;
-    long compressedBytes = 0;
-    String l;
-    int round = 0;
-    long nanos = 0;
-    long x1 = System.nanoTime();
-    while((l = reader.readLine()) != null){
-      
-      byte[] bytes = l.getBytes();
-      keys.put(bytes, 0, 10);
-      int len = bytes.length - 10;
-      originalBytes += len;
-      
-      
-      // Compress the value.
-      long n1 = System.nanoTime();
-      fromCompressBuffer.put(bytes, 10, len);
-      fromCompressBuffer.flip();
-      int newLen = Snappy.compress(fromCompressBuffer, toCompressBuffer);
-      cos.writeRawVarint32(newLen);
-      toCompressBuffer.flip();
-      values.put(toCompressBuffer);
-      fromCompressBuffer.clear();
-      toCompressBuffer.clear();
-      nanos += (System.nanoTime() - n1);
-
-      compressedBytes += newLen;
-      //valueLengths.put(newLen);
-      
-      round++;
-      
-      if(round >= blockSize){
-        // flush
-        keys.clear();
-        values.clear();
-        round = 0;
-        
-      }
-      
-      
-    }
-    
-    System.out.println("Uncompressed: " + originalBytes);
-    System.out.println("Compressed: " + compressedBytes);
-    System.out.println("CompressionTime: " + nanos/1000/1000);
-    System.out.println("Total Time: " + (System.nanoTime() - x1)/1000/1000);
-    
-  }
-  
-  private static void convertToDeltas(IntBuffer b){
-    b.flip();
-    int min = Integer.MAX_VALUE;
-    for(int i =0; i < b.limit(); i++){
-      min = Math.min(b.get(i), min);
-    }
-    
-    for(int i =0; i < b.limit(); i++){
-      int cur = b.get(i);
-      b.put(i, cur - min);
-    }
-    
-    
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/compile/ExampleExternalInterface.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/compile/ExampleExternalInterface.java b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/compile/ExampleExternalInterface.java
new file mode 100644
index 0000000..66d2fac
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/compile/ExampleExternalInterface.java
@@ -0,0 +1,23 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.compile;
+
+public interface ExampleExternalInterface {
+
+  public String getData();
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/compile/ExampleInternalInterface.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/compile/ExampleInternalInterface.java b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/compile/ExampleInternalInterface.java
new file mode 100644
index 0000000..7ddf103
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/compile/ExampleInternalInterface.java
@@ -0,0 +1,24 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.compile;
+
+public interface ExampleInternalInterface {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ExampleInternalInterface.class);
+  
+  public String getInternalData();
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/compile/ExampleTemplate.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/compile/ExampleTemplate.java b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/compile/ExampleTemplate.java
new file mode 100644
index 0000000..e8ede05
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/compile/ExampleTemplate.java
@@ -0,0 +1,30 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.compile;
+
+public abstract class ExampleTemplate implements ExampleExternalInterface{
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ExampleTemplate.class);
+
+  @Override
+  public final String getData() {
+    return this.getInternalData();
+  }
+  
+  abstract String getInternalData();
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/compile/TestClassCompilationTypes.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/compile/TestClassCompilationTypes.java b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/compile/TestClassCompilationTypes.java
new file mode 100644
index 0000000..3d5d84e
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/compile/TestClassCompilationTypes.java
@@ -0,0 +1,67 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.compile;
+
+import org.codehaus.commons.compiler.jdk.ExpressionEvaluator;
+import org.junit.Test;
+
+public class TestClassCompilationTypes {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestClassCompilationTypes.class);
+
+  @Test
+  public void comparePerfs() throws Exception {
+    for(int i =0; i < 50000; i++){
+      int r = 0;
+      long n0 = System.nanoTime();
+      r += janino();
+      long n1 = System.nanoTime();
+      //r += jdk();
+      long n2 = System.nanoTime();
+      long janinoT = (n1 - n0)/1000;
+      long jdkT = (n2 - n1)/1000;
+      System.out.println("Janino: " + janinoT + "micros.  JDK: " + jdkT + "micros. Val" + r);
+    }
+
+  }
+  
+  private int janino() throws Exception{
+    // Compile the expression once; relatively slow.
+    org.codehaus.janino.ExpressionEvaluator ee = new org.codehaus.janino.ExpressionEvaluator("c > d ? c : d", // expression
+        int.class, // expressionType
+        new String[] { "c", "d" }, // parameterNames
+        new Class[] { int.class, int.class } // parameterTypes
+    );
+
+    // Evaluate it with varying parameter values; very fast.
+    return (Integer) ee.evaluate(new Object[] { // parameterValues
+        new Integer(10), new Integer(11), });
+  }
+  
+  private int jdk() throws Exception{
+    // Compile the expression once; relatively slow.
+    ExpressionEvaluator ee = new ExpressionEvaluator("c > d ? c : d", // expression
+        int.class, // expressionType
+        new String[] { "c", "d" }, // parameterNames
+        new Class[] { int.class, int.class } // parameterTypes
+    );
+
+    // Evaluate it with varying parameter values; very fast.
+    return  (Integer) ee.evaluate(new Object[] { // parameterValues
+        new Integer(10), new Integer(11), });
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/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
new file mode 100644
index 0000000..47aee7d
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/compile/TestClassTransformation.java
@@ -0,0 +1,53 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.compile;
+
+import static org.junit.Assert.*;
+
+import org.apache.drill.exec.exception.ClassTransformationException;
+import org.junit.Test;
+
+public class TestClassTransformation {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestClassTransformation.class);
+
+  @Test
+  public void testJaninoCompilation() throws ClassTransformationException {
+    testBasicClassCompilation(true);
+  }
+  
+  @Test 
+  public void testJDKCompilation() throws ClassTransformationException{
+    testBasicClassCompilation(false);
+  }
+  
+  private void testBasicClassCompilation(boolean useJanino) throws ClassTransformationException{
+    final String output = "hello world, the time is now " + System.currentTimeMillis();
+    @SuppressWarnings("unchecked")
+    TemplateClassDefinition<ExampleExternalInterface, Object> def = new TemplateClassDefinition(
+        ExampleExternalInterface.class, "org.apache.drill.exec.compile.ExampleTemplate",
+        ExampleInternalInterface.class, Object.class);
+    ClassTransformer ct = new ClassTransformer();
+    QueryClassLoader loader = new QueryClassLoader(useJanino);
+    ExampleExternalInterface instance = ct.getImplementationClass(loader, def,
+        "public String getInternalData(){return \"" + output + "\";}",
+        new Object());
+    System.out.println(String.format("Generated a new class %s that provides the following getData response '%s'.",
+        instance.getClass().getCanonicalName(), instance.getData()));
+    assertEquals(instance.getData(), output);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/pop/CheckFragmenter.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/pop/CheckFragmenter.java b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/pop/CheckFragmenter.java
new file mode 100644
index 0000000..98bb874
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/pop/CheckFragmenter.java
@@ -0,0 +1,86 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.pop;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+
+import java.io.IOException;
+
+import org.apache.drill.common.config.DrillConfig;
+import org.apache.drill.common.physical.PhysicalPlan;
+import org.apache.drill.common.physical.pop.base.PhysicalOperator;
+import org.apache.drill.common.util.FileUtils;
+import org.apache.drill.exec.exception.FragmentSetupException;
+import org.apache.drill.exec.planner.FragmentNode;
+import org.apache.drill.exec.planner.FragmentingPhysicalVisitor;
+import org.apache.drill.exec.planner.FragmentNode.ExchangeFragmentPair;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import com.google.common.base.Charsets;
+import com.google.common.io.Files;
+
+public class CheckFragmenter {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(CheckFragmenter.class);
+  
+  static DrillConfig config;
+  
+  @BeforeClass
+  public static void setup(){
+    config = DrillConfig.create();
+  }
+  
+  @Test
+  public void ensureOneFragment() throws FragmentSetupException, IOException{
+    FragmentNode b = getRootFragment("/physical_test1.json");
+    assertEquals(1, getFragmentCount(b));
+    assertEquals(0, b.getReceivingExchangePairs().size());
+    assertNull(b.getSendingExchange());
+  }
+  
+  @Test
+  public void ensureTwoFragments() throws FragmentSetupException, IOException{
+    FragmentNode b = getRootFragment("/physical_simpleexchange.json");
+    assertEquals(2, getFragmentCount(b));
+    assertEquals(1, b.getReceivingExchangePairs().size());
+    assertNull(b.getSendingExchange());
+    
+    // get first child.
+    b = b.iterator().next().getNode();
+    assertEquals(0, b.getReceivingExchangePairs().size());
+    assertNotNull(b.getSendingExchange());
+  }
+  
+  private int getFragmentCount(FragmentNode b){
+    int i =1;
+    for(ExchangeFragmentPair p : b){
+      i += getFragmentCount(p.getNode());
+    }
+    return i;
+  }
+  
+  private FragmentNode getRootFragment(String file) throws FragmentSetupException, IOException{
+    FragmentingPhysicalVisitor f = new FragmentingPhysicalVisitor();
+    
+    PhysicalPlan plan = PhysicalPlan.parse(config.getMapper().reader(PhysicalPlan.class), Files.toString(FileUtils.getResourceAsFile(file), Charsets.UTF_8));
+    PhysicalOperator o = plan.getSortedOperators(false).iterator().next();
+    return o.accept(f, null);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/pop/CheckInjectionValue.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/pop/CheckInjectionValue.java b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/pop/CheckInjectionValue.java
new file mode 100644
index 0000000..b8fd278
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/pop/CheckInjectionValue.java
@@ -0,0 +1,61 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.pop;
+
+import static org.junit.Assert.*;
+
+import java.util.List;
+
+import org.apache.drill.common.config.DrillConfig;
+import org.apache.drill.common.physical.PhysicalPlan;
+import org.apache.drill.common.physical.pop.Screen;
+import org.apache.drill.common.physical.pop.base.PhysicalOperator;
+import org.apache.drill.common.proto.CoordinationProtos.DrillbitEndpoint;
+import org.apache.drill.common.util.FileUtils;
+import org.apache.drill.exec.planner.PhysicalPlanReader;
+import org.apache.drill.exec.server.DrillbitContext;
+import org.apache.drill.exec.store.StorageEngineRegistry;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import com.google.common.base.Charsets;
+import com.google.common.io.Files;
+
+public class CheckInjectionValue {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(CheckInjectionValue.class);
+  
+  static DrillConfig config;
+  
+  @BeforeClass
+  public static void setup(){
+    config = DrillConfig.create();
+  }
+  
+  @Test
+  public void testInjected() throws Exception{
+    PhysicalPlanReader r = new PhysicalPlanReader(config.getMapper(), DrillbitEndpoint.getDefaultInstance());
+    PhysicalPlan p = r.read(Files.toString(FileUtils.getResourceAsFile("/physical_screen.json"), Charsets.UTF_8));
+    
+    List<PhysicalOperator> o = p.getSortedOperators(false);
+    
+    PhysicalOperator op = o.iterator().next();
+    assertEquals(Screen.class, op.getClass());
+    Screen s = (Screen) op;
+    assertEquals(DrillbitEndpoint.getDefaultInstance(), s.getEndpoint());
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/rpc/user/RunRemoteQuery.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/rpc/user/RunRemoteQuery.java b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/rpc/user/RunRemoteQuery.java
new file mode 100644
index 0000000..d003373
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/rpc/user/RunRemoteQuery.java
@@ -0,0 +1,41 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.rpc.user;
+
+import io.netty.buffer.UnpooledByteBufAllocator;
+import io.netty.channel.nio.NioEventLoopGroup;
+
+import org.apache.drill.exec.proto.UserProtos.QueryHandle;
+import org.apache.drill.exec.proto.UserProtos.RunQuery;
+import org.apache.drill.exec.rpc.DrillRpcFuture;
+import org.junit.Test;
+
+public class RunRemoteQuery {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(RunRemoteQuery.class);
+  
+  @Test 
+  public void runRemoteQuery() throws Exception{
+    UserClient c = new UserClient(UnpooledByteBufAllocator.DEFAULT, new NioEventLoopGroup(1));
+    c.connectAsClient("localhost", 31010);
+    DrillRpcFuture<QueryHandle> futureHandle = c.submitQuery(RunQuery.getDefaultInstance());
+    QueryHandle h = futureHandle.checkedGet();
+    System.out.println(h);
+  }
+  
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/server/StartDrillbit.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/server/StartDrillbit.java b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/server/StartDrillbit.java
index 7b353df..ce79661 100644
--- a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/server/StartDrillbit.java
+++ b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/server/StartDrillbit.java
@@ -24,8 +24,8 @@ public class StartDrillbit {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(StartDrillbit.class);
   
   
-  @Test public void startDrillbit() throws DrillbitStartupException, InterruptedException{
+  @Test
+  public void startDrillbit() throws DrillbitStartupException, InterruptedException{
     Drillbit.main(new String[0]);
-    
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0be80dc/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/MockRecordConfig.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/MockRecordConfig.java b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/MockRecordConfig.java
new file mode 100644
index 0000000..18c6955
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/MockRecordConfig.java
@@ -0,0 +1,46 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.store;
+
+import org.apache.drill.common.expression.types.DataType;
+import org.apache.drill.common.logical.StorageEngineConfigBase;
+
+import com.fasterxml.jackson.annotation.JsonTypeName;
+
+@JsonTypeName("mock-config")
+public class MockRecordConfig extends StorageEngineConfigBase{
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(MockRecordConfig.class);
+  
+  private int recordCount;
+  private DataType[] types;
+  
+  public int getRecordCount() {
+    return recordCount;
+  }
+  public void setRecordCount(int recordCount) {
+    this.recordCount = recordCount;
+  }
+  public DataType[] getTypes() {
+    return types;
+  }
+  public void setTypes(DataType[] types) {
+    this.types = types;
+  }
+  
+  
+}


Re: [52/53] [abbrv] git commit: Updates to support SelectionVector removal

Posted by Ted Dunning <te...@gmail.com>.
On Fri, Jul 19, 2013 at 6:58 PM, <ja...@apache.org> wrote:

> Updates to support SelectionVector removal
>
>
> Project: http://git-wip-us.apache.org/repos/asf/incubator-drill/repo
> Commit:
> http://git-wip-us.apache.org/repos/asf/incubator-drill/commit/f0013215
> Tree: http://git-wip-us.apache.org/repos/asf/incubator-drill/tree/f0013215
> Diff: http://git-wip-us.apache.org/repos/asf/incubator-drill/diff/f0013215
>
> Branch: refs/heads/master
> Commit: f00132151a7946c9541f6317efb763325aef2874
> Parents: 7779a72
> Author: Jacques Nadeau <ja...@apache.org>
> Authored: Fri Jul 19 14:40:57 2013 -0700
> Committer: Jacques Nadeau <ja...@apache.org>
> Committed: Fri Jul 19 14:57:51 2013 -0700
>
> ----------------------------------------------------------------------
>  .../common/expression/fn/StringFunctions.java   |   4 +-
>  .../org/apache/drill/common/types/Types.java    |  19 +-
>  .../common/src/main/protobuf/Types.proto        |   2 -
>  .../ValueVectors/data/ValueVectorTypes.tdd      |  17 +-
>  .../templates/FixedValueVectors.java            |  32 +-
>  .../templates/NullableValueVectors.java         |  40 +--
>  .../templates/RepeatedValueVectors.java         |   5 +
>  .../templates/VariableLengthVectors.java        |  34 ++-
>  .../drill/exec/compile/QueryClassLoader.java    |  18 +-
>  .../apache/drill/exec/expr/CodeGenerator.java   |  21 +-
>  .../drill/exec/expr/annotations/Workspace.java  |  14 +
>  .../apache/drill/exec/ops/FragmentContext.java  |   2 +-
>  .../physical/config/SelectionVectorRemover.java |  60 ++++
>  .../drill/exec/physical/impl/ImplCreator.java   |  13 +
>  .../drill/exec/physical/impl/ScanBatch.java     |   2 +-
>  .../physical/impl/filter/FilterRecordBatch.java |   2 +-
>  .../impl/project/ProjectRecordBatch.java        |   4 +-
>  .../exec/physical/impl/svremover/Copier.java    |  18 ++
>  .../physical/impl/svremover/CopierTemplate.java |  43 +++
>  .../physical/impl/svremover/CopyEvaluator.java  |  11 +
>  .../impl/svremover/RemovingRecordBatch.java     | 305 +++++++++++++++++++
>  .../impl/svremover/SVRemoverCreator.java        |  23 ++
>  .../apache/drill/exec/record/RecordBatch.java   |  46 +--
>  .../drill/exec/record/RecordBatchLoader.java    |   2 +-
>  .../drill/exec/record/RecordRemapper.java       |   8 +
>  .../apache/drill/exec/record/WritableBatch.java |  39 +--
>  .../drill/exec/vector/BaseDataValueVector.java  |   4 +-
>  .../drill/exec/vector/BaseValueVector.java      |   3 +
>  .../org/apache/drill/exec/vector/BitVector.java |  21 +-
>  .../drill/exec/vector/FixedWidthVector.java     |   2 +
>  .../drill/exec/vector/NonRepeatedMutator.java   |   2 +-
>  .../apache/drill/exec/vector/ValueVector.java   |   2 +
>  .../drill/exec/vector/VariableWidthVector.java  |   2 +
>  .../exec/physical/impl/SimpleRootExec.java      |  13 +-
>  .../physical/impl/filter/TestSimpleFilter.java  |   4 +-
>  .../impl/project/TestSimpleProjection.java      |   3 +-
>  .../physical/impl/svremover/TestSVRemover.java  |  64 ++++
>  .../src/test/resources/remover/test1.json       |  39 +++
>  .../src/test/resources/scan_screen_logical.json |   2 +-
>  39 files changed, 813 insertions(+), 132 deletions(-)
> ----------------------------------------------------------------------
>
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0013215/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/fn/StringFunctions.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/fn/StringFunctions.java
> b/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/fn/StringFunctions.java
> index 568b209..565843f 100644
> ---
> a/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/fn/StringFunctions.java
> +++
> b/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/fn/StringFunctions.java
> @@ -31,8 +31,8 @@ public class StringFunctions implements CallProvider{
>    public FunctionDefinition[] getFunctionDefintions() {
>      return new FunctionDefinition[]{
>          FunctionDefinition.simple("regex_like", new
> BasicArgumentValidator( //
> -            new Arg(true, false, "pattern", MinorType.VARCHAR1,
> MinorType.VARCHAR2, MinorType.VARCHAR4), //
> -            new Arg(false, true, "value", MinorType.FIXEDCHAR,
> MinorType.VARCHAR1, MinorType.VARCHAR2, MinorType.VARCHAR4) ),
> FixedType.FIXED_BOOLEAN),
> +            new Arg(true, false, "pattern", MinorType.VARCHAR2,
> MinorType.VARCHAR4), //
> +            new Arg(false, true, "value", MinorType.FIXEDCHAR,
> MinorType.VARCHAR2, MinorType.VARCHAR4) ), FixedType.FIXED_BOOLEAN),
>      };
>
>    }
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0013215/sandbox/prototype/common/src/main/java/org/apache/drill/common/types/Types.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/common/src/main/java/org/apache/drill/common/types/Types.java
> b/sandbox/prototype/common/src/main/java/org/apache/drill/common/types/Types.java
> index e1343ab..757bed0 100644
> ---
> a/sandbox/prototype/common/src/main/java/org/apache/drill/common/types/Types.java
> +++
> b/sandbox/prototype/common/src/main/java/org/apache/drill/common/types/Types.java
> @@ -39,11 +39,27 @@ public class Types {
>      }
>    }
>
> +  public static boolean isFixedWidthType(MajorType type){
> +    switch(type.getMinorType()){
> +    case MSGPACK2:
> +    case MSGPACK4:
> +    case PROTO2:
> +    case PROTO4:
> +    case VARBINARY2:
> +    case VARBINARY4:
> +    case VARCHAR2:
> +    case VARCHAR4:
> +      return false;
> +    default:
> +      return true;
> +    }
> +  }
> +
> +
>    public static boolean isStringScalarType(MajorType type){
>      if(type.getMode() == DataMode.REPEATED) return false;
>      switch(type.getMinorType()){
>      case FIXEDCHAR:
> -    case VARCHAR1:
>      case VARCHAR2:
>      case VARCHAR4:
>        return true;
> @@ -56,7 +72,6 @@ public class Types {
>      if(type.getMode() == DataMode.REPEATED) return false;
>      switch(type.getMinorType()){
>      case FIXEDBINARY:
> -    case VARBINARY1:
>      case VARBINARY2:
>      case VARBINARY4:
>        return true;
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0013215/sandbox/prototype/common/src/main/protobuf/Types.proto
> ----------------------------------------------------------------------
> diff --git a/sandbox/prototype/common/src/main/protobuf/Types.proto
> b/sandbox/prototype/common/src/main/protobuf/Types.proto
> index 17e371d..05a70c5 100644
> --- a/sandbox/prototype/common/src/main/protobuf/Types.proto
> +++ b/sandbox/prototype/common/src/main/protobuf/Types.proto
> @@ -28,11 +28,9 @@ enum MinorType {
>      FLOAT8 = 19;   //  8 byte ieee 754
>      BOOLEAN = 20;   //  single bit value
>      FIXEDCHAR = 21;   //  utf8 fixed length string, padded with spaces
> -    VARCHAR1 = 22;   //  utf8 variable length string (up to 2^8 in length)
>      VARCHAR2 = 23;   //  utf8 variable length string (up to 2^16 in
> length)
>      VARCHAR4 = 24;   //  utf8 variable length string (up to 2^32 in
> length)
>      FIXEDBINARY = 25;   //  fixed length binary, padded with 0 bytes
> -    VARBINARY1 = 26;   //  variable length binary (up to 2^8 in length)
>      VARBINARY2 = 27;   //  variable length binary (up to 2^16 in length)
>      VARBINARY4 = 28;   //  variable length binary (up to 2^32 in length)
>      UINT1 = 29;   //  unsigned 1 byte integer
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0013215/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/data/ValueVectorTypes.tdd
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/data/ValueVectorTypes.tdd
> b/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/data/ValueVectorTypes.tdd
> index 42153b6..69cc277 100644
> ---
> a/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/data/ValueVectorTypes.tdd
> +++
> b/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/data/ValueVectorTypes.tdd
> @@ -12,10 +12,16 @@
>      {
>        major: "Fixed",
>        width: 2,
> +      javaType: "char",
> +      minor: [
> +        { class: "UInt2" }
> +      ]
> +    },    {
> +      major: "Fixed",
> +      width: 2,
>        javaType: "short",
>        minor: [
>          { class: "SmallInt" },
> -        { class: "UInt2" }
>        ]
>      },
>      {
> @@ -65,15 +71,6 @@
>      },
>      {
>        major: "VarLen",
> -      width: 1,
> -      javaType: "byte",
> -      minor: [
> -        { class: "VarBinary1" },
> -        { class: "VarChar1" }
> -      ]
> -    },
> -    {
> -      major: "VarLen",
>        width: 2,
>        javaType: "short",
>        minor: [
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0013215/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/FixedValueVectors.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/FixedValueVectors.java
> b/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/FixedValueVectors.java
> index e0c8406..61515b0 100644
> ---
> a/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/FixedValueVectors.java
> +++
> b/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/FixedValueVectors.java
> @@ -51,7 +51,7 @@ public final class ${minor.class}Vector extends
> BaseDataValueVector implements F
>      return mutator;
>    }
>
> -
> +
>
>    /**
>     * Allocate a new buffer that supports setting at least the provided
> number of values.  May actually be sized bigger depending on underlying
> buffer rounding size. Must be called prior to using the ValueVector.
> @@ -68,16 +68,16 @@ public final class ${minor.class}Vector extends
> BaseDataValueVector implements F
>    public FieldMetadata getMetadata() {
>      return FieldMetadata.newBuilder()
>               .setDef(getField().getDef())
> -             .setValueCount(recordCount)
> -             .setBufferLength(recordCount * ${type.width})
> +             .setValueCount(valueCount)
> +             .setBufferLength(valueCount * ${type.width})
>               .build();
>    }
>
>    @Override
>    public int load(int valueCount, ByteBuf buf){
>      clear();
> -    this.recordCount = valueCount;
> -    int len = recordCount * ${type.width};
> +    this.valueCount = valueCount;
> +    int len = valueCount * ${type.width};
>      data = buf.slice(0, len);
>      data.retain();
>      return len;
> @@ -97,7 +97,7 @@ public final class ${minor.class}Vector extends
> BaseDataValueVector implements F
>    public void transferTo(${minor.class}Vector target){
>      target.data = data;
>      target.data.retain();
> -    target.recordCount = recordCount;
> +    target.valueCount = valueCount;
>      clear();
>    }
>
> @@ -117,10 +117,20 @@ public final class ${minor.class}Vector extends
> BaseDataValueVector implements F
>      }
>    }
>
> +  public void copyValue(int inIndex, int outIndex, ${minor.class}Vector
> v){
> +    <#if (type.width > 8)>
> +    data.getBytes(inIndex * ${type.width}, v.data, outIndex *
> ${type.width}, ${type.width});
> +    <#else> <#-- type.width <= 8 -->
> +    data.set${(minor.javaType!type.javaType)?cap_first}(outIndex *
> ${type.width},
> +        data.get${(minor.javaType!type.javaType)?cap_first}(inIndex *
> ${type.width})
> +    );
> +    </#if> <#-- type.width -->
> +  }
> +
>    public final class Accessor extends BaseValueVector.BaseAccessor{
>
> -    public int getRecordCount() {
> -      return recordCount;
> +    public int getValueCount() {
> +      return valueCount;
>      }
>
>      <#if (type.width > 8)>
> @@ -207,9 +217,9 @@ public final class ${minor.class}Vector extends
> BaseDataValueVector implements F
>     }
>    </#if> <#-- type.width -->
>
> -   public void setValueCount(int recordCount) {
> -     ${minor.class}Vector.this.recordCount = recordCount;
> -     data.writerIndex(${type.width} * recordCount);
> +   public void setValueCount(int valueCount) {
> +     ${minor.class}Vector.this.valueCount = valueCount;
> +     data.writerIndex(${type.width} * valueCount);
>     }
>
>
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0013215/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/NullableValueVectors.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/NullableValueVectors.java
> b/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/NullableValueVectors.java
> index d2d436a..dfec62e 100644
> ---
> a/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/NullableValueVectors.java
> +++
> b/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/NullableValueVectors.java
> @@ -33,7 +33,7 @@ import org.apache.drill.exec.vector.UInt4Vector;
>  @SuppressWarnings("unused")
>  public final class Nullable${minor.class}Vector extends BaseValueVector
> implements <#if type.major ==
> "VarLen">VariableWidth<#else>FixedWidth</#if>Vector {
>
> -  private int recordCount;
> +  private int valueCount;
>    private final BitVector bits;
>    private final ${minor.class}Vector values;
>    private final Accessor accessor = new Accessor();
> @@ -56,7 +56,7 @@ public final class Nullable${minor.class}Vector extends
> BaseValueVector implemen
>
>    @Override
>    public void clear() {
> -    recordCount = 0;
> +    valueCount = 0;
>      bits.clear();
>      values.clear();
>    }
> @@ -70,7 +70,7 @@ public final class Nullable${minor.class}Vector extends
> BaseValueVector implemen
>    public FieldMetadata getMetadata() {
>      return FieldMetadata.newBuilder()
>               .setDef(getField().getDef())
> -             .setValueCount(recordCount)
> +             .setValueCount(valueCount)
>               .setVarByteLength(values.getVarByteLength())
>               .setBufferLength(getBufferSize())
>               .build();
> @@ -87,7 +87,7 @@ public final class Nullable${minor.class}Vector extends
> BaseValueVector implemen
>    @Override
>    public int load(int dataBytes, int valueCount, ByteBuf buf){
>      clear();
> -    this.recordCount = valueCount;
> +    this.valueCount = valueCount;
>      int loaded = bits.load(valueCount, buf);
>
>      // remove bits part of buffer.
> @@ -113,7 +113,7 @@ public final class Nullable${minor.class}Vector
> extends BaseValueVector implemen
>    public FieldMetadata getMetadata() {
>      return FieldMetadata.newBuilder()
>               .setDef(getField().getDef())
> -             .setValueCount(recordCount)
> +             .setValueCount(valueCount)
>               .setBufferLength(getBufferSize())
>               .build();
>    }
> @@ -129,7 +129,7 @@ public final class Nullable${minor.class}Vector
> extends BaseValueVector implemen
>    @Override
>    public int load(int valueCount, ByteBuf buf){
>      clear();
> -    this.recordCount = valueCount;
> +    this.valueCount = valueCount;
>      int loaded = bits.load(valueCount, buf);
>
>      // remove bits part of buffer.
> @@ -154,7 +154,7 @@ public final class Nullable${minor.class}Vector
> extends BaseValueVector implemen
>    public void transferTo(Nullable${minor.class}Vector target){
>      bits.transferTo(target.bits);
>      values.transferTo(target.values);
> -    target.recordCount = recordCount;
> +    target.valueCount = valueCount;
>      clear();
>    }
>
> @@ -185,13 +185,17 @@ public final class Nullable${minor.class}Vector
> extends BaseValueVector implemen
>    public ${minor.class}Vector convertToRequiredVector(){
>      ${minor.class}Vector v = new
> ${minor.class}Vector(getField().getOtherNullableVersion(), allocator);
>      v.data = values.data;
> -    v.recordCount = this.recordCount;
> +    v.valueCount = this.valueCount;
>      v.data.retain();
>      clear();
>      return v;
>    }
> +
>
> -
> +  public void copyValue(int inIndex, int outIndex,
> Nullable${minor.class}Vector v){
> +    bits.copyValue(inIndex, outIndex, v.bits);
> +    values.copyValue(inIndex, outIndex, v.values);
> +  }
>
>    public final class Accessor implements ValueVector.Accessor{
>
> @@ -220,14 +224,14 @@ public final class Nullable${minor.class}Vector
> extends BaseValueVector implemen
>        return isNull(index) ? null : values.getAccessor().getObject(index);
>      }
>
> -    public int getRecordCount(){
> -      return recordCount;
> +    public int getValueCount(){
> +      return valueCount;
>      }
>
>      public void reset(){}
>    }
>
> -  public final class Mutator implements ValueVector.Mutator{
> +  public final class Mutator implements NonRepeatedMutator{
>
>      private int setCount;
>
> @@ -246,15 +250,15 @@ public final class Nullable${minor.class}Vector
> extends BaseValueVector implemen
>        values.getMutator().set(index, value);
>      }
>
> -    public void setValueCount(int recordCount) {
> -      assert recordCount >= 0;
> -      Nullable${minor.class}Vector.this.recordCount = recordCount;
> -      values.getMutator().setValueCount(recordCount);
> -      bits.getMutator().setValueCount(recordCount);
> +    public void setValueCount(int valueCount) {
> +      assert valueCount >= 0;
> +      Nullable${minor.class}Vector.this.valueCount = valueCount;
> +      values.getMutator().setValueCount(valueCount);
> +      bits.getMutator().setValueCount(valueCount);
>      }
>
>      public boolean noNulls(){
> -      return recordCount == setCount;
> +      return valueCount == setCount;
>      }
>
>      public void randomizeData(){
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0013215/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/RepeatedValueVectors.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/RepeatedValueVectors.java
> b/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/RepeatedValueVectors.java
> index 1976340..30bc086 100644
> ---
> a/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/RepeatedValueVectors.java
> +++
> b/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/RepeatedValueVectors.java
> @@ -90,6 +90,11 @@ import org.apache.drill.exec.record.TransferPair;
>      }
>    }
>
> +  public void copyValue(int inIndex, int outIndex,
> Repeated${minor.class}Vector v){
> +    throw new UnsupportedOperationException();
> +  }
> +
> +
>    <#if type.major == "VarLen">
>    @Override
>    public FieldMetadata getMetadata() {
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0013215/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/VariableLengthVectors.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/VariableLengthVectors.java
> b/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/VariableLengthVectors.java
> index 9d723ab..7daadee 100644
> ---
> a/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/VariableLengthVectors.java
> +++
> b/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/VariableLengthVectors.java
> @@ -22,6 +22,8 @@ import org.apache.drill.exec.record.MaterializedField;
>  import org.apache.drill.exec.record.TransferPair;
>  import org.apache.drill.exec.vector.ByteHolder;
>
> +import antlr.collections.impl.Vector;
> +
>  /**
>   * ${minor.class}Vector implements a vector of variable width values.
>  Elements in the vector
>   * are accessed by position from the logical start of the vector.  A
> fixed width offsetVector
> @@ -63,22 +65,22 @@ public final class ${minor.class}Vector extends
> BaseDataValueVector implements V
>     * @return
>     */
>    public int getVarByteLength(){
> -    return offsetVector.getAccessor().get(recordCount);
> +    return offsetVector.getAccessor().get(valueCount);
>    }
>
>    @Override
>    public FieldMetadata getMetadata() {
> -    int len = recordCount * ${type.width} + getVarByteLength();
> +    int len = valueCount * ${type.width} + getVarByteLength();
>      return FieldMetadata.newBuilder()
>               .setDef(getField().getDef())
> -             .setValueCount(recordCount)
> +             .setValueCount(valueCount)
>               .setVarByteLength(getVarByteLength())
>               .setBufferLength(len)
>               .build();
>    }
>
>    public int load(int dataBytes, int valueCount, ByteBuf buf){
> -    this.recordCount = valueCount;
> +    this.valueCount = valueCount;
>      int loaded = offsetVector.load(valueCount+1, buf);
>      data = buf.slice(loaded, dataBytes);
>      data.retain();
> @@ -111,10 +113,20 @@ public final class ${minor.class}Vector extends
> BaseDataValueVector implements V
>      this.offsetVector.transferTo(target.offsetVector);
>      target.data = data;
>      target.data.retain();
> -    target.recordCount = recordCount;
> +    target.valueCount = valueCount;
>      clear();
>    }
>
> +  public void copyValue(int inIndex, int outIndex, ${minor.class}Vector
> v){
> +    int start =
> offsetVector.data.get${(minor.javaType!type.javaType)?cap_first}(inIndex);
> +    int end =
> offsetVector.data.get${(minor.javaType!type.javaType)?cap_first}(inIndex+1);
> +    int len = end - start;
> +
> +    int outputStart = outIndex == 0 ? 0 :
> v.offsetVector.data.get${(minor.javaType!type.javaType)?cap_first}(outIndex
> * ${type.width});
> +    data.getBytes(start, v.data, outputStart, len);
> +    v.offsetVector.data.set${(minor.javaType!type.javaType)?cap_first}(
> (outIndex+1) * ${type.width}, len);
> +  }
> +
>    private class TransferImpl implements TransferPair{
>      ${minor.class}Vector to;
>
> @@ -172,8 +184,8 @@ public final class ${minor.class}Vector extends
> BaseDataValueVector implements V
>        return get(index);
>      }
>
> -    public int getRecordCount() {
> -      return recordCount;
> +    public int getValueCount() {
> +      return valueCount;
>      }
>    }
>
> @@ -210,10 +222,10 @@ public final class ${minor.class}Vector extends
> BaseDataValueVector implements V
>        data.setBytes(currentOffset, bb);
>      }
>
> -    public void setValueCount(int recordCount) {
> -      ${minor.class}Vector.this.recordCount = recordCount;
> -      data.writerIndex(recordCount * ${type.width});
> -      offsetVector.getMutator().setValueCount(recordCount+1);
> +    public void setValueCount(int valueCount) {
> +      ${minor.class}Vector.this.valueCount = valueCount;
> +      data.writerIndex(valueCount * ${type.width});
> +      offsetVector.getMutator().setValueCount(valueCount+1);
>      }
>
>      @Override
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0013215/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/compile/QueryClassLoader.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/compile/QueryClassLoader.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/compile/QueryClassLoader.java
> index 60aa8f3..7feca09 100644
> ---
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/compile/QueryClassLoader.java
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/compile/QueryClassLoader.java
> @@ -18,32 +18,22 @@
>  package org.apache.drill.exec.compile;
>
>  import java.io.IOException;
> -import java.io.InputStream;
> -import java.io.OutputStream;
>  import java.net.URL;
>  import java.net.URLClassLoader;
>  import java.util.concurrent.ConcurrentMap;
> -
> -import javax.tools.JavaCompiler;
> -import javax.tools.JavaFileManager;
> -import javax.tools.JavaFileObject;
> -import javax.tools.JavaFileObject.Kind;
> -import javax.tools.StandardLocation;
> -import javax.tools.ToolProvider;
> +import java.util.concurrent.atomic.AtomicLong;
>
>  import org.apache.drill.exec.exception.ClassTransformationException;
>  import org.codehaus.commons.compiler.CompileException;
> -import org.codehaus.commons.compiler.jdk.ByteArrayJavaFileManager;
> -import
> org.codehaus.commons.compiler.jdk.ByteArrayJavaFileManager.ByteArrayJavaFileObject;
>
>  import com.google.common.collect.MapMaker;
> -import com.google.common.io.ByteStreams;
>
>  public class QueryClassLoader extends URLClassLoader {
>
>    static final org.slf4j.Logger logger =
> org.slf4j.LoggerFactory.getLogger(QueryClassLoader.class);
>
>    private final ClassCompiler classCompiler;
> +  private AtomicLong index = new AtomicLong(0);
>    private ConcurrentMap<String, byte[]> customClasses = new
> MapMaker().concurrencyLevel(4).makeMap();
>
>    public QueryClassLoader(boolean useJanino) {
> @@ -55,6 +45,10 @@ public class QueryClassLoader extends URLClassLoader {
>      }
>    }
>
> +  public long getNextClassIndex(){
> +    return index.getAndIncrement();
> +  }
> +
>    public void injectByteCode(String className, byte[] classBytes) throws
> IOException {
>      if(customClasses.containsKey(className)) throw new
> IOException(String.format("The class defined {} has already been loaded.",
> className));
>      customClasses.put(className, classBytes);
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0013215/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 6e31dec..b99af19 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
> @@ -1,7 +1,7 @@
>  package org.apache.drill.exec.expr;
>
>  import java.io.IOException;
> -import java.lang.reflect.Method;
> +import java.util.concurrent.atomic.AtomicLong;
>
>  import org.apache.drill.common.expression.LogicalExpression;
>  import org.apache.drill.common.types.TypeProtos.DataMode;
> @@ -42,13 +42,19 @@ public class CodeGenerator<T> {
>    private final TemplateClassDefinition<T> definition;
>    private JCodeModel model;
>    private int index = 0;
> -
> +  private static AtomicLong classCreator = new AtomicLong(0);
> +  private String className;
> +  private String fqcn;
> +  private String packageName = "org.apache.drill.exec.test.generated";
> +
>    public CodeGenerator(TemplateClassDefinition<T> definition,
> FunctionImplementationRegistry funcRegistry) {
>      super();
> +    className = "Gen" + classCreator.incrementAndGet();
> +    fqcn = packageName + "." + className;
>      try{
>        this.definition = definition;
>        this.model = new JCodeModel();
> -      this.clazz =
> model._package("org.apache.drill.exec.test.generated")._class("Test1");
> +      this.clazz = model._package(packageName)._class(className);
>        clazz._implements(definition.getInternalInterface());
>        this.parentEvalBlock = new JBlock();
>        this.parentSetupBlock = new JBlock();
> @@ -62,11 +68,16 @@ public class CodeGenerator<T> {
>
>    public void addExpr(LogicalExpression ex){
>      logger.debug("Adding next write {}", ex);
> +    rotateBlock();
> +    ex.accept(evaluationVisitor, this);
> +  }
> +
> +  public void rotateBlock(){
>      currentEvalBlock = new JBlock();
>      parentEvalBlock.add(currentEvalBlock);
>      currentSetupBlock = new JBlock();
>      parentSetupBlock.add(currentSetupBlock);
> -    ex.accept(evaluationVisitor, this);
> +
>    }
>
>    public JBlock getBlock() {
> @@ -74,7 +85,7 @@ public class CodeGenerator<T> {
>    }
>
>    public String getMaterializedClassName(){
> -    return "org.apache.drill.exec.test.generated.Test1";
> +    return fqcn;
>    }
>
>    public JBlock getSetupBlock(){
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0013215/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/annotations/Workspace.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/annotations/Workspace.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/annotations/Workspace.java
> new file mode 100644
> index 0000000..75c97b5
> --- /dev/null
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/annotations/Workspace.java
> @@ -0,0 +1,14 @@
> +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;
> +
> +/**
> + * Describes the field will provide output from the given function.
> + */
> +@Retention(RetentionPolicy.RUNTIME)
> +@Target({ElementType.FIELD})
> +public @interface Workspace {
> +}
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0013215/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 2ae4afa..64f3eb4 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
> @@ -106,7 +106,7 @@ public class FragmentContext {
>    public <T> T getImplementationClass(CodeGenerator<T> cg) throws
> ClassTransformationException, IOException{
>      long t1 = System.nanoTime();
>      T t= transformer.getImplementationClass(this.loader,
> cg.getDefinition(), cg.generate(), cg.getMaterializedClassName());
> -    System.out.println( (System.nanoTime() - t1)/1000/1000 );
> +    logger.debug("Compile time: {} micros.", (System.nanoTime() -
> t1)/1000/1000 );
>      return t;
>
>    }
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0013215/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/SelectionVectorRemover.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/SelectionVectorRemover.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/SelectionVectorRemover.java
> new file mode 100644
> index 0000000..2feae02
> --- /dev/null
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/SelectionVectorRemover.java
> @@ -0,0 +1,60 @@
>
> +/*******************************************************************************
> + * Licensed to the Apache Software Foundation (ASF) under one
> + * or more contributor license agreements.  See the NOTICE file
> + * distributed with this work for additional information
> + * regarding copyright ownership.  The ASF licenses this file
> + * to you under the Apache License, Version 2.0 (the
> + * "License"); you may not use this file except in compliance
> + * with the License.  You may obtain a copy of the License at
> + *
> + * http://www.apache.org/licenses/LICENSE-2.0
> + *
> + * Unless required by applicable law or agreed to in writing, software
> + * distributed under the License is distributed on an "AS IS" BASIS,
> + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
> implied.
> + * See the License for the specific language governing permissions and
> + * limitations under the License.
> +
> ******************************************************************************/
> +package org.apache.drill.exec.physical.config;
> +
> +import org.apache.drill.exec.physical.OperatorCost;
> +import org.apache.drill.exec.physical.base.AbstractSingle;
> +import org.apache.drill.exec.physical.base.PhysicalOperator;
> +import org.apache.drill.exec.physical.base.PhysicalVisitor;
> +import org.apache.drill.exec.physical.base.Size;
> +
> +import com.fasterxml.jackson.annotation.JsonCreator;
> +import com.fasterxml.jackson.annotation.JsonProperty;
> +import com.fasterxml.jackson.annotation.JsonTypeName;
> +
> +@JsonTypeName("selection-vector-remover")
> +public class SelectionVectorRemover extends AbstractSingle {
> +
> +  static final org.slf4j.Logger logger =
> org.slf4j.LoggerFactory.getLogger(SelectionVectorRemover.class);
> +
> +  @JsonCreator
> +  public SelectionVectorRemover(@JsonProperty("child") PhysicalOperator
> child) {
> +    super(child);
> +  }
> +
> +  @Override
> +  public <T, X, E extends Throwable> T accept(PhysicalVisitor<T, X, E>
> physicalVisitor, X value) throws E{
> +    return physicalVisitor.visitOp(this, value);
> +  }
> +
> +  @Override
> +  public OperatorCost getCost() {
> +    return child.getCost();
> +  }
> +
> +  @Override
> +  protected PhysicalOperator getNewWithChild(PhysicalOperator child) {
> +    return new SelectionVectorRemover(child);
> +  }
> +
> +  @Override
> +  public Size getSize() {
> +    return new Size( (long) (child.getSize().getRecordCount()),
> child.getSize().getRecordSize());
> +  }
> +
> +}
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0013215/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 f96d6f3..7740955 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
> @@ -32,9 +32,11 @@ import
> org.apache.drill.exec.physical.config.MockScanPOP;
>  import org.apache.drill.exec.physical.config.Project;
>  import org.apache.drill.exec.physical.config.RandomReceiver;
>  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.impl.filter.FilterBatchCreator;
>  import org.apache.drill.exec.physical.impl.project.ProjectBatchCreator;
> +import org.apache.drill.exec.physical.impl.svremover.SVRemoverCreator;
>  import org.apache.drill.exec.record.RecordBatch;
>
>  import com.google.common.base.Preconditions;
> @@ -49,6 +51,7 @@ public class ImplCreator extends
> AbstractPhysicalVisitor<RecordBatch, FragmentCo
>    private SingleSenderCreator ssc = new SingleSenderCreator();
>    private ProjectBatchCreator pbc = new ProjectBatchCreator();
>    private FilterBatchCreator fbc = new FilterBatchCreator();
> +  private SVRemoverCreator svc = new SVRemoverCreator();
>    private RootExec root = null;
>
>    private ImplCreator(){}
> @@ -75,6 +78,16 @@ public class ImplCreator extends
> AbstractPhysicalVisitor<RecordBatch, FragmentCo
>
>    }
>
> +
> +  @Override
> +  public RecordBatch visitOp(PhysicalOperator op, FragmentContext
> context) throws ExecutionSetupException {
> +    if(op instanceof SelectionVectorRemover){
> +      return svc.getBatch(context, (SelectionVectorRemover) op,
> getChildren(op, context));
> +    }else{
> +      return super.visitOp(op, context);
> +    }
> +  }
> +
>    @Override
>    public RecordBatch visitScreen(Screen op, FragmentContext context)
> throws ExecutionSetupException {
>      Preconditions.checkArgument(root == null);
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0013215/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScanBatch.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScanBatch.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScanBatch.java
> index 084db54..a8a62ca 100644
> ---
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScanBatch.java
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScanBatch.java
> @@ -175,7 +175,7 @@ public class ScanBatch implements RecordBatch {
>
>    @Override
>    public WritableBatch getWritableBatch() {
> -    return WritableBatch.get(this.getRecordCount(), vectors);
> +    return WritableBatch.get(this);
>    }
>
>  }
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0013215/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/filter/FilterRecordBatch.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/filter/FilterRecordBatch.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/filter/FilterRecordBatch.java
> index fc9dbc6..60e8f42 100644
> ---
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/filter/FilterRecordBatch.java
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/filter/FilterRecordBatch.java
> @@ -193,7 +193,7 @@ public class FilterRecordBatch implements RecordBatch{
>
>    @Override
>    public WritableBatch getWritableBatch() {
> -    return WritableBatch.get(sv.getCount(), outputVectors);
> +    return WritableBatch.get(this);
>    }
>
>
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0013215/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/ProjectRecordBatch.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/ProjectRecordBatch.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/ProjectRecordBatch.java
> index 060cd92..4e2b820 100644
> ---
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/ProjectRecordBatch.java
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/ProjectRecordBatch.java
> @@ -190,7 +190,7 @@ public class ProjectRecordBatch implements RecordBatch{
>
>      }
>
> -    SchemaBuilder bldr =
> BatchSchema.newBuilder().setSelectionVectorMode(SelectionVectorMode.NONE);
> +    SchemaBuilder bldr =
> BatchSchema.newBuilder().setSelectionVectorMode(incoming.getSchema().getSelectionVector());
>      for(ValueVector v : outputVectors){
>        bldr.addField(v.getField());
>      }
> @@ -207,7 +207,7 @@ public class ProjectRecordBatch implements RecordBatch{
>
>    @Override
>    public WritableBatch getWritableBatch() {
> -    return WritableBatch.get(incoming.getRecordCount(), outputVectors);
> +    return WritableBatch.get(this);
>    }
>
>    private MaterializedField getMaterializedField(FieldReference
> reference, LogicalExpression expr){
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0013215/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
> new file mode 100644
> index 0000000..55d6ba2
> --- /dev/null
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/svremover/Copier.java
> @@ -0,0 +1,18 @@
> +package org.apache.drill.exec.physical.impl.svremover;
> +
> +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.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_DEFINITION = new
> TemplateClassDefinition<Copier>( //
> +      Copier.class,
> "org.apache.drill.exec.physical.impl.svremover.CopierTemplate",
> CopyEvaluator.class, null);
> +
> +  public void setupRemover(FragmentContext context, RecordBatch incoming,
> RecordBatch outgoing, VectorAllocator[] allocators) throws
> SchemaChangeException;
> +  public abstract void copyRecords();
> +
> +
> +}
> \ No newline at end of file
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0013215/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/svremover/CopierTemplate.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/svremover/CopierTemplate.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/svremover/CopierTemplate.java
> new file mode 100644
> index 0000000..12a1e0a
> --- /dev/null
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/svremover/CopierTemplate.java
> @@ -0,0 +1,43 @@
> +package org.apache.drill.exec.physical.impl.svremover;
> +
> +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 abstract class CopierTemplate implements Copier{
> +  static final org.slf4j.Logger logger =
> org.slf4j.LoggerFactory.getLogger(CopierTemplate.class);
> +
> +  private SelectionVector2 sv2;
> +  private VectorAllocator[] allocators;
> +
> +  @Override
> +  public void setupRemover(FragmentContext context, RecordBatch incoming,
> RecordBatch outgoing, VectorAllocator[] allocators) throws
> SchemaChangeException{
> +    this.allocators = allocators;
> +    this.sv2 = incoming.getSelectionVector2();
> +    doSetup(context, incoming, outgoing);
> +  }
> +
> +  private void allocateVectors(int recordCount){
> +    for(VectorAllocator a : allocators){
> +      a.alloc(recordCount);
> +    }
> +  }
> +
> +  @Override
> +  public void copyRecords(){
> +    final int recordCount = sv2.getCount();
> +    allocateVectors(recordCount);
> +    int outgoingPosition = 0;
> +
> +    for(int svIndex = 0; svIndex < recordCount * 2; svIndex++,
> outgoingPosition++){
> +      doEval(svIndex, outgoingPosition);
> +    }
> +  }
> +
> +  public abstract void doSetup(FragmentContext context, RecordBatch
> incoming, RecordBatch outgoing) throws SchemaChangeException;
> +  public abstract void doEval(int incoming, int outgoing);
> +
> +
> +}
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0013215/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/svremover/CopyEvaluator.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/svremover/CopyEvaluator.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/svremover/CopyEvaluator.java
> new file mode 100644
> index 0000000..25c51bb
> --- /dev/null
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/svremover/CopyEvaluator.java
> @@ -0,0 +1,11 @@
> +package org.apache.drill.exec.physical.impl.svremover;
> +
> +import org.apache.drill.exec.ops.FragmentContext;
> +import org.apache.drill.exec.record.RecordBatch;
> +
> +public interface CopyEvaluator {
> +  static final org.slf4j.Logger logger =
> org.slf4j.LoggerFactory.getLogger(CopyEvaluator.class);
> +
> +  public abstract void doSetup(FragmentContext context, RecordBatch
> incoming, RecordBatch outgoing);
> +  public abstract void doEval(int incoming, int outgoing);
> +}
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0013215/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
> new file mode 100644
> index 0000000..208ebc6
> --- /dev/null
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/svremover/RemovingRecordBatch.java
> @@ -0,0 +1,305 @@
> +package org.apache.drill.exec.physical.impl.svremover;
> +
> +import java.io.IOException;
> +import java.util.Iterator;
> +import java.util.List;
> +
> +import org.apache.drill.common.expression.SchemaPath;
> +import org.apache.drill.exec.exception.ClassTransformationException;
> +import org.apache.drill.exec.exception.SchemaChangeException;
> +import org.apache.drill.exec.expr.CodeGenerator;
> +import org.apache.drill.exec.ops.FragmentContext;
> +import org.apache.drill.exec.physical.impl.VectorHolder;
> +import org.apache.drill.exec.record.BatchSchema;
> +import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
> +import org.apache.drill.exec.record.RecordBatch;
> +import org.apache.drill.exec.record.SchemaBuilder;
> +import org.apache.drill.exec.record.TransferPair;
> +import org.apache.drill.exec.record.WritableBatch;
> +import org.apache.drill.exec.record.selection.SelectionVector2;
> +import org.apache.drill.exec.record.selection.SelectionVector4;
> +import org.apache.drill.exec.vector.FixedWidthVector;
> +import org.apache.drill.exec.vector.NonRepeatedMutator;
> +import org.apache.drill.exec.vector.ValueVector;
> +import org.apache.drill.exec.vector.VariableWidthVector;
> +
> +import com.google.common.base.Preconditions;
> +import com.google.common.collect.Lists;
> +import com.sun.codemodel.JBlock;
> +import com.sun.codemodel.JClass;
> +import com.sun.codemodel.JExpr;
> +import com.sun.codemodel.JExpression;
> +import com.sun.codemodel.JType;
> +import com.sun.codemodel.JVar;
> +
> +public class RemovingRecordBatch implements RecordBatch{
> +  static final org.slf4j.Logger logger =
> org.slf4j.LoggerFactory.getLogger(RemovingRecordBatch.class);
> +
> +  private final RecordBatch incoming;
> +  private final FragmentContext context;
> +  private BatchSchema outSchema;
> +  private Copier copier;
> +  private List<ValueVector> outputVectors;
> +  private VectorHolder vh;
> +
> +
> +  public RemovingRecordBatch(RecordBatch incoming, FragmentContext
> context){
> +    this.incoming = incoming;
> +    this.context = context;
> +  }
> +
> +  @Override
> +  public Iterator<ValueVector> iterator() {
> +    return outputVectors.iterator();
> +  }
> +
> +  @Override
> +  public FragmentContext getContext() {
> +    return context;
> +  }
> +
> +  @Override
> +  public BatchSchema getSchema() {
> +    Preconditions.checkNotNull(outSchema);
> +    return outSchema;
> +  }
> +
> +  @Override
> +  public int getRecordCount() {
> +    return incoming.getRecordCount();
> +  }
> +
> +  @Override
> +  public void kill() {
> +    incoming.kill();
> +  }
> +
> +  @Override
> +  public SelectionVector2 getSelectionVector2() {
> +    throw new UnsupportedOperationException();
> +  }
> +
> +  @Override
> +  public SelectionVector4 getSelectionVector4() {
> +    throw new UnsupportedOperationException();
> +  }
> +
> +  @Override
> +  public TypedFieldId getValueVectorId(SchemaPath path) {
> +    return vh.getValueVector(path);
> +  }
> +
> +  @Override
> +  public <T extends ValueVector> T getValueVectorById(int fieldId,
> Class<?> clazz) {
> +    return vh.getValueVector(fieldId, clazz);
> +  }
> +
> +  @Override
> +  public IterOutcome next() {
> +
> +    IterOutcome upstream = incoming.next();
> +    logger.debug("Upstream... {}", upstream);
> +    switch(upstream){
> +    case NONE:
> +    case NOT_YET:
> +    case STOP:
> +      return upstream;
> +    case OK_NEW_SCHEMA:
> +      try{
> +        copier = createCopier();
> +      }catch(SchemaChangeException ex){
> +        incoming.kill();
> +        logger.error("Failure during query", ex);
> +        context.fail(ex);
> +        return IterOutcome.STOP;
> +      }
> +      // fall through.
> +    case OK:
> +      int recordCount = incoming.getRecordCount();
> +      copier.copyRecords();
> +      for(ValueVector v : this.outputVectors){
> +        ValueVector.Mutator m = v.getMutator();
> +        if(m instanceof NonRepeatedMutator){
> +          ((NonRepeatedMutator) m).setValueCount(recordCount);
> +        }else{
> +          throw new UnsupportedOperationException();
> +        }
> +      }
> +      return upstream; // change if upstream changed, otherwise normal.
> +    default:
> +      throw new UnsupportedOperationException();
> +    }
> +  }
> +
> +
> +
> +
> +  private class StraightCopier implements Copier{
> +
> +    private List<TransferPair> pairs = Lists.newArrayList();
> +    private List<ValueVector> out = Lists.newArrayList();
> +
> +    @Override
> +    public void setupRemover(FragmentContext context, RecordBatch
> incoming, RecordBatch outgoing, VectorAllocator[] allocators){
> +      for(ValueVector vv : incoming){
> +        TransferPair tp = vv.getTransferPair();
> +        pairs.add(tp);
> +        out.add(tp.getTo());
> +      }
> +    }
> +
> +    @Override
> +    public void copyRecords() {
> +      for(TransferPair tp : pairs){
> +        tp.transfer();
> +      }
> +    }
> +
> +    public List<ValueVector> getOut() {
> +      return out;
> +    }
> +
> +  }
> +
> +  private Copier getStraightCopier(){
> +    StraightCopier copier = new StraightCopier();
> +    copier.setupRemover(context, incoming, this, null);
> +    outputVectors.addAll(copier.getOut());
> +    return copier;
> +  }
> +
> +  private Copier getGeneratedCopier() throws SchemaChangeException{
> +    Preconditions.checkArgument(incoming.getSchema().getSelectionVector()
> == SelectionVectorMode.TWO_BYTE);
> +
> +    List<VectorAllocator> allocators = Lists.newArrayList();
> +    for(ValueVector i : incoming){
> +      TransferPair t = i.getTransferPair();
> +      outputVectors.add(t.getTo());
> +      allocators.add(getAllocator(i, t.getTo()));
> +    }
> +
> +    try {
> +      final CodeGenerator<Copier> cg = new
> CodeGenerator<Copier>(Copier.TEMPLATE_DEFINITION,
> context.getFunctionRegistry());
> +      generateCopies(cg);
> +      Copier copier = context.getImplementationClass(cg);
> +      copier.setupRemover(context, incoming, this, allocators.toArray(new
> VectorAllocator[allocators.size()]));
> +      return copier;
> +    } catch (ClassTransformationException | IOException e) {
> +      throw new SchemaChangeException("Failure while attempting to load
> generated class", e);
> +    }
> +  }
> +
> +
> +  private Copier createCopier() throws SchemaChangeException{
> +    if(outputVectors != null){
> +      for(ValueVector v : outputVectors){
> +        v.close();
> +      }
> +    }
> +    this.outputVectors = Lists.newArrayList();
> +    this.vh = new VectorHolder(outputVectors);
> +
> +    SchemaBuilder bldr =
> BatchSchema.newBuilder().setSelectionVectorMode(SelectionVectorMode.NONE);
> +    for(ValueVector v : outputVectors){
> +      bldr.addField(v.getField());
> +    }
> +    this.outSchema = bldr.build();
> +
> +    switch(incoming.getSchema().getSelectionVector()){
> +    case NONE:
> +      return getStraightCopier();
> +    case TWO_BYTE:
> +      return getGeneratedCopier();
> +    default:
> +      throw new UnsupportedOperationException();
> +    }
> +
> +  }
> +
> +  private void generateCopies(CodeGenerator<Copier> g){
> +    // we have parallel ids for each value vector so we don't actually
> have to deal with managing the ids at all.
> +    int fieldId = 0;
> +
> +
> +
> +    JExpression inIndex = JExpr.direct("inIndex");
> +    JExpression outIndex = JExpr.direct("outIndex");
> +    g.rotateBlock();
> +    for(ValueVector vv : incoming){
> +      JClass vvClass = (JClass) g.getModel()._ref(vv.getClass());
> +      JVar inVV = declareVVSetup("incoming", g, fieldId, vvClass);
> +      JVar outVV = declareVVSetup("outgoing", g, fieldId, vvClass);
> +
> +
>  g.getBlock().add(inVV.invoke("copyValue").arg(inIndex).arg(outIndex).arg(outVV));
> +
> +      fieldId++;
> +    }
> +  }
> +
> +  private JVar declareVVSetup(String varName, CodeGenerator<?> g, int
> fieldId, JClass vvClass){
> +    JVar vv = g.declareClassField("vv", vvClass);
> +    JClass t = (JClass) g.getModel()._ref(SchemaChangeException.class);
> +    JType objClass = g.getModel()._ref(Object.class);
> +    JBlock b = g.getSetupBlock();
> +    JVar obj = b.decl( //
> +        objClass, //
> +        g.getNextVar("tmp"), //
> +
>  JExpr.direct(varName).invoke("getValueVectorById").arg(JExpr.lit(fieldId)).arg(
> vvClass.dotclass()));
> +
>  b._if(obj.eq(JExpr._null()))._then()._throw(JExpr._new(t).arg(JExpr.lit(String.format("Failure
> while loading vector %s with id %d", vv.name(), fieldId))));
> +        b.assign(vv, JExpr.cast(vvClass, obj));
> +
> +    return vv;
> +  }
> +
> +  @Override
> +  public WritableBatch getWritableBatch() {
> +    return WritableBatch.get(this);
> +  }
> +
> +  private VectorAllocator getAllocator(ValueVector in, ValueVector
> outgoing){
> +    if(outgoing instanceof FixedWidthVector){
> +      return new FixedVectorAllocator((FixedWidthVector) outgoing);
> +    }else if(outgoing instanceof VariableWidthVector && in instanceof
> VariableWidthVector){
> +      return new VariableVectorAllocator( (VariableWidthVector) in,
> (VariableWidthVector) outgoing);
> +    }else{
> +      throw new UnsupportedOperationException();
> +    }
> +  }
> +
> +  private class FixedVectorAllocator implements VectorAllocator{
> +    FixedWidthVector out;
> +
> +    public FixedVectorAllocator(FixedWidthVector out) {
> +      super();
> +      this.out = out;
> +    }
> +
> +    public void alloc(int recordCount){
> +      out.allocateNew(recordCount);
> +      out.getMutator().setValueCount(recordCount);
> +    }
> +
> +
> +
> +  }
> +
> +  private class VariableVectorAllocator implements VectorAllocator{
> +    VariableWidthVector in;
> +    VariableWidthVector out;
> +
> +    public VariableVectorAllocator(VariableWidthVector in,
> VariableWidthVector out) {
> +      super();
> +      this.in = in;
> +      this.out = out;
> +    }
> +
> +    public void alloc(int recordCount){
> +      out.allocateNew(in.getByteCapacity(), recordCount);
> +      out.getMutator().setValueCount(recordCount);
> +    }
> +  }
> +
> +  public interface VectorAllocator{
> +    public void alloc(int recordCount);
> +  }
> +}
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0013215/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/svremover/SVRemoverCreator.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/svremover/SVRemoverCreator.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/svremover/SVRemoverCreator.java
> new file mode 100644
> index 0000000..4671baa
> --- /dev/null
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/svremover/SVRemoverCreator.java
> @@ -0,0 +1,23 @@
> +package org.apache.drill.exec.physical.impl.svremover;
> +
> +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.SelectionVectorRemover;
> +import org.apache.drill.exec.physical.impl.BatchCreator;
> +import org.apache.drill.exec.record.RecordBatch;
> +
> +import com.google.common.base.Preconditions;
> +
> +public class SVRemoverCreator implements
> BatchCreator<SelectionVectorRemover>{
> +  static final org.slf4j.Logger logger =
> org.slf4j.LoggerFactory.getLogger(SVRemoverCreator.class);
> +
> +  @Override
> +  public RecordBatch getBatch(FragmentContext context,
> SelectionVectorRemover config, List<RecordBatch> children) throws
> ExecutionSetupException {
> +    Preconditions.checkArgument(children.size() == 1);
> +    return new RemovingRecordBatch(children.iterator().next(), context);
> +  }
> +
> +
> +}
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0013215/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatch.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatch.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatch.java
> index ff856d4..fc4e759 100644
> ---
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatch.java
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatch.java
> @@ -28,8 +28,11 @@ import org.apache.drill.exec.vector.ValueVector;
>   * A record batch contains a set of field values for a particular range
> of records. In the case of a record batch
>   * composed of ValueVectors, ideally a batch fits within L2 cache (~256k
> per core). The set of value vectors do not
>   * change unless the next() IterOutcome is a *_NEW_SCHEMA type.
> + *
> + * A key thing to know is that the Iterator provided by record batch must
> align with the rank positions of the field ids
> + * provided utilizing getValueVectorId();
>   */
> -public interface RecordBatch extends Iterable<ValueVector>{
> +public interface RecordBatch extends Iterable<ValueVector> {
>
>    /**
>     * Describes the outcome of a RecordBatch being incremented forward.
> @@ -39,16 +42,14 @@ public interface RecordBatch extends
> Iterable<ValueVector>{
>      OK, // A new range of records have been provided.
>      OK_NEW_SCHEMA, // A full collection of records
>      STOP, // Informs parent nodes that the query has terminated. In this
> case, a consumer can consume their QueryContext
> -         // to understand the current state of things.
> +          // to understand the current state of things.
>      NOT_YET // used by batches that haven't received incoming data yet.
>    }
>
>    public static enum SetupOutcome {
> -    OK,
> -    OK_NEW_SCHEMA,
> -    FAILED
> +    OK, OK_NEW_SCHEMA, FAILED
>    }
> -
> +
>    /**
>     * Access the FragmentContext of the current query fragment. Useful for
> reporting failure information or other query
>     * level information.
> @@ -78,17 +79,21 @@ public interface RecordBatch extends
> Iterable<ValueVector>{
>    public void kill();
>
>    public abstract SelectionVector2 getSelectionVector2();
> +
>    public abstract SelectionVector4 getSelectionVector4();
> -
> +
>    /**
> -   * Get the value vector
> -   * @param path The path where the vector should be located.
> -   * @return The local field id associated with this vector.
> +   * Get the value vector type and id for the given schema path. The
> TypedFieldId should store a fieldId which is the
> +   * same as the ordinal position of the field within the Iterator
> provided this classes implementation of
> +   * Iterable<ValueVector>.
> +   *
> +   * @param path
> +   *          The path where the vector should be located.
> +   * @return The local field id associated with this vector. If no field
> matches this path, this will return a null
> +   *         TypedFieldId
>     */
>    public abstract TypedFieldId getValueVectorId(SchemaPath path);
> -
> -
> -
> +
>    public abstract <T extends ValueVector> T getValueVectorById(int
> fieldId, Class<?> clazz);
>
>    /**
> @@ -99,24 +104,28 @@ public interface RecordBatch extends
> Iterable<ValueVector>{
>     * @return An IterOutcome describing the result of the iteration.
>     */
>    public IterOutcome next();
> -
> +
>    /**
> -   * Get a writable version of this batch.  Takes over owernship of
> existing buffers.
> +   * Get a writable version of this batch. Takes over owernship of
> existing buffers.
> +   *
>     * @return
>     */
>    public WritableBatch getWritableBatch();
>
> -  public static class TypedFieldId{
> +  public static class TypedFieldId {
>      final MajorType type;
>      final int fieldId;
> +
>      public TypedFieldId(MajorType type, int fieldId) {
>        super();
>        this.type = type;
>        this.fieldId = fieldId;
>      }
> +
>      public MajorType getType() {
>        return type;
>      }
> +
>      public int getFieldId() {
>        return fieldId;
>      }
> @@ -139,8 +148,7 @@ public interface RecordBatch extends
> Iterable<ValueVector>{
>          return false;
>        return true;
>      }
> -
> -
> +
>    }
> -
> +
>  }
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0013215/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatchLoader.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatchLoader.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatchLoader.java
> index 5f7648b..4a12af6 100644
> ---
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatchLoader.java
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatchLoader.java
> @@ -137,7 +137,7 @@ public class RecordBatchLoader implements
> Iterable<ValueVector>{
>
>
>    public WritableBatch getWritableBatch(){
> -    return WritableBatch.get(recordCount, vectors);
> +    return WritableBatch.getBatchNoSV(recordCount, vectors);
>    }
>
>    @Override
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0013215/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordRemapper.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordRemapper.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordRemapper.java
> new file mode 100644
> index 0000000..86c963d
> --- /dev/null
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordRemapper.java
> @@ -0,0 +1,8 @@
> +package org.apache.drill.exec.record;
> +
> +/**
> + * Remove the selection vector from a record batch.
> + */
> +public class RecordRemapper {
> +  static final org.slf4j.Logger logger =
> org.slf4j.LoggerFactory.getLogger(RecordRemapper.class);
> +}
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0013215/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/WritableBatch.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/WritableBatch.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/WritableBatch.java
> index 8335e91..f49e77f 100644
> ---
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/WritableBatch.java
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/WritableBatch.java
> @@ -23,59 +23,62 @@ import java.util.List;
>
>  import org.apache.drill.exec.proto.UserBitShared.FieldMetadata;
>  import org.apache.drill.exec.proto.UserBitShared.RecordBatchDef;
> +import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
>  import org.apache.drill.exec.vector.ValueVector;
>
>  import com.google.common.collect.Lists;
>
>  /**
> - * A specialized version of record batch that can moves out buffers and
> preps them for writing.
> + * A specialized version of record batch that can moves out buffers and
> preps them for writing.
>   */
>  public class WritableBatch {
>    static final org.slf4j.Logger logger =
> org.slf4j.LoggerFactory.getLogger(WritableBatch.class);
> -
> +
>    private final RecordBatchDef def;
>    private final ByteBuf[] buffers;
> -
> -  public WritableBatch(RecordBatchDef def, List<ByteBuf> buffers) {
> +
> +  private WritableBatch(RecordBatchDef def, List<ByteBuf> buffers) {
>      logger.debug("Created new writable batch with def {} and buffers {}",
> def, buffers);
>      this.def = def;
>      this.buffers = buffers.toArray(new ByteBuf[buffers.size()]);
>    }
> -
> -  public WritableBatch(RecordBatchDef def, ByteBuf[] buffers) {
> +
> +  private WritableBatch(RecordBatchDef def, ByteBuf[] buffers) {
>      super();
>      this.def = def;
>      this.buffers = buffers;
>    }
> -
> -
> -  public RecordBatchDef getDef(){
> +
> +  public RecordBatchDef getDef() {
>      return def;
>    }
> -  public ByteBuf[] getBuffers(){
> +
> +  public ByteBuf[] getBuffers() {
>      return buffers;
>    }
>
> -  public static WritableBatch get(int recordCount, List<ValueVector>
> vectors){
> -
> +  public static WritableBatch getBatchNoSV(int recordCount,
> Iterable<ValueVector> vectors) {
>      List<ByteBuf> buffers = Lists.newArrayList();
>      List<FieldMetadata> metadata = Lists.newArrayList();
> -
>
> -    for(ValueVector vv : vectors){
> +    for (ValueVector vv : vectors) {
>        metadata.add(vv.getMetadata());
> -      for(ByteBuf b : vv.getBuffers()){
> +      for (ByteBuf b : vv.getBuffers()) {
>          buffers.add(b);
>          b.retain();
>        }
> -      // allocate new buffer to release hold on old buffer.
> +      // remove vv access to buffers.
>        vv.clear();
>      }
>
>      RecordBatchDef batchDef =
> RecordBatchDef.newBuilder().addAllField(metadata).setRecordCount(recordCount).build();
>      WritableBatch b = new WritableBatch(batchDef, buffers);
> -      return b;
> -
> +    return b;
>    }
>
> +  public static WritableBatch get(RecordBatch batch) {
> +    if(batch.getSchema() != null &&
> batch.getSchema().getSelectionVector() != SelectionVectorMode.NONE) throw
> new UnsupportedOperationException("Only batches without selections vectors
> are writable.");
> +    return getBatchNoSV(batch.getRecordCount(), batch);
> +  }
> +
>  }
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0013215/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/BaseDataValueVector.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/BaseDataValueVector.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/BaseDataValueVector.java
> index dd2b504..646640a 100644
> ---
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/BaseDataValueVector.java
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/BaseDataValueVector.java
> @@ -10,7 +10,7 @@ import org.apache.drill.exec.record.MaterializedField;
>  abstract class BaseDataValueVector extends BaseValueVector{
>
>    protected ByteBuf data = DeadBuf.DEAD_BUFFER;
> -  protected int recordCount;
> +  protected int valueCount;
>
>    public BaseDataValueVector(MaterializedField field, BufferAllocator
> allocator) {
>      super(field, allocator);
> @@ -25,7 +25,7 @@ abstract class BaseDataValueVector extends
> BaseValueVector{
>      if (data != DeadBuf.DEAD_BUFFER) {
>        data.release();
>        data = DeadBuf.DEAD_BUFFER;
> -      recordCount = 0;
> +      valueCount = 0;
>      }
>    }
>
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0013215/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/BaseValueVector.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/BaseValueVector.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/BaseValueVector.java
> index e87e132..512c13e 100644
> ---
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/BaseValueVector.java
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/BaseValueVector.java
> @@ -25,6 +25,7 @@ abstract class BaseValueVector implements ValueVector{
>    }
>
>    abstract class BaseAccessor implements ValueVector.Accessor{
> +    public abstract int getValueCount();
>      public void reset(){}
>    }
>
> @@ -32,5 +33,7 @@ abstract class BaseValueVector implements ValueVector{
>      public void reset(){}
>    }
>
> +
> +
>  }
>
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0013215/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/BitVector.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/BitVector.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/BitVector.java
> index 9a9f438..0f5b7d0 100644
> ---
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/BitVector.java
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/BitVector.java
> @@ -20,6 +20,9 @@ import org.apache.drill.exec.record.TransferPair;
>  public final class BitVector extends BaseDataValueVector implements
> FixedWidthVector{
>    static final org.slf4j.Logger logger =
> org.slf4j.LoggerFactory.getLogger(BitVector.class);
>
> +  private final Accessor accessor = new Accessor();
> +  private final Mutator mutator = new Mutator();
> +
>    private int valueCapacity;
>
>    public BitVector(MaterializedField field, BufferAllocator allocator) {
> @@ -48,13 +51,17 @@ public final class BitVector extends
> BaseDataValueVector implements FixedWidthVe
>    @Override
>    public int load(int valueCount, ByteBuf buf){
>      clear();
> -    this.recordCount = valueCount;
> +    this.valueCount = valueCount;
>      int len = getSizeFromCount(valueCount);
>      data = buf.slice(0, len);
>      data.retain();
>      return len;
>    }
>
> +  public void copyValue(int inIndex, int outIndex, BitVector target){
> +    target.mutator.set(outIndex, this.accessor.get(inIndex));
> +  }
> +
>    @Override
>    public void load(FieldMetadata metadata, ByteBuf buffer) {
>      assert this.field.getDef().equals(metadata.getDef());
> @@ -82,7 +89,7 @@ public final class BitVector extends BaseDataValueVector
> implements FixedWidthVe
>    public void transferTo(BitVector target){
>      target.data = data;
>      target.data.retain();
> -    target.recordCount = recordCount;
> +    target.valueCount = valueCount;
>      clear();
>    }
>
> @@ -125,8 +132,8 @@ public final class BitVector extends
> BaseDataValueVector implements FixedWidthVe
>        return new Boolean(get(index) != 0);
>      }
>
> -    public int getRecordCount() {
> -      return recordCount;
> +    public int getValueCount() {
> +      return valueCount;
>      }
>
>    }
> @@ -161,9 +168,9 @@ public final class BitVector extends
> BaseDataValueVector implements FixedWidthVe
>        data.setByte((int) Math.floor(index/8), currentByte);
>      }
>
> -    public void setValueCount(int recordCount) {
> -      BitVector.this.recordCount = recordCount;
> -      data.writerIndex(getSizeFromCount(recordCount));
> +    public void setValueCount(int valueCount) {
> +      BitVector.this.valueCount = valueCount;
> +      data.writerIndex(getSizeFromCount(valueCount));
>      }
>
>      @Override
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0013215/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/FixedWidthVector.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/FixedWidthVector.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/FixedWidthVector.java
> index 0e3e3e9..008d17d 100644
> ---
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/FixedWidthVector.java
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/FixedWidthVector.java
> @@ -20,4 +20,6 @@ public interface FixedWidthVector extends ValueVector{
>     */
>    public int load(int valueCount, ByteBuf buf);
>
> +
> +  public abstract NonRepeatedMutator getMutator();
>  }
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0013215/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/NonRepeatedMutator.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/NonRepeatedMutator.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/NonRepeatedMutator.java
> index e9bdcbd..eba5e2e 100644
> ---
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/NonRepeatedMutator.java
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/NonRepeatedMutator.java
> @@ -3,5 +3,5 @@ package org.apache.drill.exec.vector;
>  import org.apache.drill.exec.vector.ValueVector.Mutator;
>
>  public interface NonRepeatedMutator extends Mutator{
> -  public void setValueCount(int recordCount);
> +  public void setValueCount(int valueCount);
>  }
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0013215/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/ValueVector.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/ValueVector.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/ValueVector.java
> index b188d5b..9328a80 100644
> ---
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/ValueVector.java
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/ValueVector.java
> @@ -116,6 +116,8 @@ public interface ValueVector extends Closeable {
>       * @param index   Index of the value to get
>       */
>      public abstract Object getObject(int index);
> +
> +    public int getValueCount();
>      public void reset();
>    }
>
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0013215/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/VariableWidthVector.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/VariableWidthVector.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/VariableWidthVector.java
> index c26cbab..8229870 100644
> ---
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/VariableWidthVector.java
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/VariableWidthVector.java
> @@ -26,4 +26,6 @@ public interface VariableWidthVector extends ValueVector{
>     * @return The number of bytes of the buffer that were consumed.
>     */
>    public int load(int dataBytes, int valueCount, ByteBuf buf);
> +
> +  public abstract NonRepeatedMutator getMutator();
>  }
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0013215/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/SimpleRootExec.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/SimpleRootExec.java
> b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/SimpleRootExec.java
> index c6434f7..5e7c599 100644
> ---
> a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/SimpleRootExec.java
> +++
> b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/SimpleRootExec.java
> @@ -1,5 +1,7 @@
>  package org.apache.drill.exec.physical.impl;
>
> +import java.util.Iterator;
> +
>  import org.apache.drill.common.expression.SchemaPath;
>  import org.apache.drill.exec.physical.impl.ScreenCreator.ScreenRoot;
>  import org.apache.drill.exec.record.RecordBatch;
> @@ -8,7 +10,7 @@ import
> org.apache.drill.exec.record.RecordBatch.TypedFieldId;
>  import org.apache.drill.exec.record.selection.SelectionVector2;
>  import org.apache.drill.exec.vector.ValueVector;
>
> -public class SimpleRootExec implements RootExec{
> +public class SimpleRootExec implements RootExec, Iterable<ValueVector>{
>    static final org.slf4j.Logger logger =
> org.slf4j.LoggerFactory.getLogger(SimpleRootExec.class);
>
>    private RecordBatch incoming;
> @@ -40,5 +42,14 @@ public class SimpleRootExec implements RootExec{
>    public void stop() {
>    }
>
> +  @Override
> +  public Iterator<ValueVector> iterator() {
> +    return incoming.iterator();
> +  }
> +
> +  public int getRecordCount(){
> +    return incoming.getRecordCount();
> +  }
> +
>
>  }
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0013215/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 e2bf25b..96a6139 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
> @@ -31,8 +31,8 @@ public class TestSimpleFilter {
>
>
>    @Test
> -  public void project(@Injectable final DrillbitContext bitContext,
> @Injectable UserClientConnection connection) throws Exception{
> -    System.out.println(System.getProperty("java.class.path"));
> +  public void testFilter(@Injectable final DrillbitContext bitContext,
> @Injectable UserClientConnection connection) throws Exception{
> +//    System.out.println(System.getProperty("java.class.path"));
>
>
>      new NonStrictExpectations(){{
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0013215/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/project/TestSimpleProjection.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/project/TestSimpleProjection.java
> b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/project/TestSimpleProjection.java
> index e3f03f2..f4900e1 100644
> ---
> a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/project/TestSimpleProjection.java
> +++
> b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/project/TestSimpleProjection.java
> @@ -35,7 +35,6 @@ public class TestSimpleProjection {
>
>    @Test
>    public void project(@Injectable final DrillbitContext bitContext,
> @Injectable UserClientConnection connection) throws Exception{
> -    System.out.println(System.getProperty("java.class.path"));
>
>
>      new NonStrictExpectations(){{
> @@ -57,7 +56,7 @@ public class TestSimpleProjection {
>        a1 = c1.getAccessor();
>        a2 = c2.getAccessor();
>
> -      for(int i =0; i < c1.getAccessor().getRecordCount(); i++){
> +      for(int i =0; i < c1.getAccessor().getValueCount(); i++){
>          assertEquals(a1.get(i)+1, a2.get(i));
>          x += a1.get(i);
>        }
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0013215/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/svremover/TestSVRemover.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/svremover/TestSVRemover.java
> b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/svremover/TestSVRemover.java
> new file mode 100644
> index 0000000..f417b91
> --- /dev/null
> +++
> b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/svremover/TestSVRemover.java
> @@ -0,0 +1,64 @@
> +package org.apache.drill.exec.physical.impl.svremover;
> +
> +import static org.junit.Assert.assertEquals;
> +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.FragmentHandle;
> +import org.apache.drill.exec.rpc.user.UserServer.UserClientConnection;
> +import org.apache.drill.exec.server.DrillbitContext;
> +import org.apache.drill.exec.vector.ValueVector;
> +import org.junit.After;
> +import org.junit.Test;
> +
> +import com.google.common.base.Charsets;
> +import com.google.common.io.Files;
> +import com.yammer.metrics.MetricRegistry;
> +
> +public class TestSVRemover {
> +  static final org.slf4j.Logger logger =
> org.slf4j.LoggerFactory.getLogger(TestSVRemover.class);
> +  DrillConfig c = DrillConfig.create();
> +
> +
> +  @Test
> +  public void testSelectionVectorRemoval(@Injectable final
> DrillbitContext bitContext, @Injectable UserClientConnection connection)
> throws Exception{
> +//    System.out.println(System.getProperty("java.class.path"));
> +
> +
> +    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("/remover/test1.json"),
> Charsets.UTF_8));
> +    FunctionImplementationRegistry registry = new
> FunctionImplementationRegistry(c);
> +    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()){
> +      int count = exec.getRecordCount();
> +      for(ValueVector v : exec){
> +        ValueVector.Accessor a = v.getAccessor();
> +        assertEquals(count, a.getValueCount());
> +      }
> +    }
> +  }
> +
> +  @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/f0013215/sandbox/prototype/exec/java-exec/src/test/resources/remover/test1.json
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/test/resources/remover/test1.json
> b/sandbox/prototype/exec/java-exec/src/test/resources/remover/test1.json
> new file mode 100644
> index 0000000..7ef8fd6
> --- /dev/null
> +++
> b/sandbox/prototype/exec/java-exec/src/test/resources/remover/test1.json
> @@ -0,0 +1,39 @@
> +{
> +    head:{
> +        type:"APACHE_DRILL_PHYSICAL",
> +        version:"1",
> +        generator:{
> +            type:"manual"
> +        }
> +    },
> +       graph:[
> +        {
> +            @id:1,
> +            pop:"mock-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,
> +            child: 1,
> +            pop:"filter",
> +            expr: "alternate()"
> +        },
> +        {
> +            @id:3,
> +            child: 2,
> +            pop:"selection-vector-remover"
> +        },
> +        {
> +            @id: 4,
> +            child: 3,
> +            pop: "screen"
> +        }
> +    ]
> +}
> \ No newline at end of file
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0013215/sandbox/prototype/exec/java-exec/src/test/resources/scan_screen_logical.json
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/test/resources/scan_screen_logical.json
> b/sandbox/prototype/exec/java-exec/src/test/resources/scan_screen_logical.json
> index 3e5ebd7..5b6c19b 100644
> ---
> a/sandbox/prototype/exec/java-exec/src/test/resources/scan_screen_logical.json
> +++
> b/sandbox/prototype/exec/java-exec/src/test/resources/scan_screen_logical.json
> @@ -20,7 +20,7 @@
>
>                      {
>                          name : "superhero_name",
> -                        type : "VARCHAR1",
> +                        type : "VARCHAR2",
>                          mode : "REQUIRED",
>                          width : 100,
>                          precision : 1,
>
>

[52/53] [abbrv] git commit: Updates to support SelectionVector removal

Posted by ja...@apache.org.
Updates to support SelectionVector removal


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

Branch: refs/heads/master
Commit: f00132151a7946c9541f6317efb763325aef2874
Parents: 7779a72
Author: Jacques Nadeau <ja...@apache.org>
Authored: Fri Jul 19 14:40:57 2013 -0700
Committer: Jacques Nadeau <ja...@apache.org>
Committed: Fri Jul 19 14:57:51 2013 -0700

----------------------------------------------------------------------
 .../common/expression/fn/StringFunctions.java   |   4 +-
 .../org/apache/drill/common/types/Types.java    |  19 +-
 .../common/src/main/protobuf/Types.proto        |   2 -
 .../ValueVectors/data/ValueVectorTypes.tdd      |  17 +-
 .../templates/FixedValueVectors.java            |  32 +-
 .../templates/NullableValueVectors.java         |  40 +--
 .../templates/RepeatedValueVectors.java         |   5 +
 .../templates/VariableLengthVectors.java        |  34 ++-
 .../drill/exec/compile/QueryClassLoader.java    |  18 +-
 .../apache/drill/exec/expr/CodeGenerator.java   |  21 +-
 .../drill/exec/expr/annotations/Workspace.java  |  14 +
 .../apache/drill/exec/ops/FragmentContext.java  |   2 +-
 .../physical/config/SelectionVectorRemover.java |  60 ++++
 .../drill/exec/physical/impl/ImplCreator.java   |  13 +
 .../drill/exec/physical/impl/ScanBatch.java     |   2 +-
 .../physical/impl/filter/FilterRecordBatch.java |   2 +-
 .../impl/project/ProjectRecordBatch.java        |   4 +-
 .../exec/physical/impl/svremover/Copier.java    |  18 ++
 .../physical/impl/svremover/CopierTemplate.java |  43 +++
 .../physical/impl/svremover/CopyEvaluator.java  |  11 +
 .../impl/svremover/RemovingRecordBatch.java     | 305 +++++++++++++++++++
 .../impl/svremover/SVRemoverCreator.java        |  23 ++
 .../apache/drill/exec/record/RecordBatch.java   |  46 +--
 .../drill/exec/record/RecordBatchLoader.java    |   2 +-
 .../drill/exec/record/RecordRemapper.java       |   8 +
 .../apache/drill/exec/record/WritableBatch.java |  39 +--
 .../drill/exec/vector/BaseDataValueVector.java  |   4 +-
 .../drill/exec/vector/BaseValueVector.java      |   3 +
 .../org/apache/drill/exec/vector/BitVector.java |  21 +-
 .../drill/exec/vector/FixedWidthVector.java     |   2 +
 .../drill/exec/vector/NonRepeatedMutator.java   |   2 +-
 .../apache/drill/exec/vector/ValueVector.java   |   2 +
 .../drill/exec/vector/VariableWidthVector.java  |   2 +
 .../exec/physical/impl/SimpleRootExec.java      |  13 +-
 .../physical/impl/filter/TestSimpleFilter.java  |   4 +-
 .../impl/project/TestSimpleProjection.java      |   3 +-
 .../physical/impl/svremover/TestSVRemover.java  |  64 ++++
 .../src/test/resources/remover/test1.json       |  39 +++
 .../src/test/resources/scan_screen_logical.json |   2 +-
 39 files changed, 813 insertions(+), 132 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0013215/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/fn/StringFunctions.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/fn/StringFunctions.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/fn/StringFunctions.java
index 568b209..565843f 100644
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/fn/StringFunctions.java
+++ b/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/fn/StringFunctions.java
@@ -31,8 +31,8 @@ public class StringFunctions implements CallProvider{
   public FunctionDefinition[] getFunctionDefintions() {
     return new FunctionDefinition[]{
         FunctionDefinition.simple("regex_like", new BasicArgumentValidator( //
-            new Arg(true, false, "pattern", MinorType.VARCHAR1, MinorType.VARCHAR2, MinorType.VARCHAR4), //
-            new Arg(false, true, "value", MinorType.FIXEDCHAR, MinorType.VARCHAR1, MinorType.VARCHAR2, MinorType.VARCHAR4) ), FixedType.FIXED_BOOLEAN),
+            new Arg(true, false, "pattern", MinorType.VARCHAR2, MinorType.VARCHAR4), //
+            new Arg(false, true, "value", MinorType.FIXEDCHAR, MinorType.VARCHAR2, MinorType.VARCHAR4) ), FixedType.FIXED_BOOLEAN),
     };
 
   }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0013215/sandbox/prototype/common/src/main/java/org/apache/drill/common/types/Types.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/types/Types.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/types/Types.java
index e1343ab..757bed0 100644
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/types/Types.java
+++ b/sandbox/prototype/common/src/main/java/org/apache/drill/common/types/Types.java
@@ -39,11 +39,27 @@ public class Types {
     }
   }
   
+  public static boolean isFixedWidthType(MajorType type){
+    switch(type.getMinorType()){
+    case MSGPACK2:
+    case MSGPACK4:
+    case PROTO2:
+    case PROTO4:
+    case VARBINARY2:
+    case VARBINARY4:
+    case VARCHAR2:
+    case VARCHAR4:
+      return false;
+    default:
+      return true;
+    }
+  }
+  
+  
   public static boolean isStringScalarType(MajorType type){
     if(type.getMode() == DataMode.REPEATED) return false;
     switch(type.getMinorType()){
     case FIXEDCHAR:
-    case VARCHAR1:
     case VARCHAR2:
     case VARCHAR4:
       return true;
@@ -56,7 +72,6 @@ public class Types {
     if(type.getMode() == DataMode.REPEATED) return false;
     switch(type.getMinorType()){
     case FIXEDBINARY:
-    case VARBINARY1:
     case VARBINARY2:
     case VARBINARY4:
       return true;

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0013215/sandbox/prototype/common/src/main/protobuf/Types.proto
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/protobuf/Types.proto b/sandbox/prototype/common/src/main/protobuf/Types.proto
index 17e371d..05a70c5 100644
--- a/sandbox/prototype/common/src/main/protobuf/Types.proto
+++ b/sandbox/prototype/common/src/main/protobuf/Types.proto
@@ -28,11 +28,9 @@ enum MinorType {
     FLOAT8 = 19;   //  8 byte ieee 754
     BOOLEAN = 20;   //  single bit value
     FIXEDCHAR = 21;   //  utf8 fixed length string, padded with spaces
-    VARCHAR1 = 22;   //  utf8 variable length string (up to 2^8 in length)
     VARCHAR2 = 23;   //  utf8 variable length string (up to 2^16 in length)
     VARCHAR4 = 24;   //  utf8 variable length string (up to 2^32 in length)
     FIXEDBINARY = 25;   //  fixed length binary, padded with 0 bytes
-    VARBINARY1 = 26;   //  variable length binary (up to 2^8 in length)
     VARBINARY2 = 27;   //  variable length binary (up to 2^16 in length)
     VARBINARY4 = 28;   //  variable length binary (up to 2^32 in length)
     UINT1 = 29;   //  unsigned 1 byte integer

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0013215/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/data/ValueVectorTypes.tdd
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/data/ValueVectorTypes.tdd b/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/data/ValueVectorTypes.tdd
index 42153b6..69cc277 100644
--- a/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/data/ValueVectorTypes.tdd
+++ b/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/data/ValueVectorTypes.tdd
@@ -12,10 +12,16 @@
     {
       major: "Fixed",
       width: 2,
+      javaType: "char",
+      minor: [
+        { class: "UInt2" }
+      ]
+    },    {
+      major: "Fixed",
+      width: 2,
       javaType: "short",
       minor: [
         { class: "SmallInt" },
-        { class: "UInt2" }
       ]
     },
     {
@@ -65,15 +71,6 @@
     },
     {
       major: "VarLen",
-      width: 1,
-      javaType: "byte",
-      minor: [
-        { class: "VarBinary1" },
-        { class: "VarChar1" }
-      ]
-    },
-    {
-      major: "VarLen",
       width: 2,
       javaType: "short",
       minor: [

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0013215/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/FixedValueVectors.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/FixedValueVectors.java b/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/FixedValueVectors.java
index e0c8406..61515b0 100644
--- a/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/FixedValueVectors.java
+++ b/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/FixedValueVectors.java
@@ -51,7 +51,7 @@ public final class ${minor.class}Vector extends BaseDataValueVector implements F
     return mutator;
   }
   
-
+  
 
   /**
    * Allocate a new buffer that supports setting at least the provided number of values.  May actually be sized bigger depending on underlying buffer rounding size. Must be called prior to using the ValueVector.
@@ -68,16 +68,16 @@ public final class ${minor.class}Vector extends BaseDataValueVector implements F
   public FieldMetadata getMetadata() {
     return FieldMetadata.newBuilder()
              .setDef(getField().getDef())
-             .setValueCount(recordCount)
-             .setBufferLength(recordCount * ${type.width})
+             .setValueCount(valueCount)
+             .setBufferLength(valueCount * ${type.width})
              .build();
   }
 
   @Override
   public int load(int valueCount, ByteBuf buf){
     clear();
-    this.recordCount = valueCount;
-    int len = recordCount * ${type.width};
+    this.valueCount = valueCount;
+    int len = valueCount * ${type.width};
     data = buf.slice(0, len);
     data.retain();
     return len;
@@ -97,7 +97,7 @@ public final class ${minor.class}Vector extends BaseDataValueVector implements F
   public void transferTo(${minor.class}Vector target){
     target.data = data;
     target.data.retain();
-    target.recordCount = recordCount;
+    target.valueCount = valueCount;
     clear();
   }
   
@@ -117,10 +117,20 @@ public final class ${minor.class}Vector extends BaseDataValueVector implements F
     }
   }
   
+  public void copyValue(int inIndex, int outIndex, ${minor.class}Vector v){
+    <#if (type.width > 8)>
+    data.getBytes(inIndex * ${type.width}, v.data, outIndex * ${type.width}, ${type.width});
+    <#else> <#-- type.width <= 8 -->
+    data.set${(minor.javaType!type.javaType)?cap_first}(outIndex * ${type.width}, 
+        data.get${(minor.javaType!type.javaType)?cap_first}(inIndex * ${type.width})
+    );
+    </#if> <#-- type.width -->
+  }
+  
   public final class Accessor extends BaseValueVector.BaseAccessor{
 
-    public int getRecordCount() {
-      return recordCount;
+    public int getValueCount() {
+      return valueCount;
     }
     
     <#if (type.width > 8)>
@@ -207,9 +217,9 @@ public final class ${minor.class}Vector extends BaseDataValueVector implements F
    }
   </#if> <#-- type.width -->
   
-   public void setValueCount(int recordCount) {
-     ${minor.class}Vector.this.recordCount = recordCount;
-     data.writerIndex(${type.width} * recordCount);
+   public void setValueCount(int valueCount) {
+     ${minor.class}Vector.this.valueCount = valueCount;
+     data.writerIndex(${type.width} * valueCount);
    }
 
 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0013215/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/NullableValueVectors.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/NullableValueVectors.java b/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/NullableValueVectors.java
index d2d436a..dfec62e 100644
--- a/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/NullableValueVectors.java
+++ b/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/NullableValueVectors.java
@@ -33,7 +33,7 @@ import org.apache.drill.exec.vector.UInt4Vector;
 @SuppressWarnings("unused")
 public final class Nullable${minor.class}Vector extends BaseValueVector implements <#if type.major == "VarLen">VariableWidth<#else>FixedWidth</#if>Vector {
 
-  private int recordCount;
+  private int valueCount;
   private final BitVector bits;
   private final ${minor.class}Vector values;
   private final Accessor accessor = new Accessor();
@@ -56,7 +56,7 @@ public final class Nullable${minor.class}Vector extends BaseValueVector implemen
 
   @Override
   public void clear() {
-    recordCount = 0;
+    valueCount = 0;
     bits.clear();
     values.clear();
   }
@@ -70,7 +70,7 @@ public final class Nullable${minor.class}Vector extends BaseValueVector implemen
   public FieldMetadata getMetadata() {
     return FieldMetadata.newBuilder()
              .setDef(getField().getDef())
-             .setValueCount(recordCount)
+             .setValueCount(valueCount)
              .setVarByteLength(values.getVarByteLength())
              .setBufferLength(getBufferSize())
              .build();
@@ -87,7 +87,7 @@ public final class Nullable${minor.class}Vector extends BaseValueVector implemen
   @Override
   public int load(int dataBytes, int valueCount, ByteBuf buf){
     clear();
-    this.recordCount = valueCount;
+    this.valueCount = valueCount;
     int loaded = bits.load(valueCount, buf);
     
     // remove bits part of buffer.
@@ -113,7 +113,7 @@ public final class Nullable${minor.class}Vector extends BaseValueVector implemen
   public FieldMetadata getMetadata() {
     return FieldMetadata.newBuilder()
              .setDef(getField().getDef())
-             .setValueCount(recordCount)
+             .setValueCount(valueCount)
              .setBufferLength(getBufferSize())
              .build();
   }
@@ -129,7 +129,7 @@ public final class Nullable${minor.class}Vector extends BaseValueVector implemen
   @Override
   public int load(int valueCount, ByteBuf buf){
     clear();
-    this.recordCount = valueCount;
+    this.valueCount = valueCount;
     int loaded = bits.load(valueCount, buf);
     
     // remove bits part of buffer.
@@ -154,7 +154,7 @@ public final class Nullable${minor.class}Vector extends BaseValueVector implemen
   public void transferTo(Nullable${minor.class}Vector target){
     bits.transferTo(target.bits);
     values.transferTo(target.values);
-    target.recordCount = recordCount;
+    target.valueCount = valueCount;
     clear();
   }
   
@@ -185,13 +185,17 @@ public final class Nullable${minor.class}Vector extends BaseValueVector implemen
   public ${minor.class}Vector convertToRequiredVector(){
     ${minor.class}Vector v = new ${minor.class}Vector(getField().getOtherNullableVersion(), allocator);
     v.data = values.data;
-    v.recordCount = this.recordCount;
+    v.valueCount = this.valueCount;
     v.data.retain();
     clear();
     return v;
   }
+
   
-  
+  public void copyValue(int inIndex, int outIndex, Nullable${minor.class}Vector v){
+    bits.copyValue(inIndex, outIndex, v.bits);
+    values.copyValue(inIndex, outIndex, v.values);
+  }
   
   public final class Accessor implements ValueVector.Accessor{
 
@@ -220,14 +224,14 @@ public final class Nullable${minor.class}Vector extends BaseValueVector implemen
       return isNull(index) ? null : values.getAccessor().getObject(index);
     }
     
-    public int getRecordCount(){
-      return recordCount;
+    public int getValueCount(){
+      return valueCount;
     }
     
     public void reset(){}
   }
   
-  public final class Mutator implements ValueVector.Mutator{
+  public final class Mutator implements NonRepeatedMutator{
     
     private int setCount;
     
@@ -246,15 +250,15 @@ public final class Nullable${minor.class}Vector extends BaseValueVector implemen
       values.getMutator().set(index, value);
     }
 
-    public void setValueCount(int recordCount) {
-      assert recordCount >= 0;
-      Nullable${minor.class}Vector.this.recordCount = recordCount;
-      values.getMutator().setValueCount(recordCount);
-      bits.getMutator().setValueCount(recordCount);
+    public void setValueCount(int valueCount) {
+      assert valueCount >= 0;
+      Nullable${minor.class}Vector.this.valueCount = valueCount;
+      values.getMutator().setValueCount(valueCount);
+      bits.getMutator().setValueCount(valueCount);
     }
     
     public boolean noNulls(){
-      return recordCount == setCount;
+      return valueCount == setCount;
     }
     
     public void randomizeData(){

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0013215/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/RepeatedValueVectors.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/RepeatedValueVectors.java b/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/RepeatedValueVectors.java
index 1976340..30bc086 100644
--- a/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/RepeatedValueVectors.java
+++ b/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/RepeatedValueVectors.java
@@ -90,6 +90,11 @@ import org.apache.drill.exec.record.TransferPair;
     }
   }
   
+  public void copyValue(int inIndex, int outIndex, Repeated${minor.class}Vector v){
+    throw new UnsupportedOperationException();
+  }
+
+  
   <#if type.major == "VarLen">
   @Override
   public FieldMetadata getMetadata() {

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0013215/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/VariableLengthVectors.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/VariableLengthVectors.java b/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/VariableLengthVectors.java
index 9d723ab..7daadee 100644
--- a/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/VariableLengthVectors.java
+++ b/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/VariableLengthVectors.java
@@ -22,6 +22,8 @@ import org.apache.drill.exec.record.MaterializedField;
 import org.apache.drill.exec.record.TransferPair;
 import org.apache.drill.exec.vector.ByteHolder;
 
+import antlr.collections.impl.Vector;
+
 /**
  * ${minor.class}Vector implements a vector of variable width values.  Elements in the vector
  * are accessed by position from the logical start of the vector.  A fixed width offsetVector
@@ -63,22 +65,22 @@ public final class ${minor.class}Vector extends BaseDataValueVector implements V
    * @return
    */
   public int getVarByteLength(){
-    return offsetVector.getAccessor().get(recordCount); 
+    return offsetVector.getAccessor().get(valueCount); 
   }
   
   @Override
   public FieldMetadata getMetadata() {
-    int len = recordCount * ${type.width} + getVarByteLength();
+    int len = valueCount * ${type.width} + getVarByteLength();
     return FieldMetadata.newBuilder()
              .setDef(getField().getDef())
-             .setValueCount(recordCount)
+             .setValueCount(valueCount)
              .setVarByteLength(getVarByteLength())
              .setBufferLength(len)
              .build();
   }
 
   public int load(int dataBytes, int valueCount, ByteBuf buf){
-    this.recordCount = valueCount;
+    this.valueCount = valueCount;
     int loaded = offsetVector.load(valueCount+1, buf);
     data = buf.slice(loaded, dataBytes);
     data.retain();
@@ -111,10 +113,20 @@ public final class ${minor.class}Vector extends BaseDataValueVector implements V
     this.offsetVector.transferTo(target.offsetVector);
     target.data = data;
     target.data.retain();
-    target.recordCount = recordCount;
+    target.valueCount = valueCount;
     clear();
   }
   
+  public void copyValue(int inIndex, int outIndex, ${minor.class}Vector v){
+    int start = offsetVector.data.get${(minor.javaType!type.javaType)?cap_first}(inIndex);
+    int end =   offsetVector.data.get${(minor.javaType!type.javaType)?cap_first}(inIndex+1);
+    int len = end - start;
+    
+    int outputStart = outIndex == 0 ? 0 : v.offsetVector.data.get${(minor.javaType!type.javaType)?cap_first}(outIndex * ${type.width});
+    data.getBytes(start, v.data, outputStart, len);
+    v.offsetVector.data.set${(minor.javaType!type.javaType)?cap_first}( (outIndex+1) * ${type.width}, len);
+  }
+  
   private class TransferImpl implements TransferPair{
     ${minor.class}Vector to;
     
@@ -172,8 +184,8 @@ public final class ${minor.class}Vector extends BaseDataValueVector implements V
       return get(index);
     }
     
-    public int getRecordCount() {
-      return recordCount;
+    public int getValueCount() {
+      return valueCount;
     }
   }
   
@@ -210,10 +222,10 @@ public final class ${minor.class}Vector extends BaseDataValueVector implements V
       data.setBytes(currentOffset, bb);
     }
 
-    public void setValueCount(int recordCount) {
-      ${minor.class}Vector.this.recordCount = recordCount;
-      data.writerIndex(recordCount * ${type.width});
-      offsetVector.getMutator().setValueCount(recordCount+1);
+    public void setValueCount(int valueCount) {
+      ${minor.class}Vector.this.valueCount = valueCount;
+      data.writerIndex(valueCount * ${type.width});
+      offsetVector.getMutator().setValueCount(valueCount+1);
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0013215/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/compile/QueryClassLoader.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/compile/QueryClassLoader.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/compile/QueryClassLoader.java
index 60aa8f3..7feca09 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/compile/QueryClassLoader.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/compile/QueryClassLoader.java
@@ -18,32 +18,22 @@
 package org.apache.drill.exec.compile;
 
 import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
 import java.net.URL;
 import java.net.URLClassLoader;
 import java.util.concurrent.ConcurrentMap;
-
-import javax.tools.JavaCompiler;
-import javax.tools.JavaFileManager;
-import javax.tools.JavaFileObject;
-import javax.tools.JavaFileObject.Kind;
-import javax.tools.StandardLocation;
-import javax.tools.ToolProvider;
+import java.util.concurrent.atomic.AtomicLong;
 
 import org.apache.drill.exec.exception.ClassTransformationException;
 import org.codehaus.commons.compiler.CompileException;
-import org.codehaus.commons.compiler.jdk.ByteArrayJavaFileManager;
-import org.codehaus.commons.compiler.jdk.ByteArrayJavaFileManager.ByteArrayJavaFileObject;
 
 import com.google.common.collect.MapMaker;
-import com.google.common.io.ByteStreams;
 
 public class QueryClassLoader extends URLClassLoader {
 
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(QueryClassLoader.class);
 
   private final ClassCompiler classCompiler;
+  private AtomicLong index = new AtomicLong(0);
   private ConcurrentMap<String, byte[]> customClasses = new MapMaker().concurrencyLevel(4).makeMap();
 
   public QueryClassLoader(boolean useJanino) {
@@ -55,6 +45,10 @@ public class QueryClassLoader extends URLClassLoader {
     }
   }
 
+  public long getNextClassIndex(){
+    return index.getAndIncrement();
+  }
+  
   public void injectByteCode(String className, byte[] classBytes) throws IOException {
     if(customClasses.containsKey(className)) throw new IOException(String.format("The class defined {} has already been loaded.", className));
     customClasses.put(className, classBytes);

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0013215/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 6e31dec..b99af19 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
@@ -1,7 +1,7 @@
 package org.apache.drill.exec.expr;
 
 import java.io.IOException;
-import java.lang.reflect.Method;
+import java.util.concurrent.atomic.AtomicLong;
 
 import org.apache.drill.common.expression.LogicalExpression;
 import org.apache.drill.common.types.TypeProtos.DataMode;
@@ -42,13 +42,19 @@ public class CodeGenerator<T> {
   private final TemplateClassDefinition<T> definition;
   private JCodeModel model;
   private int index = 0;
-
+  private static AtomicLong classCreator = new AtomicLong(0);
+  private String className;
+  private String fqcn;
+  private String packageName = "org.apache.drill.exec.test.generated";
+  
   public CodeGenerator(TemplateClassDefinition<T> definition, FunctionImplementationRegistry funcRegistry) {
     super();
+    className = "Gen" + classCreator.incrementAndGet();
+    fqcn = packageName + "." + className;
     try{
       this.definition = definition;
       this.model = new JCodeModel();
-      this.clazz = model._package("org.apache.drill.exec.test.generated")._class("Test1");
+      this.clazz = model._package(packageName)._class(className);
       clazz._implements(definition.getInternalInterface());
       this.parentEvalBlock = new JBlock();
       this.parentSetupBlock = new JBlock();
@@ -62,11 +68,16 @@ public class CodeGenerator<T> {
 
   public void addExpr(LogicalExpression ex){
     logger.debug("Adding next write {}", ex);
+    rotateBlock();
+    ex.accept(evaluationVisitor, this);
+  }
+  
+  public void rotateBlock(){
     currentEvalBlock = new JBlock();
     parentEvalBlock.add(currentEvalBlock);
     currentSetupBlock = new JBlock();
     parentSetupBlock.add(currentSetupBlock);
-    ex.accept(evaluationVisitor, this);
+
   }
   
   public JBlock getBlock() {
@@ -74,7 +85,7 @@ public class CodeGenerator<T> {
   }
 
   public String getMaterializedClassName(){
-    return "org.apache.drill.exec.test.generated.Test1";
+    return fqcn;
   }
   
   public JBlock getSetupBlock(){

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0013215/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/annotations/Workspace.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/annotations/Workspace.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/annotations/Workspace.java
new file mode 100644
index 0000000..75c97b5
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/annotations/Workspace.java
@@ -0,0 +1,14 @@
+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;
+
+/**
+ * Describes the field will provide output from the given function.
+ */
+@Retention(RetentionPolicy.RUNTIME)
+@Target({ElementType.FIELD})
+public @interface Workspace {
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0013215/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 2ae4afa..64f3eb4 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
@@ -106,7 +106,7 @@ public class FragmentContext {
   public <T> T getImplementationClass(CodeGenerator<T> cg) throws ClassTransformationException, IOException{
     long t1 = System.nanoTime();
     T t= transformer.getImplementationClass(this.loader, cg.getDefinition(), cg.generate(), cg.getMaterializedClassName());
-    System.out.println( (System.nanoTime() - t1)/1000/1000 );
+    logger.debug("Compile time: {} micros.", (System.nanoTime() - t1)/1000/1000 );
     return t;
     
   }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0013215/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/SelectionVectorRemover.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/SelectionVectorRemover.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/SelectionVectorRemover.java
new file mode 100644
index 0000000..2feae02
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/SelectionVectorRemover.java
@@ -0,0 +1,60 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.physical.config;
+
+import org.apache.drill.exec.physical.OperatorCost;
+import org.apache.drill.exec.physical.base.AbstractSingle;
+import org.apache.drill.exec.physical.base.PhysicalOperator;
+import org.apache.drill.exec.physical.base.PhysicalVisitor;
+import org.apache.drill.exec.physical.base.Size;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.annotation.JsonTypeName;
+
+@JsonTypeName("selection-vector-remover")
+public class SelectionVectorRemover extends AbstractSingle {
+
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(SelectionVectorRemover.class);
+  
+  @JsonCreator
+  public SelectionVectorRemover(@JsonProperty("child") PhysicalOperator child) {
+    super(child);
+  }
+
+  @Override
+  public <T, X, E extends Throwable> T accept(PhysicalVisitor<T, X, E> physicalVisitor, X value) throws E{
+    return physicalVisitor.visitOp(this, value);
+  }
+
+  @Override
+  public OperatorCost getCost() {
+    return child.getCost();
+  }
+
+  @Override
+  protected PhysicalOperator getNewWithChild(PhysicalOperator child) {
+    return new SelectionVectorRemover(child);
+  }
+
+  @Override
+  public Size getSize() {
+    return new Size( (long) (child.getSize().getRecordCount()), child.getSize().getRecordSize());
+  }
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0013215/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 f96d6f3..7740955 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
@@ -32,9 +32,11 @@ import org.apache.drill.exec.physical.config.MockScanPOP;
 import org.apache.drill.exec.physical.config.Project;
 import org.apache.drill.exec.physical.config.RandomReceiver;
 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.impl.filter.FilterBatchCreator;
 import org.apache.drill.exec.physical.impl.project.ProjectBatchCreator;
+import org.apache.drill.exec.physical.impl.svremover.SVRemoverCreator;
 import org.apache.drill.exec.record.RecordBatch;
 
 import com.google.common.base.Preconditions;
@@ -49,6 +51,7 @@ public class ImplCreator extends AbstractPhysicalVisitor<RecordBatch, FragmentCo
   private SingleSenderCreator ssc = new SingleSenderCreator();
   private ProjectBatchCreator pbc = new ProjectBatchCreator();
   private FilterBatchCreator fbc = new FilterBatchCreator();
+  private SVRemoverCreator svc = new SVRemoverCreator();
   private RootExec root = null;
   
   private ImplCreator(){}
@@ -75,6 +78,16 @@ public class ImplCreator extends AbstractPhysicalVisitor<RecordBatch, FragmentCo
     
   }
 
+  
+  @Override
+  public RecordBatch visitOp(PhysicalOperator op, FragmentContext context) throws ExecutionSetupException {
+    if(op instanceof SelectionVectorRemover){
+      return svc.getBatch(context, (SelectionVectorRemover) op, getChildren(op, context));
+    }else{
+      return super.visitOp(op, context);  
+    }
+  }
+
   @Override
   public RecordBatch visitScreen(Screen op, FragmentContext context) throws ExecutionSetupException {
     Preconditions.checkArgument(root == null);

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0013215/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScanBatch.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScanBatch.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScanBatch.java
index 084db54..a8a62ca 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScanBatch.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScanBatch.java
@@ -175,7 +175,7 @@ public class ScanBatch implements RecordBatch {
 
   @Override
   public WritableBatch getWritableBatch() {
-    return WritableBatch.get(this.getRecordCount(), vectors);
+    return WritableBatch.get(this);
   }
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0013215/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/filter/FilterRecordBatch.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/filter/FilterRecordBatch.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/filter/FilterRecordBatch.java
index fc9dbc6..60e8f42 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/filter/FilterRecordBatch.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/filter/FilterRecordBatch.java
@@ -193,7 +193,7 @@ public class FilterRecordBatch implements RecordBatch{
   
   @Override
   public WritableBatch getWritableBatch() {
-    return WritableBatch.get(sv.getCount(), outputVectors);
+    return WritableBatch.get(this);
   }
   
   

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0013215/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/ProjectRecordBatch.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/ProjectRecordBatch.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/ProjectRecordBatch.java
index 060cd92..4e2b820 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/ProjectRecordBatch.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/ProjectRecordBatch.java
@@ -190,7 +190,7 @@ public class ProjectRecordBatch implements RecordBatch{
       
     }
     
-    SchemaBuilder bldr = BatchSchema.newBuilder().setSelectionVectorMode(SelectionVectorMode.NONE);
+    SchemaBuilder bldr = BatchSchema.newBuilder().setSelectionVectorMode(incoming.getSchema().getSelectionVector());
     for(ValueVector v : outputVectors){
       bldr.addField(v.getField());
     }
@@ -207,7 +207,7 @@ public class ProjectRecordBatch implements RecordBatch{
   
   @Override
   public WritableBatch getWritableBatch() {
-    return WritableBatch.get(incoming.getRecordCount(), outputVectors);
+    return WritableBatch.get(this);
   }
   
   private MaterializedField getMaterializedField(FieldReference reference, LogicalExpression expr){

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0013215/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
new file mode 100644
index 0000000..55d6ba2
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/svremover/Copier.java
@@ -0,0 +1,18 @@
+package org.apache.drill.exec.physical.impl.svremover;
+
+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.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_DEFINITION = new TemplateClassDefinition<Copier>( //
+      Copier.class, "org.apache.drill.exec.physical.impl.svremover.CopierTemplate", CopyEvaluator.class, null);
+
+  public void setupRemover(FragmentContext context, RecordBatch incoming, RecordBatch outgoing, VectorAllocator[] allocators) throws SchemaChangeException;
+  public abstract void copyRecords();
+
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0013215/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/svremover/CopierTemplate.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/svremover/CopierTemplate.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/svremover/CopierTemplate.java
new file mode 100644
index 0000000..12a1e0a
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/svremover/CopierTemplate.java
@@ -0,0 +1,43 @@
+package org.apache.drill.exec.physical.impl.svremover;
+
+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 abstract class CopierTemplate implements Copier{
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(CopierTemplate.class);
+  
+  private SelectionVector2 sv2;
+  private VectorAllocator[] allocators;
+  
+  @Override
+  public void setupRemover(FragmentContext context, RecordBatch incoming, RecordBatch outgoing, VectorAllocator[] allocators) throws SchemaChangeException{
+    this.allocators = allocators;
+    this.sv2 = incoming.getSelectionVector2();
+    doSetup(context, incoming, outgoing);
+  }
+  
+  private void allocateVectors(int recordCount){
+    for(VectorAllocator a : allocators){
+      a.alloc(recordCount);
+    }
+  }
+  
+  @Override
+  public void copyRecords(){
+    final int recordCount = sv2.getCount();
+    allocateVectors(recordCount);
+    int outgoingPosition = 0;
+    
+    for(int svIndex = 0; svIndex < recordCount * 2; svIndex++, outgoingPosition++){
+      doEval(svIndex, outgoingPosition);
+    }
+  }
+  
+  public abstract void doSetup(FragmentContext context, RecordBatch incoming, RecordBatch outgoing) throws SchemaChangeException;
+  public abstract void doEval(int incoming, int outgoing);
+        
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0013215/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/svremover/CopyEvaluator.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/svremover/CopyEvaluator.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/svremover/CopyEvaluator.java
new file mode 100644
index 0000000..25c51bb
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/svremover/CopyEvaluator.java
@@ -0,0 +1,11 @@
+package org.apache.drill.exec.physical.impl.svremover;
+
+import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.record.RecordBatch;
+
+public interface CopyEvaluator {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(CopyEvaluator.class);
+  
+  public abstract void doSetup(FragmentContext context, RecordBatch incoming, RecordBatch outgoing);
+  public abstract void doEval(int incoming, int outgoing);
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0013215/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
new file mode 100644
index 0000000..208ebc6
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/svremover/RemovingRecordBatch.java
@@ -0,0 +1,305 @@
+package org.apache.drill.exec.physical.impl.svremover;
+
+import java.io.IOException;
+import java.util.Iterator;
+import java.util.List;
+
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.exec.exception.ClassTransformationException;
+import org.apache.drill.exec.exception.SchemaChangeException;
+import org.apache.drill.exec.expr.CodeGenerator;
+import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.physical.impl.VectorHolder;
+import org.apache.drill.exec.record.BatchSchema;
+import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
+import org.apache.drill.exec.record.RecordBatch;
+import org.apache.drill.exec.record.SchemaBuilder;
+import org.apache.drill.exec.record.TransferPair;
+import org.apache.drill.exec.record.WritableBatch;
+import org.apache.drill.exec.record.selection.SelectionVector2;
+import org.apache.drill.exec.record.selection.SelectionVector4;
+import org.apache.drill.exec.vector.FixedWidthVector;
+import org.apache.drill.exec.vector.NonRepeatedMutator;
+import org.apache.drill.exec.vector.ValueVector;
+import org.apache.drill.exec.vector.VariableWidthVector;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+import com.sun.codemodel.JBlock;
+import com.sun.codemodel.JClass;
+import com.sun.codemodel.JExpr;
+import com.sun.codemodel.JExpression;
+import com.sun.codemodel.JType;
+import com.sun.codemodel.JVar;
+
+public class RemovingRecordBatch implements RecordBatch{
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(RemovingRecordBatch.class);
+
+  private final RecordBatch incoming;
+  private final FragmentContext context;
+  private BatchSchema outSchema;
+  private Copier copier;
+  private List<ValueVector> outputVectors;
+  private VectorHolder vh;
+  
+  
+  public RemovingRecordBatch(RecordBatch incoming, FragmentContext context){
+    this.incoming = incoming;
+    this.context = context;
+  }
+  
+  @Override
+  public Iterator<ValueVector> iterator() {
+    return outputVectors.iterator();
+  }
+
+  @Override
+  public FragmentContext getContext() {
+    return context;
+  }
+
+  @Override
+  public BatchSchema getSchema() {
+    Preconditions.checkNotNull(outSchema);
+    return outSchema;
+  }
+
+  @Override
+  public int getRecordCount() {
+    return incoming.getRecordCount();
+  }
+
+  @Override
+  public void kill() {
+    incoming.kill();
+  }
+
+  @Override
+  public SelectionVector2 getSelectionVector2() {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public SelectionVector4 getSelectionVector4() {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public TypedFieldId getValueVectorId(SchemaPath path) {
+    return vh.getValueVector(path);
+  }
+
+  @Override
+  public <T extends ValueVector> T getValueVectorById(int fieldId, Class<?> clazz) {
+    return vh.getValueVector(fieldId, clazz);
+  }
+
+  @Override
+  public IterOutcome next() {
+    
+    IterOutcome upstream = incoming.next();
+    logger.debug("Upstream... {}", upstream);
+    switch(upstream){
+    case NONE:
+    case NOT_YET:
+    case STOP:
+      return upstream;
+    case OK_NEW_SCHEMA:
+      try{
+        copier = createCopier();
+      }catch(SchemaChangeException ex){
+        incoming.kill();
+        logger.error("Failure during query", ex);
+        context.fail(ex);
+        return IterOutcome.STOP;
+      }
+      // fall through.
+    case OK:
+      int recordCount = incoming.getRecordCount();
+      copier.copyRecords();
+      for(ValueVector v : this.outputVectors){
+        ValueVector.Mutator m = v.getMutator();
+        if(m instanceof NonRepeatedMutator){
+          ((NonRepeatedMutator) m).setValueCount(recordCount);
+        }else{
+          throw new UnsupportedOperationException();
+        }
+      }
+      return upstream; // change if upstream changed, otherwise normal.
+    default:
+      throw new UnsupportedOperationException();
+    }
+  }
+  
+  
+
+  
+  private class StraightCopier implements Copier{
+
+    private List<TransferPair> pairs = Lists.newArrayList();
+    private List<ValueVector> out = Lists.newArrayList();
+    
+    @Override
+    public void setupRemover(FragmentContext context, RecordBatch incoming, RecordBatch outgoing, VectorAllocator[] allocators){
+      for(ValueVector vv : incoming){
+        TransferPair tp = vv.getTransferPair();
+        pairs.add(tp);
+        out.add(tp.getTo());
+      }
+    }
+
+    @Override
+    public void copyRecords() {
+      for(TransferPair tp : pairs){
+        tp.transfer();
+      }
+    }
+
+    public List<ValueVector> getOut() {
+      return out;
+    }
+    
+  }
+
+  private Copier getStraightCopier(){
+    StraightCopier copier = new StraightCopier();
+    copier.setupRemover(context, incoming, this, null);
+    outputVectors.addAll(copier.getOut());
+    return copier;
+  }
+  
+  private Copier getGeneratedCopier() throws SchemaChangeException{
+    Preconditions.checkArgument(incoming.getSchema().getSelectionVector() == SelectionVectorMode.TWO_BYTE);
+    
+    List<VectorAllocator> allocators = Lists.newArrayList();
+    for(ValueVector i : incoming){
+      TransferPair t = i.getTransferPair();
+      outputVectors.add(t.getTo());
+      allocators.add(getAllocator(i, t.getTo()));
+    }
+
+    try {
+      final CodeGenerator<Copier> cg = new CodeGenerator<Copier>(Copier.TEMPLATE_DEFINITION, context.getFunctionRegistry());
+      generateCopies(cg);
+      Copier copier = context.getImplementationClass(cg);
+      copier.setupRemover(context, incoming, this, allocators.toArray(new VectorAllocator[allocators.size()]));
+      return copier;
+    } catch (ClassTransformationException | IOException e) {
+      throw new SchemaChangeException("Failure while attempting to load generated class", e);
+    }
+  }
+  
+  
+  private Copier createCopier() throws SchemaChangeException{
+    if(outputVectors != null){
+      for(ValueVector v : outputVectors){
+        v.close();
+      }
+    }
+    this.outputVectors = Lists.newArrayList();
+    this.vh = new VectorHolder(outputVectors);
+
+    SchemaBuilder bldr = BatchSchema.newBuilder().setSelectionVectorMode(SelectionVectorMode.NONE);
+    for(ValueVector v : outputVectors){
+      bldr.addField(v.getField());
+    }
+    this.outSchema = bldr.build();
+    
+    switch(incoming.getSchema().getSelectionVector()){
+    case NONE:
+      return getStraightCopier();
+    case TWO_BYTE:
+      return getGeneratedCopier();
+    default:
+      throw new UnsupportedOperationException();
+    }
+
+  }
+  
+  private void generateCopies(CodeGenerator<Copier> g){
+    // we have parallel ids for each value vector so we don't actually have to deal with managing the ids at all.
+    int fieldId = 0;
+    
+
+
+    JExpression inIndex = JExpr.direct("inIndex");
+    JExpression outIndex = JExpr.direct("outIndex");
+    g.rotateBlock();
+    for(ValueVector vv : incoming){
+      JClass vvClass = (JClass) g.getModel()._ref(vv.getClass());
+      JVar inVV = declareVVSetup("incoming", g, fieldId, vvClass);
+      JVar outVV = declareVVSetup("outgoing", g, fieldId, vvClass);
+      
+      g.getBlock().add(inVV.invoke("copyValue").arg(inIndex).arg(outIndex).arg(outVV));
+      
+      fieldId++;
+    }
+  }
+  
+  private JVar declareVVSetup(String varName, CodeGenerator<?> g, int fieldId, JClass vvClass){
+    JVar vv = g.declareClassField("vv", vvClass);
+    JClass t = (JClass) g.getModel()._ref(SchemaChangeException.class);
+    JType objClass = g.getModel()._ref(Object.class);
+    JBlock b = g.getSetupBlock();
+    JVar obj = b.decl( //
+        objClass, //
+        g.getNextVar("tmp"), // 
+        JExpr.direct(varName).invoke("getValueVectorById").arg(JExpr.lit(fieldId)).arg( vvClass.dotclass()));
+        b._if(obj.eq(JExpr._null()))._then()._throw(JExpr._new(t).arg(JExpr.lit(String.format("Failure while loading vector %s with id %d", vv.name(), fieldId))));
+        b.assign(vv, JExpr.cast(vvClass, obj));
+        
+    return vv;
+  }
+  
+  @Override
+  public WritableBatch getWritableBatch() {
+    return WritableBatch.get(this);
+  }
+  
+  private VectorAllocator getAllocator(ValueVector in, ValueVector outgoing){
+    if(outgoing instanceof FixedWidthVector){
+      return new FixedVectorAllocator((FixedWidthVector) outgoing);
+    }else if(outgoing instanceof VariableWidthVector && in instanceof VariableWidthVector){
+      return new VariableVectorAllocator( (VariableWidthVector) in, (VariableWidthVector) outgoing);
+    }else{
+      throw new UnsupportedOperationException();
+    }
+  }
+  
+  private class FixedVectorAllocator implements VectorAllocator{
+    FixedWidthVector out;
+    
+    public FixedVectorAllocator(FixedWidthVector out) {
+      super();
+      this.out = out;
+    }
+
+    public void alloc(int recordCount){
+      out.allocateNew(recordCount);
+      out.getMutator().setValueCount(recordCount);
+    }
+
+    
+    
+  }
+  
+  private class VariableVectorAllocator implements VectorAllocator{
+    VariableWidthVector in;
+    VariableWidthVector out;
+    
+    public VariableVectorAllocator(VariableWidthVector in, VariableWidthVector out) {
+      super();
+      this.in = in;
+      this.out = out;
+    }
+
+    public void alloc(int recordCount){
+      out.allocateNew(in.getByteCapacity(), recordCount);
+      out.getMutator().setValueCount(recordCount);
+    }
+  }
+  
+  public interface VectorAllocator{
+    public void alloc(int recordCount);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0013215/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/svremover/SVRemoverCreator.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/svremover/SVRemoverCreator.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/svremover/SVRemoverCreator.java
new file mode 100644
index 0000000..4671baa
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/svremover/SVRemoverCreator.java
@@ -0,0 +1,23 @@
+package org.apache.drill.exec.physical.impl.svremover;
+
+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.SelectionVectorRemover;
+import org.apache.drill.exec.physical.impl.BatchCreator;
+import org.apache.drill.exec.record.RecordBatch;
+
+import com.google.common.base.Preconditions;
+
+public class SVRemoverCreator implements BatchCreator<SelectionVectorRemover>{
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(SVRemoverCreator.class);
+
+  @Override
+  public RecordBatch getBatch(FragmentContext context, SelectionVectorRemover config, List<RecordBatch> children) throws ExecutionSetupException {
+    Preconditions.checkArgument(children.size() == 1);
+    return new RemovingRecordBatch(children.iterator().next(), context);
+  }
+  
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0013215/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatch.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatch.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatch.java
index ff856d4..fc4e759 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatch.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatch.java
@@ -28,8 +28,11 @@ import org.apache.drill.exec.vector.ValueVector;
  * A record batch contains a set of field values for a particular range of records. In the case of a record batch
  * composed of ValueVectors, ideally a batch fits within L2 cache (~256k per core). The set of value vectors do not
  * change unless the next() IterOutcome is a *_NEW_SCHEMA type.
+ * 
+ * A key thing to know is that the Iterator provided by record batch must align with the rank positions of the field ids
+ * provided utilizing getValueVectorId();
  */
-public interface RecordBatch extends Iterable<ValueVector>{
+public interface RecordBatch extends Iterable<ValueVector> {
 
   /**
    * Describes the outcome of a RecordBatch being incremented forward.
@@ -39,16 +42,14 @@ public interface RecordBatch extends Iterable<ValueVector>{
     OK, // A new range of records have been provided.
     OK_NEW_SCHEMA, // A full collection of records
     STOP, // Informs parent nodes that the query has terminated. In this case, a consumer can consume their QueryContext
-         // to understand the current state of things.
+          // to understand the current state of things.
     NOT_YET // used by batches that haven't received incoming data yet.
   }
 
   public static enum SetupOutcome {
-    OK,
-    OK_NEW_SCHEMA,
-    FAILED
+    OK, OK_NEW_SCHEMA, FAILED
   }
-  
+
   /**
    * Access the FragmentContext of the current query fragment. Useful for reporting failure information or other query
    * level information.
@@ -78,17 +79,21 @@ public interface RecordBatch extends Iterable<ValueVector>{
   public void kill();
 
   public abstract SelectionVector2 getSelectionVector2();
+
   public abstract SelectionVector4 getSelectionVector4();
-  
+
   /**
-   * Get the value vector 
-   * @param path The path where the vector should be located.
-   * @return The local field id associated with this vector.
+   * Get the value vector type and id for the given schema path. The TypedFieldId should store a fieldId which is the
+   * same as the ordinal position of the field within the Iterator provided this classes implementation of
+   * Iterable<ValueVector>.
+   * 
+   * @param path
+   *          The path where the vector should be located.
+   * @return The local field id associated with this vector. If no field matches this path, this will return a null
+   *         TypedFieldId
    */
   public abstract TypedFieldId getValueVectorId(SchemaPath path);
-  
-  
-  
+
   public abstract <T extends ValueVector> T getValueVectorById(int fieldId, Class<?> clazz);
 
   /**
@@ -99,24 +104,28 @@ public interface RecordBatch extends Iterable<ValueVector>{
    * @return An IterOutcome describing the result of the iteration.
    */
   public IterOutcome next();
-  
+
   /**
-   * Get a writable version of this batch.  Takes over owernship of existing buffers.
+   * Get a writable version of this batch. Takes over owernship of existing buffers.
+   * 
    * @return
    */
   public WritableBatch getWritableBatch();
 
-  public static class TypedFieldId{
+  public static class TypedFieldId {
     final MajorType type;
     final int fieldId;
+
     public TypedFieldId(MajorType type, int fieldId) {
       super();
       this.type = type;
       this.fieldId = fieldId;
     }
+
     public MajorType getType() {
       return type;
     }
+
     public int getFieldId() {
       return fieldId;
     }
@@ -139,8 +148,7 @@ public interface RecordBatch extends Iterable<ValueVector>{
         return false;
       return true;
     }
-    
-    
+
   }
-  
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0013215/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatchLoader.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatchLoader.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatchLoader.java
index 5f7648b..4a12af6 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatchLoader.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatchLoader.java
@@ -137,7 +137,7 @@ public class RecordBatchLoader implements Iterable<ValueVector>{
 
 
   public WritableBatch getWritableBatch(){
-    return WritableBatch.get(recordCount, vectors);
+    return WritableBatch.getBatchNoSV(recordCount, vectors);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0013215/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordRemapper.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordRemapper.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordRemapper.java
new file mode 100644
index 0000000..86c963d
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordRemapper.java
@@ -0,0 +1,8 @@
+package org.apache.drill.exec.record;
+
+/**
+ * Remove the selection vector from a record batch.
+ */
+public class RecordRemapper {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(RecordRemapper.class);
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0013215/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/WritableBatch.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/WritableBatch.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/WritableBatch.java
index 8335e91..f49e77f 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/WritableBatch.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/WritableBatch.java
@@ -23,59 +23,62 @@ import java.util.List;
 
 import org.apache.drill.exec.proto.UserBitShared.FieldMetadata;
 import org.apache.drill.exec.proto.UserBitShared.RecordBatchDef;
+import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
 import org.apache.drill.exec.vector.ValueVector;
 
 import com.google.common.collect.Lists;
 
 /**
- * A specialized version of record batch that can moves out buffers and preps them for writing. 
+ * A specialized version of record batch that can moves out buffers and preps them for writing.
  */
 public class WritableBatch {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(WritableBatch.class);
-  
+
   private final RecordBatchDef def;
   private final ByteBuf[] buffers;
-  
-  public WritableBatch(RecordBatchDef def, List<ByteBuf> buffers) {
+
+  private WritableBatch(RecordBatchDef def, List<ByteBuf> buffers) {
     logger.debug("Created new writable batch with def {} and buffers {}", def, buffers);
     this.def = def;
     this.buffers = buffers.toArray(new ByteBuf[buffers.size()]);
   }
-  
-  public WritableBatch(RecordBatchDef def, ByteBuf[] buffers) {
+
+  private WritableBatch(RecordBatchDef def, ByteBuf[] buffers) {
     super();
     this.def = def;
     this.buffers = buffers;
   }
-  
-  
-  public RecordBatchDef getDef(){
+
+  public RecordBatchDef getDef() {
     return def;
   }
-  public ByteBuf[] getBuffers(){
+
+  public ByteBuf[] getBuffers() {
     return buffers;
   }
 
-  public static WritableBatch get(int recordCount, List<ValueVector> vectors){
-    
+  public static WritableBatch getBatchNoSV(int recordCount, Iterable<ValueVector> vectors) {
     List<ByteBuf> buffers = Lists.newArrayList();
     List<FieldMetadata> metadata = Lists.newArrayList();
-    
 
-    for(ValueVector vv : vectors){
+    for (ValueVector vv : vectors) {
       metadata.add(vv.getMetadata());
-      for(ByteBuf b : vv.getBuffers()){
+      for (ByteBuf b : vv.getBuffers()) {
         buffers.add(b);
         b.retain();
       }
-      // allocate new buffer to release hold on old buffer.
+      // remove vv access to buffers.
       vv.clear();
     }
 
     RecordBatchDef batchDef = RecordBatchDef.newBuilder().addAllField(metadata).setRecordCount(recordCount).build();
     WritableBatch b = new WritableBatch(batchDef, buffers);
-      return b;
-    
+    return b;
   }
   
+  public static WritableBatch get(RecordBatch batch) {
+    if(batch.getSchema() != null && batch.getSchema().getSelectionVector() != SelectionVectorMode.NONE) throw new UnsupportedOperationException("Only batches without selections vectors are writable.");
+    return getBatchNoSV(batch.getRecordCount(), batch);
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0013215/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/BaseDataValueVector.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/BaseDataValueVector.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/BaseDataValueVector.java
index dd2b504..646640a 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/BaseDataValueVector.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/BaseDataValueVector.java
@@ -10,7 +10,7 @@ import org.apache.drill.exec.record.MaterializedField;
 abstract class BaseDataValueVector extends BaseValueVector{
 
   protected ByteBuf data = DeadBuf.DEAD_BUFFER;
-  protected int recordCount;
+  protected int valueCount;
   
   public BaseDataValueVector(MaterializedField field, BufferAllocator allocator) {
     super(field, allocator);
@@ -25,7 +25,7 @@ abstract class BaseDataValueVector extends BaseValueVector{
     if (data != DeadBuf.DEAD_BUFFER) {
       data.release();
       data = DeadBuf.DEAD_BUFFER;
-      recordCount = 0;
+      valueCount = 0;
     }
   }
   

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0013215/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/BaseValueVector.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/BaseValueVector.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/BaseValueVector.java
index e87e132..512c13e 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/BaseValueVector.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/BaseValueVector.java
@@ -25,6 +25,7 @@ abstract class BaseValueVector implements ValueVector{
   }
   
   abstract class BaseAccessor implements ValueVector.Accessor{
+    public abstract int getValueCount();
     public void reset(){}
   }
   
@@ -32,5 +33,7 @@ abstract class BaseValueVector implements ValueVector{
     public void reset(){}
   }
   
+  
+  
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0013215/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/BitVector.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/BitVector.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/BitVector.java
index 9a9f438..0f5b7d0 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/BitVector.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/BitVector.java
@@ -20,6 +20,9 @@ import org.apache.drill.exec.record.TransferPair;
 public final class BitVector extends BaseDataValueVector implements FixedWidthVector{
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(BitVector.class);
 
+  private final Accessor accessor = new Accessor();
+  private final Mutator mutator = new Mutator();
+
   private int valueCapacity;
   
   public BitVector(MaterializedField field, BufferAllocator allocator) {
@@ -48,13 +51,17 @@ public final class BitVector extends BaseDataValueVector implements FixedWidthVe
   @Override
   public int load(int valueCount, ByteBuf buf){
     clear();
-    this.recordCount = valueCount;
+    this.valueCount = valueCount;
     int len = getSizeFromCount(valueCount);
     data = buf.slice(0, len);
     data.retain();
     return len;
   }
   
+  public void copyValue(int inIndex, int outIndex, BitVector target){
+    target.mutator.set(outIndex, this.accessor.get(inIndex));
+  }
+  
   @Override
   public void load(FieldMetadata metadata, ByteBuf buffer) {
     assert this.field.getDef().equals(metadata.getDef());
@@ -82,7 +89,7 @@ public final class BitVector extends BaseDataValueVector implements FixedWidthVe
   public void transferTo(BitVector target){
     target.data = data;
     target.data.retain();
-    target.recordCount = recordCount;
+    target.valueCount = valueCount;
     clear();
   }
   
@@ -125,8 +132,8 @@ public final class BitVector extends BaseDataValueVector implements FixedWidthVe
       return new Boolean(get(index) != 0);
     }
     
-    public int getRecordCount() {
-      return recordCount;
+    public int getValueCount() {
+      return valueCount;
     }
     
   }
@@ -161,9 +168,9 @@ public final class BitVector extends BaseDataValueVector implements FixedWidthVe
       data.setByte((int) Math.floor(index/8), currentByte);
     }
 
-    public void setValueCount(int recordCount) {
-      BitVector.this.recordCount = recordCount;
-      data.writerIndex(getSizeFromCount(recordCount));
+    public void setValueCount(int valueCount) {
+      BitVector.this.valueCount = valueCount;
+      data.writerIndex(getSizeFromCount(valueCount));
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0013215/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/FixedWidthVector.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/FixedWidthVector.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/FixedWidthVector.java
index 0e3e3e9..008d17d 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/FixedWidthVector.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/FixedWidthVector.java
@@ -20,4 +20,6 @@ public interface FixedWidthVector extends ValueVector{
    */
   public int load(int valueCount, ByteBuf buf);
   
+  
+  public abstract NonRepeatedMutator getMutator();
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0013215/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/NonRepeatedMutator.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/NonRepeatedMutator.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/NonRepeatedMutator.java
index e9bdcbd..eba5e2e 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/NonRepeatedMutator.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/NonRepeatedMutator.java
@@ -3,5 +3,5 @@ package org.apache.drill.exec.vector;
 import org.apache.drill.exec.vector.ValueVector.Mutator;
 
 public interface NonRepeatedMutator extends Mutator{
-  public void setValueCount(int recordCount);
+  public void setValueCount(int valueCount);
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0013215/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/ValueVector.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/ValueVector.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/ValueVector.java
index b188d5b..9328a80 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/ValueVector.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/ValueVector.java
@@ -116,6 +116,8 @@ public interface ValueVector extends Closeable {
      * @param index   Index of the value to get
      */
     public abstract Object getObject(int index);
+    
+    public int getValueCount();
     public void reset();
   }
   

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0013215/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/VariableWidthVector.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/VariableWidthVector.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/VariableWidthVector.java
index c26cbab..8229870 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/VariableWidthVector.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/VariableWidthVector.java
@@ -26,4 +26,6 @@ public interface VariableWidthVector extends ValueVector{
    * @return The number of bytes of the buffer that were consumed.
    */
   public int load(int dataBytes, int valueCount, ByteBuf buf);
+  
+  public abstract NonRepeatedMutator getMutator();
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0013215/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/SimpleRootExec.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/SimpleRootExec.java b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/SimpleRootExec.java
index c6434f7..5e7c599 100644
--- a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/SimpleRootExec.java
+++ b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/SimpleRootExec.java
@@ -1,5 +1,7 @@
 package org.apache.drill.exec.physical.impl;
 
+import java.util.Iterator;
+
 import org.apache.drill.common.expression.SchemaPath;
 import org.apache.drill.exec.physical.impl.ScreenCreator.ScreenRoot;
 import org.apache.drill.exec.record.RecordBatch;
@@ -8,7 +10,7 @@ import org.apache.drill.exec.record.RecordBatch.TypedFieldId;
 import org.apache.drill.exec.record.selection.SelectionVector2;
 import org.apache.drill.exec.vector.ValueVector;
 
-public class SimpleRootExec implements RootExec{
+public class SimpleRootExec implements RootExec, Iterable<ValueVector>{
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(SimpleRootExec.class);
 
   private RecordBatch incoming;
@@ -40,5 +42,14 @@ public class SimpleRootExec implements RootExec{
   public void stop() {
   }
 
+  @Override
+  public Iterator<ValueVector> iterator() {
+    return incoming.iterator();
+  }
+
+  public int getRecordCount(){
+    return incoming.getRecordCount();
+  }
+  
   
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0013215/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 e2bf25b..96a6139 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
@@ -31,8 +31,8 @@ public class TestSimpleFilter {
   
   
   @Test
-  public void project(@Injectable final DrillbitContext bitContext, @Injectable UserClientConnection connection) throws Exception{
-    System.out.println(System.getProperty("java.class.path"));
+  public void testFilter(@Injectable final DrillbitContext bitContext, @Injectable UserClientConnection connection) throws Exception{
+//    System.out.println(System.getProperty("java.class.path"));
 
     
     new NonStrictExpectations(){{

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0013215/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/project/TestSimpleProjection.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/project/TestSimpleProjection.java b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/project/TestSimpleProjection.java
index e3f03f2..f4900e1 100644
--- a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/project/TestSimpleProjection.java
+++ b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/project/TestSimpleProjection.java
@@ -35,7 +35,6 @@ public class TestSimpleProjection {
   
   @Test
   public void project(@Injectable final DrillbitContext bitContext, @Injectable UserClientConnection connection) throws Exception{
-    System.out.println(System.getProperty("java.class.path"));
 
 
     new NonStrictExpectations(){{
@@ -57,7 +56,7 @@ public class TestSimpleProjection {
       a1 = c1.getAccessor();
       a2 = c2.getAccessor();
       
-      for(int i =0; i < c1.getAccessor().getRecordCount(); i++){
+      for(int i =0; i < c1.getAccessor().getValueCount(); i++){
         assertEquals(a1.get(i)+1, a2.get(i));
         x += a1.get(i);
       }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0013215/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/svremover/TestSVRemover.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/svremover/TestSVRemover.java b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/svremover/TestSVRemover.java
new file mode 100644
index 0000000..f417b91
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/svremover/TestSVRemover.java
@@ -0,0 +1,64 @@
+package org.apache.drill.exec.physical.impl.svremover;
+
+import static org.junit.Assert.assertEquals;
+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.FragmentHandle;
+import org.apache.drill.exec.rpc.user.UserServer.UserClientConnection;
+import org.apache.drill.exec.server.DrillbitContext;
+import org.apache.drill.exec.vector.ValueVector;
+import org.junit.After;
+import org.junit.Test;
+
+import com.google.common.base.Charsets;
+import com.google.common.io.Files;
+import com.yammer.metrics.MetricRegistry;
+
+public class TestSVRemover {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestSVRemover.class);
+  DrillConfig c = DrillConfig.create();
+  
+  
+  @Test
+  public void testSelectionVectorRemoval(@Injectable final DrillbitContext bitContext, @Injectable UserClientConnection connection) throws Exception{
+//    System.out.println(System.getProperty("java.class.path"));
+
+
+    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("/remover/test1.json"), Charsets.UTF_8));
+    FunctionImplementationRegistry registry = new FunctionImplementationRegistry(c);
+    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()){
+      int count = exec.getRecordCount();
+      for(ValueVector v : exec){
+        ValueVector.Accessor a = v.getAccessor();
+        assertEquals(count, a.getValueCount());
+      }
+    }
+  }
+  
+  @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/f0013215/sandbox/prototype/exec/java-exec/src/test/resources/remover/test1.json
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/resources/remover/test1.json b/sandbox/prototype/exec/java-exec/src/test/resources/remover/test1.json
new file mode 100644
index 0000000..7ef8fd6
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/test/resources/remover/test1.json
@@ -0,0 +1,39 @@
+{
+    head:{
+        type:"APACHE_DRILL_PHYSICAL",
+        version:"1",
+        generator:{
+            type:"manual"
+        }
+    },
+	graph:[
+        {
+            @id:1,
+            pop:"mock-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,
+            child: 1,
+            pop:"filter",
+            expr: "alternate()"
+        },
+        {
+            @id:3,
+            child: 2,
+            pop:"selection-vector-remover"
+        },
+        {
+            @id: 4,
+            child: 3,
+            pop: "screen"
+        }
+    ]
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f0013215/sandbox/prototype/exec/java-exec/src/test/resources/scan_screen_logical.json
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/resources/scan_screen_logical.json b/sandbox/prototype/exec/java-exec/src/test/resources/scan_screen_logical.json
index 3e5ebd7..5b6c19b 100644
--- a/sandbox/prototype/exec/java-exec/src/test/resources/scan_screen_logical.json
+++ b/sandbox/prototype/exec/java-exec/src/test/resources/scan_screen_logical.json
@@ -20,7 +20,7 @@
 
                     {
                         name : "superhero_name",
-                        type : "VARCHAR1",
+                        type : "VARCHAR2",
                         mode : "REQUIRED",
                         width : 100,
                         precision : 1,


[43/53] [abbrv] Types transition

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/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 dfae6fd..19d1069 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,133 +17,123 @@
  ******************************************************************************/
 package org.apache.drill.common.expression;
 
+import java.util.List;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 
 import org.apache.drill.common.expression.ValueExpressions.CollisionBehavior;
-import org.apache.drill.common.expression.types.DataType;
 import org.apache.drill.common.expression.visitors.ExprVisitor;
+import org.apache.drill.common.types.TypeProtos.MajorType;
+import org.apache.drill.common.types.Types;
 
-public class SchemaPath extends LogicalExpressionBase{
+import com.google.protobuf.DescriptorProtos.UninterpretedOption.NamePart;
+
+public class SchemaPath extends LogicalExpressionBase {
 
   // reads well in RegexBuddy
-  private static final String ENTIRE_REGEX = "^\n" +
-      "(?:                # first match required\n" +
-      "\\[\\d+\\]             # array index only\n" +
-      "|\n" +
-      "'?\n" +
-      "[^\\.\\[\\+\\-\\!\\]\\}]+  # identifier\n" +
-      "'?\n" +
-      "(?:\\[\\d+\\])?\n" +
-      ")\n" +
-      "[\\+\\-\\!\\]\\}]?\n" +
-
-      "# secondary matches (starts with dot)\n" +
-      "(?:\n" +
-      "\\.\n" +
-      "(?:                # first match required\n" +
-      "\\[\\d+\\]             # array index only\n" +
-      "|\n" +
-      "'?\n" +
-      "[^\\.\\[\\+\\-\\!\\]\\}]+  # identifier\n" +
-      "'?\n" +
-      "(?:\\[\\d+\\])?\n" +
-      ")\n" +
-      "[\\+\\-\\!\\]\\}]?\n" +
+  private static final String ENTIRE_REGEX = "^\n" + "(?:                # first match required\n"
+      + "\\[\\d+\\]             # array index only\n" + "|\n" + "'?\n" + "[^\\.\\[\\+\\-\\!\\]\\}]+  # identifier\n"
+      + "'?\n" + "(?:\\[\\d+\\])?\n" + ")\n" + "[\\+\\-\\!\\]\\}]?\n" +
+
+      "# secondary matches (starts with dot)\n" + "(?:\n" + "\\.\n" + "(?:                # first match required\n"
+      + "\\[\\d+\\]             # array index only\n" + "|\n" + "'?\n" + "[^\\.\\[\\+\\-\\!\\]\\}]+  # identifier\n"
+      + "'?\n" + "(?:\\[\\d+\\])?\n" + ")\n" + "[\\+\\-\\!\\]\\}]?\n" +
 
       ")*$";
-  
+
   // reads well in RegexBuddy
-  private static final String SEGMENT_REGEX = "(?:\n" +
-      "(\\[\\d+\\])\n" +
-      "|\n" +
-      "'?\n" +
-      "([^\\.\\[\\+\\-\\!\\]\\}]+)  # identifier\n" +
-      "'?\n" +
-      ")\n" +
-      "([\\+\\-\\!\\]\\}]?)         # collision type";
+  private static final String SEGMENT_REGEX = "(?:\n" + "(\\[\\d+\\])\n" + "|\n" + "'?\n"
+      + "([^\\.\\[\\+\\-\\!\\]\\}]+)  # identifier\n" + "'?\n" + ")\n"
+      + "([\\+\\-\\!\\]\\}]?)         # collision type";
   private static final int GROUP_INDEX = 1;
   private static final int GROUP_PATH_SEGMENT = 2;
   private static final int GROUP_COLLISION = 3;
-  
-  
+
   private final static Pattern SEGMENT_PATTERN = Pattern.compile(SEGMENT_REGEX, Pattern.COMMENTS);
   private final static Pattern ENTIRE_PATTERN = Pattern.compile(ENTIRE_REGEX, Pattern.COMMENTS);
-  
+
   private final CharSequence originalPath;
   private final PathSegment rootSegment;
-  
-  
-  
-	public SchemaPath(CharSequence str) {
-	  if(!ENTIRE_PATTERN.matcher(str).matches()) throw new IllegalArgumentException("Identifier doesn't match expected pattern.");
-	  this.originalPath = str;
-	  Matcher m = SEGMENT_PATTERN.matcher(str);
-		PathSegment r = null;
-		PathSegment previous = null;
-		PathSegment current;
-		while(m.find()){
-		  CollisionBehavior col =  (m.start(GROUP_COLLISION) != -1) ? CollisionBehavior.find(m.group(GROUP_COLLISION)) : CollisionBehavior.DEFAULT;
-
-      if(m.start(GROUP_INDEX) != -1){
+
+  public SchemaPath(CharSequence str, ExpressionPosition pos) {
+    super(pos);
+
+    if (!ENTIRE_PATTERN.matcher(str).matches())
+      throw new IllegalArgumentException("Identifier doesn't match expected pattern.");
+    this.originalPath = str;
+    Matcher m = SEGMENT_PATTERN.matcher(str);
+    PathSegment r = null;
+    PathSegment previous = null;
+    PathSegment current;
+    while (m.find()) {
+      CollisionBehavior col = (m.start(GROUP_COLLISION) != -1) ? CollisionBehavior.find(m.group(GROUP_COLLISION))
+          : CollisionBehavior.DEFAULT;
+
+      if (m.start(GROUP_INDEX) != -1) {
         String d = m.group(GROUP_INDEX);
         current = new PathSegment.ArraySegment(Integer.parseInt(d), col);
-      }else{
+      } else {
         String i = m.group(GROUP_PATH_SEGMENT);
         current = new PathSegment.NameSegment(i, col);
       }
-		  if(previous == null){
-		    r = current;
-		  }else{
-		    previous.setChild(current);
-		  }
-		  previous = current;
-		}
-		
-		rootSegment = r;
-		
-
-	}
-		
-
-	
-	@Override
-  public <T> T accept(ExprVisitor<T> visitor) {
-    return visitor.visitSchemaPath(this);
+      if (previous == null) {
+        r = current;
+      } else {
+        previous.setChild(current);
+      }
+      previous = current;
+    }
+
+    rootSegment = r;
+
   }
 
+  @Override
+  public <T, V, E extends Exception> T accept(ExprVisitor<T, V, E> visitor, V value) throws E {
+    return visitor.visitSchemaPath(this, value);
+  }
 
-  public PathSegment getRootSegment(){
-	  return rootSegment;
-	}
-	
-	public CharSequence getPath(){
-	  return originalPath;
-	}
+  public PathSegment getRootSegment() {
+    return rootSegment;
+  }
 
-    @Override
-    public DataType getDataType() {
-        return DataType.LATEBIND;
-    }
+  public CharSequence getPath() {
+    return originalPath;
+  }
 
-    @Override
-  public void addToString(StringBuilder sb) {
-    sb.append("'");
-    sb.append(originalPath);
-    sb.append("'");
+  @Override
+  public MajorType getMajorType() {
+    return Types.LATE_BIND_TYPE;
   }
 
-    @Override
-    public void resolveAndValidate(String expr, ErrorCollector errors) {
-    }
+  @Override
+  public int hashCode() {
+    final int prime = 31;
+    int result = 1;
+    result = prime * result + ((rootSegment == null) ? 0 : rootSegment.hashCode());
+    return result;
+  }
 
+  @Override
+  public boolean equals(Object obj) {
+    if (this == obj)
+      return true;
+    if (obj == null)
+      return false;
+    if (getClass() != obj.getClass())
+      return false;
+    SchemaPath other = (SchemaPath) obj;
+    if (rootSegment == null) {
+      if (other.rootSegment != null)
+        return false;
+    } else if (!rootSegment.equals(other.rootSegment))
+      return false;
+    return true;
+  }
 
-    @Override
+  @Override
   public String toString() {
     return "SchemaPath [rootSegment=" + rootSegment + "]";
   }
-  
-  
-  
 
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/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 f07f2a7..1acffcb 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,197 +17,181 @@
  ******************************************************************************/
 package org.apache.drill.common.expression;
 
-import org.apache.drill.common.expression.types.DataType;
 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;
 
 public class ValueExpressions {
 
-  public static LogicalExpression getNumericExpression(String s){
-    try{
+  public static LogicalExpression getNumericExpression(String s, ExpressionPosition ep) {
+    try {
       long l = Long.parseLong(s);
-      return new LongExpression(l);
-    }catch(Exception e){
-      
+      return new LongExpression(l, ep);
+    } catch (Exception e) {
+
     }
-    
-    try{
+
+    try {
       double d = Double.parseDouble(s);
-      return new DoubleExpression(d);
-    }catch(Exception e){
-      
+      return new DoubleExpression(d, ep);
+    } catch (Exception e) {
+
     }
+
+    throw new IllegalArgumentException(String.format("Unable to parse string %s as integer or floating point number.",
+        s));
+
+  }
+
+  protected static abstract class ValueExpression<V> extends LogicalExpressionBase {
+    public final V value;
+
+    protected ValueExpression(String value, ExpressionPosition pos) {
+      super(pos);
+      this.value = parseValue(value);
+    }
+
+    protected abstract V parseValue(String s);
+
+  }
+
+  public static class BooleanExpression extends ValueExpression<Boolean> {
     
-    throw new IllegalArgumentException(String.format("Unable to parse string %s as integer or floating point number.", s));
     
+    public BooleanExpression(String value, ExpressionPosition pos) {
+      super(value, pos);
+    }
+
+    @Override
+    protected Boolean parseValue(String s) {
+      return Boolean.parseBoolean(s);
+    }
+
+    @Override
+    public MajorType getMajorType() {
+      return Types.REQUIRED_BOOLEAN;
+    }
+
+    @Override
+    public <T, V, E extends Exception> T accept(ExprVisitor<T, V, E> visitor, V value) throws E {
+      return visitor.visitBooleanConstant(this, value);
+    }
+
+    public boolean getBoolean() {
+      return value;
+    }
+
   }
-  
-	protected static abstract class ValueExpression<V> extends LogicalExpressionBase {
-		public final V value;
 
-		protected ValueExpression(String value) {
-			this.value = parseValue(value);
-		}
+  public static class DoubleExpression extends LogicalExpressionBase {
+    private double d;
 
-		protected abstract V parseValue(String s);
+    private static final MajorType DOUBLE_CONSTANT = MajorType.newBuilder().setMinorType(MinorType.FLOAT8)
+        .setMode(DataMode.REQUIRED).build();
 
+    public DoubleExpression(double d, ExpressionPosition pos) {
+      super(pos);
+      this.d = d;
+    }
 
-	}
+    public double getDouble() {
+      return d;
+    }
+
+    @Override
+    public MajorType getMajorType() {
+      return DOUBLE_CONSTANT;
+    }
+
+    @Override
+    public <T, V, E extends Exception> T accept(ExprVisitor<T, V, E> visitor, V value) throws E {
+      return visitor.visitDoubleConstant(this, value);
+    }
+
+  }
 
-	public static class BooleanExpression extends ValueExpression<Boolean> {
-		public BooleanExpression(String value) {
-			super(value);
-		}
+  public static class LongExpression extends LogicalExpressionBase {
 
-		@Override
-		protected Boolean parseValue(String s) {
-			return Boolean.parseBoolean(s);
-		}
+    private static final MajorType LONG_CONSTANT = MajorType.newBuilder().setMinorType(MinorType.BIGINT)
+        .setMode(DataMode.REQUIRED).build();
 
-        @Override
-        public DataType getDataType() {
-            return DataType.BOOLEAN;
-        }
+    private long l;
 
-        @Override
-    public void addToString(StringBuilder sb) {
-      sb.append(value.toString());
+    public LongExpression(long l, ExpressionPosition pos) {
+      super(pos);
+      this.l = l;
     }
 
-        @Override
-        public void resolveAndValidate(String expr, ErrorCollector errors) {
-        }
+    public long getLong() {
+      return l;
+    }
 
+    @Override
+    public MajorType getMajorType() {
+      return LONG_CONSTANT;
+    }
 
-        @Override
-    public <T> T accept(ExprVisitor<T> visitor) {
-      return visitor.visitBoolean(this);
+    @Override
+    public <T, V, E extends Exception> T accept(ExprVisitor<T, V, E> visitor, V value) throws E {
+      return visitor.visitLongConstant(this, value);
     }
-    
-    public boolean getBoolean(){
-      return value;
+  }
+
+  public static class QuotedString extends ValueExpression<String> {
+
+    private static final MajorType QUOTED_STRING_CONSTANT = MajorType.newBuilder().setMinorType(MinorType.VARCHAR2)
+        .setMode(DataMode.REQUIRED).build();
+
+    public QuotedString(String value, ExpressionPosition pos) {
+      super(value, pos);
     }
-		
-	}
-
-	 public static class DoubleExpression extends LogicalExpressionBase  {
-	   private double d;
-	    public DoubleExpression(double d) {
-	      this.d = d;
-	    }
-
-	    public double getDouble(){
-	      return d;
-	    }
-
-         @Override
-         public DataType getDataType() {
-             return DataType.FLOAT32;
-         }
-
-         @Override
-	    public void addToString(StringBuilder sb) {
-	      sb.append(d);
-	    }
-
-         @Override
-         public void resolveAndValidate(String expr, ErrorCollector errors) {
-         }
-
-         @Override
-	    public <T> T accept(ExprVisitor<T> visitor) {
-	      return visitor.visitDoubleExpression(this);
-	    }
-	  }
-	 
-	public static class LongExpression extends LogicalExpressionBase {
-	  private long l;
-		public LongExpression(long l) {
-		  this.l = l;
-		}
-		
-		public long getLong(){
-		  return l;
-		}
-
-        @Override
-        public DataType getDataType() {
-            return DataType.INT64;
-        }
-
-        @Override
-    public void addToString(StringBuilder sb) {
-      sb.append(l);
-    }
-
-        @Override
-        public void resolveAndValidate(String expr, ErrorCollector errors) {
-        }
-
-        @Override
-    public <T> T accept(ExprVisitor<T> visitor) {
-      return visitor.visitLongExpression(this);
-    }
-	}
-
-	public static class QuotedString extends ValueExpression<String> {
-		public QuotedString(String value) {
-			super(value);
-		}
-
-		@Override
-		protected String parseValue(String s) {
-			return s;
-		}
-
-        @Override
-        public DataType getDataType() {
-            return DataType.NVARCHAR;
-        }
-
-        @Override
-    public void addToString(StringBuilder sb) {
-      sb.append("\"");
-      sb.append(value.toString());
-      sb.append("\"");
-    }
-
-        @Override
-        public void resolveAndValidate(String expr, ErrorCollector errors) {
-        }
-
-        @Override
-    public <T> T accept(ExprVisitor<T> visitor) {
-      return visitor.visitQuotedString(this);
-    }
-	}
-
-	
-	public static enum CollisionBehavior{
-	  SKIP("-"),  // keep the old value.
-	  FAIL("!"), // give up on the record
-	  REPLACE("+"), // replace the old value with the new value.
-	  ARRAYIFY("]"), // replace the current position with an array.  Then place the old and new value in the array. 
-	  OBJECTIFY("}"),  // replace the current position with a map.  Give the two values names of 'old' and 'new'. 
-	  MERGE_OVERRIDE("%"); // do your best to do a deep merge of the old and new values.
-	  
-	  private String identifier;
-	  
-	  private CollisionBehavior(String identifier){
-	    this.identifier = identifier;
-	  }
-	  public static final CollisionBehavior DEFAULT = FAIL;
-	  
-	  public static final CollisionBehavior find(String c){
-	    if(c == null || c.isEmpty()) return DEFAULT;
-	    
-	    for(CollisionBehavior b : values()){
-	      if(b.identifier.equals(c)) return b;
-	    }
-	    return DEFAULT;
-	  }
-	}
+
+    @Override
+    protected String parseValue(String s) {
+      return s;
+    }
+
+    @Override
+    public MajorType getMajorType() {
+      return QUOTED_STRING_CONSTANT;
+    }
+
+    @Override
+    public <T, V, E extends Exception> T accept(ExprVisitor<T, V, E> visitor, V value) throws E {
+      return visitor.visitQuotedStringConstant(this, value);
+    }
+  }
+
+  public static enum CollisionBehavior {
+    SKIP("-"), // keep the old value.
+    FAIL("!"), // give up on the record
+    REPLACE("+"), // replace the old value with the new value.
+    ARRAYIFY("]"), // replace the current position with an array. Then place the
+                   // old and new value in the array.
+    OBJECTIFY("}"), // replace the current position with a map. Give the two
+                    // values names of 'old' and 'new'.
+    MERGE_OVERRIDE("%"); // do your best to do a deep merge of the old and new
+                         // values.
+
+    private String identifier;
+
+    private CollisionBehavior(String identifier) {
+      this.identifier = identifier;
+    }
+
+    public static final CollisionBehavior DEFAULT = FAIL;
+
+    public static final CollisionBehavior find(String c) {
+      if (c == null || c.isEmpty())
+        return DEFAULT;
+
+      for (CollisionBehavior b : values()) {
+        if (b.identifier.equals(c))
+          return b;
+      }
+      return DEFAULT;
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/fn/BooleanFunctions.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/fn/BooleanFunctions.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/fn/BooleanFunctions.java
index 9163a5b..65d4c76 100644
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/fn/BooleanFunctions.java
+++ b/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/fn/BooleanFunctions.java
@@ -22,7 +22,7 @@ import org.apache.drill.common.expression.ArgumentValidators.ComparableArguments
 import org.apache.drill.common.expression.CallProvider;
 import org.apache.drill.common.expression.FunctionDefinition;
 import org.apache.drill.common.expression.OutputTypeDeterminer;
-import org.apache.drill.common.expression.types.DataType;
+import org.apache.drill.common.types.Types;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -32,8 +32,8 @@ public class BooleanFunctions implements CallProvider {
   @Override
   public FunctionDefinition[] getFunctionDefintions() {
     return new FunctionDefinition[] {
-        FunctionDefinition.operator("or", new AllowedTypeList(2, Integer.MAX_VALUE, DataType.BOOLEAN), OutputTypeDeterminer.FIXED_BOOLEAN, "or", "||"),
-        FunctionDefinition.operator("and", new AllowedTypeList(2, Integer.MAX_VALUE, DataType.BOOLEAN), OutputTypeDeterminer.FIXED_BOOLEAN, "and", "&&"),
+        FunctionDefinition.operator("or", new AllowedTypeList(2, Integer.MAX_VALUE, Types.REQUIRED_BOOLEAN), OutputTypeDeterminer.FIXED_BOOLEAN, "or", "||"),
+        FunctionDefinition.operator("and", new AllowedTypeList(2, Integer.MAX_VALUE, Types.REQUIRED_BOOLEAN), OutputTypeDeterminer.FIXED_BOOLEAN, "and", "&&"),
         FunctionDefinition.operator("greater than", new ComparableArguments(2), OutputTypeDeterminer.FIXED_BOOLEAN, ">"),
         FunctionDefinition.operator("less than", new ComparableArguments(2), OutputTypeDeterminer.FIXED_BOOLEAN, "<"),
         FunctionDefinition.operator("equal", new ComparableArguments(2), OutputTypeDeterminer.FIXED_BOOLEAN, "==", "<>"),

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/fn/MathFunctions.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/fn/MathFunctions.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/fn/MathFunctions.java
index 75f8cdb..4e530b8 100644
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/fn/MathFunctions.java
+++ b/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/fn/MathFunctions.java
@@ -30,11 +30,11 @@ public class MathFunctions implements CallProvider{
   @Override
   public FunctionDefinition[] getFunctionDefintions() {
     return new FunctionDefinition[]{
-        FunctionDefinition.operator("add", new ArgumentValidators.NumericTypeAllowed(1, Integer.MAX_VALUE, true), new OutputTypeDeterminer.SameAsFirstInput(), "+"),
-        FunctionDefinition.operator("subtract", new ArgumentValidators.NumericTypeAllowed(1, Integer.MAX_VALUE, true), new OutputTypeDeterminer.SameAsFirstInput(), "-"),
-        FunctionDefinition.operator("divide", new ArgumentValidators.NumericTypeAllowed(1, Integer.MAX_VALUE, true), new OutputTypeDeterminer.SameAsFirstInput(), "/"),
-        FunctionDefinition.operator("multiply", new ArgumentValidators.NumericTypeAllowed(1, Integer.MAX_VALUE, true), new OutputTypeDeterminer.SameAsFirstInput(), "*"),
-        FunctionDefinition.operator("modulo", new ArgumentValidators.NumericTypeAllowed(1, Integer.MAX_VALUE, true), new OutputTypeDeterminer.SameAsFirstInput(), "%"),
+        FunctionDefinition.operator("add", new ArgumentValidators.NumericTypeAllowed(1, Integer.MAX_VALUE, true), new OutputTypeDeterminer.SameAsAnySoft(), "+"),
+        FunctionDefinition.operator("subtract", new ArgumentValidators.NumericTypeAllowed(1, Integer.MAX_VALUE, true), new OutputTypeDeterminer.SameAsAnySoft(), "-"),
+        FunctionDefinition.operator("divide", new ArgumentValidators.NumericTypeAllowed(1, Integer.MAX_VALUE, true), new OutputTypeDeterminer.SameAsAnySoft(), "/"),
+        FunctionDefinition.operator("multiply", new ArgumentValidators.NumericTypeAllowed(1, Integer.MAX_VALUE, true), new OutputTypeDeterminer.SameAsAnySoft(), "*"),
+        FunctionDefinition.operator("modulo", new ArgumentValidators.NumericTypeAllowed(1, Integer.MAX_VALUE, true), new OutputTypeDeterminer.SameAsAnySoft(), "%"),
         
     };
 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/fn/StringFunctions.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/fn/StringFunctions.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/fn/StringFunctions.java
index 494ba38..568b209 100644
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/fn/StringFunctions.java
+++ b/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/fn/StringFunctions.java
@@ -22,7 +22,7 @@ import org.apache.drill.common.expression.BasicArgumentValidator;
 import org.apache.drill.common.expression.CallProvider;
 import org.apache.drill.common.expression.FunctionDefinition;
 import org.apache.drill.common.expression.OutputTypeDeterminer.FixedType;
-import org.apache.drill.common.expression.types.DataType;
+import org.apache.drill.common.types.TypeProtos.MinorType;
 
 public class StringFunctions implements CallProvider{
 
@@ -30,8 +30,9 @@ public class StringFunctions implements CallProvider{
   @Override
   public FunctionDefinition[] getFunctionDefintions() {
     return new FunctionDefinition[]{
-        FunctionDefinition.simple("regex_like", new BasicArgumentValidator(new Arg(true, "pattern", DataType.NVARCHAR), new Arg("value", DataType.NVARCHAR) ), new FixedType(DataType.BOOLEAN)),
-        //new FunctionDefinition("startsWith", new ArgumentValidators.BasicArgumentValidator(DataTypeImpls.NVARCHAR), new FixedType(DataTypeImpls.BOOLEAN), false, false),
+        FunctionDefinition.simple("regex_like", new BasicArgumentValidator( //
+            new Arg(true, false, "pattern", MinorType.VARCHAR1, MinorType.VARCHAR2, MinorType.VARCHAR4), //
+            new Arg(false, true, "value", MinorType.FIXEDCHAR, MinorType.VARCHAR1, MinorType.VARCHAR2, MinorType.VARCHAR4) ), FixedType.FIXED_BOOLEAN),
     };
 
   }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/fn/UnaryFunctions.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/fn/UnaryFunctions.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/fn/UnaryFunctions.java
index 8660c5f..5570d10 100644
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/fn/UnaryFunctions.java
+++ b/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/fn/UnaryFunctions.java
@@ -22,15 +22,15 @@ import org.apache.drill.common.expression.BasicArgumentValidator;
 import org.apache.drill.common.expression.CallProvider;
 import org.apache.drill.common.expression.FunctionDefinition;
 import org.apache.drill.common.expression.OutputTypeDeterminer;
-import org.apache.drill.common.expression.types.DataType;
+import org.apache.drill.common.types.Types;
 
 public class UnaryFunctions implements CallProvider{
 
   @Override
   public FunctionDefinition[] getFunctionDefintions() {
     return new FunctionDefinition[]{
-        FunctionDefinition.operator("isNull", new ArgumentValidators.AnyTypeAllowed(1), new OutputTypeDeterminer.FixedType(DataType.BOOLEAN)),
-        FunctionDefinition.operator("not", new BasicArgumentValidator(DataType.BOOLEAN), new OutputTypeDeterminer.FixedType(DataType.BOOLEAN), "!"),
+        FunctionDefinition.operator("isNull", new ArgumentValidators.AnyTypeAllowed(1), new OutputTypeDeterminer.FixedType(Types.REQUIRED_BOOLEAN)),
+        FunctionDefinition.operator("not", new BasicArgumentValidator(Types.REQUIRED_BOOLEAN), new OutputTypeDeterminer.FixedType(Types.REQUIRED_BOOLEAN), "!"),
         FunctionDefinition.operator("negative", new ArgumentValidators.NumericTypeAllowed(1, true), new OutputTypeDeterminer.SameAsFirstInput(), "u-"),
     };
 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/types/AtomType.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/types/AtomType.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/types/AtomType.java
new file mode 100644
index 0000000..f21ddb9
--- /dev/null
+++ b/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/types/AtomType.java
@@ -0,0 +1,67 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.common.expression.types;
+
+
+public class AtomType extends DataType {
+  private String name;
+  private Comparability comparability;
+  private boolean isNumericType;
+  
+  public AtomType(String name, Comparability comparability, boolean isNumericType) {
+    super();
+    this.name = name;
+    this.comparability = comparability;
+    this.isNumericType = isNumericType;
+  }
+
+  
+  public boolean isNumericType() {
+    return isNumericType;
+  }
+
+
+  @Override
+  public String getName() {
+    return name;
+  }
+
+  @Override
+  public boolean isLateBind() {
+    return false;
+  }
+
+  @Override
+  public boolean hasChildType() {
+    return false;
+  }
+
+  @Override
+  public DataType getChildType() {
+    return null;
+  }
+
+  @Override
+  public Comparability getComparability() {
+    return comparability;
+  }
+  
+  
+  
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/types/DataType.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/types/DataType.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/types/DataType.java
index 56e2485..84ff054 100644
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/types/DataType.java
+++ b/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/types/DataType.java
@@ -37,7 +37,7 @@ import com.fasterxml.jackson.databind.ser.std.StdSerializer;
 
 @JsonSerialize(using = DataType.Se.class)
 @JsonDeserialize(using = DataType.De.class)
-public abstract class DataType {
+abstract class DataType {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DataType.class);
   
   public static enum Comparability{

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/visitors/AbstractExprVisitor.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/visitors/AbstractExprVisitor.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/visitors/AbstractExprVisitor.java
new file mode 100644
index 0000000..d561f7e
--- /dev/null
+++ b/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/visitors/AbstractExprVisitor.java
@@ -0,0 +1,56 @@
+package org.apache.drill.common.expression.visitors;
+
+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;
+
+public abstract class AbstractExprVisitor<T, VAL, EXCEP extends Exception> implements ExprVisitor<T, VAL, EXCEP> {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(AbstractExprVisitor.class);
+
+  @Override
+  public T visitFunctionCall(FunctionCall call, VAL value) throws EXCEP {
+    return null;
+  }
+
+  @Override
+  public T visitIfExpression(IfExpression ifExpr, VAL value) throws EXCEP {
+    return visitUnknown(ifExpr, value);
+  }
+
+  @Override
+  public T visitSchemaPath(SchemaPath path, VAL value) throws EXCEP {
+    return visitUnknown(path, value);
+  }
+
+  @Override
+  public T visitLongConstant(LongExpression intExpr, VAL value) throws EXCEP {
+    return visitUnknown(intExpr, value);
+  }
+
+  @Override
+  public T visitDoubleConstant(DoubleExpression dExpr, VAL value) throws EXCEP {
+    return visitUnknown(dExpr, value);
+  }
+
+  @Override
+  public T visitBooleanConstant(BooleanExpression e, VAL value) throws EXCEP {
+    return visitUnknown(e, value);
+  }
+
+  @Override
+  public T visitQuotedStringConstant(QuotedString e, VAL value) throws EXCEP {
+    return visitUnknown(e, value);
+  }
+
+  @Override
+  public T visitUnknown(LogicalExpression e, VAL value) throws EXCEP {
+    throw new UnsupportedOperationException(String.format("Expression of type %s not handled by visitor type %s.", e.getClass().getCanonicalName(), this.getClass().getCanonicalName()));
+  }
+  
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/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 9860bdf..4d99b15 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
@@ -27,26 +27,26 @@ import org.apache.drill.common.expression.ValueExpressions.DoubleExpression;
 import org.apache.drill.common.expression.ValueExpressions.LongExpression;
 import org.apache.drill.common.expression.ValueExpressions.QuotedString;
 
-public final class AggregateChecker implements ExprVisitor<Boolean>{
+public final class AggregateChecker extends SimpleExprVisitor<Boolean>{
 	
   public static final AggregateChecker INSTANCE = new AggregateChecker();
   
   private AggregateChecker(){};
   
   public static boolean isAggregating(LogicalExpression e){
-    return e.accept(INSTANCE);
+    return e.accept(INSTANCE, null);
   }
 
   @Override
   public Boolean visitFunctionCall(FunctionCall call) {
     if(call.getDefinition().isAggregating()){
       for(LogicalExpression e : call){
-        if(e.accept(this)) throw new IllegalArgumentException(String.format("Your aggregating function call %s also includes arguments that contain aggregations.  This isn't allowed.", call.getDefinition().toString()));
+        if(e.accept(this, null)) throw new IllegalArgumentException(String.format("Your aggregating function call %s also includes arguments that contain aggregations.  This isn't allowed.", call.getDefinition().toString()));
       }
       return true;
     }else{
       for(LogicalExpression e : call){
-        if(e.accept(this)) return true;
+        if(e.accept(this, null)) return true;
       }
       return false;
     }
@@ -55,9 +55,9 @@ public final class AggregateChecker implements ExprVisitor<Boolean>{
   @Override
   public Boolean visitIfExpression(IfExpression ifExpr) {
     for(IfCondition c : ifExpr){
-      if(c.condition.accept(this) || c.expression.accept(this)) return true;
+      if(c.condition.accept(this, null) || c.expression.accept(this, null)) return true;
     }
-    return ifExpr.elseExpression.accept(this);
+    return ifExpr.elseExpression.accept(this, null);
   }
 
   @Override
@@ -66,22 +66,27 @@ public final class AggregateChecker implements ExprVisitor<Boolean>{
   }
 
   @Override
-  public Boolean visitLongExpression(LongExpression intExpr) {
+  public Boolean visitLongConstant(LongExpression intExpr) {
     return false;
   }
 
   @Override
-  public Boolean visitDoubleExpression(DoubleExpression dExpr) {
+  public Boolean visitDoubleConstant(DoubleExpression dExpr) {
     return false;
   }
 
   @Override
-  public Boolean visitBoolean(BooleanExpression e) {
+  public Boolean visitBooleanConstant(BooleanExpression e) {
     return false;
   }
 
   @Override
-  public Boolean visitQuotedString(QuotedString e) {
+  public Boolean visitQuotedStringConstant(QuotedString e) {
+    return false;
+  }
+
+  @Override
+  public Boolean visitUnknown(LogicalExpression e, Void value) throws RuntimeException {
     return false;
   }
 	

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/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 307b2cf..038fb85 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
@@ -27,20 +27,20 @@ import org.apache.drill.common.expression.ValueExpressions.DoubleExpression;
 import org.apache.drill.common.expression.ValueExpressions.LongExpression;
 import org.apache.drill.common.expression.ValueExpressions.QuotedString;
 
-public final class ConstantChecker implements ExprVisitor<Boolean>{
+public final class ConstantChecker extends SimpleExprVisitor<Boolean>{
 	
   private final static ConstantChecker INSTANCE = new ConstantChecker();
   
   private ConstantChecker(){}
   
   public static boolean onlyIncludesConstants(LogicalExpression e){
-    return e.accept(INSTANCE);
+    return e.accept(INSTANCE, null);
   }
 
   @Override
   public Boolean visitFunctionCall(FunctionCall call) {
     for(LogicalExpression e : call){
-      if(!e.accept(this)) return false;
+      if(!e.accept(this, null)) return false;
     }
     return true;
   }
@@ -48,7 +48,7 @@ public final class ConstantChecker implements ExprVisitor<Boolean>{
   @Override
   public Boolean visitIfExpression(IfExpression ifExpr) {
     for(IfCondition c : ifExpr){
-      if(!c.condition.accept(this) || !c.expression.accept(this)) return false;
+      if(!c.condition.accept(this, null) || !c.expression.accept(this, null)) return false;
     }
     return true;
   }
@@ -59,24 +59,29 @@ public final class ConstantChecker implements ExprVisitor<Boolean>{
   }
 
   @Override
-  public Boolean visitLongExpression(LongExpression intExpr) {
+  public Boolean visitLongConstant(LongExpression intExpr) {
     return true;
   }
 
   @Override
-  public Boolean visitDoubleExpression(DoubleExpression dExpr) {
+  public Boolean visitDoubleConstant(DoubleExpression dExpr) {
     return true;
   }
 
   @Override
-  public Boolean visitBoolean(BooleanExpression e) {
+  public Boolean visitBooleanConstant(BooleanExpression e) {
     return true;
   }
 
   @Override
-  public Boolean visitQuotedString(QuotedString e) {
+  public Boolean visitQuotedStringConstant(QuotedString e) {
     return true;
   }
+
+  @Override
+  public Boolean visitUnknown(LogicalExpression e, Void value) throws RuntimeException {
+    return false;
+  }
 	
   
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/visitors/ExprVisitor.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/visitors/ExprVisitor.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/visitors/ExprVisitor.java
index 5ce0ef5..cd27157 100644
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/visitors/ExprVisitor.java
+++ b/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/visitors/ExprVisitor.java
@@ -19,18 +19,20 @@ package org.apache.drill.common.expression.visitors;
 
 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;
 
-public interface ExprVisitor<T> {
-	public T visitFunctionCall(FunctionCall call);
-	public T visitIfExpression(IfExpression ifExpr);
-	public T visitSchemaPath(SchemaPath path);
-	public T visitLongExpression(LongExpression intExpr);
-	public T visitDoubleExpression(DoubleExpression dExpr);
-	public T visitBoolean(BooleanExpression e);
-	public T visitQuotedString(QuotedString e);	
+public interface ExprVisitor<T, VAL, EXCEP extends Exception> {
+	public T visitFunctionCall(FunctionCall call, VAL value) throws EXCEP;
+	public T visitIfExpression(IfExpression ifExpr, VAL value) throws EXCEP;
+	public T visitSchemaPath(SchemaPath path, VAL value) throws EXCEP;	
+	public T visitLongConstant(LongExpression intExpr, VAL value) throws EXCEP;
+	public T visitDoubleConstant(DoubleExpression dExpr, VAL value) throws EXCEP;
+	public T visitBooleanConstant(BooleanExpression e, VAL value) throws EXCEP;
+	public T visitQuotedStringConstant(QuotedString e, VAL value) throws EXCEP;	
+	public T visitUnknown(LogicalExpression e, VAL value) throws EXCEP;
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/visitors/SimpleExprVisitor.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/visitors/SimpleExprVisitor.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/visitors/SimpleExprVisitor.java
new file mode 100644
index 0000000..434bdaf
--- /dev/null
+++ b/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/visitors/SimpleExprVisitor.java
@@ -0,0 +1,56 @@
+package org.apache.drill.common.expression.visitors;
+
+import org.apache.drill.common.expression.FunctionCall;
+import org.apache.drill.common.expression.IfExpression;
+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;
+
+public abstract class SimpleExprVisitor<T> implements ExprVisitor<T, Void, RuntimeException>{
+
+  @Override
+  public T visitFunctionCall(FunctionCall call, Void value) throws RuntimeException {
+    return visitFunctionCall(call);
+  }
+
+  @Override
+  public T visitIfExpression(IfExpression ifExpr, Void value) throws RuntimeException {
+    return visitIfExpression(ifExpr);
+  }
+
+  @Override
+  public T visitSchemaPath(SchemaPath path, Void value) throws RuntimeException {
+    return visitSchemaPath(path);
+  }
+
+  @Override
+  public T visitLongConstant(LongExpression intExpr, Void value) throws RuntimeException {
+    return visitLongConstant(intExpr);
+  }
+
+  @Override
+  public T visitDoubleConstant(DoubleExpression dExpr, Void value) throws RuntimeException {
+    return visitDoubleConstant(dExpr);
+  }
+
+  @Override
+  public T visitBooleanConstant(BooleanExpression e, Void value) throws RuntimeException {
+    return visitBooleanConstant(e);
+  }
+
+  @Override
+  public T visitQuotedStringConstant(QuotedString e, Void value) throws RuntimeException {
+    return visitQuotedStringConstant(e);
+  }
+
+  
+  public abstract T visitFunctionCall(FunctionCall call);
+  public abstract T visitIfExpression(IfExpression ifExpr);
+  public abstract T visitSchemaPath(SchemaPath path);
+  public abstract T visitLongConstant(LongExpression intExpr);
+  public abstract T visitDoubleConstant(DoubleExpression dExpr);
+  public abstract T visitBooleanConstant(BooleanExpression e);
+  public abstract T visitQuotedStringConstant(QuotedString e); 
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/sandbox/prototype/common/src/main/java/org/apache/drill/common/types/Types.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/types/Types.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/types/Types.java
new file mode 100644
index 0000000..e1343ab
--- /dev/null
+++ b/sandbox/prototype/common/src/main/java/org/apache/drill/common/types/Types.java
@@ -0,0 +1,134 @@
+package org.apache.drill.common.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;
+
+public class Types {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(Types.class);
+  
+  public static final MajorType NULL = required(MinorType.NULL);
+  public static final MajorType LATE_BIND_TYPE = optional(MinorType.LATE);
+  public static final MajorType REQUIRED_BOOLEAN = required(MinorType.BOOLEAN);
+  
+  public static enum Comparability{
+    UNKNOWN, NONE, EQUAL, ORDERED;
+  }
+  
+  public static boolean isNumericType(MajorType type){
+    if(type.getMode() == DataMode.REPEATED) return false;
+    
+    switch(type.getMinorType()){
+    case BIGINT:
+    case DECIMAL16:
+    case DECIMAL4:
+    case DECIMAL8:
+    case FLOAT4:
+    case FLOAT8:
+    case INT:
+    case MONEY:
+    case SMALLINT:
+    case TINYINT:
+    case UINT1:
+    case UINT2:
+    case UINT4:
+    case UINT8:
+      return true;
+      default:
+        return false;
+    }
+  }
+  
+  public static boolean isStringScalarType(MajorType type){
+    if(type.getMode() == DataMode.REPEATED) return false;
+    switch(type.getMinorType()){
+    case FIXEDCHAR:
+    case VARCHAR1:
+    case VARCHAR2:
+    case VARCHAR4:
+      return true;
+    default: 
+      return false;
+    }
+  }
+  
+  public static boolean isBytesScalarType(MajorType type){
+    if(type.getMode() == DataMode.REPEATED) return false;
+    switch(type.getMinorType()){
+    case FIXEDBINARY:
+    case VARBINARY1:
+    case VARBINARY2:
+    case VARBINARY4:
+      return true;
+    default: 
+      return false;
+    }
+  }
+  
+  public static Comparability getComparability(MajorType type){
+    if(type.getMode() == DataMode.REPEATED) return Comparability.NONE;
+    if(type.getMinorType() == MinorType.LATE) return Comparability.UNKNOWN;
+    
+    switch(type.getMinorType()){
+    case LATE:
+      return Comparability.UNKNOWN;
+    case MAP:
+    case REPEATMAP:
+      return Comparability.NONE;
+    case INTERVAL:
+    case BOOLEAN:
+    case MSGPACK2:
+    case MSGPACK4:
+    case PROTO2:
+    case PROTO4:
+      return Comparability.EQUAL;
+    default:
+      return Comparability.ORDERED;
+    }
+    
+  }
+  
+  
+  public static boolean softEquals(MajorType a, MajorType b, boolean allowNullSwap){
+    if(a.getMinorType() != b.getMinorType()) return false;
+    if(allowNullSwap){
+      switch(a.getMode()){
+      case OPTIONAL:
+      case REQUIRED:
+        switch(b.getMode()){
+        case OPTIONAL:
+        case REQUIRED:
+          return true;
+        default:
+          return false;
+        }
+      default:
+        return false;
+      }
+    }else{
+      if(a.getMode() != b.getMode()){
+        return false;
+      }else{
+        return true;
+      }
+    }
+  }
+  
+  public static boolean isLateBind(MajorType type){
+    return type.getMinorType() == MinorType.LATE;
+  }
+  
+  public static MajorType required(MinorType type){
+    return MajorType.newBuilder().setMode(DataMode.REQUIRED).setMinorType(type).build();
+  }
+  
+  public static MajorType repeated(MinorType type){
+    return MajorType.newBuilder().setMode(DataMode.REPEATED).setMinorType(type).build();
+  }
+  
+  public static MajorType optional(MinorType type){
+    return MajorType.newBuilder().setMode(DataMode.OPTIONAL).setMinorType(type).build();
+  }
+  
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/sandbox/prototype/common/src/main/protobuf/Types.proto
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/protobuf/Types.proto b/sandbox/prototype/common/src/main/protobuf/Types.proto
new file mode 100644
index 0000000..58399de
--- /dev/null
+++ b/sandbox/prototype/common/src/main/protobuf/Types.proto
@@ -0,0 +1,64 @@
+package common;
+
+option java_package = "org.apache.drill.common.types";
+option java_outer_classname = "TypeProtos";
+option optimize_for = SPEED;
+
+
+enum MinorType {
+    LATE = 0;   //  late binding type
+    MAP = 1;   //  an empty map column.  Useful for conceptual setup.  Children listed within here
+    REPEATMAP = 2;   //  a repeated map column (means that multiple children sit below this)
+    TINYINT = 3;   //  single byte signed integer
+    SMALLINT = 4;   //  two byte signed integer
+    INT = 5;   //  four byte signed integer
+    BIGINT = 6;   //  eight byte signed integer
+    DECIMAL4 = 7;   //  a decimal supporting precision between 1 and 8 (4 bits for decimal location, 1 sign)
+    DECIMAL8 = 8;   //  a decimal supporting precision between 9 and 18 (5 bits for decimal location, 1 sign)
+    DECIMAL16 = 10;   //  a decimal supporting precision between 19 and 37 (6 bits for decimal location, 1 sign)
+    MONEY = 11;   //  signed decimal with two digit precision
+    DATE = 12;   //  days since 4713bc 
+    TIME = 13;   //  time in micros before or after 2000/1/1
+    TIMETZ = 14;   //  time in micros before or after 2000/1/1 with timezone
+    TIMESTAMP = 15;   //  unix epoch time in millis
+    DATETIME = 16;   //  TBD
+    INTERVAL = 17;   //  TBD
+    FLOAT4 = 18;   //  4 byte ieee 754 
+    FLOAT8 = 19;   //  8 byte ieee 754
+    BOOLEAN = 20;   //  single bit value
+    FIXEDCHAR = 21;   //  utf8 fixed length string, padded with spaces
+    VARCHAR1 = 22;   //  utf8 variable length string (up to 2^8 in length)
+    VARCHAR2 = 23;   //  utf8 variable length string (up to 2^16 in length)
+    VARCHAR4 = 24;   //  utf8 variable length string (up to 2^32 in length)
+    FIXEDBINARY = 25;   //  fixed length binary, padded with 0 bytes
+    VARBINARY1 = 26;   //  variable length binary (up to 2^8 in length)
+    VARBINARY2 = 27;   //  variable length binary (up to 2^16 in length)
+    VARBINARY4 = 28;   //  variable length binary (up to 2^32 in length)
+    UINT1 = 29;   //  unsigned 1 byte integer
+    UINT2 = 30;   //  unsigned 2 byte integer
+    UINT4 = 31;   //  unsigned 4 byte integer
+    UINT8 = 32;   //  unsigned 8 byte integer
+    PROTO2 = 33;   //  protobuf encoded complex type. (up to 2^16 in length)
+    PROTO4 = 34;   //  protobuf encoded complex type. (up to 2^32 in length)
+    MSGPACK2 = 35;   //  msgpack encoded complex type. (up to 2^16 in length)
+    MSGPACK4 = 36;   //  msgpack encoded complex type. (up to 2^32 in length)
+    NULL = 37; // a value of unknown type (e.g. a missing reference).
+    
+}
+
+message MajorType {
+  optional MinorType minor_type = 1;
+  optional DataMode mode = 2;
+  optional int32 width = 3; // optional width for fixed size values.
+  optional int32 precision = 4; // used for decimal types
+  optional int32 scale = 5; // used for decimal types 
+}
+
+
+
+enum DataMode {
+  OPTIONAL = 0; // nullable
+  REQUIRED = 1; // non-nullable
+  REPEATED = 2; // single, repeated-field
+}
+

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/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 1b6dac0..e348bc7 100644
--- a/sandbox/prototype/exec/java-exec/pom.xml
+++ b/sandbox/prototype/exec/java-exec/pom.xml
@@ -1,161 +1,186 @@
 <?xml version="1.0"?>
 <project
-	xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"
-	xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance">
-	<modelVersion>4.0.0</modelVersion>
-	<parent>
-		<artifactId>exec-parent</artifactId>
-		<groupId>org.apache.drill.exec</groupId>
-		<version>1.0-SNAPSHOT</version>
-	</parent>
-	<artifactId>java-exec</artifactId>
-	<name>java-exec</name>
+  xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"
+  xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance">
+  <modelVersion>4.0.0</modelVersion>
+  <parent>
+    <artifactId>exec-parent</artifactId>
+    <groupId>org.apache.drill.exec</groupId>
+    <version>1.0-SNAPSHOT</version>
+  </parent>
+  <artifactId>java-exec</artifactId>
+  <name>java-exec</name>
 
-	<properties>
-		<target.gen.source.path>${project.basedir}/target/generated-sources</target.gen.source.path>
-		<proto.cas.path>${project.basedir}/src/main/protobuf/</proto.cas.path>
-	</properties>
+  <dependencies>
+    <dependency>
+      <groupId>asm</groupId>
+      <artifactId>asm-util</artifactId>
+      <version>3.3.1</version>
+    </dependency>
+    <dependency>
+      <groupId>asm</groupId>
+      <artifactId>asm-commons</artifactId>
+      <version>3.3.1</version>
+    </dependency>
+    <dependency>
+      <groupId>com.sun.codemodel</groupId>
+      <artifactId>codemodel</artifactId>
+      <version>2.6</version>
+    </dependency>
+    <dependency>
+      <groupId>org.codehaus.janino</groupId>
+      <artifactId>commons-compiler-jdk</artifactId>
+      <version>2.6.1</version>
+    </dependency>
+    <dependency>
+      <groupId>net.hydromatic</groupId>
+      <artifactId>optiq</artifactId>
+      <version>0.3.2</version>
+    </dependency>
+    <dependency>
+      <groupId>org.freemarker</groupId>
+      <artifactId>freemarker</artifactId>
+      <version>2.3.19</version>
+    </dependency>
+    <dependency>
+      <groupId>com.google.caliper</groupId>
+      <artifactId>caliper</artifactId>
+      <version>1.0-beta-1</version>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>com.twitter</groupId>
+      <artifactId>parquet-column</artifactId>
+      <version>1.0.0-SNAPSHOT</version>
+    </dependency>
+    <dependency>
+      <groupId>com.yammer.metrics</groupId>
+      <artifactId>metrics-core</artifactId>
+      <version>3.0.0-BETA1</version>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.drill</groupId>
+      <artifactId>common</artifactId>
+      <version>1.0-SNAPSHOT</version>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.drill</groupId>
+      <artifactId>common</artifactId>
+      <version>1.0-SNAPSHOT</version>
+      <classifier>tests</classifier>
+    </dependency>
+    <dependency>
+      <groupId>com.beust</groupId>
+      <artifactId>jcommander</artifactId>
+      <version>1.30</version>
+    </dependency>
+    <dependency>
+      <groupId>com.netflix.curator</groupId>
+      <artifactId>curator-x-discovery</artifactId>
+      <version>1.1.9</version>
+      <exclusions>
+        <!-- <exclusion> -->
+        <!-- <artifactId>netty</artifactId> -->
+        <!-- <groupId>org.jboss.netty</groupId> -->
+        <!-- </exclusion> -->
+        <exclusion>
+          <artifactId>slf4j-log4j12</artifactId>
+          <groupId>org.slf4j</groupId>
+        </exclusion>
+        <exclusion>
+          <artifactId>log4j</artifactId>
+          <groupId>log4j</groupId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+    <dependency>
+      <groupId>org.xerial.snappy</groupId>
+      <artifactId>snappy-java</artifactId>
+      <version>1.0.5-M3</version>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-core</artifactId>
+      <version>1.1.0</version>
+      <exclusions>
+        <exclusion>
+          <artifactId>jets3t</artifactId>
+          <groupId>net.java.dev.jets3t</groupId>
+        </exclusion>
+        <exclusion>
+          <artifactId>commons-logging</artifactId>
+          <groupId>commons-logging</groupId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+    <dependency>
+      <groupId>com.carrotsearch</groupId>
+      <artifactId>hppc</artifactId>
+      <version>0.4.2</version>
+    </dependency>
+    <dependency>
+      <groupId>io.netty</groupId>
+      <artifactId>netty-all</artifactId>
+      <version>4.0.0.CR2</version>
+    </dependency>
+    <dependency>
+      <groupId>com.google.protobuf</groupId>
+      <artifactId>protobuf-java</artifactId>
+      <version>2.5.0</version>
+    </dependency>
+    <dependency>
+      <groupId>com.hazelcast</groupId>
+      <artifactId>hazelcast</artifactId>
+      <version>2.5.1</version>
+    </dependency>
+  </dependencies>
 
-	<dependencies>
-		<dependency>
-			<groupId>asm</groupId>
-			<artifactId>asm-util</artifactId>
-			<version>3.3.1</version>
-		</dependency>
-		<dependency>
-			<groupId>asm</groupId>
-			<artifactId>asm-commons</artifactId>
-			<version>3.3.1</version>
-		</dependency>
-		<dependency>
-			<groupId>org.codehaus.janino</groupId>
-			<artifactId>commons-compiler-jdk</artifactId>
-			<version>2.6.1</version>
-		</dependency>
-		<dependency>
-			<groupId>net.hydromatic</groupId>
-			<artifactId>optiq</artifactId>
-			<version>0.3.2</version>
-		</dependency>
-		<dependency>
-			<groupId>com.twitter</groupId>
-			<artifactId>parquet-column</artifactId>
-			<version>1.0.0-SNAPSHOT</version>
-		</dependency>
-		<dependency>
-			<groupId>com.yammer.metrics</groupId>
-			<artifactId>metrics-core</artifactId>
-			<version>3.0.0-BETA1</version>
-		</dependency>
-		<dependency>
-			<groupId>org.apache.drill</groupId>
-			<artifactId>common</artifactId>
-			<version>1.0-SNAPSHOT</version>
-		</dependency>
-		<dependency>
-			<groupId>org.apache.drill</groupId>
-			<artifactId>common</artifactId>
-			<version>1.0-SNAPSHOT</version>
-			<classifier>tests</classifier>
-		</dependency>
-		<dependency>
-			<groupId>com.beust</groupId>
-			<artifactId>jcommander</artifactId>
-			<version>1.30</version>
-		</dependency>
-		<dependency>
-			<groupId>com.netflix.curator</groupId>
-			<artifactId>curator-x-discovery</artifactId>
-			<version>1.1.9</version>
-			<exclusions>
-				<!-- <exclusion> -->
-				<!-- <artifactId>netty</artifactId> -->
-				<!-- <groupId>org.jboss.netty</groupId> -->
-				<!-- </exclusion> -->
-				<exclusion>
-					<artifactId>slf4j-log4j12</artifactId>
-					<groupId>org.slf4j</groupId>
-				</exclusion>
-				<exclusion>
-					<artifactId>log4j</artifactId>
-					<groupId>log4j</groupId>
-				</exclusion>
-			</exclusions>
-		</dependency>
-		<dependency>
-			<groupId>org.xerial.snappy</groupId>
-			<artifactId>snappy-java</artifactId>
-			<version>1.0.5-M3</version>
-		</dependency>
-		<dependency>
-			<groupId>org.apache.hadoop</groupId>
-			<artifactId>hadoop-core</artifactId>
-			<version>1.1.0</version>
-			<exclusions>
-				<exclusion>
-					<artifactId>jets3t</artifactId>
-					<groupId>net.java.dev.jets3t</groupId>
-				</exclusion>
-				<exclusion>
-					<artifactId>commons-logging</artifactId>
-					<groupId>commons-logging</groupId>
-				</exclusion>
-			</exclusions>
-		</dependency>
-		<dependency>
-			<groupId>com.carrotsearch</groupId>
-			<artifactId>hppc</artifactId>
-			<version>0.4.2</version>
-		</dependency>
-		<dependency>
-			<groupId>io.netty</groupId>
-			<artifactId>netty-all</artifactId>
-			<version>4.0.0.CR2</version>
-		</dependency>
-		<dependency>
-			<groupId>com.google.protobuf</groupId>
-			<artifactId>protobuf-java</artifactId>
-			<version>2.5.0</version>
-		</dependency>
-		<dependency>
-			<groupId>com.hazelcast</groupId>
-			<artifactId>hazelcast</artifactId>
-			<version>2.5.1</version>
-		</dependency>
-	</dependencies>
+  <build>
+    <plugins>
+      <plugin>
+        <artifactId>maven-resources-plugin</artifactId>
+        <version>2.6</version>
+        <executions>
+          <execution>
+            <id>copy-resources</id>
+            <phase>validate</phase>
+            <goals>
+              <goal>copy-resources</goal>
+            </goals>
+            <configuration>
+              <outputDirectory>${basedir}/target/classes/</outputDirectory>
+              <resources>
+                <resource>
+                  <directory>src/main/java</directory>
+                  <filtering>true</filtering>
+                </resource>
+              </resources>
+            </configuration>
+          </execution>
+        </executions>
+      </plugin>
+      <plugin>
+        <artifactId>maven-antrun-plugin</artifactId>
+        <executions>
+          <execution>
+            <id>generate-sources</id>
+            <phase>generate-sources</phase>
+            <configuration>
+              <tasks>
+                <mkdir dir="${target.gen.source.path}" />
+                <path id="proto.path.files">
+                  <fileset dir="${proto.cas.path}">
+                    <include name="*.proto" />
+                  </fileset>
+                </path>
+                <pathconvert pathsep=" " property="proto.files"
+                  refid="proto.path.files" />
 
-	<build>
-		<plugins>
-			<plugin>
-				<groupId>org.codehaus.mojo</groupId>
-				<artifactId>native-maven-plugin</artifactId>
-				<version>1.0-alpha-7</version>
-				<configuration>
-					<javahClassNames>
-						<javahClassName>org.apache.drill.exec.mem.ByteBufferAllocator</javahClassName>
-					</javahClassNames>
-				</configuration>
-			</plugin>
-			<plugin>
-				<artifactId>maven-antrun-plugin</artifactId>
-				<executions>
-					<execution>
-						<id>generate-sources</id>
-						<phase>generate-sources</phase>
-						<configuration>
-							<tasks>
-								<mkdir dir="${target.gen.source.path}" />
-								<path id="proto.path.files">
-									<fileset dir="${proto.cas.path}">
-										<include name="*.proto" />
-									</fileset>
-								</path>
-								<pathconvert pathsep=" " property="proto.files"
-									refid="proto.path.files" />
-
-								<exec executable="protoc">
-									<arg value="--java_out=${target.gen.source.path}" />
-									<arg value="--proto_path=${proto.cas.path}" />
+                <exec executable="protoc">
+                  <arg value="--java_out=${target.gen.source.path}" />
+                  <arg value="--proto_path=${proto.cas.path}" />
+<<<<<<< HEAD
+                  <arg value="--proto_path=${project.basedir}/../../common/src/main/protobuf/" />
 									<arg line="${proto.files}" />
 								</exec>
 							</tasks>
@@ -201,5 +226,34 @@
 			<!-- </plugin> -->
 		</plugins>
 	</build>
+=======
+                  <arg
+                    value="--proto_path=${project.basedir}/../../common/src/main/protobuf/" />
+                  <arg line="${proto.files}" />
+                </exec>
+              </tasks>
+              <sourceRoot>${target.gen.source.path}</sourceRoot>
+            </configuration>
+            <goals>
+              <goal>run</goal>
+            </goals>
+          </execution>
+        </executions>
+      </plugin>
+      <!-- <plugin> -->
+      <!-- <groupId>com.github.igor-petruk.protobuf</groupId> -->
+      <!-- <artifactId>protobuf-maven-plugin</artifactId> -->
+      <!-- <version>0.6.2</version> -->
+      <!-- <executions> -->
+      <!-- <execution> -->
+      <!-- <goals> -->
+      <!-- <goal>run</goal> -->
+      <!-- </goals> -->
+      <!-- </execution> -->
+      <!-- </executions> -->
+      <!-- </plugin> -->
+    </plugins>
+  </build>
+>>>>>>> Build working
 
 </project>

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ExecConstants.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ExecConstants.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ExecConstants.java
index 0ccaa22..4eb0f4c 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ExecConstants.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ExecConstants.java
@@ -29,4 +29,5 @@ public interface ExecConstants {
   public static final String INITIAL_BIT_PORT = "drill.exec.rpc.bit.port";
   public static final String INITIAL_USER_PORT = "drill.exec.rpc.user.port";
   public static final String METRICS_CONTEXT_NAME = "drill.exec.metrics.context";
+  public static final String FUNCTION_PACKAGES = "drill.exec.functions";
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/compile/ClassTransformer.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/compile/ClassTransformer.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/compile/ClassTransformer.java
index 814b239..079ab6c 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/compile/ClassTransformer.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/compile/ClassTransformer.java
@@ -79,6 +79,7 @@ public class ClassTransformer {
   }
 
   
+  
   @SuppressWarnings("unchecked")
   public <T, I> T getImplementationClass(QueryClassLoader classLoader,
       TemplateClassDefinition<T, I> templateDefinition, String internalClassBody, I initObject)

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/compile/CodeModelTools.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/compile/CodeModelTools.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/compile/CodeModelTools.java
new file mode 100644
index 0000000..5e7f1bd
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/compile/CodeModelTools.java
@@ -0,0 +1,31 @@
+package org.apache.drill.exec.compile;
+
+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.exec.expr.CodeGenerator;
+
+import com.sun.codemodel.JType;
+
+public class CodeModelTools {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(CodeModelTools.class);
+  
+  public static JType getType(MinorType mt, DataMode mode, CodeGenerator g){
+    switch (mt) {
+    case BOOLEAN:
+      return g.getModel().BOOLEAN;
+    case INT:
+      return g.getModel().INT;
+    case BIGINT:
+      return g.getModel().LONG;
+    default:
+      throw new UnsupportedOperationException();
+    }
+  }
+  
+  
+  public static JType getType(MajorType mt, CodeGenerator g) {
+    return getType(mt.getMinorType(), mt.getMode(), g);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/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
new file mode 100644
index 0000000..77ae77d
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/CodeGenerator.java
@@ -0,0 +1,201 @@
+package org.apache.drill.exec.expr;
+
+import java.io.IOException;
+
+import org.apache.drill.common.types.TypeProtos.DataMode;
+import org.apache.drill.common.types.TypeProtos.MajorType;
+import org.apache.drill.exec.expr.fn.FunctionImplementationRegistry;
+import org.apache.drill.exec.expr.holders.ValueHolderImplmenetations.BooleanHolder;
+import org.apache.drill.exec.expr.holders.ValueHolderImplmenetations.IntHolder;
+import org.apache.drill.exec.expr.holders.ValueHolderImplmenetations.LongHolder;
+import org.apache.drill.exec.expr.holders.ValueHolderImplmenetations.NullableBooleanHolder;
+import org.apache.drill.exec.expr.holders.ValueHolderImplmenetations.NullableIntHolder;
+import org.apache.drill.exec.expr.holders.ValueHolderImplmenetations.NullableLongHolder;
+import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.record.RecordBatch;
+
+import com.google.common.base.Preconditions;
+import com.sun.codemodel.JBlock;
+import com.sun.codemodel.JClassAlreadyExistsException;
+import com.sun.codemodel.JCodeModel;
+import com.sun.codemodel.JDefinedClass;
+import com.sun.codemodel.JExpr;
+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 {
+  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 final String setupName;
+  private final String perRecordName;
+  
+  private JCodeModel model;
+  private int index = 0;
+
+  public CodeGenerator(String setupName, String perRecordName, FunctionImplementationRegistry funcRegistry) {
+    super();
+    try{
+      this.setupName = setupName;
+      this.perRecordName = perRecordName;
+      this.model = new JCodeModel();
+      this.clazz = model._package("org.apache.drill.exec.test.generated")._class("Test1");
+      this.parentEvalBlock = new JBlock();
+      this.parentSetupBlock = new JBlock();
+      this.evaluationVisitor = new EvaluationVisitor(funcRegistry);
+    } catch (JClassAlreadyExistsException e) {
+      throw new IllegalStateException(e);
+    }
+  }
+
+  public void addNextWrite(ValueVectorWriteExpression ex){
+    currentEvalBlock = new JBlock();
+    parentEvalBlock.add(currentEvalBlock);
+    currentSetupBlock = new JBlock();
+    parentSetupBlock.add(currentSetupBlock);
+    ex.accept(evaluationVisitor, this);
+  }
+  
+  public JBlock getBlock() {
+    return currentEvalBlock;
+  }
+
+  public JBlock getSetupBlock(){
+    return currentSetupBlock;
+  }
+  
+  public String generate() throws IOException{
+
+    {
+      //setup method
+      JMethod m = clazz.method(JMod.PUBLIC, model.VOID, this.setupName);
+      m.param(model._ref(FragmentContext.class), "context");
+      m.param(model._ref(RecordBatch.class), "incoming");
+      m.param(model._ref(RecordBatch.class), "outgoing");
+      m.body().add(parentSetupBlock);
+    }
+    
+    {
+      // eval method.
+      JMethod m = clazz.method(JMod.PUBLIC, model.VOID, this.perRecordName);
+      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;
+  }
+
+  public String getNextVar() {
+    return "v" + index++;
+  }
+  
+  public String getNextVar(String prefix){
+    return prefix + index++;
+  }
+  
+  public JVar declareClassField(String prefix, JType t){
+    return clazz.field(JMod.NONE, t, prefix + index++);
+  }
+  
+  public HoldingContainer declare(MajorType t){
+    return declare(t, true);
+  }
+  
+  public HoldingContainer declare(MajorType t, boolean includeNewInstance){
+    JType holderType = getHolderType(t);
+    JVar var;
+    if(includeNewInstance){
+      var = currentEvalBlock.decl(holderType, "out" + index, JExpr._new(holderType));
+    }else{
+      var = currentEvalBlock.decl(holderType, "out" + index);
+    }
+    JFieldRef outputSet = null;
+    if(t.getMode() == DataMode.OPTIONAL){
+      outputSet = var.ref("isSet");  
+    }
+    index++;
+    return new HoldingContainer(t, var, var.ref("value"), outputSet);
+  }
+  
+  
+  public class HoldingContainer{
+    private final JVar holder;
+    private final JFieldRef value;
+    private final JFieldRef isSet;
+    private final MajorType type;
+    
+    public HoldingContainer(MajorType t, JVar holder, JFieldRef value, JFieldRef isSet) {
+      super();
+      this.holder = holder;
+      this.value = value;
+      this.isSet = isSet;
+      this.type = t;
+    }
+
+    public JVar getHolder() {
+      return holder;
+    }
+
+    public JFieldRef getValue() {
+      return value;
+    }
+
+    public JFieldRef getIsSet() {
+      Preconditions.checkNotNull(isSet, "You cannot access the isSet variable when operating on a non-nullable output value.");
+      return isSet;
+    }
+    
+    public boolean isOptional(){
+      return type.getMode() == DataMode.OPTIONAL;
+    }
+    
+    public boolean isRepeated(){
+      return type.getMode() == DataMode.REPEATED;
+    }
+  }
+  
+  public JType getHolderType(MajorType t){
+    switch(t.getMode()){
+    case REQUIRED:
+      switch(t.getMinorType()){
+      case BOOLEAN:
+        return model._ref(BooleanHolder.class);
+      case INT:
+        return model._ref(IntHolder.class);
+      case BIGINT:  
+        return model._ref(LongHolder.class);
+      
+      }
+      
+    case OPTIONAL:
+      switch(t.getMinorType()){
+      case BOOLEAN:
+        return model._ref(NullableBooleanHolder.class);
+      case INT:
+        return model._ref(NullableIntHolder.class);
+      case BIGINT:  
+        return model._ref(NullableLongHolder.class);
+      }
+
+    }
+    
+    
+    throw new UnsupportedOperationException();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/DrillAggrFunc.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/DrillAggrFunc.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/DrillAggrFunc.java
new file mode 100644
index 0000000..d4433e4
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/DrillAggrFunc.java
@@ -0,0 +1,9 @@
+package org.apache.drill.exec.expr;
+
+import org.apache.drill.exec.record.RecordBatch;
+
+public interface DrillAggrFunc {
+  public void setup(RecordBatch incoming);
+  public void add();
+  public void eval();
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/DrillFunc.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/DrillFunc.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/DrillFunc.java
new file mode 100644
index 0000000..83c5ed9
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/DrillFunc.java
@@ -0,0 +1,9 @@
+package org.apache.drill.exec.expr;
+
+import org.apache.drill.exec.record.RecordBatch;
+
+public interface DrillFunc {
+  public void setup(RecordBatch incoming);
+  public void eval();
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/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
new file mode 100644
index 0000000..a5bc5fa
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/EvaluationVisitor.java
@@ -0,0 +1,201 @@
+package org.apache.drill.exec.expr;
+
+import org.apache.drill.common.expression.FunctionCall;
+import org.apache.drill.common.expression.IfExpression;
+import org.apache.drill.common.expression.IfExpression.IfCondition;
+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.AbstractExprVisitor;
+import org.apache.drill.exec.expr.CodeGenerator.HoldingContainer;
+import org.apache.drill.exec.expr.fn.FunctionHolder;
+import org.apache.drill.exec.expr.fn.FunctionImplementationRegistry;
+import org.apache.drill.exec.record.vector.TypeHelper;
+
+import com.sun.codemodel.JBlock;
+import com.sun.codemodel.JClass;
+import com.sun.codemodel.JConditional;
+import com.sun.codemodel.JExpr;
+import com.sun.codemodel.JInvocation;
+import com.sun.codemodel.JType;
+import com.sun.codemodel.JVar;
+
+public class EvaluationVisitor extends AbstractExprVisitor<HoldingContainer, CodeGenerator, RuntimeException> {
+
+  private FunctionImplementationRegistry registry;
+  
+  
+  public EvaluationVisitor(FunctionImplementationRegistry registry) {
+    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);
+    return holder.generateEvalBody(generator, args);
+  }
+  
+  @Override
+  public HoldingContainer visitIfExpression(IfExpression ifExpr, CodeGenerator generator) throws RuntimeException {
+    JBlock local = generator.getBlock();
+    
+    HoldingContainer output = generator.declare(ifExpr.getMajorType());
+    
+    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()));
+        } else {
+          jc = conditionalBlock._if(HoldingContainer.getValue());
+        }
+      } else {
+        if (HoldingContainer.isOptional()) {
+          jc = jc._else()._if(HoldingContainer.getIsSet().cand(HoldingContainer.getValue()));
+        } else {
+          jc = jc._else()._if(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._then().assign(output.getValue(), thenExpr.getValue());
+      }
+
+    }
+
+    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());
+
+    }
+    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 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 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()));
+    return out;
+  }
+
+  
+  
+  @Override
+  public HoldingContainer visitUnknown(LogicalExpression e, CodeGenerator generator) throws RuntimeException {
+    if(e instanceof ValueVectorReadExpression){
+      return visitValueVectorExpression((ValueVectorReadExpression) e, generator);
+    }else if(e instanceof ValueVectorWriteExpression){
+      return visitValueVectorWriteExpression((ValueVectorWriteExpression) 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.getBlock();
+    
+    Class<?> vvClass = TypeHelper.getValueVectorClass(child.getMajorType().getMinorType(), child.getMajorType().getMode());
+    JType vvType = generator.getModel()._ref(vvClass);
+    JVar vv = generator.declareClassField("vv", vvType);
+    
+    // get value vector in setup block.
+    generator.getSetupBlock().assign(vv, JExpr.direct("outgoing").invoke("getValueVector") //
+      .arg(JExpr.lit(e.getFieldId())) //
+      .arg( ((JClass)vvType).dotclass()));
+    
+    if(hc.isOptional()){
+      vv.invoke("set").arg(JExpr.direct("outIndex"));
+      JConditional jc = block._if(hc.getIsSet().eq(JExpr.lit(0)).not());
+      block = jc._then();
+    }
+    block.add(vv.invoke("set").arg(JExpr.direct("outIndex")).arg(hc.getValue()));
+
+    return null;
+  }
+  
+  private HoldingContainer visitValueVectorExpression(ValueVectorReadExpression e, CodeGenerator generator) throws RuntimeException{
+    // declare value vector
+    Class<?> vvClass = TypeHelper.getValueVectorClass(e.getMajorType().getMinorType(), e.getMajorType().getMode());
+    JType vvType = generator.getModel()._ref(vvClass);
+    JVar vv1 = generator.declareClassField("vv", vvType);
+    
+    // get value vector from incoming batch and 
+    JInvocation getValue = JExpr //
+        .invoke(JExpr.direct("incoming"), "getValueVector") //
+        .arg(JExpr.lit(e.getFieldId())) //
+        .arg( ((JClass)vvType).dotclass());
+    generator.getSetupBlock().assign(vv1, getValue);
+
+    // evaluation work.
+    HoldingContainer out = generator.declare(e.getMajorType());
+    
+    
+    if(out.isOptional()){
+      JBlock blk = generator.getBlock();
+      blk.assign(out.getIsSet(), vv1.invoke("isSet").arg(JExpr.direct("index")));
+      JConditional jc = blk._if(out.getIsSet());
+      jc._then() //
+        .assign(out.getValue(), vv1.invoke("get").arg(JExpr.direct("index"))); //
+        //.assign(out.getIsSet(), JExpr.lit(1));
+      //jc._else()
+        //.assign(out.getIsSet(), JExpr.lit(0));
+      
+    }else{
+      generator.getBlock().assign(out.getValue(), vv1.invoke("get").arg(JExpr.direct("index")));
+    }
+    return out;
+  }
+  
+  
+  
+  @Override
+  public HoldingContainer visitQuotedStringConstant(QuotedString e, CodeGenerator CodeGenerator) throws RuntimeException {
+    throw new UnsupportedOperationException("We don't yet support string literals as we need to use the valuevector classes and internal vectors.");
+//    JExpr stringLiteral = JExpr.lit(e.value);
+//    CodeGenerator.block.decl(stringLiteral.invoke("getBytes").arg(JExpr.ref(Charsets.UTF_8));
+  }
+
+}


[36/53] [abbrv] git commit: Merge commit '5052b64d9953857575f8f40995b8da05160e5457' into execwork

Posted by ja...@apache.org.
Merge commit '5052b64d9953857575f8f40995b8da05160e5457' into execwork


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

Branch: refs/heads/master
Commit: c941874d73f1d387c68daa40d6089e3068cd0073
Parents: 9ca9eb9 5052b64
Author: Jacques Nadeau <ja...@apache.org>
Authored: Mon Jul 15 13:17:54 2013 -0700
Committer: Jacques Nadeau <ja...@apache.org>
Committed: Mon Jul 15 13:17:54 2013 -0700

----------------------------------------------------------------------
 .../apache/drill/synth/ChineseRestaurant.java   | 118 +++++++++++++++++++
 .../java/org/apache/drill/synth/LongTail.java   |   9 +-
 2 files changed, 121 insertions(+), 6 deletions(-)
----------------------------------------------------------------------



[17/53] [abbrv] Update typing system. Update RPC system. Add Fragmenting Implementation. Working single node. Distributed failing due to threading issues.

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/coord/ZKClusterCoordinator.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/coord/ZKClusterCoordinator.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/coord/ZKClusterCoordinator.java
index 85c573d..75dce2c 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/coord/ZKClusterCoordinator.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/coord/ZKClusterCoordinator.java
@@ -23,16 +23,19 @@ import static com.google.common.collect.Collections2.transform;
 import java.io.IOException;
 import java.util.Collection;
 import java.util.Collections;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
 
 import org.apache.drill.common.config.DrillConfig;
-import org.apache.drill.common.proto.CoordinationProtos.DrillbitEndpoint;
 import org.apache.drill.exec.ExecConstants;
+import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
 
 import com.google.common.base.Function;
 import com.netflix.curator.RetryPolicy;
 import com.netflix.curator.framework.CuratorFramework;
 import com.netflix.curator.framework.CuratorFrameworkFactory;
 import com.netflix.curator.framework.state.ConnectionState;
+import com.netflix.curator.framework.state.ConnectionStateListener;
 import com.netflix.curator.retry.RetryNTimes;
 import com.netflix.curator.x.discovery.ServiceDiscovery;
 import com.netflix.curator.x.discovery.ServiceDiscoveryBuilder;
@@ -52,6 +55,7 @@ public class ZKClusterCoordinator extends ClusterCoordinator {
   private ServiceCache<DrillbitEndpoint> serviceCache;
   private volatile Collection<DrillbitEndpoint> endpoints = Collections.emptyList();
   private final String serviceName;
+  private final CountDownLatch initialConnection = new CountDownLatch(1);
 
   public ZKClusterCoordinator(DrillConfig config) throws IOException {
 
@@ -64,6 +68,7 @@ public class ZKClusterCoordinator extends ClusterCoordinator {
       .retryPolicy(rp)
       .connectString(config.getString(ExecConstants.ZK_CONNECTION))
       .build();
+    curator.getConnectionStateListenable().addListener(new InitialConnectionListener());
     discovery = getDiscovery();
     serviceCache = discovery.
       serviceCacheBuilder()
@@ -72,15 +77,36 @@ public class ZKClusterCoordinator extends ClusterCoordinator {
       .build();
   }
 
-  public void start() throws Exception {
+  public void start(long millisToWait) throws Exception {
     logger.debug("Starting ZKClusterCoordination.");
     curator.start();
     discovery.start();
     serviceCache.start();
     serviceCache.addListener(new ZKListener());
+    
+    if(millisToWait != 0){
+      boolean success = this.initialConnection.await(millisToWait, TimeUnit.MILLISECONDS);
+      if(!success) throw new IOException(String.format("Failure to connect to the zookeeper cluster service within the allotted time of %d milliseconds.", millisToWait));
+    }else{
+      this.initialConnection.await();
+    }
+    
+    
     updateEndpoints();
   }
+  
+  private class InitialConnectionListener implements ConnectionStateListener{
 
+    @Override
+    public void stateChanged(CuratorFramework client, ConnectionState newState) {
+      if(newState == ConnectionState.CONNECTED){
+        ZKClusterCoordinator.this.initialConnection.countDown();
+        client.getConnectionStateListenable().removeListener(this);
+      }
+    }
+    
+  }
+  
   private class ZKListener implements ServiceCacheListener {
 
     @Override

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/exception/BitComException.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/exception/BitComException.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/exception/BitComException.java
new file mode 100644
index 0000000..9c18e51
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/exception/BitComException.java
@@ -0,0 +1,45 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.exception;
+
+import org.apache.drill.common.exceptions.DrillException;
+
+public class BitComException extends DrillException{
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(BitComException.class);
+
+  public BitComException() {
+    super();
+  }
+
+  public BitComException(String message, Throwable cause, boolean enableSuppression, boolean writableStackTrace) {
+    super(message, cause, enableSuppression, writableStackTrace);
+  }
+
+  public BitComException(String message, Throwable cause) {
+    super(message, cause);
+  }
+
+  public BitComException(String message) {
+    super(message);
+  }
+
+  public BitComException(Throwable cause) {
+    super(cause);
+  }
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/exception/ExecutionSetupException.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/exception/ExecutionSetupException.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/exception/ExecutionSetupException.java
deleted file mode 100644
index a4899bd..0000000
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/exception/ExecutionSetupException.java
+++ /dev/null
@@ -1,45 +0,0 @@
-/*******************************************************************************
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- ******************************************************************************/
-package org.apache.drill.exec.exception;
-
-import org.apache.drill.common.exceptions.DrillException;
-
-public class ExecutionSetupException extends DrillException{
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ExecutionSetupException.class);
-  
-  public ExecutionSetupException() {
-    super();
-  }
-
-  public ExecutionSetupException(String message, Throwable cause, boolean enableSuppression, boolean writableStackTrace) {
-    super(message, cause, enableSuppression, writableStackTrace);
-  }
-
-  public ExecutionSetupException(String message, Throwable cause) {
-    super(message, cause);
-  }
-
-  public ExecutionSetupException(String message) {
-    super(message);
-  }
-
-  public ExecutionSetupException(Throwable cause) {
-    super(cause);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/exception/FragmentSetupException.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/exception/FragmentSetupException.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/exception/FragmentSetupException.java
index c273463..dbd66b9 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/exception/FragmentSetupException.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/exception/FragmentSetupException.java
@@ -17,6 +17,8 @@
  ******************************************************************************/
 package org.apache.drill.exec.exception;
 
+import org.apache.drill.common.exceptions.ExecutionSetupException;
+
 public class FragmentSetupException extends ExecutionSetupException{
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(FragmentSetupException.class);
 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/foreman/CancelableQuery.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/foreman/CancelableQuery.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/foreman/CancelableQuery.java
deleted file mode 100644
index 30e7a63..0000000
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/foreman/CancelableQuery.java
+++ /dev/null
@@ -1,22 +0,0 @@
-/*******************************************************************************
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- ******************************************************************************/
-package org.apache.drill.exec.foreman;
-
-public interface CancelableQuery {
-  public boolean cancel(long queryid);
-}

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

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/foreman/Foreman.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/foreman/Foreman.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/foreman/Foreman.java
deleted file mode 100644
index f138171..0000000
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/foreman/Foreman.java
+++ /dev/null
@@ -1,39 +0,0 @@
-/*******************************************************************************
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- ******************************************************************************/
-package org.apache.drill.exec.foreman;
-
-
-public class Foreman extends Thread{
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(Foreman.class);
-  
-  public Foreman(){
-    
-  }
-  
-  public void doWork(QueryWorkUnit work){
-    // generate fragment structure. 
-    // store fragments in distributed grid.
-    // generate any codegen required and store in grid.
-    // drop 
-    // do get on the result set you're looking for.  Do the initial get on the result node you're looking for.  This will return either data or a metadata record set
-  }
-
-  public boolean checkStatus(long queryId){
-    return false;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/foreman/QueryWorkUnit.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/foreman/QueryWorkUnit.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/foreman/QueryWorkUnit.java
deleted file mode 100644
index bdf4a1e..0000000
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/foreman/QueryWorkUnit.java
+++ /dev/null
@@ -1,54 +0,0 @@
-/*******************************************************************************
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- ******************************************************************************/
-package org.apache.drill.exec.foreman;
-
-import java.util.Collection;
-import java.util.List;
-
-import org.apache.drill.exec.proto.ExecProtos.PlanFragment;
-
-import com.google.common.base.Preconditions;
-
-public class QueryWorkUnit {
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(QueryWorkUnit.class);
-  
-  private PlanFragment rootFragment; // for local
-  private List<PlanFragment> fragments;
-  
-  public QueryWorkUnit(PlanFragment rootFragment, List<PlanFragment> fragments) {
-    super();
-    Preconditions.checkNotNull(rootFragment);
-    Preconditions.checkNotNull(fragments);
-    this.rootFragment = rootFragment;
-    this.fragments = fragments;
-  }
-
-  public PlanFragment getRootFragment() {
-    return rootFragment;
-  }
-
-  public List<PlanFragment> getFragments() {
-    return fragments;
-  }
-  
-  
-  
-  
-  
-  
-}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/foreman/ResourceRequest.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/foreman/ResourceRequest.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/foreman/ResourceRequest.java
deleted file mode 100644
index 96d7d1e..0000000
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/foreman/ResourceRequest.java
+++ /dev/null
@@ -1,30 +0,0 @@
-/*******************************************************************************
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- ******************************************************************************/
-package org.apache.drill.exec.foreman;
-
-public class ResourceRequest {
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ResourceRequest.class);
-  
-  public long memoryMin;
-  public long memoryDesired;
-  
-
-  public static class ResourceAllocation {
-    public long memory;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/foreman/StatusProvider.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/foreman/StatusProvider.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/foreman/StatusProvider.java
deleted file mode 100644
index fee6172..0000000
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/foreman/StatusProvider.java
+++ /dev/null
@@ -1,24 +0,0 @@
-/*******************************************************************************
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- ******************************************************************************/
-package org.apache.drill.exec.foreman;
-
-import org.apache.drill.exec.proto.ExecProtos.FragmentStatus;
-
-public interface StatusProvider {
-  public FragmentStatus getStatus();
-}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/memory/BufferAllocator.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/memory/BufferAllocator.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/memory/BufferAllocator.java
index 2b3f574..6bddab7 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/memory/BufferAllocator.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/memory/BufferAllocator.java
@@ -19,11 +19,10 @@ package org.apache.drill.exec.memory;
 
 import io.netty.buffer.ByteBuf;
 import io.netty.buffer.ByteBufAllocator;
-import io.netty.buffer.PooledByteBufAllocator;
 
 import java.io.Closeable;
 
-import org.apache.drill.exec.server.DrillbitContext;
+import org.apache.drill.common.config.DrillConfig;
 
 /**
  * Wrapper class to deal with byte buffer allocation. Ensures users only use designated methods.  Also allows inser 
@@ -48,7 +47,7 @@ public abstract class BufferAllocator implements Closeable{
   @Override
   public abstract void close(); 
   
-  public static BufferAllocator getAllocator(DrillbitContext context){
+  public static BufferAllocator getAllocator(DrillConfig config){
     // TODO: support alternative allocators (including a debugging allocator that records all allocation locations for each buffer).
     return new DirectBufferAllocator();
   }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/metrics/SingleThreadNestedCounter.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/metrics/SingleThreadNestedCounter.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/metrics/SingleThreadNestedCounter.java
index 6b89c12..1f47041 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/metrics/SingleThreadNestedCounter.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/metrics/SingleThreadNestedCounter.java
@@ -22,34 +22,34 @@ import org.apache.drill.exec.server.DrillbitContext;
 import com.yammer.metrics.Counter;
 
 /**
- * Wraps a parent counter so that local in thread metrics can be collected while collecting for a global counter.
+ * Wraps a parent counter so that local in-thread metrics can be collected while collecting for a global counter. Note
+ * that this one writer, many reader safe.
  */
 public class SingleThreadNestedCounter {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(SingleThreadNestedCounter.class);
-  
+
   private volatile long count;
   private final Counter counter;
-  
-  
+
   public SingleThreadNestedCounter(DrillbitContext context, String name) {
     super();
     this.counter = context.getMetrics().counter(name);
   }
 
-  public long inc(long n){
+  public long inc(long n) {
     counter.inc(n);
-    count+= n;
+    count += n;
     return count;
   }
-  
-  public long dec(long n){
+
+  public long dec(long n) {
     counter.dec(n);
     count -= n;
     return count;
   }
-  
-  public long get(){
+
+  public long get() {
     return count;
   }
-  
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/FilteringRecordBatchTransformer.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/FilteringRecordBatchTransformer.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/FilteringRecordBatchTransformer.java
deleted file mode 100644
index f626cea..0000000
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/FilteringRecordBatchTransformer.java
+++ /dev/null
@@ -1,58 +0,0 @@
-/*******************************************************************************
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- ******************************************************************************/
-package org.apache.drill.exec.ops;
-
-import org.apache.drill.exec.record.BatchSchema;
-import org.apache.drill.exec.record.RecordBatch;
-import org.apache.drill.exec.record.vector.SelectionVector;
-
-public abstract class FilteringRecordBatchTransformer {
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(FilteringRecordBatchTransformer.class);
-  
-  final RecordBatch incoming;
-  final SelectionVector selectionVector;
-  final BatchSchema schema;
-  
-  public FilteringRecordBatchTransformer(RecordBatch incoming, OutputMutator output, SelectionVector selectionVector) {
-    super();
-    this.incoming = incoming;
-    this.selectionVector = selectionVector;
-    this.schema = innerSetup();
-  }
-
-  public abstract BatchSchema innerSetup();
-  
-  /**
-   * Applies the filter to the selection index.  Ignores any values in the selection vector, instead creating a.
-   * @return
-   */
-  public abstract int apply();
-  
-  /**
-   * Applies the filter to the selection index.  Utilizes the existing selection index and only evaluates on those records.
-   * @return
-   */
-  public abstract int applyWithSelection();
-
-  public BatchSchema getSchema() {
-    return schema;
-  }
-  
-  
-  
-}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/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 0cf17e9..e64453c 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
@@ -20,10 +20,15 @@ package org.apache.drill.exec.ops;
 import org.apache.drill.common.expression.LogicalExpression;
 import org.apache.drill.exec.memory.BufferAllocator;
 import org.apache.drill.exec.metrics.SingleThreadNestedCounter;
-import org.apache.drill.exec.planner.FragmentRunnable;
-import org.apache.drill.exec.proto.ExecProtos.PlanFragment;
+import org.apache.drill.exec.physical.impl.FilteringRecordBatchTransformer;
+import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
+import org.apache.drill.exec.proto.ExecProtos.FragmentHandle;
+import org.apache.drill.exec.proto.ExecProtos.FragmentStatus;
 import org.apache.drill.exec.rpc.bit.BitCom;
+import org.apache.drill.exec.rpc.user.UserServer.UserClientConnection;
 import org.apache.drill.exec.server.DrillbitContext;
+import org.apache.drill.exec.work.FragmentRunner;
+import org.apache.drill.exec.work.batch.IncomingBuffers;
 
 import com.yammer.metrics.MetricRegistry;
 import com.yammer.metrics.Timer;
@@ -34,51 +39,72 @@ import com.yammer.metrics.Timer;
 public class FragmentContext {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(FragmentContext.class);
 
-  private final static String METRIC_TIMER_FRAGMENT_TIME = MetricRegistry.name(FragmentRunnable.class, "completionTimes");
-  private final static String METRIC_BATCHES_COMPLETED = MetricRegistry.name(FragmentRunnable.class, "batchesCompleted");
-  private final static String METRIC_RECORDS_COMPLETED = MetricRegistry.name(FragmentRunnable.class, "recordsCompleted");
-  private final static String METRIC_DATA_PROCESSED = MetricRegistry.name(FragmentRunnable.class, "dataProcessed");
+  private final static String METRIC_TIMER_FRAGMENT_TIME = MetricRegistry.name(FragmentRunner.class, "completionTimes");
+  private final static String METRIC_BATCHES_COMPLETED = MetricRegistry.name(FragmentRunner.class, "batchesCompleted");
+  private final static String METRIC_RECORDS_COMPLETED = MetricRegistry.name(FragmentRunner.class, "recordsCompleted");
+  private final static String METRIC_DATA_PROCESSED = MetricRegistry.name(FragmentRunner.class, "dataProcessed");
 
   private final DrillbitContext context;
-  private final PlanFragment fragment;
   public final SingleThreadNestedCounter batchesCompleted;
   public final SingleThreadNestedCounter recordsCompleted;
   public final SingleThreadNestedCounter dataProcessed;
   public final Timer fragmentTime;
+  private final FragmentHandle handle;
+  private final UserClientConnection connection;
+  private final IncomingBuffers buffers;
 
-  public FragmentContext(DrillbitContext dbContext, PlanFragment fragment) {
+  public FragmentContext(DrillbitContext dbContext, FragmentHandle handle, UserClientConnection connection, IncomingBuffers buffers) {
     this.fragmentTime = dbContext.getMetrics().timer(METRIC_TIMER_FRAGMENT_TIME);
     this.batchesCompleted = new SingleThreadNestedCounter(dbContext, METRIC_BATCHES_COMPLETED);
     this.recordsCompleted = new SingleThreadNestedCounter(dbContext, METRIC_RECORDS_COMPLETED);
     this.dataProcessed = new SingleThreadNestedCounter(dbContext, METRIC_DATA_PROCESSED);
     this.context = dbContext;
-    this.fragment = fragment;
+    this.connection = connection;
+    this.handle = handle;
+    this.buffers = buffers;
   }
 
   public void fail(Throwable cause) {
 
   }
 
+  
   public DrillbitContext getDrillbitContext(){
     return context;
   }
-  
-  public PlanFragment getFragment() {
-    return fragment;
+
+  public DrillbitEndpoint getIdentity(){
+    return context.getEndpoint();
   }
   
+  public FragmentHandle getHandle() {
+    return handle;
+  }
+
   public BufferAllocator getAllocator(){
     // TODO: A local query allocator to ensure memory limits and accurately gauge memory usage.
     return context.getAllocator();
   }
 
-  
   public FilteringRecordBatchTransformer getFilteringExpression(LogicalExpression expr){
     return null;
   }
   
+  public void addMetricsToStatus(FragmentStatus.Builder stats){
+    stats.setBatchesCompleted(batchesCompleted.get());
+    stats.setDataProcessed(dataProcessed.get());
+    stats.setRecordsCompleted(recordsCompleted.get());
+  }
   
+  public UserClientConnection getConnection() {
+    return connection;
+  }
+
   public BitCom getCommunicator(){
-    return null;
+    return context.getBitCom();
+  }
+  
+  public IncomingBuffers getBuffers(){
+    return buffers;
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/FragmentConverter.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/FragmentConverter.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/FragmentConverter.java
deleted file mode 100644
index 3c75648..0000000
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/FragmentConverter.java
+++ /dev/null
@@ -1,30 +0,0 @@
-/*******************************************************************************
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- ******************************************************************************/
-package org.apache.drill.exec.ops;
-
-import org.apache.drill.exec.proto.ExecProtos.PlanFragment;
-
-public class FragmentConverter {
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(FragmentConverter.class);
-  
-  public static FragmentRoot getFragment(FragmentContext context){
-    PlanFragment m = context.getFragment();
-    
-    return null;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/FragmentRoot.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/FragmentRoot.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/FragmentRoot.java
deleted file mode 100644
index ddacb41..0000000
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/FragmentRoot.java
+++ /dev/null
@@ -1,37 +0,0 @@
-/*******************************************************************************
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- ******************************************************************************/
-package org.apache.drill.exec.ops;
-
-import org.apache.drill.exec.exception.FragmentSetupException;
-
-/**
- * A FragmentRoot is a node which is the last processing node in a query plan. FragmentTerminals include Exchange
- * output nodes and storage nodes.  They are there driving force behind the completion of a query.
- */
-public interface FragmentRoot {
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(FragmentRoot.class);
-  
-  /**
-   * Do the next batch of work.  
-   * @return Whether or not additional batches of work are necessary.
-   */
-  public boolean next();
-  
-  
-  public void setup() throws FragmentSetupException;
-}

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

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/OutputMutator.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/OutputMutator.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/OutputMutator.java
deleted file mode 100644
index 59abdc4..0000000
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/OutputMutator.java
+++ /dev/null
@@ -1,28 +0,0 @@
-/*******************************************************************************
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- ******************************************************************************/
-package org.apache.drill.exec.ops;
-
-import org.apache.drill.exec.exception.SchemaChangeException;
-import org.apache.drill.exec.record.BatchSchema;
-import org.apache.drill.exec.record.vector.ValueVector;
-
-public interface OutputMutator {
-  public void removeField(int fieldId) throws SchemaChangeException;
-  public void addField(int fieldId, ValueVector<?> vector) throws SchemaChangeException ;
-  public void setNewSchema(BatchSchema schema) throws SchemaChangeException ;
-}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/QueryContext.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/QueryContext.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/QueryContext.java
index fe37e70..fd24deb 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/QueryContext.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/QueryContext.java
@@ -19,33 +19,42 @@ package org.apache.drill.exec.ops;
 
 import java.util.Collection;
 
-import org.apache.drill.common.proto.CoordinationProtos.DrillbitEndpoint;
+import org.apache.drill.exec.cache.DistributedCache;
+import org.apache.drill.exec.planner.PhysicalPlanReader;
+import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
+import org.apache.drill.exec.proto.UserBitShared.QueryId;
 import org.apache.drill.exec.server.DrillbitContext;
 
-import com.fasterxml.jackson.databind.ObjectMapper;
-
 public class QueryContext {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(QueryContext.class);
   
-  private long queryId;
+  private QueryId queryId;
   private DrillbitContext drillbitContext;
   
-  public QueryContext(long queryId, DrillbitContext drllbitContext) {
+  public QueryContext(QueryId queryId, DrillbitContext drllbitContext) {
     super();
     this.queryId = queryId;
     this.drillbitContext = drllbitContext;
   }
   
-  public long getQueryId() {
-    return queryId;
+  public DrillbitEndpoint getCurrentEndpoint(){
+    return drillbitContext.getEndpoint();
   }
   
-  public ObjectMapper getMapper(){
-    return drillbitContext.getConfig().getMapper();
+  public QueryId getQueryId() {
+    return queryId;
+  }
+
+  public DistributedCache getCache(){
+    return drillbitContext.getCache();
   }
   
   public Collection<DrillbitEndpoint> getActiveEndpoints(){
     return drillbitContext.getBits();
   }
   
+  public PhysicalPlanReader getPlanReader(){
+    return drillbitContext.getPlanReader();
+  }
+  
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/ScanBatch.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/ScanBatch.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/ScanBatch.java
deleted file mode 100644
index b46804f..0000000
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/ScanBatch.java
+++ /dev/null
@@ -1,157 +0,0 @@
-/*******************************************************************************
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- ******************************************************************************/
-package org.apache.drill.exec.ops;
-
-import java.util.Iterator;
-
-import org.apache.drill.exec.exception.ExecutionSetupException;
-import org.apache.drill.exec.exception.SchemaChangeException;
-import org.apache.drill.exec.record.BatchSchema;
-import org.apache.drill.exec.record.InvalidValueAccessor;
-import org.apache.drill.exec.record.RecordBatch;
-import org.apache.drill.exec.record.vector.ValueVector;
-import org.apache.drill.exec.store.RecordReader;
-
-import com.carrotsearch.hppc.IntObjectOpenHashMap;
-import com.carrotsearch.hppc.procedures.IntObjectProcedure;
-
-/**
- * Record batch used for a particular scan. Operators against one or more
- */
-public abstract class ScanBatch implements RecordBatch {
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ScanBatch.class);
-
-  private IntObjectOpenHashMap<ValueVector<?>> fields = new IntObjectOpenHashMap<ValueVector<?>>();
-  private BatchSchema schema;
-  private int recordCount;
-  private boolean schemaChanged = true;
-  private final FragmentContext context;
-  private Iterator<RecordReader> readers;
-  private RecordReader currentReader;
-  private final BatchSchema expectedSchema;
-  private final Mutator mutator = new Mutator();
-
-  public ScanBatch(BatchSchema expectedSchema, Iterator<RecordReader> readers, FragmentContext context)
-      throws ExecutionSetupException {
-    this.expectedSchema = expectedSchema;
-    this.context = context;
-    this.readers = readers;
-    if (!readers.hasNext()) throw new ExecutionSetupException("A scan batch must contain at least one reader.");
-    this.currentReader = readers.next();
-    this.currentReader.setup(expectedSchema, mutator);
-  }
-
-  private void schemaChanged() {
-    schema = null;
-    schemaChanged = true;
-  }
-
-  @Override
-  public FragmentContext getContext() {
-    return context;
-  }
-
-  @Override
-  public BatchSchema getSchema() {
-    return schema;
-  }
-
-  @Override
-  public int getRecordCount() {
-    return recordCount;
-  }
-
-  @Override
-  public void kill() {
-    releaseAssets();
-  }
-
-  private void releaseAssets() {
-    fields.forEach(new IntObjectProcedure<ValueVector<?>>() {
-      @Override
-      public void apply(int key, ValueVector<?> value) {
-        value.close();
-      }
-    });
-  }
-
-  @SuppressWarnings("unchecked")
-  @Override
-  public <T extends ValueVector<T>> T getValueVector(int fieldId, Class<T> clazz) throws InvalidValueAccessor {
-    if (fields.containsKey(fieldId))
-      throw new InvalidValueAccessor(String.format("Unknown value accesor for field id %d."));
-    ValueVector<?> vector = this.fields.lget();
-    if (vector.getClass().isAssignableFrom(clazz)) {
-      return (T) vector;
-    } else {
-      throw new InvalidValueAccessor(String.format(
-          "You requested a field accessor of type %s for field id %d but the actual type was %s.",
-          clazz.getCanonicalName(), fieldId, vector.getClass().getCanonicalName()));
-    }
-  }
-
-  @Override
-  public IterOutcome next() {
-    while ((recordCount = currentReader.next()) == 0) {
-      try {
-        if (!readers.hasNext()) {
-          currentReader.cleanup();
-          releaseAssets();
-          return IterOutcome.NONE;
-        }
-        currentReader.cleanup();
-        currentReader = readers.next();
-        currentReader.setup(expectedSchema, mutator);
-      } catch (ExecutionSetupException e) {
-        this.context.fail(e);
-        releaseAssets();
-        return IterOutcome.STOP;
-      }
-    }
-
-    if (schemaChanged) {
-      schemaChanged = false;
-      return IterOutcome.OK_NEW_SCHEMA;
-    } else {
-      return IterOutcome.OK;
-    }
-  }
-
-  private class Mutator implements OutputMutator {
-
-    public void removeField(int fieldId) throws SchemaChangeException {
-      schemaChanged();
-      ValueVector<?> v = fields.remove(fieldId);
-      if (v == null) throw new SchemaChangeException("Failure attempting to remove an unknown field.");
-      v.close();
-    }
-
-    public void addField(int fieldId, ValueVector<?> vector) {
-      schemaChanged();
-      ValueVector<?> v = fields.put(fieldId, vector);
-      if (v != null) v.close();
-    }
-
-    @Override
-    public void setNewSchema(BatchSchema schema) {
-      ScanBatch.this.schema = schema;
-      ScanBatch.this.schemaChanged = true;
-    }
-
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/StreamingRecordBatch.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/StreamingRecordBatch.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/StreamingRecordBatch.java
deleted file mode 100644
index 0fc7a1f..0000000
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/StreamingRecordBatch.java
+++ /dev/null
@@ -1,25 +0,0 @@
-/*******************************************************************************
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- ******************************************************************************/
-package org.apache.drill.exec.ops;
-
-/**
- * Works on one incoming batch at a time.  Creates one output batch for each input batch.
- */
-public class StreamingRecordBatch {
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(StreamingRecordBatch.class);
-}

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

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

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/filter/FilterRecordBatch.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/filter/FilterRecordBatch.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/filter/FilterRecordBatch.java
deleted file mode 100644
index 5bef612..0000000
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/filter/FilterRecordBatch.java
+++ /dev/null
@@ -1,109 +0,0 @@
-/*******************************************************************************
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- ******************************************************************************/
-package org.apache.drill.exec.ops.filter;
-
-import org.apache.drill.exec.ops.FilteringRecordBatchTransformer;
-import org.apache.drill.exec.ops.FragmentContext;
-import org.apache.drill.exec.record.BatchSchema;
-import org.apache.drill.exec.record.InvalidValueAccessor;
-import org.apache.drill.exec.record.RecordBatch;
-import org.apache.drill.exec.record.vector.SelectionVector;
-import org.apache.drill.exec.record.vector.ValueVector;
-
-public abstract class FilterRecordBatch implements RecordBatch {
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(FilterRecordBatch.class);
-
-  private RecordBatch incoming;
-  private SelectionVector selectionVector;
-  private BatchSchema schema;
-  private FilteringRecordBatchTransformer transformer;
-  private int outstanding;
-
-  public FilterRecordBatch(RecordBatch batch) {
-    this.incoming = batch;
-  }
-
-  @Override
-  public FragmentContext getContext() {
-    return incoming.getContext();
-  }
-
-  @Override
-  public BatchSchema getSchema() {
-    return schema;
-  }
-
-  @Override
-  public int getRecordCount() {
-    return 0;
-  }
-
-  @Override
-  public void kill() {
-    incoming.kill();
-  }
-
-  @Override
-  public <T extends ValueVector<T>> T getValueVector(int fieldId, Class<T> clazz) throws InvalidValueAccessor {
-    return null;
-  }
-
-  abstract int applyFilter(SelectionVector vector, int count);
-
-  /**
-   * Release all assets.
-   */
-  private void close() {
-
-  }
-
-  @Override
-  public IterOutcome next() {
-    while (true) {
-      IterOutcome o = incoming.next();
-      switch (o) {
-      case OK_NEW_SCHEMA:
-        transformer = incoming.getContext().getFilteringExpression(null);
-        schema = transformer.getSchema();
-        // fall through to ok.
-      case OK:
-
-      case NONE:
-      case STOP:
-        close();
-        return IterOutcome.STOP;
-      }
-
-      if (outstanding > 0) {
-        // move data to output location.
-
-        for (int i = incoming.getRecordCount() - outstanding; i < incoming.getRecordCount(); i++) {
-
-        }
-      }
-
-      // make sure the bit vector is as large as the current record batch.
-      if (selectionVector.size() < incoming.getRecordCount()) {
-        selectionVector.allocateNew(incoming.getRecordCount());
-      }
-
-      return null;
-    }
-
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/filter/SelectionVectorUpdater.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/filter/SelectionVectorUpdater.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/filter/SelectionVectorUpdater.java
deleted file mode 100644
index 218a19a..0000000
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/filter/SelectionVectorUpdater.java
+++ /dev/null
@@ -1,80 +0,0 @@
-/*******************************************************************************
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- ******************************************************************************/
-package org.apache.drill.exec.ops.filter;
-
-import org.apache.drill.exec.memory.BufferAllocator;
-import org.apache.drill.exec.memory.DirectBufferAllocator;
-import org.apache.drill.exec.record.vector.NullableInt32Vector;
-import org.apache.drill.exec.record.vector.UInt16Vector;
-import org.codehaus.janino.ExpressionEvaluator;
-
-public class SelectionVectorUpdater {
-  //static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(SelectionVectorUpdater.class);
-
-  // Add a selection vector to a record batch.
-  /**
-   * where a + b < 10
-   */
-
-  public static int applyToBatch(final int recordCount, final NullableInt32Vector a, final NullableInt32Vector b,
-      final UInt16Vector selectionVector) {
-    int selectionIndex = 0;
-    for (int i = 0; i < recordCount; i++) {
-      int isNotNull = a.isNull(i) * b.isNull(i);
-      if (isNotNull > 0 && a.get(i) + b.get(i) < 10) {
-        selectionVector.set(selectionIndex, (char) i);
-        selectionIndex++;
-      }
-    }
-    return selectionIndex;
-  }
-
-  public static void mai2n(String[] args) {
-    int size = 1024;
-    BufferAllocator allocator = new DirectBufferAllocator();
-    NullableInt32Vector a = new NullableInt32Vector(0, allocator);
-    NullableInt32Vector b = new NullableInt32Vector(1, allocator);
-    UInt16Vector select = new UInt16Vector(2, allocator);
-    a.allocateNew(size);
-    b.allocateNew(size);
-    select.allocateNew(size);
-    int r = 0;
-    for (int i = 0; i < 1500; i++) {
-      r += applyToBatch(size, a, b, select);
-    }
-
-    System.out.println(r);
-  }
-  
-public static void main(String[] args) throws Exception{
-  ExpressionEvaluator ee = new ExpressionEvaluator(
-      "c > d ? c : d",                     // expression
-      int.class,                           // expressionType
-      new String[] { "c", "d" },           // parameterNames
-      new Class[] { int.class, int.class } // parameterTypes
-  );
-  
-  Integer res = (Integer) ee.evaluate(
-      new Object[] {          // parameterValues
-          new Integer(10),
-          new Integer(11),
-      }
-  );
-  System.out.println("res = " + res);
-}
-}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/opt/IdentityOptimizer.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/opt/IdentityOptimizer.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/opt/IdentityOptimizer.java
index 70a42be..d2aaca3 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/opt/IdentityOptimizer.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/opt/IdentityOptimizer.java
@@ -19,8 +19,7 @@ package org.apache.drill.exec.opt;
 
 import org.apache.drill.common.config.DrillConfig;
 import org.apache.drill.common.logical.LogicalPlan;
-import org.apache.drill.common.optimize.Optimizer;
-import org.apache.drill.common.physical.PhysicalPlan;
+import org.apache.drill.exec.physical.PhysicalPlan;
 
 public class IdentityOptimizer extends Optimizer {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(IdentityOptimizer.class);

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/opt/Optimizer.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/opt/Optimizer.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/opt/Optimizer.java
new file mode 100644
index 0000000..9f506c1
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/opt/Optimizer.java
@@ -0,0 +1,45 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.opt;
+
+import java.io.Closeable;
+
+import org.apache.drill.common.config.DrillConfig;
+import org.apache.drill.common.exceptions.DrillConfigurationException;
+import org.apache.drill.common.logical.LogicalPlan;
+import org.apache.drill.exec.physical.PhysicalPlan;
+
+public abstract class Optimizer implements Closeable{
+  
+  public static String OPTIMIZER_IMPL_KEY = "drill.exec.optimizer.implementation";
+  
+  public abstract void init(DrillConfig config);
+  
+  public abstract PhysicalPlan optimize(OptimizationContext context, LogicalPlan plan);
+  public abstract void close();
+  
+  public static Optimizer getOptimizer(DrillConfig config) throws DrillConfigurationException{
+    Optimizer o = config.getInstanceOf(OPTIMIZER_IMPL_KEY, Optimizer.class);
+    o.init(config);
+    return o;
+  }
+  
+  public interface OptimizationContext{
+    public int getPriority();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/DataValidationMode.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/DataValidationMode.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/DataValidationMode.java
new file mode 100644
index 0000000..334119d
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/DataValidationMode.java
@@ -0,0 +1,24 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.physical;
+
+public enum DataValidationMode {
+  TERMINATE, // terminate the query if the data doesn't match expected.
+  DROP_RECORD, // drop the record that doesn't match the expected situation.
+  SINK_RECORD // record the failed record along with the rule violation in a secondary location.
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/EndpointAffinity.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/EndpointAffinity.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/EndpointAffinity.java
new file mode 100644
index 0000000..d7b21db
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/EndpointAffinity.java
@@ -0,0 +1,60 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.physical;
+
+import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
+
+
+public class EndpointAffinity implements Comparable<EndpointAffinity>{
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(EndpointAffinity.class);
+  
+  private DrillbitEndpoint endpoint;
+  private float affinity = 0.0f;
+  
+  public EndpointAffinity(DrillbitEndpoint endpoint) {
+    super();
+    this.endpoint = endpoint;
+  }
+  
+  public EndpointAffinity(DrillbitEndpoint endpoint, float affinity) {
+    super();
+    this.endpoint = endpoint;
+    this.affinity = affinity;
+  }
+
+  public DrillbitEndpoint getEndpoint() {
+    return endpoint;
+  }
+  public void setEndpoint(DrillbitEndpoint endpoint) {
+    this.endpoint = endpoint;
+  }
+  public float getAffinity() {
+    return affinity;
+  }
+  
+  @Override
+  public int compareTo(EndpointAffinity o) {
+    return Float.compare(affinity, o.affinity);
+  }
+  
+  public void addAffinity(float f){
+    affinity += f;
+  }
+  
+  
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/OperatorCost.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/OperatorCost.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/OperatorCost.java
new file mode 100644
index 0000000..ebe6446
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/OperatorCost.java
@@ -0,0 +1,66 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.physical;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+
+public class OperatorCost {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(OperatorCost.class);
+  
+  private final float network; 
+  private final float disk;
+  private final float memory;
+  private final float cpu;
+  
+  
+  
+  @JsonCreator
+  public OperatorCost(@JsonProperty("network") float network, @JsonProperty("disk") float disk, @JsonProperty("memory") float memory, @JsonProperty("cpu") float cpu) {
+    super();
+    this.network = network;
+    this.disk = disk;
+    this.memory = memory;
+    this.cpu = cpu;
+  }
+
+  public float getNetwork() {
+    return network;
+  }
+
+  public float getDisk() {
+    return disk;
+  }
+
+  public float getMemory() {
+    return memory;
+  }
+
+  public float getCpu() {
+    return cpu;
+  }
+  
+  public static OperatorCost combine(OperatorCost c1, OperatorCost c2){
+    return new OperatorCost(c1.network + c2.network, c1.disk + c2.disk, c1.memory + c2.memory, c1.cpu + c2.cpu);
+  }
+
+  public OperatorCost add(OperatorCost c2){
+    return combine(this, c2);
+  }
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/PhysicalPlan.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/PhysicalPlan.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/PhysicalPlan.java
new file mode 100644
index 0000000..84bfc87
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/PhysicalPlan.java
@@ -0,0 +1,94 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.physical;
+
+import java.io.IOException;
+import java.util.List;
+
+import org.apache.drill.common.PlanProperties;
+import org.apache.drill.common.config.DrillConfig;
+import org.apache.drill.common.graph.Graph;
+import org.apache.drill.common.graph.GraphAlgos;
+import org.apache.drill.exec.physical.base.Leaf;
+import org.apache.drill.exec.physical.base.PhysicalOperator;
+import org.apache.drill.exec.physical.base.Root;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.annotation.JsonPropertyOrder;
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.ObjectReader;
+import com.fasterxml.jackson.databind.ObjectWriter;
+import com.google.common.collect.Lists;
+
+@JsonPropertyOrder({ "head", "graph" })
+public class PhysicalPlan {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(PhysicalPlan.class);
+  
+  PlanProperties properties;
+  
+  Graph<PhysicalOperator, Root, Leaf> graph;
+  
+  @JsonCreator
+  public PhysicalPlan(@JsonProperty("head") PlanProperties properties, @JsonProperty("graph") List<PhysicalOperator> operators){
+    this.properties = properties;
+    this.graph = Graph.newGraph(operators, Root.class, Leaf.class);
+  }
+  
+  @JsonProperty("graph")
+  public List<PhysicalOperator> getSortedOperators(){
+    // reverse the list so that nested references are flattened rather than nested.
+    return getSortedOperators(true);
+  }
+  
+  public List<PhysicalOperator> getSortedOperators(boolean reverse){
+    List<PhysicalOperator> list = GraphAlgos.TopoSorter.sort(graph);
+    if(reverse){
+      return Lists.reverse(list);
+    }else{
+      return list;
+    }
+    
+  }
+
+
+  @JsonProperty("head")
+  public PlanProperties getProperties() {
+    return properties;
+  }
+
+  /** Parses a physical plan. */
+  public static PhysicalPlan parse(ObjectReader reader, String planString) {
+    try {
+      PhysicalPlan plan = reader.readValue(planString);
+      return plan;
+    } catch (IOException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  /** Converts a physical plan to a string. (Opposite of {@link #parse}.) */
+  public String unparse(ObjectWriter writer) {
+    try {
+      return writer.writeValueAsString(this);
+    } catch (JsonProcessingException e) {
+      throw new RuntimeException(e);
+    }
+  }
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/ReadEntry.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/ReadEntry.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/ReadEntry.java
new file mode 100644
index 0000000..02fe025
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/ReadEntry.java
@@ -0,0 +1,33 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.physical;
+
+import org.apache.drill.exec.physical.base.Size;
+
+import com.fasterxml.jackson.annotation.JsonIgnore;
+
+/**
+ * Describes a chunk of read work that will be done.
+ */
+public interface ReadEntry {
+  @JsonIgnore
+  public OperatorCost getCost();
+
+  @JsonIgnore
+  public Size getSize();
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/RecordField.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/RecordField.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/RecordField.java
new file mode 100644
index 0000000..db3390a
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/RecordField.java
@@ -0,0 +1,60 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.physical;
+
+import org.apache.drill.common.expression.types.DataType;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+
+public class RecordField {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(RecordField.class);
+
+  
+  private DataType type;
+  private ValueMode mode;
+  
+  @JsonCreator
+  public RecordField(@JsonProperty("type") DataType type, @JsonProperty("mode") ValueMode mode) {
+    super();
+    this.type = type;
+    this.mode = mode;
+  }
+
+  public DataType getType() {
+    return type;
+  }
+
+  public ValueMode getMode() {
+    return mode;
+  }
+  
+  public static enum ValueMode {
+    VECTOR,
+    DICT,
+    RLE
+  }
+  
+  public static enum ValueType {
+    OPTIONAL,
+    REQUIRED, 
+    REPEATED
+  }
+  
+  
+}

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

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractBase.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractBase.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractBase.java
new file mode 100644
index 0000000..e91257e
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractBase.java
@@ -0,0 +1,43 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.physical.base;
+
+import org.apache.drill.common.graph.GraphVisitor;
+import org.apache.drill.exec.physical.OperatorCost;
+
+public abstract class AbstractBase implements PhysicalOperator{
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(AbstractBase.class);
+
+
+
+  @Override
+  public void accept(GraphVisitor<PhysicalOperator> visitor) {
+    visitor.enter(this);
+    if(this.iterator() == null) throw new IllegalArgumentException("Null iterator for pop." + this);
+    for(PhysicalOperator o : this){
+      o.accept(visitor);  
+    }
+    visitor.leave(this);
+  }
+  
+  @Override
+  public boolean isExecutable() {
+    return true;
+  }
+  
+}


[20/53] [abbrv] Clean up threading of client/server. Utilize command pattern for BitCom stuff to abstract away connection failures. Works on one bit single exchange remote query now. Next up, two bit single exchange query.

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/b8db98ad/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/WorkManager.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/WorkManager.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/WorkManager.java
index d3664a0..1170a1e 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/WorkManager.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/WorkManager.java
@@ -33,6 +33,7 @@ import org.apache.drill.exec.coord.ClusterCoordinator;
 import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
 import org.apache.drill.exec.proto.ExecProtos.FragmentHandle;
 import org.apache.drill.exec.proto.UserBitShared.QueryId;
+import org.apache.drill.exec.rpc.NamedThreadFactory;
 import org.apache.drill.exec.rpc.bit.BitCom;
 import org.apache.drill.exec.server.BootStrapContext;
 import org.apache.drill.exec.server.DrillbitContext;
@@ -40,7 +41,6 @@ import org.apache.drill.exec.work.batch.BitComHandler;
 import org.apache.drill.exec.work.batch.BitComHandlerImpl;
 import org.apache.drill.exec.work.foreman.Foreman;
 import org.apache.drill.exec.work.fragment.IncomingFragmentHandler;
-import org.apache.drill.exec.work.fragment.RemoteFragmentHandler;
 import org.apache.drill.exec.work.user.UserWorker;
 
 import com.google.common.collect.Maps;
@@ -63,7 +63,7 @@ public class WorkManager implements Closeable{
   private final BitComHandler bitComWorker;
   private final UserWorker userWorker;
   private final WorkerBee bee;
-  private Executor executor = Executors.newFixedThreadPool(4);
+  private Executor executor = Executors.newFixedThreadPool(4, new NamedThreadFactory("Working Thread - "));
   private final EventThread eventThread;
   
   public WorkManager(BootStrapContext context){
@@ -148,9 +148,10 @@ public class WorkManager implements Closeable{
   public void run() {
     try {
     while(true){
-      logger.debug("Checking for pending work tasks.");
+      logger.debug("Polling for pending work tasks.");
       Runnable r = pendingTasks.poll(10, TimeUnit.SECONDS);
       if(r != null){
+        logger.debug("Starting pending task {}", r);
         executor.execute(r);  
       }
       

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/b8db98ad/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/AbstractFragmentCollector.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/AbstractFragmentCollector.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/AbstractFragmentCollector.java
index 5dacb71..ec03392 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/AbstractFragmentCollector.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/AbstractFragmentCollector.java
@@ -67,18 +67,23 @@ public abstract class AbstractFragmentCollector implements BatchCollector{
   
   public abstract void streamFinished(int minorFragmentId);
   
-  public void batchArrived(ConnectionThrottle throttle, int minorFragmentId, RawFragmentBatch batch) {
+  public boolean batchArrived(ConnectionThrottle throttle, int minorFragmentId, RawFragmentBatch batch) {
+    boolean decremented = false;
     if (remainders.compareAndSet(minorFragmentId, 0, 1)) {
       int rem = remainingRequired.decrementAndGet();
       if (rem == 0) {
         parentAccounter.decrementAndGet();
+        decremented = true;
       }
     }
     if(batch.getHeader().getIsLastBatch()){
       streamFinished(minorFragmentId);
     }
     getBuffer(minorFragmentId).enqueue(throttle, batch);
+    return decremented;
   }
 
+  
+
   protected abstract RawBatchBuffer getBuffer(int minorFragmentId);
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/b8db98ad/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/BatchCollector.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/BatchCollector.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/BatchCollector.java
index ff091d7..b5a497e 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/BatchCollector.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/BatchCollector.java
@@ -24,8 +24,7 @@ import org.apache.drill.exec.rpc.RemoteConnection.ConnectionThrottle;
 
 interface BatchCollector {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(BatchCollector.class);
-
-  public void batchArrived(ConnectionThrottle throttle, int minorFragmentId, RawFragmentBatch batch);
+  public boolean batchArrived(ConnectionThrottle throttle, int minorFragmentId, RawFragmentBatch batch);
   public int getOppositeMajorFragmentId();
   public RawBatchBuffer[] getBuffers();
 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/b8db98ad/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 9b227da..edda714 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
@@ -159,14 +159,11 @@ public class BitComHandlerImpl implements BitComHandler {
     // Create a handler if there isn't already one.
     if(handler == null){
       
-      
-      
       PlanFragment fragment = bee.getContext().getCache().getFragment(handle);
       if(fragment == null){
         logger.error("Received batch where fragment was not in cache.");
         return Acks.FAIL;
       }
-      
 
       IncomingFragmentHandler newHandler = new RemoteFragmentHandler(fragment, bee.getContext(), bee.getContext().getBitCom().getTunnel(fragment.getForeman()));
       
@@ -174,7 +171,7 @@ public class BitComHandlerImpl implements BitComHandler {
       handler = handlers.putIfAbsent(fragment.getHandle(), newHandler);
           
       if(handler == null){
-        // we added a handler, inform foreman that we did so.  This way, the foreman can track status.  We also tell foreman that we don't need inform ourself.
+        // we added a handler, inform the bee that we did so.  This way, the foreman can track status. 
         bee.addFragmentPendingRemote(newHandler);
         handler = newHandler;
       }
@@ -182,10 +179,12 @@ public class BitComHandlerImpl implements BitComHandler {
     
     boolean canRun = handler.handle(connection.getConnectionThrottle(), new RawFragmentBatch(fragmentBatch, body));
     if(canRun){
+      logger.debug("Arriving batch means local batch can run, starting local batch.");
       // if we've reached the canRun threshold, we'll proceed.  This expects handler.handle() to only return a single true.
       bee.startFragmentPendingRemote(handler);
     }
-    if(handler.isDone()){
+    if(fragmentBatch.getIsLastBatch() && !handler.isWaiting()){
+      logger.debug("Removing handler.  Is Last Batch {}.  Is Waiting for more {}", fragmentBatch.getIsLastBatch(), handler.isWaiting());
       handlers.remove(handler.getHandle());
     }
     

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/b8db98ad/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/IncomingBuffers.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/IncomingBuffers.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/IncomingBuffers.java
index 20775c5..264c4b9 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/IncomingBuffers.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/IncomingBuffers.java
@@ -17,6 +17,7 @@
  ******************************************************************************/
 package org.apache.drill.exec.work.batch;
 
+import java.util.List;
 import java.util.Map;
 import java.util.concurrent.atomic.AtomicInteger;
 
@@ -28,6 +29,7 @@ import org.apache.drill.exec.record.RawFragmentBatch;
 import org.apache.drill.exec.rpc.RemoteConnection.ConnectionThrottle;
 
 import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
 
 /**
@@ -42,7 +44,10 @@ public class IncomingBuffers {
 
   public IncomingBuffers(PhysicalOperator root) {
     Map<Integer, BatchCollector> counts = Maps.newHashMap();
-    root.accept(new CountRequiredFragments(), counts);
+    CountRequiredFragments reqFrags = new CountRequiredFragments();
+    root.accept(reqFrags, counts);
+    
+    logger.debug("Came up with a list of {} required fragments.  Fragments {}", remainingRequired.get(), counts);
     streamsRemaining.set(remainingRequired.get());
     fragCounts = ImmutableMap.copyOf(counts);
   }
@@ -53,11 +58,13 @@ public class IncomingBuffers {
     if(batch.getHeader().getIsLastBatch()){
       streamsRemaining.decrementAndGet();
     }
+    int sendMajorFragmentId = batch.getHeader().getSendingMajorFragmentId();
+    BatchCollector fSet = fragCounts.get(sendMajorFragmentId);
+    if (fSet == null) throw new FragmentSetupException(String.format("We received a major fragment id that we were not expecting.  The id was %d.", sendMajorFragmentId));
+    boolean decremented = fSet.batchArrived(throttle, batch.getHeader().getSendingMinorFragmentId(), batch);
     
-    BatchCollector fSet = fragCounts.get(batch.getHeader().getSendingMajorFragmentId());
-    if (fSet == null) throw new FragmentSetupException("We received a major fragment id that we were not expecting.");
-    fSet.batchArrived(throttle, batch.getHeader().getSendingMinorFragmentId(), batch);
-    return remainingRequired.get() == 0;
+    // we should only return true if remaining required has been decremented and is currently equal to zero.
+    return decremented && remainingRequired.get() == 0;
   }
 
   public int getRemainingRequired() {
@@ -75,7 +82,7 @@ public class IncomingBuffers {
    * Designed to setup initial values for arriving fragment accounting.
    */
   public class CountRequiredFragments extends AbstractPhysicalVisitor<Void, Map<Integer, BatchCollector>, RuntimeException> {
-
+    
     @Override
     public Void visitReceiver(Receiver receiver, Map<Integer, BatchCollector> counts) throws RuntimeException {
       BatchCollector set;
@@ -84,7 +91,7 @@ public class IncomingBuffers {
       } else {
         set = new PartitionedCollector(remainingRequired, receiver);
       }
-
+      
       counts.put(set.getOppositeMajorFragmentId(), set);
       remainingRequired.incrementAndGet();
       return null;

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/b8db98ad/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/MergingCollector.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/MergingCollector.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/MergingCollector.java
index e21d69a..93868a7 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/MergingCollector.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/MergingCollector.java
@@ -27,7 +27,7 @@ public class MergingCollector extends AbstractFragmentCollector{
   
   public MergingCollector(AtomicInteger parentAccounter, Receiver receiver) {
     super(parentAccounter, receiver, 1);
-    streamsRunning = new AtomicInteger(parentAccounter.get());
+    streamsRunning = new AtomicInteger(receiver.getProvidingEndpoints().size());
   }
 
   @Override
@@ -35,10 +35,11 @@ public class MergingCollector extends AbstractFragmentCollector{
     return buffers[0];
   }
 
-  @Override
+  
   public void streamFinished(int minorFragmentId) {
     if(streamsRunning.decrementAndGet() == 0) buffers[0].finished();
   }
+
   
   
 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/b8db98ad/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/PartitionedCollector.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/PartitionedCollector.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/PartitionedCollector.java
index 116ca26..25b5884 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/PartitionedCollector.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/PartitionedCollector.java
@@ -36,6 +36,7 @@ public class PartitionedCollector extends AbstractFragmentCollector{
   public void streamFinished(int minorFragmentId) {
     buffers[minorFragmentId].finished();
   }
+
   
   
 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/b8db98ad/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/UnlmitedRawBatchBuffer.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/UnlmitedRawBatchBuffer.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/UnlmitedRawBatchBuffer.java
index f97d878..71ae576 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/UnlmitedRawBatchBuffer.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/UnlmitedRawBatchBuffer.java
@@ -65,7 +65,7 @@ public class UnlmitedRawBatchBuffer implements RawBatchBuffer{
       }
     }
     
-    return null;
+    return b;
     
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/b8db98ad/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 dea8282..f86c4fb 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
@@ -24,12 +24,9 @@ import java.util.List;
 import org.apache.drill.common.exceptions.ExecutionSetupException;
 import org.apache.drill.common.logical.LogicalPlan;
 import org.apache.drill.exec.exception.FragmentSetupException;
-import org.apache.drill.exec.ops.FragmentContext;
 import org.apache.drill.exec.ops.QueryContext;
 import org.apache.drill.exec.physical.PhysicalPlan;
 import org.apache.drill.exec.physical.base.PhysicalOperator;
-import org.apache.drill.exec.physical.impl.ImplCreator;
-import org.apache.drill.exec.physical.impl.RootExec;
 import org.apache.drill.exec.physical.impl.materialize.QueryWritableBatch;
 import org.apache.drill.exec.planner.fragment.Fragment;
 import org.apache.drill.exec.planner.fragment.MakeFragmentsVisitor;
@@ -45,8 +42,8 @@ import org.apache.drill.exec.proto.UserProtos.QueryResult;
 import org.apache.drill.exec.proto.UserProtos.QueryResult.QueryState;
 import org.apache.drill.exec.proto.UserProtos.RequestResults;
 import org.apache.drill.exec.proto.UserProtos.RunQuery;
+import org.apache.drill.exec.rpc.BaseRpcOutcomeListener;
 import org.apache.drill.exec.rpc.RpcException;
-import org.apache.drill.exec.rpc.RpcOutcomeListener;
 import org.apache.drill.exec.rpc.user.UserServer.UserClientConnection;
 import org.apache.drill.exec.server.DrillbitContext;
 import org.apache.drill.exec.util.AtomicState;
@@ -126,14 +123,13 @@ public class Foreman implements Runnable, Closeable, Comparable<Object>{
   
   void cleanupAndSendResult(QueryResult result){
     bee.retireForeman(this);
-    initiatingClient.sendResult(new QueryWritableBatch(result)).addLightListener(new ResponseSendListener());
+    initiatingClient.sendResult(new ResponseSendListener(), new QueryWritableBatch(result));
   }
 
-  private class ResponseSendListener extends RpcOutcomeListener<Ack> {
+  private class ResponseSendListener extends BaseRpcOutcomeListener<Ack> {
     @Override
     public void failed(RpcException ex) {
-      logger
-          .info(
+      logger.info(
               "Failure while trying communicate query result to initating client.  This would happen if a client is disconnected before response notice can be sent.",
               ex);
     }
@@ -193,12 +189,17 @@ public class Foreman implements Runnable, Closeable, Comparable<Object>{
       fail("Failure while fragmenting query.", e);
       return;
     }
+    
+    
+
+    
     PlanningSet planningSet = StatsCollector.collectStats(rootFragment);
     SimpleParallelizer parallelizer = new SimpleParallelizer();
 
     try {
       QueryWorkUnit work = parallelizer.getFragments(context.getCurrentEndpoint(), queryId, context.getActiveEndpoints(), context.getPlanReader(), rootFragment, planningSet, 10);
 
+      this.context.getBitCom().getListeners().addFragmentStatusListener(work.getRootFragment().getHandle(), fragmentManager);
       List<PlanFragment> leafFragments = Lists.newArrayList();
 
       // store fragments in distributed grid.
@@ -213,7 +214,7 @@ public class Foreman implements Runnable, Closeable, Comparable<Object>{
       fragmentManager.runFragments(bee, work.getRootFragment(), work.getRootOperator(), initiatingClient, leafFragments);
 
     
-    } catch (ExecutionSetupException e) {
+    } catch (ExecutionSetupException | RpcException e) {
       fail("Failure while setting up query.", e);
     }
 
@@ -245,9 +246,6 @@ public class Foreman implements Runnable, Closeable, Comparable<Object>{
     return this.state.getState();
   }
 
-  public boolean rootCoorespondsTo(FragmentHandle handle){
-    throw new UnsupportedOperationException();
-  }
   
   class ForemanManagerListener{
     void fail(String message, Throwable t) {

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/b8db98ad/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 20797b8..f069db7 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
@@ -64,12 +64,13 @@ class RunningFragmentManager implements FragmentStatusListener{
     this.foreman = foreman;
     this.tun = tun;
     this.remainingFragmentCount = new AtomicInteger(0);
+    
   }
 
   public void runFragments(WorkerBee bee, PlanFragment rootFragment, FragmentRoot rootOperator, UserClientConnection rootClient, List<PlanFragment> leafFragments) throws ExecutionSetupException{
     remainingFragmentCount.set(leafFragments.size()+1);
 
-    // set up the root framgnet first so we'll have incoming buffers available.
+    // set up the root fragment first so we'll have incoming buffers available.
     {
       IncomingBuffers buffers = new IncomingBuffers(rootOperator);
       
@@ -97,13 +98,13 @@ class RunningFragmentManager implements FragmentStatusListener{
   private void sendRemoteFragment(PlanFragment fragment){
     map.put(fragment.getHandle(), new FragmentData(fragment.getHandle(), fragment.getAssignment(), false));
     FragmentSubmitListener listener = new FragmentSubmitListener(fragment.getAssignment(), fragment);
-    tun.get(fragment.getAssignment()).sendFragment(fragment).addLightListener(listener);
+    tun.get(fragment.getAssignment()).sendFragment(listener, fragment);
   }
   
   
   @Override
   public void statusUpdate(FragmentStatus status) {
-    
+    logger.debug("New fragment status was provided to Foreman of {}", status);
     switch(status.getState()){
     case AWAITING_ALLOCATION:
       updateStatus(status);
@@ -205,6 +206,7 @@ class RunningFragmentManager implements FragmentStatusListener{
 
     @Override
     public void failed(RpcException ex) {
+      logger.debug("Failure while sending fragment.  Stopping query.", ex);
       stopQuery();
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/b8db98ad/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/IncomingFragmentHandler.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/IncomingFragmentHandler.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/IncomingFragmentHandler.java
index b4e9308..b23f003 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/IncomingFragmentHandler.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/IncomingFragmentHandler.java
@@ -44,6 +44,6 @@ public interface IncomingFragmentHandler {
   public abstract FragmentRunner getRunnable();
 
   public abstract void cancel();
-  public boolean isDone();
+  public boolean isWaiting();
   public abstract FragmentHandle getHandle();
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/b8db98ad/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/LocalFragmentHandler.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/LocalFragmentHandler.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/LocalFragmentHandler.java
index 3f710ed..5ffd09a 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/LocalFragmentHandler.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/LocalFragmentHandler.java
@@ -60,8 +60,8 @@ public class LocalFragmentHandler implements IncomingFragmentHandler{
   }
 
   @Override
-  public boolean isDone() {
-    return cancel || isDone();
+  public boolean isWaiting() {
+    return !buffers.isDone() && !cancel;
   }
   
   

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/b8db98ad/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/RemoteFragmentHandler.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/RemoteFragmentHandler.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/RemoteFragmentHandler.java
index 70d7e93..4a5dbf2 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/RemoteFragmentHandler.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/RemoteFragmentHandler.java
@@ -113,8 +113,8 @@ public class RemoteFragmentHandler implements IncomingFragmentHandler {
   }
 
   @Override
-  public boolean isDone() {
-    return cancel || buffers.isDone();
+  public boolean isWaiting() {
+    return !buffers.isDone() && !cancel;
   }
   
   

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/b8db98ad/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/DistributedFragmentRun.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/DistributedFragmentRun.java b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/DistributedFragmentRun.java
index 7c6bfe5..586ccf6 100644
--- a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/DistributedFragmentRun.java
+++ b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/DistributedFragmentRun.java
@@ -17,6 +17,8 @@
  ******************************************************************************/
 package org.apache.drill.exec.physical.impl;
 
+import static org.junit.Assert.*;
+
 import java.util.List;
 
 import org.apache.drill.common.util.FileUtils;
@@ -26,26 +28,29 @@ import org.apache.drill.exec.proto.UserProtos.QueryType;
 import org.apache.drill.exec.rpc.user.QueryResultBatch;
 import org.apache.drill.exec.server.Drillbit;
 import org.apache.drill.exec.server.RemoteServiceSet;
-import org.junit.Ignore;
 import org.junit.Test;
 
 import com.google.common.base.Charsets;
 import com.google.common.io.Files;
 
-@Ignore
+//@Ignore
 public class DistributedFragmentRun extends PopUnitTestBase{
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DistributedFragmentRun.class);
   
   
   @Test 
-  public void simpleDistributedQuery() throws Exception{
+  public void oneBitOneExchangeRun() throws Exception{
     RemoteServiceSet serviceSet = RemoteServiceSet.getLocalServiceSet();
-    try(Drillbit bit1 = new Drillbit(CONFIG, serviceSet); Drillbit bit2 = new Drillbit(CONFIG, serviceSet); DrillClient client = new DrillClient(CONFIG, serviceSet.getCoordinator());){
+
+    try(Drillbit bit1 = new Drillbit(CONFIG, serviceSet); DrillClient client = new DrillClient(CONFIG, serviceSet.getCoordinator());){
       bit1.run();
-      bit2.run();
       client.connect();
       List<QueryResultBatch> results = client.runQuery(QueryType.PHYSICAL, Files.toString(FileUtils.getResourceAsFile("/physical_single_exchange.json"), Charsets.UTF_8));
-      System.out.println(results);
+      int count = 0;
+      for(QueryResultBatch b : results){
+        count += b.getHeader().getRowCount();
+      }
+      assertEquals(100, count);
     }
     
 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/b8db98ad/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/pop/CheckFragmenter.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/pop/CheckFragmenter.java b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/pop/CheckFragmenter.java
index 7b7ab8e..1e0c5b6 100644
--- a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/pop/CheckFragmenter.java
+++ b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/pop/CheckFragmenter.java
@@ -23,28 +23,12 @@ import static org.junit.Assert.assertNull;
 
 import java.io.IOException;
 
-import org.apache.drill.common.config.DrillConfig;
-import org.apache.drill.common.util.FileUtils;
 import org.apache.drill.exec.exception.FragmentSetupException;
-import org.apache.drill.exec.ops.QueryContext;
-import org.apache.drill.exec.physical.PhysicalPlan;
-import org.apache.drill.exec.physical.base.PhysicalOperator;
-import org.apache.drill.exec.planner.fragment.Fragment;
-import org.apache.drill.exec.planner.fragment.PlanningSet;
-import org.apache.drill.exec.planner.fragment.StatsCollector;
-import org.apache.drill.exec.planner.fragment.MakeFragmentsVisitor;
-import org.apache.drill.exec.planner.fragment.Fragment.ExchangeFragmentPair;
 import org.apache.drill.exec.planner.PhysicalPlanReader;
-import org.apache.drill.exec.planner.SimpleExecPlanner;
+import org.apache.drill.exec.planner.fragment.Fragment;
 import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
-import org.apache.drill.exec.server.DrillbitContext;
-import org.apache.drill.exec.work.QueryWorkUnit;
-import org.junit.BeforeClass;
 import org.junit.Test;
 
-import com.google.common.base.Charsets;
-import com.google.common.io.Files;
-
 public class CheckFragmenter extends PopUnitTestBase {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(CheckFragmenter.class);
 
@@ -77,10 +61,11 @@ public class CheckFragmenter extends PopUnitTestBase {
     assertNotNull(b.getSendingExchange());
   }
 
-  
 
 
   
+
+  
   
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/b8db98ad/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/pop/FragmentChecker.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/pop/FragmentChecker.java b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/pop/FragmentChecker.java
index 6f229a3..e1db639 100644
--- a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/pop/FragmentChecker.java
+++ b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/pop/FragmentChecker.java
@@ -17,16 +17,15 @@
  ******************************************************************************/
 package org.apache.drill.exec.pop;
 
-import static org.junit.Assert.*;
+import static org.junit.Assert.assertEquals;
 
-import java.util.Collection;
-import java.util.Collections;
+import java.util.List;
 
 import org.apache.drill.exec.planner.PhysicalPlanReader;
 import org.apache.drill.exec.planner.fragment.Fragment;
 import org.apache.drill.exec.planner.fragment.PlanningSet;
-import org.apache.drill.exec.planner.fragment.StatsCollector;
 import org.apache.drill.exec.planner.fragment.SimpleParallelizer;
+import org.apache.drill.exec.planner.fragment.StatsCollector;
 import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
 import org.apache.drill.exec.proto.ExecProtos.PlanFragment;
 import org.apache.drill.exec.proto.UserBitShared.QueryId;
@@ -41,26 +40,46 @@ public class FragmentChecker extends PopUnitTestBase{
   
   @Test
   public void checkSimpleExchangePlan() throws Exception{
+    print("/physical_simpleexchange.json", 2, 3);
+
+  }
+  
+  
+  private void print(String fragmentFile, int bitCount, int exepectedFragmentCount) throws Exception{
     
+    System.out.println(String.format("=================Building plan fragments for [%s].  Allowing %d total Drillbits.==================", fragmentFile, bitCount));
     PhysicalPlanReader ppr = new PhysicalPlanReader(CONFIG, CONFIG.getMapper(), DrillbitEndpoint.getDefaultInstance());
-    Fragment fragmentRoot = getRootFragment(ppr, "/physical_simpleexchange.json");
+    Fragment fragmentRoot = getRootFragment(ppr, fragmentFile);
     PlanningSet planningSet = StatsCollector.collectStats(fragmentRoot);
     SimpleParallelizer par = new SimpleParallelizer();
+    List<DrillbitEndpoint> endpoints = Lists.newArrayList();
+    DrillbitEndpoint localBit = null;
+    for(int i =0; i < bitCount; i++){
+      DrillbitEndpoint b1 = DrillbitEndpoint.newBuilder().setAddress("localhost").setBitPort(1234+i).build();
+      if(i ==0) localBit = b1; 
+      endpoints.add(b1);
+    }
     
-    DrillbitEndpoint b1 = DrillbitEndpoint.newBuilder().setAddress("localhost").setBitPort(1234).build();
-    DrillbitEndpoint b2 = DrillbitEndpoint.newBuilder().setAddress("localhost").setBitPort(2345).build();
     
-    QueryWorkUnit qwu = par.getFragments(b1, QueryId.getDefaultInstance(), Lists.newArrayList(b1, b2), ppr, fragmentRoot, planningSet, 10);
-    assertEquals(qwu.getFragments().size(), 3);
-    System.out.println("=========ROOT FRAGMENT=========");
+    QueryWorkUnit qwu = par.getFragments(localBit, QueryId.getDefaultInstance(), endpoints, ppr, fragmentRoot, planningSet, 10);
+    System.out.println(String.format("=========ROOT FRAGMENT [%d:%d] =========", qwu.getRootFragment().getHandle().getMajorFragmentId(), qwu.getRootFragment().getHandle().getMinorFragmentId()));
+    
     System.out.print(qwu.getRootFragment().getFragmentJson());
     
     
     for(PlanFragment f : qwu.getFragments()){
-      System.out.println("=========");
+      System.out.println(String.format("=========Fragment [%d:%d]=====", f.getHandle().getMajorFragmentId(), f.getHandle().getMinorFragmentId()));
       System.out.print(f.getFragmentJson());
     }
+    
+    //assertEquals(exepectedFragmentCount, qwu.getFragments().size());
+
     logger.debug("Planning Set {}", planningSet);
+  }
+  
+  @Test
+  public void validateSingleExchangeFragment() throws Exception{
+    print("/physical_single_exchange.json", 1, 2);
 
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/b8db98ad/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/server/TestBitRpc.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/server/TestBitRpc.java b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/server/TestBitRpc.java
index 9684e9f..038b093 100644
--- a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/server/TestBitRpc.java
+++ b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/server/TestBitRpc.java
@@ -19,27 +19,25 @@ package org.apache.drill.exec.server;
 
 import io.netty.buffer.ByteBuf;
 
-import java.util.concurrent.ConcurrentMap;
-
 import org.apache.drill.common.config.DrillConfig;
 import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
 import org.apache.drill.exec.proto.ExecProtos.FragmentHandle;
+import org.apache.drill.exec.proto.ExecProtos.FragmentStatus;
 import org.apache.drill.exec.proto.ExecProtos.PlanFragment;
 import org.apache.drill.exec.proto.GeneralRPCProtos.Ack;
 import org.apache.drill.exec.rpc.Response;
 import org.apache.drill.exec.rpc.RpcException;
-import org.apache.drill.exec.rpc.bit.BitClient;
-import org.apache.drill.exec.rpc.bit.BitComImpl;
 import org.apache.drill.exec.rpc.bit.BitConnection;
+import org.apache.drill.exec.rpc.bit.BitConnectionManager;
 import org.apache.drill.exec.rpc.bit.BitRpcConfig;
 import org.apache.drill.exec.rpc.bit.BitServer;
+import org.apache.drill.exec.rpc.bit.BitTunnel.SendFragmentStatus;
+import org.apache.drill.exec.rpc.bit.ConnectionManagerRegistry;
 import org.apache.drill.exec.rpc.bit.ListenerPool;
 import org.apache.drill.exec.work.batch.BitComHandler;
 import org.apache.drill.exec.work.fragment.IncomingFragmentHandler;
 import org.junit.Test;
 
-import com.google.common.collect.Maps;
-
 public class TestBitRpc {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestBitRpc.class);
   
@@ -47,13 +45,19 @@ public class TestBitRpc {
   public void testBasicConnectionAndHandshake() throws Exception{
     int port = 1234;
     BootStrapContext c = new BootStrapContext(DrillConfig.create());
-    ConcurrentMap<DrillbitEndpoint, BitConnection> registry = Maps.newConcurrentMap();
-    BitServer server = new BitServer(new BitComTestHandler(), c, registry, new ListenerPool(2));
+    final BitComTestHandler handler = new BitComTestHandler();
+    final ListenerPool listeners = new ListenerPool(2);
+    ConnectionManagerRegistry registry = new ConnectionManagerRegistry(handler, c, listeners);
+    BitServer server = new BitServer(handler, c, registry, listeners);
     port = server.bind(port);
+    DrillbitEndpoint ep = DrillbitEndpoint.newBuilder().setAddress("localhost").setBitPort(port).build();
+    registry.setEndpoint(ep);
     for(int i =0; i < 10; i++){
-      BitClient client = new BitClient(DrillbitEndpoint.newBuilder().setAddress("localhost").setBitPort(port).build(), null, new BitComTestHandler(), c, registry, new ListenerPool(2));
-      client.connect();
-      
+      try(BitConnectionManager cm = new BitConnectionManager(ep, ep, handler, c, listeners)){
+        SendFragmentStatus cmd = new SendFragmentStatus(FragmentStatus.getDefaultInstance());
+        cm.runCommand(cmd);
+        cmd.getFuture().checkedGet();
+      }
     }
     System.out.println("connected");
   }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/b8db98ad/sandbox/prototype/exec/java-exec/src/test/resources/physical_single_exchange.json
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/resources/physical_single_exchange.json b/sandbox/prototype/exec/java-exec/src/test/resources/physical_single_exchange.json
index 675ecfb..0e1921e 100644
--- a/sandbox/prototype/exec/java-exec/src/test/resources/physical_single_exchange.json
+++ b/sandbox/prototype/exec/java-exec/src/test/resources/physical_single_exchange.json
@@ -17,7 +17,6 @@
             	  {name: "red", type: "BIGINT", mode: "REQUIRED"},
             	  {name: "green", type: "INT", mode: "REQUIRED"}
             	]}
-            	
             ]
         },
         {


[53/53] [abbrv] git commit: Remove logging fields from annotations. (Okay with 1.7u9, not with 1.7u21)

Posted by ja...@apache.org.
Remove logging fields from annotations.  (Okay with 1.7u9, not with 1.7u21)


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

Branch: refs/heads/master
Commit: 8186b5a1ac8b380234c85a65d54884ab794d7f92
Parents: f001321
Author: Jacques Nadeau <ja...@apache.org>
Authored: Fri Jul 19 18:06:38 2013 -0700
Committer: Jacques Nadeau <ja...@apache.org>
Committed: Fri Jul 19 18:10:02 2013 -0700

----------------------------------------------------------------------
 .../main/java/org/apache/drill/exec/expr/annotations/Output.java | 4 ----
 .../main/java/org/apache/drill/exec/expr/annotations/Param.java  | 1 -
 2 files changed, 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/8186b5a1/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/annotations/Output.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/annotations/Output.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/annotations/Output.java
index 2d77dfc..d45f726 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/annotations/Output.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/annotations/Output.java
@@ -11,8 +11,4 @@ import java.lang.annotation.Target;
 @Retention(RetentionPolicy.RUNTIME)
 @Target({ElementType.FIELD})
 public @interface Output {
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(Output.class);
-
-  
-
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/8186b5a1/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/annotations/Param.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/annotations/Param.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/annotations/Param.java
index 9ba7611..87aac57 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/annotations/Param.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/annotations/Param.java
@@ -14,5 +14,4 @@ import org.apache.drill.common.types.TypeProtos.MinorType;
 @Retention(RetentionPolicy.RUNTIME)
 @Target({ElementType.FIELD})
 public @interface Param {
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(Param.class);
 }


[12/53] [abbrv] Update typing system. Update RPC system. Add Fragmenting Implementation. Working single node. Distributed failing due to threading issues.

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/BitVector.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/BitVector.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/BitVector.java
deleted file mode 100644
index 83ad599..0000000
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/BitVector.java
+++ /dev/null
@@ -1,166 +0,0 @@
-/*******************************************************************************
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- ******************************************************************************/
-package org.apache.drill.exec.record.vector;
-
-import org.apache.drill.common.expression.types.DataType;
-import org.apache.drill.common.physical.RecordField.ValueMode;
-import org.apache.drill.exec.memory.BufferAllocator;
-import org.apache.drill.exec.record.MaterializedField;
-
-/**
- * Describes a vector which holds a number of true/false values.
- */
-public class BitVector extends AbstractFixedValueVector<BitVector> {
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(BitVector.class);
-
-  private final MaterializedField field;
-  
-  public BitVector(int fieldId, BufferAllocator allocator) {
-    super(fieldId, allocator, 1);
-    this.field = new MaterializedField(fieldId, DataType.BOOLEAN, false, ValueMode.VECTOR, this.getClass());
-  }
-
-  @Override
-  public MaterializedField getField() {
-    return field;
-  }
-  
-//  /** Returns true or false for the specified bit index.
-//   * The index should be less than the OpenBitSet size
-//   */
-//  public boolean get(int index) {
-//    assert index >= 0 && index < this.valueCount;
-//    int i = index >> 3;               // div 8
-//    // signed shift will keep a negative index and force an
-//    // array-index-out-of-bounds-exception, removing the need for an explicit check.
-//    int bit = index & 0x3f;           // mod 64
-//    long bitmask = 1L << bit;
-//    return (data.getLong(i) & bitmask) != 0;
-//  }
-  
-  public int getBit(int index) {
-    
-    assert index >= 0 && index < this.valueCount;
-    int i = 8*(index >> 6); // div 8
-    int bit = index & 0x3f; // mod 64
-    return ((int) (data.getLong(i) >>> bit)) & 0x01;
-  }
-  
-  /** Sets the bit at the specified index.
-   * The index should be less than the OpenBitSet size.
-   */
-   public void set(int index) {
-     assert index >= 0 && index < this.valueCount;
-     int wordNum = index >> 3;   
-     int bit = index & 0x3f;
-     long bitmask = 1L << bit;
-     data.setLong(wordNum, data.getLong(wordNum) | bitmask);
-   }
-   
-   public void clear(int index) {
-     assert index >= 0 && index < this.valueCount;
-     int wordNum = index >> 3;
-     int bit = index & 0x03f;
-     long bitmask = 1L << bit;
-     data.setLong(wordNum, data.getLong(wordNum) & ~bitmask);
-   }
-   
-   
-   
-   /** Clears a range of bits.  Clearing past the end does not change the size of the set.
-   *
-   * @param startBitIndex lower index
-   * @param lastBitIndex one-past the last bit to clear
-   */
-  private void clear2(int startBitIndex, int lastBitIndex) {
-    if (lastBitIndex <= startBitIndex) return;
-
-    int firstWordStart = (startBitIndex>>3);
-    if (firstWordStart >= this.longWords) return;
-
-    // since endIndex is one past the end, this is index of the last
-    // word to be changed.
-    int lastWordStart   = ((lastBitIndex-1)>>3);
-
-    long startmask = -1L << startBitIndex;
-    long endmask = -1L >>> -lastBitIndex;  // 64-(endIndex&0x3f) is the same as -endIndex due to wrap
-
-    // invert masks since we are clearing
-    startmask = ~startmask;
-    endmask = ~endmask;
-
-    if (firstWordStart == lastWordStart) {
-      data.setLong(firstWordStart,  data.getLong(firstWordStart) & (startmask | endmask));
-      return;
-    }
-    data.setLong(firstWordStart,  data.getLong(firstWordStart) & startmask);
-
-    int middle = Math.min(this.longWords, lastWordStart);
-    
-    for(int i =firstWordStart+8; i < middle; i += 8){
-      data.setLong(i, 0L);
-    }
-    if (lastWordStart < this.longWords) {
-      data.setLong(lastWordStart,  data.getLong(lastWordStart) & endmask);
-    }
-  }
-  
-  public void setAllFalse(){
-    clear(0, valueCount);
-  }
-
-  
-  public void clear(int startIndex, int endIndex) {
-    if (endIndex <= startIndex) return;
-
-    int startWord = (startIndex >> 6);
-    if (startWord >= longWords) return;
-
-    // since endIndex is one past the end, this is index of the last
-    // word to be changed.
-    int endWord = ((endIndex - 1) >> 6);
-
-    long startmask = -1L << startIndex;
-    long endmask = -1L >>> -endIndex; // 64-(endIndex&0x3f) is the same as -endIndex due to wrap
-
-    // invert masks since we are clearing
-    startmask = ~startmask;
-    endmask = ~endmask;
-    
-    int startWordPos = startWord * 8;
-    if (startWord == endWord) {
-      data.setLong(startWordPos, data.getLong(startWordPos) & (startmask | endmask));
-      return;
-    }
-
-    int endWordPos = endWord * 8;
-
-    data.setLong(startWordPos, data.getLong(startWordPos) & startmask);
-
-    int middle = Math.min(longWords, endWord)*8;
-    
-    
-    for(int i =startWordPos+8; i < middle; i += 8){
-      data.setLong(i, 0L);
-    }
-    
-    if (endWordPos < startWordPos) {
-      data.setLong(endWordPos, data.getLong(endWordPos) & endmask);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/ByteVector.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/ByteVector.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/ByteVector.java
deleted file mode 100644
index d8e1c80..0000000
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/ByteVector.java
+++ /dev/null
@@ -1,48 +0,0 @@
-/*******************************************************************************
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- ******************************************************************************/
-package org.apache.drill.exec.record.vector;
-
-import org.apache.drill.common.expression.types.DataType;
-import org.apache.drill.common.physical.RecordField.ValueMode;
-import org.apache.drill.exec.memory.BufferAllocator;
-import org.apache.drill.exec.record.MaterializedField;
-
-
-public class ByteVector extends AbstractFixedValueVector<ByteVector>{
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ByteVector.class);
-
-  private final MaterializedField field;
-
-  public ByteVector(int fieldId, BufferAllocator allocator) {
-    super(fieldId, allocator, 8);
-    this.field = new MaterializedField(fieldId, DataType.SIGNED_BYTE, false, ValueMode.VECTOR, this.getClass());
-  }
-
-  @Override
-  public MaterializedField getField() {
-    return field;
-  }
-
-  public void setByte(int index, byte b){
-    data.setByte(index, b);
-  }
-
-  public byte getByte(int index){
-    return data.getByte(index);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/Fixed1.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/Fixed1.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/Fixed1.java
new file mode 100644
index 0000000..82c86d1
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/Fixed1.java
@@ -0,0 +1,43 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.record.vector;
+
+import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.record.MaterializedField;
+
+public class Fixed1 extends AbstractFixedValueVector<Fixed1>{
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(Fixed1.class);
+
+  public Fixed1(MaterializedField field, BufferAllocator allocator) {
+    super(field, allocator, 8);
+  }
+  
+  public void setByte(int index, byte b){
+    data.setByte(index, b);
+  }
+
+  public byte getByte(int index){
+    return data.getByte(index);
+  }
+  
+  @Override
+  public Object getObject(int index) {
+    return getByte(index);
+  }
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/Fixed12.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/Fixed12.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/Fixed12.java
new file mode 100644
index 0000000..c5f641a
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/Fixed12.java
@@ -0,0 +1,35 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.record.vector;
+
+import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.record.MaterializedField;
+
+public class Fixed12 extends AbstractFixedValueVector<Fixed12>{
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(Fixed12.class);
+
+  public Fixed12(MaterializedField field, BufferAllocator allocator) {
+    super(field, allocator, 12*8);
+  }
+
+  
+  @Override
+  public Object getObject(int index) {
+    return null;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/Fixed16.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/Fixed16.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/Fixed16.java
new file mode 100644
index 0000000..649832b
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/Fixed16.java
@@ -0,0 +1,37 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.record.vector;
+
+import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.record.MaterializedField;
+
+public class Fixed16 extends AbstractFixedValueVector<Fixed16>{
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(Fixed16.class);
+
+  public Fixed16(MaterializedField field, BufferAllocator allocator) {
+    super(field, allocator, 16*8);
+  }
+
+  @Override
+  public Object getObject(int index) {
+    return null;
+  }
+  
+  
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/Fixed2.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/Fixed2.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/Fixed2.java
new file mode 100644
index 0000000..bd0e313
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/Fixed2.java
@@ -0,0 +1,53 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.record.vector;
+
+import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.record.MaterializedField;
+
+public class Fixed2 extends AbstractFixedValueVector<Fixed2>{
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(Fixed2.class);
+
+  public Fixed2(MaterializedField field, BufferAllocator allocator) {
+    super(field, allocator, 2*8);
+  }
+
+  public final void setSmallInt(int index, short value){
+    index*=2;
+    data.setShort(index, value);
+  }
+  
+  public final short getSmallInt(int index){
+    index*=2;
+    return data.getShort(index);
+  }
+  
+  public final void setUInt2(int index, short value){
+    setSmallInt(index, value);
+  }
+  
+  public final short getUInt2(int index){
+    return getSmallInt(index);
+  }
+  
+  @Override
+  public Object getObject(int index) {
+    return getSmallInt(index);
+  }
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/Fixed4.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/Fixed4.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/Fixed4.java
new file mode 100644
index 0000000..650029d
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/Fixed4.java
@@ -0,0 +1,55 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.record.vector;
+
+import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.record.MaterializedField;
+
+public class Fixed4 extends AbstractFixedValueVector<Fixed4>{
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(Fixed4.class);
+
+  public Fixed4(MaterializedField field, BufferAllocator allocator) {
+    super(field, allocator, 4*8);
+  }
+
+  public final void setInt(int index, int value){
+    index*=4;
+    data.setInt(index, value);
+  }
+  
+  public final int getInt(int index){
+    index*=4;
+    return data.getInt(index);
+  }
+  
+  public final void setFloat4(int index, float value){
+    index*=8;
+    data.setFloat(index, value);
+  }
+  
+  public final float getFloat4(int index){
+    index*=8;
+    return data.getFloat(index);
+  }
+  
+  @Override
+  public Object getObject(int index) {
+    return getInt(index);
+  }
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/Fixed8.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/Fixed8.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/Fixed8.java
new file mode 100644
index 0000000..3629f5c
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/Fixed8.java
@@ -0,0 +1,58 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.record.vector;
+
+import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.record.MaterializedField;
+
+public class Fixed8 extends AbstractFixedValueVector<Fixed8>{
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(Fixed8.class);
+
+  public Fixed8(MaterializedField field, BufferAllocator allocator) {
+    super(field, allocator, 8*8);
+  }
+
+  public final void setBigInt(int index, long value){
+    index*=8;
+    data.setLong(index, value);
+  }
+  
+  public final long getBigInt(int index){
+    index*=8;
+    return data.getLong(index);
+  }
+  
+  public final void setFloat8(int index, double value){
+    index*=8;
+    data.setDouble(index, value);
+  }
+  
+  public final double getFloat8(int index){
+    index*=8;
+    return data.getDouble(index);
+  }
+
+  @Override
+  public Object getObject(int index) {
+    return getBigInt(index);
+  }
+  
+  
+  
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/FixedLen.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/FixedLen.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/FixedLen.java
new file mode 100644
index 0000000..594af23
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/FixedLen.java
@@ -0,0 +1,45 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.record.vector;
+
+import io.netty.buffer.ByteBuf;
+
+import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.record.MaterializedField;
+
+public class FixedLen extends AbstractFixedValueVector<FixedLen>{
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(FixedLen.class);
+
+  
+  public FixedLen(MaterializedField field, BufferAllocator allocator) {
+    super(field, allocator, field.getWidth());
+  }
+
+  public void set(ByteBuf b){
+     
+  }
+  
+  public void get(ByteBuf b){
+    
+  }
+
+  @Override
+  public Object getObject(int index) {
+    return null;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/Int16Vector.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/Int16Vector.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/Int16Vector.java
deleted file mode 100644
index 779b01b..0000000
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/Int16Vector.java
+++ /dev/null
@@ -1,52 +0,0 @@
-/*******************************************************************************
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- ******************************************************************************/
-package org.apache.drill.exec.record.vector;
-
-import org.apache.drill.common.expression.types.DataType;
-import org.apache.drill.common.physical.RecordField.ValueMode;
-import org.apache.drill.exec.memory.BufferAllocator;
-import org.apache.drill.exec.record.MaterializedField;
-
-public final class Int16Vector extends AbstractFixedValueVector<Int16Vector>{
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(Int16Vector.class);
-  
-  private final MaterializedField field;
-
-  public Int16Vector(int fieldId, BufferAllocator allocator) {
-    super(fieldId, allocator, 32);
-    this.field = new MaterializedField(fieldId, DataType.INT16, false, ValueMode.VECTOR, this.getClass());
-  }
-
-  @Override
-  public MaterializedField getField() {
-    return field;
-  }
-
-  public final void set(int index, short value){
-    index*=2;
-    data.setShort(index, value);
-  }
-  
-  public final short get(int index){
-    index*=2;
-    return data.getShort(index);
-  }
-  
-  
-  
-}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/Int32Vector.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/Int32Vector.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/Int32Vector.java
deleted file mode 100644
index d142367..0000000
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/Int32Vector.java
+++ /dev/null
@@ -1,52 +0,0 @@
-/*******************************************************************************
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- ******************************************************************************/
-package org.apache.drill.exec.record.vector;
-
-import org.apache.drill.common.expression.types.DataType;
-import org.apache.drill.common.physical.RecordField.ValueMode;
-import org.apache.drill.exec.memory.BufferAllocator;
-import org.apache.drill.exec.record.MaterializedField;
-
-public final class Int32Vector extends AbstractFixedValueVector<Int32Vector>{
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(Int32Vector.class);
-  
-  private final MaterializedField field;
-
-  public Int32Vector(int fieldId, BufferAllocator allocator) {
-    super(fieldId, allocator, 32);
-    this.field = new MaterializedField(fieldId, DataType.INT32, false, ValueMode.VECTOR, this.getClass());
-  }
-
-  @Override
-  public MaterializedField getField() {
-    return field;
-  }
-
-  public final void set(int index, int value){
-    index*=4;
-    data.setInt(index, value);
-  }
-  
-  public final int get(int index){
-    index*=4;
-    return data.getInt(index);
-  }
-  
-  
-  
-}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/NullableFixed4.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/NullableFixed4.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/NullableFixed4.java
new file mode 100644
index 0000000..cc18538
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/NullableFixed4.java
@@ -0,0 +1,37 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.record.vector;
+
+import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.record.MaterializedField;
+
+public final class NullableFixed4 extends NullableValueVector<NullableFixed4, Fixed4>{
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(NullableFixed4.class);
+
+  public NullableFixed4(MaterializedField field, BufferAllocator allocator) {
+    super(field, allocator, NullableFixed4.class);
+  }
+
+  @Override
+  protected Fixed4 getNewValueVector(BufferAllocator allocator) {
+    return new Fixed4(null, allocator);
+  }
+
+
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/NullableInt32Vector.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/NullableInt32Vector.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/NullableInt32Vector.java
deleted file mode 100644
index 372de13..0000000
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/NullableInt32Vector.java
+++ /dev/null
@@ -1,47 +0,0 @@
-/*******************************************************************************
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- ******************************************************************************/
-package org.apache.drill.exec.record.vector;
-
-import org.apache.drill.exec.memory.BufferAllocator;
-import org.apache.drill.exec.record.MaterializedField;
-
-public final class NullableInt32Vector extends NullableValueVector<NullableInt32Vector, Int32Vector>{
-
-  public NullableInt32Vector(int fieldId, BufferAllocator allocator) {
-    super(fieldId, allocator, NullableInt32Vector.class);
-  }
-
-
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(NullableInt32Vector.class);
-  
-  
-  public int get(int index){
-    return this.value.get(index);
-  }
-  
-  public void set(int index, int value){
-    this.value.set(index, value);
-  }
-
-
-  @Override
-  protected Int32Vector getNewValueVector(int fieldId, BufferAllocator allocator) {
-    return new Int32Vector(fieldId, allocator);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/NullableValueVector.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/NullableValueVector.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/NullableValueVector.java
index 8e714ed..692ab87 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/NullableValueVector.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/NullableValueVector.java
@@ -20,6 +20,7 @@ package org.apache.drill.exec.record.vector;
 import io.netty.buffer.ByteBuf;
 
 import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.proto.UserBitShared.FieldMetadata;
 import org.apache.drill.exec.record.MaterializedField;
 
 /**
@@ -28,18 +29,16 @@ import org.apache.drill.exec.record.MaterializedField;
 abstract class NullableValueVector<T extends NullableValueVector<T, E>, E extends BaseValueVector<E>> extends BaseValueVector<T> {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(NullableValueVector.class);
 
-  protected BitVector bits;
+  protected Bit bits;
   protected E value;
-  private final MaterializedField field;
 
-  public NullableValueVector(int fieldId, BufferAllocator allocator, Class<T> valueClass) {
-    super(fieldId, allocator);
-    bits = new BitVector(fieldId, allocator);
-    value = getNewValueVector(fieldId, allocator);
-    this.field = value.getField().getNullableVersion(valueClass);
+  public NullableValueVector(MaterializedField field, BufferAllocator allocator, Class<T> valueClass) {
+    super(field, allocator);
+    bits = new Bit(null, allocator);
+    value = getNewValueVector(allocator);
   }
   
-  protected abstract E getNewValueVector(int fieldId, BufferAllocator allocator);
+  protected abstract E getNewValueVector(BufferAllocator allocator);
 
   public int isNull(int index){
     return bits.getBit(index);
@@ -76,5 +75,26 @@ abstract class NullableValueVector<T extends NullableValueVector<T, E>, E extend
   }
 
   
+  @Override
+  public ByteBuf[] getBuffers() {
+    return new ByteBuf[]{bits.data, value.data};
+  }
+
+  @Override
+  public void setRecordCount(int recordCount) {
+    super.setRecordCount(recordCount);
+    bits.setRecordCount(recordCount);
+    value.setRecordCount(recordCount);
+  }
+
+  @Override
+  public Object getObject(int index) {
+    if(isNull(index) == 0){
+      return null;
+    }else{
+      return value.getObject(index);
+    }
+  }
+
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/RepeatMap.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/RepeatMap.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/RepeatMap.java
new file mode 100644
index 0000000..2c08551
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/RepeatMap.java
@@ -0,0 +1,57 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.record.vector;
+
+import io.netty.buffer.ByteBuf;
+
+import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.record.MaterializedField;
+
+public class RepeatMap extends BaseValueVector<RepeatMap>{
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(RepeatMap.class);
+
+  
+  public RepeatMap(MaterializedField field, BufferAllocator allocator) {
+    super(field, allocator);
+  }
+
+  @Override
+  protected int getAllocationSize(int valueCount) {
+    return 4 * valueCount;
+  }
+
+  @Override
+  protected void childResetAllocation(int valueCount, ByteBuf buf) {
+  }
+
+  @Override
+  protected void childCloneMetadata(RepeatMap other) {
+  }
+
+  @Override
+  protected void childClear() {
+  }
+
+  @Override
+  public Object getObject(int index) {
+    return null;
+  }
+
+  
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/SelectionVector.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/SelectionVector.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/SelectionVector.java
index e9faa93..323b55f 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/SelectionVector.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/SelectionVector.java
@@ -20,12 +20,16 @@ package org.apache.drill.exec.record.vector;
 import io.netty.buffer.ByteBufAllocator;
 
 import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.record.MaterializedField;
 
-public class SelectionVector extends UInt16Vector{
+/**
+ * Convenience/Clarification Fixed2 wrapper.
+ */
+public class SelectionVector extends Fixed2{
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(SelectionVector.class);
 
-  public SelectionVector(int fieldId, BufferAllocator allocator) {
-    super(fieldId, allocator);
+  public SelectionVector(MaterializedField field, BufferAllocator allocator) {
+    super(field, allocator);
   }
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/TypeHelper.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/TypeHelper.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/TypeHelper.java
new file mode 100644
index 0000000..8e89c41
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/TypeHelper.java
@@ -0,0 +1,250 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.record.vector;
+
+import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.proto.SchemaDefProtos.DataMode;
+import org.apache.drill.exec.proto.SchemaDefProtos.MajorType;
+import org.apache.drill.exec.proto.SchemaDefProtos.MinorType;
+import org.apache.drill.exec.record.MaterializedField;
+
+public class TypeHelper {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TypeHelper.class);
+  
+  private static final int WIDTH_ESTIMATE_1 = 10;
+  private static final int WIDTH_ESTIMATE_2 = 50000;
+  private static final int WIDTH_ESTIMATE_4 = 1024*1024;
+  
+  public static int getSize(MajorType major){
+    switch(major.getMinorType()){
+    case TINYINT: return 1;
+    case SMALLINT: return 2;
+    case INT: return 4;
+    case BIGINT: return 8;
+    case DECIMAL4: return 4;
+    case DECIMAL8: return 8;
+    case DECIMAL12: return 12;
+    case DECIMAL16: return 16;
+    case MONEY: return 8;
+    case DATE: return 4;
+    case TIME: return 8;
+    case TIMETZ: return 12;
+    case TIMESTAMP: return 8;
+    case DATETIME: return 8;
+    case INTERVAL: return 12;
+    case FLOAT4: return 4;
+    case FLOAT8: return 8;
+    case BOOLEAN: return 1/8;
+    case FIXEDCHAR: return major.getWidth();
+    case VARCHAR1: return 1 + WIDTH_ESTIMATE_1;
+    case VARCHAR2: return 2 + WIDTH_ESTIMATE_2;
+    case VARCHAR4: return 4 + WIDTH_ESTIMATE_4;
+    case FIXEDBINARY: return major.getWidth();
+    case VARBINARY1: return 1 + WIDTH_ESTIMATE_1;
+    case VARBINARY2: return 2 + WIDTH_ESTIMATE_2;
+    case VARBINARY4: return 4 + WIDTH_ESTIMATE_4;
+    case UINT1: return 1;
+    case UINT2: return 2;
+    case UINT4: return 4;
+    case UINT8: return 8;
+    case PROTO2: return 2 + WIDTH_ESTIMATE_2;
+    case PROTO4: return 4 + WIDTH_ESTIMATE_4;
+    case MSGPACK2: return 2 + WIDTH_ESTIMATE_2;
+    case MSGPACK4: return 4 + WIDTH_ESTIMATE_4;    
+    }
+    return 4;
+  }
+  
+  public static Class<?> getValueVectorClass(MinorType type, DataMode mode){
+    switch(mode){
+    case OPTIONAL:
+      switch(type){
+        case REPEATMAP: return RepeatMap.class;
+        case TINYINT: return Fixed1.class;
+        case SMALLINT: return Fixed2.class;
+        case INT: return Fixed4.class;
+        case BIGINT: return Fixed8.class;
+        case DECIMAL4: return Fixed4.class;
+        case DECIMAL8: return Fixed8.class;
+        case DECIMAL12: return Fixed12.class;
+        case DECIMAL16: return Fixed16.class;
+        case MONEY: return Fixed8.class;
+        case DATE: return Fixed4.class;
+        case TIME: return Fixed8.class;
+        case TIMETZ: return Fixed12.class;
+        case TIMESTAMP: return Fixed8.class;
+        case DATETIME: return Fixed8.class;
+        case INTERVAL: return Fixed12.class;
+        case FLOAT4: return Fixed4.class;
+        case FLOAT8: return Fixed8.class;
+        case BOOLEAN: return Bit.class;
+        case FIXEDCHAR: return FixedLen.class;
+        case VARCHAR1: return VarLen1.class;
+        case VARCHAR2: return VarLen2.class;
+        case VARCHAR4: return VarLen4.class;
+        case FIXEDBINARY: return FixedLen.class;
+        case VARBINARY1: return VarLen1.class;
+        case VARBINARY2: return VarLen2.class;
+        case VARBINARY4: return VarLen4.class;
+        case UINT1: return Fixed1.class;
+        case UINT2: return Fixed2.class;
+        case UINT4: return Fixed4.class;
+        case UINT8: return Fixed8.class;
+        case PROTO2: return VarLen2.class;
+        case PROTO4: return VarLen4.class;
+        case MSGPACK2: return VarLen2.class;
+        case MSGPACK4: return VarLen4.class;
+      }
+      break;
+    case REQUIRED:
+      switch(type){
+//        case TINYINT: return NullableFixed1.class;
+//        case SMALLINT: return NullableFixed2.class;
+//        case INT: return NullableFixed4.class;
+//        case BIGINT: return NullableFixed8.class;
+//        case DECIMAL4: return NullableFixed4.class;
+//        case DECIMAL8: return NullableFixed8.class;
+//        case DECIMAL12: return NullableFixed12.class;
+//        case DECIMAL16: return NullableFixed16.class;
+//        case MONEY: return NullableFixed8.class;
+//        case DATE: return NullableFixed4.class;
+//        case TIME: return NullableFixed8.class;
+//        case TIMETZ: return NullableFixed12.class;
+//        case TIMESTAMP: return NullableFixed8.class;
+//        case DATETIME: return NullableFixed8.class;
+//        case INTERVAL: return NullableFixed12.class;
+//        case FLOAT4: return NullableFixed4.class;
+//        case FLOAT8: return NullableFixed8.class;
+//        case BOOLEAN: return NullableBit.class;
+//        case FIXEDCHAR: return NullableFixedLen.class;
+//        case VARCHAR1: return NullableVarLen1.class;
+//        case VARCHAR2: return NullableVarLen2.class;
+//        case VARCHAR4: return NullableVarLen4.class;
+//        case FIXEDBINARY: return NullableFixedLen.class;
+//        case VARBINARY1: return NullableVarLen1.class;
+//        case VARBINARY2: return NullableVarLen2.class;
+//        case VARBINARY4: return NullableVarLen4.class;
+//        case UINT1: return NullableFixed1.class;
+//        case UINT2: return NullableFixed2.class;
+//        case UINT4: return NullableFixed4.class;
+//        case UINT8: return NullableFixed8.class;
+//        case PROTO2: return NullableVarLen2.class;
+//        case PROTO4: return NullableVarLen4.class;
+//        case MSGPACK2: return NullableVarLen2.class;
+//        case MSGPACK4: return NullableVarLen4.class;      
+      }
+      break;
+    case REPEATED:
+      switch(type){
+//        case TINYINT: return RepeatedFixed1.class;
+//        case SMALLINT: return RepeatedFixed2.class;
+//        case INT: return RepeatedFixed4.class;
+//        case BIGINT: return RepeatedFixed8.class;
+//        case DECIMAL4: return RepeatedFixed4.class;
+//        case DECIMAL8: return RepeatedFixed8.class;
+//        case DECIMAL12: return RepeatedFixed12.class;
+//        case DECIMAL16: return RepeatedFixed16.class;
+//        case MONEY: return RepeatedFixed8.class;
+//        case DATE: return RepeatedFixed4.class;
+//        case TIME: return RepeatedFixed8.class;
+//        case TIMETZ: return RepeatedFixed12.class;
+//        case TIMESTAMP: return RepeatedFixed8.class;
+//        case DATETIME: return RepeatedFixed8.class;
+//        case INTERVAL: return RepeatedFixed12.class;
+//        case FLOAT4: return RepeatedFixed4.class;
+//        case FLOAT8: return RepeatedFixed8.class;
+//        case BOOLEAN: return RepeatedBit.class;
+//        case FIXEDCHAR: return RepeatedFixedLen.class;
+//        case VARCHAR1: return RepeatedVarLen1.class;
+//        case VARCHAR2: return RepeatedVarLen2.class;
+//        case VARCHAR4: return RepeatedVarLen4.class;
+//        case FIXEDBINARY: return RepeatedFixedLen.class;
+//        case VARBINARY1: return RepeatedVarLen1.class;
+//        case VARBINARY2: return RepeatedVarLen2.class;
+//        case VARBINARY4: return RepeatedVarLen4.class;
+//        case UINT1: return RepeatedFixed1.class;
+//        case UINT2: return RepeatedFixed2.class;
+//        case UINT4: return RepeatedFixed4.class;
+//        case UINT8: return RepeatedFixed8.class;
+//        case PROTO2: return RepeatedVarLen2.class;
+//        case PROTO4: return RepeatedVarLen4.class;
+//        case MSGPACK2: return RepeatedVarLen2.class;
+//        case MSGPACK4: return RepeatedVarLen4.class;      
+      }
+      break;
+    default:
+      break;
+    
+    }
+    throw new UnsupportedOperationException();
+  }
+  
+  
+  public static ValueVector<?> getNewVector(MaterializedField field, BufferAllocator allocator){
+    MajorType type = field.getType();
+    switch(type.getMode()){
+    case REQUIRED:
+      switch(type.getMinorType()){
+      case TINYINT: return new Fixed1(field, allocator);
+      case SMALLINT: return new Fixed2(field, allocator);
+      case INT: return new Fixed4(field, allocator);
+      case BIGINT: return new Fixed8(field, allocator);
+      case DECIMAL4: return new Fixed4(field, allocator);
+      case DECIMAL8: return new Fixed8(field, allocator);
+      case DECIMAL12: return new Fixed12(field, allocator);
+      case DECIMAL16: return new Fixed16(field, allocator);
+      case MONEY: return new Fixed8(field, allocator);
+      case DATE: return new Fixed4(field, allocator);
+      case TIME: return new Fixed8(field, allocator);
+      case TIMETZ: return new Fixed12(field, allocator);
+      case TIMESTAMP: return new Fixed8(field, allocator);
+      case DATETIME: return new Fixed8(field, allocator);
+      case INTERVAL: return new Fixed12(field, allocator);
+      case FLOAT4: return new Fixed4(field, allocator);
+      case FLOAT8: return new Fixed8(field, allocator);
+      case BOOLEAN: return new Bit(field, allocator);
+      case FIXEDCHAR: return new FixedLen(field, allocator);
+      case VARCHAR1: return new VarLen1(field, allocator);
+      case VARCHAR2: return new VarLen2(field, allocator);
+      case VARCHAR4: return new VarLen4(field, allocator);
+      case FIXEDBINARY: return new FixedLen(field, allocator);
+      case VARBINARY1: return new VarLen1(field, allocator);
+      case VARBINARY2: return new VarLen2(field, allocator);
+      case VARBINARY4: return new VarLen4(field, allocator);
+      case UINT1: return new Fixed1(field, allocator);
+      case UINT2: return new Fixed2(field, allocator);
+      case UINT4: return new Fixed4(field, allocator);
+      case UINT8: return new Fixed8(field, allocator);
+      case PROTO2: return new VarLen2(field, allocator);
+      case PROTO4: return new VarLen4(field, allocator);
+      case MSGPACK2: return new VarLen2(field, allocator);
+      case MSGPACK4: return new VarLen4(field, allocator);      
+      }
+      break;
+    case REPEATED:
+      break;
+    case OPTIONAL:
+      break;
+    default:
+      break;
+    
+    }
+    throw new UnsupportedOperationException();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/UInt16Vector.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/UInt16Vector.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/UInt16Vector.java
deleted file mode 100644
index 87c306b..0000000
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/UInt16Vector.java
+++ /dev/null
@@ -1,51 +0,0 @@
-/*******************************************************************************
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- ******************************************************************************/
-package org.apache.drill.exec.record.vector;
-
-import org.apache.drill.common.expression.types.DataType;
-import org.apache.drill.common.physical.RecordField.ValueMode;
-import org.apache.drill.exec.memory.BufferAllocator;
-import org.apache.drill.exec.record.MaterializedField;
-
-public class UInt16Vector extends AbstractFixedValueVector<Int32Vector>{
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(UInt16Vector.class);
-  
-  private final MaterializedField field;
-
-  public UInt16Vector(int fieldId, BufferAllocator allocator) {
-    super(fieldId, allocator, 16);
-    this.field = new MaterializedField(fieldId, DataType.UINT16, false, ValueMode.VECTOR, this.getClass());
-  }
-
-  @Override
-  public MaterializedField getField() {
-    return field;
-  }
-
-  public final void set(int index, char value){
-    index*=2;
-    data.setChar(index, value);
-  }
-  
-  public final char get(int index){
-    index*=2;
-    return data.getChar(index);
-  }
-  
-  
-}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/ValueVector.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/ValueVector.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/ValueVector.java
index 76b0e90..8a5a822 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/ValueVector.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/ValueVector.java
@@ -21,6 +21,7 @@ import io.netty.buffer.ByteBuf;
 
 import java.io.Closeable;
 
+import org.apache.drill.exec.proto.UserBitShared.FieldMetadata;
 import org.apache.drill.exec.record.MaterializedField;
 
 /**
@@ -44,6 +45,13 @@ public interface ValueVector<T extends ValueVector<T>> extends Closeable {
   public abstract void allocateNew(int valueCount);
 
   /**
+   * Update the value vector to the provided record information.
+   * @param metadata
+   * @param data
+   */
+  public abstract void setTo(FieldMetadata metadata, ByteBuf data);
+  
+  /**
    * Zero copy move of data from this vector to the target vector. Any future access to this vector without being
    * populated by a new vector will cause problems.
    * 
@@ -52,19 +60,19 @@ public interface ValueVector<T extends ValueVector<T>> extends Closeable {
   public abstract void transferTo(T vector);
 
   /**
-   * Return the underlying buffer. Note that this doesn't impact the reference counts for this buffer so it only should be
+   * Return the underlying buffers associated with this vector. Note that this doesn't impact the reference counts for this buffer so it only should be
    * used for in context access. Also note that this buffer changes regularly thus external classes shouldn't hold a
-   * reference to it.
+   * reference to it (unless they change it).
    * 
    * @return The underlying ByteBuf.
    */
-  public abstract ByteBuf getBuffer();
+  public abstract ByteBuf[] getBuffers();
 
   /**
-   * Returns the number of value contained within this vector.
+   * Returns the maximum number of values contained within this vector.
    * @return Vector size
    */
-  public abstract int size();
+  public abstract int capacity();
 
 
   /**
@@ -79,4 +87,32 @@ public interface ValueVector<T extends ValueVector<T>> extends Closeable {
    */
   public abstract MaterializedField getField();
 
+  /**
+   * Define the number of records that are in this value vector.
+   * @param recordCount Number of records active in this vector.  Used for purposes such as getting a writable range of the data.
+   */
+  public abstract void setRecordCount(int recordCount);
+  public abstract int getRecordCount();
+  
+  
+  /**
+   * Get the metadata for this field.
+   * @return
+   */
+  public abstract FieldMetadata getMetadata();
+  
+  /**
+   * Debug interface to get values per record.
+   * @param index The record index.
+   * @return The value in the vector.
+   */
+  public Object getObject(int index);
+  
+  
+  /**
+   * Useful for generating random data.
+   */
+  public void randomizeData();
+    
+  
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/VarLen1.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/VarLen1.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/VarLen1.java
new file mode 100644
index 0000000..d87029d
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/VarLen1.java
@@ -0,0 +1,36 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.record.vector;
+
+import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.record.MaterializedField;
+
+public class VarLen1 extends VariableVector<VarLen1, Fixed1>{
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(VarLen1.class);
+
+  public VarLen1(MaterializedField field, BufferAllocator allocator) {
+    super(field, allocator);
+  }
+
+  @Override
+  protected Fixed1 getNewLengthVector(BufferAllocator allocator) {
+    return new Fixed1(null, allocator);
+  }
+  
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/VarLen2.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/VarLen2.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/VarLen2.java
new file mode 100644
index 0000000..ebd440a
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/VarLen2.java
@@ -0,0 +1,36 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.record.vector;
+
+import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.record.MaterializedField;
+
+public class VarLen2 extends VariableVector<VarLen2, Fixed2>{
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(VarLen2.class);
+
+  public VarLen2(MaterializedField field, BufferAllocator allocator) {
+    super(field, allocator);
+  }
+
+  @Override
+  protected Fixed2 getNewLengthVector(BufferAllocator allocator) {
+    return new Fixed2(null, allocator);
+  }
+  
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/VarLen4.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/VarLen4.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/VarLen4.java
new file mode 100644
index 0000000..b3cd712
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/VarLen4.java
@@ -0,0 +1,36 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.record.vector;
+
+import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.record.MaterializedField;
+
+public class VarLen4 extends VariableVector<VarLen4, Fixed4>{
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(VarLen4.class);
+
+  public VarLen4(MaterializedField field, BufferAllocator allocator) {
+    super(field, allocator);
+  }
+
+  @Override
+  protected Fixed4 getNewLengthVector(BufferAllocator allocator) {
+    return new Fixed4(null, allocator);
+  }
+  
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/VariableVector.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/VariableVector.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/VariableVector.java
index dd84c94..4247f14 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/VariableVector.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/VariableVector.java
@@ -21,6 +21,7 @@ import io.netty.buffer.ByteBuf;
 
 import org.apache.drill.exec.memory.BufferAllocator;
 import org.apache.drill.exec.record.DeadBuf;
+import org.apache.drill.exec.record.MaterializedField;
 
 /** 
  * A vector of variable length bytes.  Constructed as a vector of lengths or positions and a vector of values.  Random access is only possible if the variable vector stores positions as opposed to lengths.
@@ -29,18 +30,16 @@ public abstract class VariableVector<T extends VariableVector<T, E>, E extends B
 
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(VariableVector.class);
   
-  protected E lengthVector;
+  protected final E lengthVector;
   private ByteBuf values = DeadBuf.DEAD_BUFFER;
   protected int expectedValueLength;
-  private final boolean hasPositions;
   
-  public VariableVector(int fieldId, BufferAllocator allocator, boolean hasPositions) {
-    super(fieldId, allocator);
-    this.lengthVector = getNewLengthVector(fieldId, allocator);
-    this.hasPositions = hasPositions;
+  public VariableVector(MaterializedField field, BufferAllocator allocator) {
+    super(field, allocator);
+    this.lengthVector = getNewLengthVector(allocator);
   }
   
-  protected abstract E getNewLengthVector(int fieldId, BufferAllocator allocator);
+  protected abstract E getNewLengthVector(BufferAllocator allocator);
   
   @Override
   protected int getAllocationSize(int valueCount) {
@@ -67,12 +66,28 @@ public abstract class VariableVector<T extends VariableVector<T, E>, E extends B
       values.release();
       values = DeadBuf.DEAD_BUFFER;
     }
-  }  
+  }
+
   
-  public boolean hasPositions(){
-    return hasPositions;
+  @Override
+  public ByteBuf[] getBuffers() {
+    return new ByteBuf[]{lengthVector.data, values};
   }
+
+  @Override
+  public void setRecordCount(int recordCount) {
+    super.setRecordCount(recordCount);
+    lengthVector.setRecordCount(recordCount);
+  }  
   
+  public void setTotalBytes(int totalBytes){
+    values.writerIndex(totalBytes);
+  }
+
+  @Override
+  public Object getObject(int index) {
+    return null;
+  }
   
   
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/AbstractHandshakeHandler.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/AbstractHandshakeHandler.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/AbstractHandshakeHandler.java
new file mode 100644
index 0000000..859d385
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/AbstractHandshakeHandler.java
@@ -0,0 +1,57 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.rpc;
+
+import io.netty.channel.Channel;
+import io.netty.channel.ChannelHandlerContext;
+import io.netty.channel.ChannelInboundMessageHandlerAdapter;
+
+import com.google.protobuf.Internal.EnumLite;
+import com.google.protobuf.MessageLite;
+import com.google.protobuf.Parser;
+
+public abstract class AbstractHandshakeHandler<T extends MessageLite> extends
+    ChannelInboundMessageHandlerAdapter<InboundRpcMessage> {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(AbstractHandshakeHandler.class);
+
+  protected final EnumLite handshakeType;
+  protected final Parser<T> parser;
+  protected int coordinationId;
+
+  public AbstractHandshakeHandler(EnumLite handshakeType, Parser<T> parser) {
+    super();
+    this.handshakeType = handshakeType;
+    this.parser = parser;
+  }
+
+  @Override
+  public final void messageReceived(ChannelHandlerContext ctx, InboundRpcMessage inbound) throws Exception {
+    coordinationId = inbound.coordinationId;
+    ctx.channel().pipeline().remove(this);
+    if (inbound.rpcType != handshakeType.getNumber())
+      throw new RpcException(String.format("Handshake failure.  Expected %s[%d] but received number [%d]",
+          handshakeType, handshakeType.getNumber(), inbound.rpcType));
+  
+    T msg = parser.parseFrom(inbound.getProtobufBodyAsIS());
+    consumeHandshake(ctx.channel(), msg);
+    
+  }
+
+  protected abstract void consumeHandshake(Channel c, T msg) throws Exception;
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/Acks.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/Acks.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/Acks.java
new file mode 100644
index 0000000..a241880
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/Acks.java
@@ -0,0 +1,27 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.rpc;
+
+import org.apache.drill.exec.proto.GeneralRPCProtos.Ack;
+
+public class Acks {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(Acks.class);
+  
+  public static final Ack OK = Ack.newBuilder().setOk(true).build();
+  public static final Ack FAIL = Ack.newBuilder().setOk(false).build();
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/BasicClient.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/BasicClient.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/BasicClient.java
index c62d445..0ff2b9d 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/BasicClient.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/BasicClient.java
@@ -18,23 +18,34 @@
 package org.apache.drill.exec.rpc;
 
 import io.netty.bootstrap.Bootstrap;
+import io.netty.buffer.ByteBuf;
 import io.netty.buffer.ByteBufAllocator;
+import io.netty.channel.Channel;
 import io.netty.channel.ChannelFuture;
 import io.netty.channel.ChannelInitializer;
 import io.netty.channel.ChannelOption;
 import io.netty.channel.EventLoopGroup;
 import io.netty.channel.socket.SocketChannel;
 import io.netty.channel.socket.nio.NioSocketChannel;
+import io.netty.util.concurrent.GenericFutureListener;
 
+import com.google.common.util.concurrent.SettableFuture;
 import com.google.protobuf.Internal.EnumLite;
+import com.google.protobuf.MessageLite;
+import com.google.protobuf.Parser;
 
-public abstract class BasicClient<T extends EnumLite> extends RpcBus<T> {
+public abstract class BasicClient<T extends EnumLite, R extends RemoteConnection> extends RpcBus<T, R> {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(BasicClient.class);
 
   private Bootstrap b;
   private volatile boolean connect = false;
+  protected R connection;
+  private EventLoopGroup eventLoop;
 
-  public BasicClient(ByteBufAllocator alloc, EventLoopGroup eventLoopGroup) {
+  public BasicClient(RpcConfig rpcMapping, ByteBufAllocator alloc, EventLoopGroup eventLoopGroup) {
+    super(rpcMapping);
+    this.eventLoop = eventLoopGroup;
+    
     b = new Bootstrap() //
         .group(eventLoopGroup) //
         .channel(NioSocketChannel.class) //
@@ -42,40 +53,132 @@ public abstract class BasicClient<T extends EnumLite> extends RpcBus<T> {
         .option(ChannelOption.SO_RCVBUF, 1 << 17) //
         .option(ChannelOption.SO_SNDBUF, 1 << 17) //
         .handler(new ChannelInitializer<SocketChannel>() {
-          
+
           @Override
           protected void initChannel(SocketChannel ch) throws Exception {
-            ch.closeFuture().addListener(getCloseHandler(ch));
-            
+            logger.debug("initializing client connection.");
+            connection = initRemoteConnection(ch);
+            ch.closeFuture().addListener(getCloseHandler(connection));
+
             ch.pipeline().addLast( //
                 new ZeroCopyProtobufLengthDecoder(), //
-                new RpcDecoder(), //
-                new RpcEncoder(), //
-                new InboundHandler(ch), //
+                new RpcDecoder(rpcConfig.getName()), //
+                new RpcEncoder(rpcConfig.getName()), //
+                getHandshakeHandler(), //
+                new InboundHandler(connection), //
                 new RpcExceptionHandler() //
                 );
-            channel = ch;
             connect = true;
           }
         }) //
-        
-        ;
+
+    ;
+  }
+
+  protected abstract ClientHandshakeHandler<?> getHandshakeHandler();
+
+  protected abstract class ClientHandshakeHandler<T extends MessageLite> extends AbstractHandshakeHandler<T> {
+    private Class<T> responseType;
+
+    public ClientHandshakeHandler(EnumLite handshakeType, Class<T> responseType, Parser<T> parser) {
+      super(handshakeType, parser);
+      this.responseType = responseType;
+    }
+
+    @Override
+    protected final void consumeHandshake(Channel c, T msg) throws Exception {
+      validateHandshake(msg);
+      queue.getFuture(handshakeType.getNumber(), coordinationId, responseType).setValue(msg);
+    }
+
+    protected abstract void validateHandshake(T msg) throws Exception;
+
+  }
+
+  protected GenericFutureListener<ChannelFuture> getCloseHandler(Channel channel) {
+    return new ChannelClosedHandler();
+  }
+
+  protected final <SEND extends MessageLite, RECEIVE extends MessageLite> DrillRpcFutureImpl<RECEIVE> send(
+      T connection, T rpcType, SEND protobufBody, Class<RECEIVE> clazz, ByteBuf... dataBodies) throws RpcException {
+    throw new UnsupportedOperationException(
+        "This shouldn't be used in client mode as a client only has a single connection.");
+  }
+
+  protected <SEND extends MessageLite, RECEIVE extends MessageLite> DrillRpcFuture<RECEIVE> send(T rpcType,
+      SEND protobufBody, Class<RECEIVE> clazz, ByteBuf... dataBodies) throws RpcException {
+    return super.send(connection, rpcType, protobufBody, clazz, dataBodies);
   }
 
   @Override
   public boolean isClient() {
     return true;
   }
-  
-  public ChannelFuture connectAsClient(String host, int port) throws InterruptedException {
-    ChannelFuture f = b.connect(host, port).sync();
-    connect = !connect;
-    return f;
+
+  /**
+   * TODO: This is a horrible hack to manage deadlock caused by creation of BitClient within BitCom.  Should be cleaned up.
+   */
+  private class HandshakeThread<SEND extends MessageLite, RECEIVE extends MessageLite> extends Thread {
+    final SettableFuture<RECEIVE> future;
+    T handshakeType;
+    SEND handshakeValue;
+    String host;
+    int port;
+    Class<RECEIVE> responseClass;
+
+    public HandshakeThread(T handshakeType, SEND handshakeValue, String host, int port, Class<RECEIVE> responseClass) {
+      super();
+      assert host != null && !host.isEmpty();
+      assert port > 0;
+      logger.debug("Creating new handshake thread to connec to {}:{}", host, port);
+      this.setName(String.format("handshake thread for %s", handshakeType.getClass().getCanonicalName()));
+      future = SettableFuture.create();
+      this.handshakeType = handshakeType;
+      this.handshakeValue = handshakeValue;
+      this.host = host;
+      this.port = port;
+      this.responseClass = responseClass;
+    }
+
+    @Override
+    public void run() {
+      try {
+        logger.debug("Starting to get client connection on host {}, port {}.", host, port);
+        
+        ChannelFuture f = b.connect(host, port);
+        f.sync();
+        if (connection == null) throw new RpcException("Failure while attempting to connect to server.");
+        connect = !connect;
+        logger.debug("Client connected, sending handshake.");
+        DrillRpcFuture<RECEIVE> fut = send(handshakeType, handshakeValue, responseClass);
+        future.set(fut.checkedGet());
+        logger.debug("Got bit client connection.");
+      } catch (Exception e) {
+        logger.debug("Failed to get client connection.", e);
+        future.setException(e);
+      }
+    }
+
+  }
+
+  protected <SEND extends MessageLite, RECEIVE extends MessageLite> RECEIVE connectAsClient(T handshakeType,
+      SEND handshakeValue, String host, int port, Class<RECEIVE> responseClass) throws InterruptedException,
+      RpcException {
+    
+    
+    HandshakeThread<SEND, RECEIVE> ht = new HandshakeThread<SEND, RECEIVE>(handshakeType, handshakeValue, host, port, responseClass);
+    ht.start();
+    try{
+      return ht.future.get();  
+    }catch(Exception e){
+      throw new RpcException(e);
+    }
+    
   }
 
   public void close() {
     logger.debug("Closing client");
-    b.shutdown();
+    connection.getChannel().close();
   }
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/BasicClientWithConnection.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/BasicClientWithConnection.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/BasicClientWithConnection.java
new file mode 100644
index 0000000..0e62f14
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/BasicClientWithConnection.java
@@ -0,0 +1,64 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.rpc;
+
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.ByteBufAllocator;
+import io.netty.channel.Channel;
+import io.netty.channel.ChannelFuture;
+import io.netty.channel.EventLoopGroup;
+import io.netty.util.concurrent.GenericFutureListener;
+
+import org.apache.drill.exec.rpc.BasicClientWithConnection.ServerConnection;
+
+import com.google.protobuf.Internal.EnumLite;
+
+public abstract class BasicClientWithConnection<T extends EnumLite> extends BasicClient<T, ServerConnection>{
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(BasicClientWithConnection.class);
+
+  public BasicClientWithConnection(RpcConfig rpcMapping, ByteBufAllocator alloc, EventLoopGroup eventLoopGroup) {
+    super(rpcMapping, alloc, eventLoopGroup);
+  }
+  
+  @Override
+  protected GenericFutureListener<ChannelFuture> getCloseHandler(ServerConnection clientConnection) {
+    return getCloseHandler(clientConnection.getChannel());
+  }
+  
+  @Override
+  protected Response handle(ServerConnection connection, int rpcType, ByteBuf pBody, ByteBuf dBody) throws RpcException {
+    return handle(rpcType, pBody, dBody);
+  }
+  
+  protected abstract Response handle(int rpcType, ByteBuf pBody, ByteBuf dBody) throws RpcException ;
+    
+  @Override
+  public ServerConnection initRemoteConnection(Channel channel) {
+    return new ServerConnection(channel);
+  }
+
+  public static class ServerConnection extends RemoteConnection{
+
+    public ServerConnection(Channel channel) {
+      super(channel);
+    }
+
+  }
+
+  
+}


[39/53] [abbrv] Types transition

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/JSONRecordReaderTest.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/JSONRecordReaderTest.java b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/JSONRecordReaderTest.java
index 4b35313..d8966ae 100644
--- a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/JSONRecordReaderTest.java
+++ b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/JSONRecordReaderTest.java
@@ -12,6 +12,7 @@ import mockit.Expectations;
 import mockit.Injectable;
 
 import org.apache.drill.common.exceptions.ExecutionSetupException;
+import org.apache.drill.common.types.TypeProtos.MinorType;
 import org.apache.drill.exec.exception.SchemaChangeException;
 import org.apache.drill.exec.memory.DirectBufferAllocator;
 import org.apache.drill.exec.ops.FragmentContext;
@@ -25,270 +26,240 @@ import org.junit.Test;
 import com.beust.jcommander.internal.Lists;
 
 public class JSONRecordReaderTest {
-    private static final Charset UTF_8 = Charset.forName("UTF-8");
+  private static final Charset UTF_8 = Charset.forName("UTF-8");
 
-    private String getResource(String resourceName) {
-        return "resource:" + resourceName;
-    }
-
-    class MockOutputMutator implements OutputMutator {
-        List<Integer> removedFields = Lists.newArrayList();
-        List<ValueVector> addFields = Lists.newArrayList();
-
-        @Override
-        public void removeField(int fieldId) throws SchemaChangeException {
-            removedFields.add(fieldId);
-        }
-
-        @Override
-        public void addField(int fieldId, ValueVector vector) throws SchemaChangeException {
-            addFields.add(vector);
-        }
-
-        @Override
-        public void setNewSchema() throws SchemaChangeException {
-        }
-
-        List<Integer> getRemovedFields() {
-            return removedFields;
-        }
-
-        List<ValueVector> getAddFields() {
-            return addFields;
-        }
-    }
+  private String getResource(String resourceName) {
+    return "resource:" + resourceName;
+  }
 
-    private <T> void assertField(ValueVector valueVector, int index, SchemaDefProtos.MinorType expectedMinorType, T value, String name) {
-        assertField(valueVector, index, expectedMinorType, value, name, 0);
-    }
-
-    private <T> void assertField(ValueVector valueVector, int index, SchemaDefProtos.MinorType expectedMinorType, T value, String name, int parentFieldId) {
-        UserBitShared.FieldMetadata metadata = valueVector.getMetadata();
-        SchemaDefProtos.FieldDef def = metadata.getDef();
-        assertEquals(expectedMinorType, def.getMajorType().getMinorType());
-        assertEquals(name, def.getNameList().get(0).getName());
-        assertEquals(parentFieldId, def.getParentId());
-
-        if(expectedMinorType == SchemaDefProtos.MinorType.MAP) {
-            return;
-        }
-
-        T val = (T) valueVector.getAccessor().getObject(index);
-        if (val instanceof byte[]) {
-            assertTrue(Arrays.equals((byte[]) value, (byte[]) val));
-        } else {
-            assertEquals(value, val);
-        }
-    }
-
-    @Test
-    public void testSameSchemaInSameBatch(@Injectable final FragmentContext context) throws IOException, ExecutionSetupException {
-        new Expectations() {
-            {
-                context.getAllocator();
-                returns(new DirectBufferAllocator());
-            }
-        };
-        JSONRecordReader jr = new JSONRecordReader(context, getResource("scan_json_test_1.json"));
-
-        MockOutputMutator mutator = new MockOutputMutator();
-        List<ValueVector> addFields = mutator.getAddFields();
-        jr.setup(mutator);
-        assertEquals(2, jr.next());
-        assertEquals(3, addFields.size());
-        assertField(addFields.get(0), 0, SchemaDefProtos.MinorType.INT, 123, "test");
-        assertField(addFields.get(1), 0, SchemaDefProtos.MinorType.BOOLEAN, true, "b");
-        assertField(addFields.get(2), 0, SchemaDefProtos.MinorType.VARCHAR4, "hi!".getBytes(UTF_8), "c");
-        assertField(addFields.get(0), 1, SchemaDefProtos.MinorType.INT, 1234, "test");
-        assertField(addFields.get(1), 1, SchemaDefProtos.MinorType.BOOLEAN, false, "b");
-        assertField(addFields.get(2), 1, SchemaDefProtos.MinorType.VARCHAR4, "drill!".getBytes(UTF_8), "c");
-
-        assertEquals(0, jr.next());
-        assertTrue(mutator.getRemovedFields().isEmpty());
-    }
-
-    @Test
-    public void testChangedSchemaInSameBatch(@Injectable final FragmentContext context) throws IOException, ExecutionSetupException {
-        new Expectations() {
-            {
-                context.getAllocator();
-                returns(new DirectBufferAllocator());
-            }
-        };
-
-        JSONRecordReader jr = new JSONRecordReader(context, getResource("scan_json_test_2.json"));
-        MockOutputMutator mutator = new MockOutputMutator();
-        List<ValueVector> addFields = mutator.getAddFields();
-
-        jr.setup(mutator);
-        assertEquals(3, jr.next());
-        assertEquals(7, addFields.size());
-        assertField(addFields.get(0), 0, SchemaDefProtos.MinorType.INT, 123, "test");
-        assertField(addFields.get(1), 0, SchemaDefProtos.MinorType.INT, 1, "b");
-        assertField(addFields.get(2), 0, SchemaDefProtos.MinorType.FLOAT4, (float) 2.15, "c");
-        assertField(addFields.get(3), 0, SchemaDefProtos.MinorType.BOOLEAN, true, "bool");
-        assertField(addFields.get(4), 0, SchemaDefProtos.MinorType.VARCHAR4, "test1".getBytes(UTF_8), "str1");
-
-        assertField(addFields.get(0), 1, SchemaDefProtos.MinorType.INT, 1234, "test");
-        assertField(addFields.get(1), 1, SchemaDefProtos.MinorType.INT, 3, "b");
-        assertField(addFields.get(3), 1, SchemaDefProtos.MinorType.BOOLEAN, false, "bool");
-        assertField(addFields.get(4), 1, SchemaDefProtos.MinorType.VARCHAR4, "test2".getBytes(UTF_8), "str1");
-        assertField(addFields.get(5), 1, SchemaDefProtos.MinorType.INT, 4, "d");
-
-        assertField(addFields.get(0), 2, SchemaDefProtos.MinorType.INT, 12345, "test");
-        assertField(addFields.get(2), 2, SchemaDefProtos.MinorType.FLOAT4, (float) 5.16, "c");
-        assertField(addFields.get(3), 2, SchemaDefProtos.MinorType.BOOLEAN, true, "bool");
-        assertField(addFields.get(5), 2, SchemaDefProtos.MinorType.INT, 6, "d");
-        assertField(addFields.get(6), 2, SchemaDefProtos.MinorType.VARCHAR4, "test3".getBytes(UTF_8), "str2");
-        assertTrue(mutator.getRemovedFields().isEmpty());
-        assertEquals(0, jr.next());
-    }
+  class MockOutputMutator implements OutputMutator {
+    List<MaterializedField> removedFields = Lists.newArrayList();
+    List<ValueVector<?>> addFields = Lists.newArrayList();
 
-    @Test @Ignore
-    public void testChangedSchemaInTwoBatches(@Injectable final FragmentContext context) throws IOException, ExecutionSetupException {
-        new Expectations() {
-            {
-                context.getAllocator();
-                returns(new DirectBufferAllocator());
-            }
-        };
-
-        JSONRecordReader jr = new JSONRecordReader(context, getResource("scan_json_test_2.json"), 64); // batch only fits 1 int
-        MockOutputMutator mutator = new MockOutputMutator();
-        List<ValueVector> addFields = mutator.getAddFields();
-        List<Integer> removedFields = mutator.getRemovedFields();
-
-        jr.setup(mutator);
-        assertEquals(1, jr.next());
-        assertEquals(5, addFields.size());
-        assertField(addFields.get(0), 0, SchemaDefProtos.MinorType.INT, 123, "test");
-        assertField(addFields.get(1), 0, SchemaDefProtos.MinorType.INT, 1, "b");
-        assertField(addFields.get(2), 0, SchemaDefProtos.MinorType.FLOAT4, (float) 2.15, "c");
-        assertField(addFields.get(3), 0, SchemaDefProtos.MinorType.BOOLEAN, true, "bool");
-        assertField(addFields.get(4), 0, SchemaDefProtos.MinorType.VARCHAR4, "test1".getBytes(UTF_8), "str1");
-        assertTrue(removedFields.isEmpty());
-        assertEquals(1, jr.next());
-        assertEquals(6, addFields.size());
-        assertField(addFields.get(0), 0, SchemaDefProtos.MinorType.INT, 1234, "test");
-        assertField(addFields.get(1), 0, SchemaDefProtos.MinorType.INT, 3, "b");
-        assertField(addFields.get(3), 0, SchemaDefProtos.MinorType.BOOLEAN, false, "bool");
-        assertField(addFields.get(4), 0, SchemaDefProtos.MinorType.VARCHAR4, "test2".getBytes(UTF_8), "str1");
-        assertField(addFields.get(5), 0, SchemaDefProtos.MinorType.INT, 4, "d");
-        assertEquals(1, removedFields.size());
-        assertEquals(3, (int) removedFields.get(0));
-        removedFields.clear();
-        assertEquals(1, jr.next());
-        assertEquals(8, addFields.size()); // The reappearing of field 'c' is also included
-        assertField(addFields.get(0), 0, SchemaDefProtos.MinorType.INT, 12345, "test");
-        assertField(addFields.get(3), 0, SchemaDefProtos.MinorType.BOOLEAN, true, "bool");
-        assertField(addFields.get(5), 0, SchemaDefProtos.MinorType.INT, 6, "d");
-        assertField(addFields.get(6), 0, SchemaDefProtos.MinorType.FLOAT4, (float) 5.16, "c");
-        assertField(addFields.get(7), 0, SchemaDefProtos.MinorType.VARCHAR4, "test3".getBytes(UTF_8), "str2");
-        assertEquals(2, removedFields.size());
-        assertTrue(removedFields.contains(5));
-        assertTrue(removedFields.contains(2));
-        assertEquals(0, jr.next());
+    @Override
+    public void removeField(MaterializedField field) throws SchemaChangeException {
+      removedFields.add(field);
     }
 
-    @Ignore("Pending repeated map implementation")
-    @Test
-    public void testNestedFieldInSameBatch(@Injectable final FragmentContext context) throws ExecutionSetupException {
-        new Expectations() {
-            {
-                context.getAllocator();
-                returns(new DirectBufferAllocator());
-            }
-        };
-
-        JSONRecordReader jr = new JSONRecordReader(context, getResource("scan_json_test_3.json"));
-
-        MockOutputMutator mutator = new MockOutputMutator();
-        List<ValueVector> addFields = mutator.getAddFields();
-        jr.setup(mutator);
-        assertEquals(2, jr.next());
-        assertEquals(5, addFields.size());
-        assertField(addFields.get(0), 0, SchemaDefProtos.MinorType.INT, 123, "test");
-        assertField(addFields.get(1), 0, SchemaDefProtos.MinorType.MAP, null, "a");
-        assertField(addFields.get(2), 0, SchemaDefProtos.MinorType.VARCHAR4, "test".getBytes(UTF_8), "b", 2);
-        assertField(addFields.get(3), 0, SchemaDefProtos.MinorType.MAP, null, "a", 2);
-        assertField(addFields.get(4), 0, SchemaDefProtos.MinorType.BOOLEAN, 1, "d", 4);
-        assertField(addFields.get(0), 1, SchemaDefProtos.MinorType.INT, 1234, "test");
-        assertField(addFields.get(1), 1, SchemaDefProtos.MinorType.MAP, null, "a");
-        assertField(addFields.get(2), 1, SchemaDefProtos.MinorType.VARCHAR4, "test2".getBytes(UTF_8), "b", 2);
-        assertField(addFields.get(3), 1, SchemaDefProtos.MinorType.MAP, null, "a", 2);
-        assertField(addFields.get(4), 1, SchemaDefProtos.MinorType.BOOLEAN, 0, "d", 4);
-
-        assertEquals(0, jr.next());
-        assertTrue(mutator.getRemovedFields().isEmpty());
+    @Override
+    public void addField(ValueVector<?> vector) throws SchemaChangeException {
+      addFields.add(vector);
     }
 
-    /*
-
-    @Test
-    public void testScanJsonRemovedOneField() throws IOException {
-        ScanJson sj = new ScanJson(getResource("scan_json_test_3.json"));
-        PhysicalOperatorIterator iterator = sj.getIterator();
-        expectSchemaChanged(iterator);
-        DiffSchema diffSchema = expectSchemaChanged(iterator).getSchemaChanges();
-        assertEquals(0, diffSchema.getAddedFields().size());
-        assertEquals(1, diffSchema.getRemovedFields().size());
-        assertEquals(PhysicalOperatorIterator.NextOutcome.NONE_LEFT, iterator.next());
+    @Override
+    public void setNewSchema() throws SchemaChangeException {
     }
 
-    @Test
-    public void testScanJsonAddOneRemoveOne() throws IOException {
-        ScanJson sj = new ScanJson(getResource("scan_json_test_4.json"));
-        PhysicalOperatorIterator iterator = sj.getIterator();
-        expectSchemaChanged(iterator);
-        DiffSchema diffSchema = expectSchemaChanged(iterator).getSchemaChanges();
-        assertEquals(1, diffSchema.getAddedFields().size());
-        assertEquals(1, diffSchema.getRemovedFields().size());
-        assertEquals(PhysicalOperatorIterator.NextOutcome.NONE_LEFT, iterator.next());
+    List<MaterializedField> getRemovedFields() {
+      return removedFields;
     }
 
-    @Test
-    public void testScanJsonCycleAdditions() throws IOException {
-        ScanJson sj = new ScanJson(getResource("scan_json_test_5.json"));
-        PhysicalOperatorIterator iterator = sj.getIterator();
-        expectSchemaChanged(iterator);
-        DiffSchema diffSchema = expectSchemaChanged(iterator).getSchemaChanges();
-        assertEquals(1, diffSchema.getAddedFields().size());
-        assertEquals(1, diffSchema.getRemovedFields().size());
-        diffSchema = expectSchemaChanged(iterator).getSchemaChanges();
-        assertEquals(1, diffSchema.getAddedFields().size());
-        assertEquals(Field.FieldType.FLOAT, diffSchema.getAddedFields().get(0).getFieldType());
-        assertEquals("test2", diffSchema.getAddedFields().get(0).getFieldName());
-        assertEquals(1, diffSchema.getRemovedFields().size());
-        assertEquals(Field.FieldType.BOOLEAN, diffSchema.getRemovedFields().get(0).getFieldType());
-        assertEquals("test3", diffSchema.getRemovedFields().get(0).getFieldName());
-        assertEquals(PhysicalOperatorIterator.NextOutcome.NONE_LEFT, iterator.next());
+    List<ValueVector<?>> getAddFields() {
+      return addFields;
     }
+  }
 
-    @Test
-    public void testScanJsonModifiedOneFieldType() throws IOException {
-        ScanJson sj = new ScanJson(getResource("scan_json_test_6.json"));
-        PhysicalOperatorIterator iterator = sj.getIterator();
-        expectSchemaChanged(iterator);
-        DiffSchema diffSchema = expectSchemaChanged(iterator).getSchemaChanges();
-        List<Field> addedFields = diffSchema.getAddedFields();
-        assertEquals(4, addedFields.size());
-        List<Field> removedFields = diffSchema.getRemovedFields();
-        assertEquals(4, removedFields.size());
-        assertFieldExists("test", Field.FieldType.STRING, addedFields);
-        assertFieldExists("test2", Field.FieldType.BOOLEAN, addedFields);
-        assertFieldExists("b", Field.FieldType.ARRAY, addedFields);
-        assertFieldExists("[0]", Field.FieldType.INTEGER, addedFields);
-        assertFieldExists("test", Field.FieldType.INTEGER, removedFields);
-        assertFieldExists("test2", Field.FieldType.ARRAY, removedFields);
-        assertFieldExists("b", Field.FieldType.INTEGER, removedFields);
-        assertFieldExists("[0]", Field.FieldType.INTEGER, removedFields);
-        assertEquals(PhysicalOperatorIterator.NextOutcome.NONE_LEFT, iterator.next());
-    }
+  private <T> void assertField(ValueVector valueVector, int index, MinorType expectedMinorType, T value, String name) {
+    UserBitShared.FieldMetadata metadata = valueVector.getMetadata();
+    SchemaDefProtos.FieldDef def = metadata.getDef();
+    assertEquals(expectedMinorType, def.getMajorType().getMinorType());
+    assertEquals(name, def.getNameList().get(0).getName());
 
-    private void expectSchemaChanged(PhysicalOperatorIterator iterator) throws IOException {
+    if (expectedMinorType == MinorType.MAP) {
+      return;
     }
 
-    private void expectDataRecord(PhysicalOperatorIterator iterator) throws IOException {
+    T val = (T) valueVector.getObject(index);
+    if (val instanceof byte[]) {
+      assertTrue(Arrays.equals((byte[]) value, (byte[]) val));
+    } else {
+      assertEquals(value, val);
     }
-*/
+  }
+
+  @Test
+  public void testSameSchemaInSameBatch(@Injectable final FragmentContext context) throws IOException,
+      ExecutionSetupException {
+    new Expectations() {
+      {
+        context.getAllocator();
+        returns(new DirectBufferAllocator());
+      }
+    };
+    JSONRecordReader jr = new JSONRecordReader(context, getResource("scan_json_test_1.json"));
+
+    MockOutputMutator mutator = new MockOutputMutator();
+    List<ValueVector<?>> addFields = mutator.getAddFields();
+    jr.setup(mutator);
+    assertEquals(2, jr.next());
+    assertEquals(3, addFields.size());
+    assertField(addFields.get(0), 0, MinorType.INT, 123, "test");
+    assertField(addFields.get(1), 0, MinorType.BOOLEAN, 1, "b");
+    assertField(addFields.get(2), 0, MinorType.VARCHAR4, "hi!".getBytes(UTF_8), "c");
+    assertField(addFields.get(0), 1, MinorType.INT, 1234, "test");
+    assertField(addFields.get(1), 1, MinorType.BOOLEAN, 0, "b");
+    assertField(addFields.get(2), 1, MinorType.VARCHAR4, "drill!".getBytes(UTF_8), "c");
+
+    assertEquals(0, jr.next());
+    assertTrue(mutator.getRemovedFields().isEmpty());
+  }
+
+  @Test
+  public void testChangedSchemaInSameBatch(@Injectable final FragmentContext context) throws IOException,
+      ExecutionSetupException {
+    new Expectations() {
+      {
+        context.getAllocator();
+        returns(new DirectBufferAllocator());
+      }
+    };
+
+    JSONRecordReader jr = new JSONRecordReader(context, getResource("scan_json_test_2.json"));
+    MockOutputMutator mutator = new MockOutputMutator();
+    List<ValueVector<?>> addFields = mutator.getAddFields();
+
+    jr.setup(mutator);
+    assertEquals(3, jr.next());
+    assertEquals(7, addFields.size());
+    assertField(addFields.get(0), 0, MinorType.INT, 123, "test");
+    assertField(addFields.get(1), 0, MinorType.INT, 1, "b");
+    assertField(addFields.get(2), 0, MinorType.FLOAT4, (float) 2.15, "c");
+    assertField(addFields.get(3), 0, MinorType.BOOLEAN, 1, "bool");
+    assertField(addFields.get(4), 0, MinorType.VARCHAR4, "test1".getBytes(UTF_8), "str1");
+
+    assertField(addFields.get(0), 1, MinorType.INT, 1234, "test");
+    assertField(addFields.get(1), 1, MinorType.INT, 3, "b");
+    assertField(addFields.get(3), 1, MinorType.BOOLEAN, 0, "bool");
+    assertField(addFields.get(4), 1, MinorType.VARCHAR4, "test2".getBytes(UTF_8), "str1");
+    assertField(addFields.get(5), 1, MinorType.INT, 4, "d");
+
+    assertField(addFields.get(0), 2, MinorType.INT, 12345, "test");
+    assertField(addFields.get(2), 2, MinorType.FLOAT4, (float) 5.16, "c");
+    assertField(addFields.get(3), 2, MinorType.BOOLEAN, 1, "bool");
+    assertField(addFields.get(5), 2, MinorType.INT, 6, "d");
+    assertField(addFields.get(6), 2, MinorType.VARCHAR4, "test3".getBytes(UTF_8), "str2");
+    assertTrue(mutator.getRemovedFields().isEmpty());
+    assertEquals(0, jr.next());
+  }
+
+  @Test
+  public void testChangedSchemaInTwoBatches(@Injectable final FragmentContext context) throws IOException,
+      ExecutionSetupException {
+    new Expectations() {
+      {
+        context.getAllocator();
+        returns(new DirectBufferAllocator());
+      }
+    };
+
+    JSONRecordReader jr = new JSONRecordReader(context, getResource("scan_json_test_2.json"), 64); // batch only fits 1
+                                                                                                   // int
+    MockOutputMutator mutator = new MockOutputMutator();
+    List<ValueVector<?>> addFields = mutator.getAddFields();
+    List<MaterializedField> removedFields = mutator.getRemovedFields();
+
+    jr.setup(mutator);
+    assertEquals(1, jr.next());
+    assertEquals(5, addFields.size());
+    assertField(addFields.get(0), 0, MinorType.INT, 123, "test");
+    assertField(addFields.get(1), 0, MinorType.INT, 1, "b");
+    assertField(addFields.get(2), 0, MinorType.FLOAT4, (float) 2.15, "c");
+    assertField(addFields.get(3), 0, MinorType.BOOLEAN, 1, "bool");
+    assertField(addFields.get(4), 0, MinorType.VARCHAR4, "test1".getBytes(UTF_8), "str1");
+    assertTrue(removedFields.isEmpty());
+    assertEquals(1, jr.next());
+    assertEquals(6, addFields.size());
+    assertField(addFields.get(0), 0, MinorType.INT, 1234, "test");
+    assertField(addFields.get(1), 0, MinorType.INT, 3, "b");
+    assertField(addFields.get(3), 0, MinorType.BOOLEAN, 0, "bool");
+    assertField(addFields.get(4), 0, MinorType.VARCHAR4, "test2".getBytes(UTF_8), "str1");
+    assertField(addFields.get(5), 0, MinorType.INT, 4, "d");
+    assertEquals(1, removedFields.size());
+    //assertEquals(3, (int) removedFields.get(0));
+    removedFields.clear();
+    assertEquals(1, jr.next());
+    assertEquals(8, addFields.size()); // The reappearing of field 'c' is also included
+    assertField(addFields.get(0), 0, MinorType.INT, 12345, "test");
+    assertField(addFields.get(3), 0, MinorType.BOOLEAN, 1, "bool");
+    assertField(addFields.get(5), 0, MinorType.INT, 6, "d");
+    assertField(addFields.get(6), 0, MinorType.FLOAT4, (float) 5.16, "c");
+    assertField(addFields.get(7), 0, MinorType.VARCHAR4, "test3".getBytes(UTF_8), "str2");
+    assertEquals(2, removedFields.size());
+//    assertTrue(removedFields.contains(5));
+//    assertTrue(removedFields.contains(2));
+    assertEquals(0, jr.next());
+  }
+
+  @Test
+  public void testNestedFieldInSameBatch(@Injectable final FragmentContext context) throws ExecutionSetupException {
+    new Expectations() {
+      {
+        context.getAllocator();
+        returns(new DirectBufferAllocator());
+      }
+    };
+
+    JSONRecordReader jr = new JSONRecordReader(context, getResource("scan_json_test_3.json"));
+
+    MockOutputMutator mutator = new MockOutputMutator();
+    List<ValueVector<?>> addFields = mutator.getAddFields();
+    jr.setup(mutator);
+    assertEquals(2, jr.next());
+    assertEquals(5, addFields.size());
+    assertField(addFields.get(0), 0, MinorType.INT, 123, "test");
+    assertField(addFields.get(1), 0, MinorType.MAP, null, "a");
+    assertField(addFields.get(2), 0, MinorType.VARCHAR4, "test".getBytes(UTF_8), "b");
+    assertField(addFields.get(3), 0, MinorType.MAP, null, "a");
+    assertField(addFields.get(4), 0, MinorType.BOOLEAN, 1, "d");
+    assertField(addFields.get(0), 1, MinorType.INT, 1234, "test");
+    assertField(addFields.get(1), 1, MinorType.MAP, null, "a");
+    assertField(addFields.get(2), 1, MinorType.VARCHAR4, "test2".getBytes(UTF_8), "b");
+    assertField(addFields.get(3), 1, MinorType.MAP, null, "a");
+    assertField(addFields.get(4), 1, MinorType.BOOLEAN, 0, "d");
+
+    assertEquals(0, jr.next());
+    assertTrue(mutator.getRemovedFields().isEmpty());
+  }
+
+  /*
+   * 
+   * @Test public void testScanJsonRemovedOneField() throws IOException { ScanJson sj = new
+   * ScanJson(getResource("scan_json_test_3.json")); PhysicalOperatorIterator iterator = sj.getIterator();
+   * expectSchemaChanged(iterator); DiffSchema diffSchema = expectSchemaChanged(iterator).getSchemaChanges();
+   * assertEquals(0, diffSchema.getAddedFields().size()); assertEquals(1, diffSchema.getRemovedFields().size());
+   * assertEquals(PhysicalOperatorIterator.NextOutcome.NONE_LEFT, iterator.next()); }
+   * 
+   * @Test public void testScanJsonAddOneRemoveOne() throws IOException { ScanJson sj = new
+   * ScanJson(getResource("scan_json_test_4.json")); PhysicalOperatorIterator iterator = sj.getIterator();
+   * expectSchemaChanged(iterator); DiffSchema diffSchema = expectSchemaChanged(iterator).getSchemaChanges();
+   * assertEquals(1, diffSchema.getAddedFields().size()); assertEquals(1, diffSchema.getRemovedFields().size());
+   * assertEquals(PhysicalOperatorIterator.NextOutcome.NONE_LEFT, iterator.next()); }
+   * 
+   * @Test public void testScanJsonCycleAdditions() throws IOException { ScanJson sj = new
+   * ScanJson(getResource("scan_json_test_5.json")); PhysicalOperatorIterator iterator = sj.getIterator();
+   * expectSchemaChanged(iterator); DiffSchema diffSchema = expectSchemaChanged(iterator).getSchemaChanges();
+   * assertEquals(1, diffSchema.getAddedFields().size()); assertEquals(1, diffSchema.getRemovedFields().size());
+   * diffSchema = expectSchemaChanged(iterator).getSchemaChanges(); assertEquals(1, diffSchema.getAddedFields().size());
+   * assertEquals(Field.FieldType.FLOAT, diffSchema.getAddedFields().get(0).getFieldType()); assertEquals("test2",
+   * diffSchema.getAddedFields().get(0).getFieldName()); assertEquals(1, diffSchema.getRemovedFields().size());
+   * assertEquals(Field.FieldType.BOOLEAN, diffSchema.getRemovedFields().get(0).getFieldType()); assertEquals("test3",
+   * diffSchema.getRemovedFields().get(0).getFieldName()); assertEquals(PhysicalOperatorIterator.NextOutcome.NONE_LEFT,
+   * iterator.next()); }
+   * 
+   * @Test public void testScanJsonModifiedOneFieldType() throws IOException { ScanJson sj = new
+   * ScanJson(getResource("scan_json_test_6.json")); PhysicalOperatorIterator iterator = sj.getIterator();
+   * expectSchemaChanged(iterator); DiffSchema diffSchema = expectSchemaChanged(iterator).getSchemaChanges();
+   * List<Field> addedFields = diffSchema.getAddedFields(); assertEquals(4, addedFields.size()); List<Field>
+   * removedFields = diffSchema.getRemovedFields(); assertEquals(4, removedFields.size()); assertFieldExists("test",
+   * Field.FieldType.STRING, addedFields); assertFieldExists("test2", Field.FieldType.BOOLEAN, addedFields);
+   * assertFieldExists("b", Field.FieldType.ARRAY, addedFields); assertFieldExists("[0]", Field.FieldType.INTEGER,
+   * addedFields); assertFieldExists("test", Field.FieldType.INTEGER, removedFields); assertFieldExists("test2",
+   * Field.FieldType.ARRAY, removedFields); assertFieldExists("b", Field.FieldType.INTEGER, removedFields);
+   * assertFieldExists("[0]", Field.FieldType.INTEGER, removedFields);
+   * assertEquals(PhysicalOperatorIterator.NextOutcome.NONE_LEFT, iterator.next()); }
+   * 
+   * private void expectSchemaChanged(PhysicalOperatorIterator iterator) throws IOException { }
+   * 
+   * private void expectDataRecord(PhysicalOperatorIterator iterator) throws IOException { }
+   */
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/sandbox/prototype/exec/ref/src/main/java/org/apache/drill/exec/ref/RunOutcome.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/ref/src/main/java/org/apache/drill/exec/ref/RunOutcome.java b/sandbox/prototype/exec/ref/src/main/java/org/apache/drill/exec/ref/RunOutcome.java
index 069892a..1e5b226 100644
--- a/sandbox/prototype/exec/ref/src/main/java/org/apache/drill/exec/ref/RunOutcome.java
+++ b/sandbox/prototype/exec/ref/src/main/java/org/apache/drill/exec/ref/RunOutcome.java
@@ -31,6 +31,7 @@ public class RunOutcome {
   
   public RunOutcome(OutcomeType outcome, long bytes, long records, Throwable exception) {
     super();
+    if(outcome != OutcomeType.SUCCESS) logger.warn("Creating failed outcome.", exception);
     this.outcome = outcome;
     this.bytes = bytes;
     this.records = records;

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/sandbox/prototype/exec/ref/src/main/java/org/apache/drill/exec/ref/eval/BasicEvaluatorFactory.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/ref/src/main/java/org/apache/drill/exec/ref/eval/BasicEvaluatorFactory.java b/sandbox/prototype/exec/ref/src/main/java/org/apache/drill/exec/ref/eval/BasicEvaluatorFactory.java
index 2718db7..ad86b73 100644
--- a/sandbox/prototype/exec/ref/src/main/java/org/apache/drill/exec/ref/eval/BasicEvaluatorFactory.java
+++ b/sandbox/prototype/exec/ref/src/main/java/org/apache/drill/exec/ref/eval/BasicEvaluatorFactory.java
@@ -20,6 +20,7 @@ package org.apache.drill.exec.ref.eval;
 import org.apache.drill.common.expression.LogicalExpression;
 import org.apache.drill.common.expression.SchemaPath;
 import org.apache.drill.common.expression.visitors.ExprVisitor;
+import org.apache.drill.common.expression.visitors.SimpleExprVisitor;
 import org.apache.drill.common.logical.data.NamedExpression;
 import org.apache.drill.exec.ref.IteratorRegistry;
 import org.apache.drill.exec.ref.RecordPointer;
@@ -38,13 +39,13 @@ public class BasicEvaluatorFactory extends EvaluatorFactory{
     
   }
   
-  private ExprVisitor<BasicEvaluator> get(RecordPointer record){
+  private SimpleExprVisitor<BasicEvaluator> get(RecordPointer record){
     return new SimpleEvaluationVisitor(record);
   }
   
   @Override
   public BasicEvaluator getBasicEvaluator(RecordPointer inputRecord, LogicalExpression e) {
-    return e.accept(get(inputRecord));
+    return e.accept(get(inputRecord), null);
   }
   
 
@@ -52,13 +53,13 @@ public class BasicEvaluatorFactory extends EvaluatorFactory{
   @Override
   public AggregatingEvaluator getAggregatingOperator(RecordPointer record, LogicalExpression e) {
     SimpleEvaluationVisitor visitor = new SimpleEvaluationVisitor(record);
-    BasicEvaluator b = e.accept(visitor);
+    BasicEvaluator b = e.accept(visitor, null);
     return new AggregatingWrapperEvaluator(visitor.getAggregators(), b);
   }
 
   @Override
   public BooleanEvaluator getBooleanEvaluator(RecordPointer record, LogicalExpression e) {
-    return new BooleanEvaluatorImpl(e.accept(get(record)));
+    return new BooleanEvaluatorImpl(e.accept(get(record),  null));
   }
 
   @Override
@@ -89,7 +90,7 @@ public class BasicEvaluatorFactory extends EvaluatorFactory{
     public ConnectedEvaluatorImpl(RecordPointer record, NamedExpression e){
       this.outputPath = e.getRef();
       this.record = record;
-      this.eval = e.getExpr().accept(get(record));
+      this.eval = e.getExpr().accept(get(record), null);
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/sandbox/prototype/exec/ref/src/main/java/org/apache/drill/exec/ref/eval/IfEvaluator.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/ref/src/main/java/org/apache/drill/exec/ref/eval/IfEvaluator.java b/sandbox/prototype/exec/ref/src/main/java/org/apache/drill/exec/ref/eval/IfEvaluator.java
index 2e5770b..c78980a 100644
--- a/sandbox/prototype/exec/ref/src/main/java/org/apache/drill/exec/ref/eval/IfEvaluator.java
+++ b/sandbox/prototype/exec/ref/src/main/java/org/apache/drill/exec/ref/eval/IfEvaluator.java
@@ -31,12 +31,12 @@ public class IfEvaluator implements BasicEvaluator{
   private final IfCond[] conditions;
   private final BasicEvaluator elseExpression;
   
-  public IfEvaluator(IfExpression expression, ExprVisitor<BasicEvaluator> evalBuilder, RecordPointer record){
+  public IfEvaluator(IfExpression expression, ExprVisitor<BasicEvaluator, Void, RuntimeException> evalBuilder, RecordPointer record){
     this.conditions = new IfCond[expression.conditions.size()];
     for(int i =0; i < conditions.length; i++){
       conditions[i] = new IfCond(expression.conditions.get(i), evalBuilder);
     }
-    elseExpression = expression.elseExpression.accept(evalBuilder);
+    elseExpression = expression.elseExpression.accept(evalBuilder, null);
   }
   
   @Override
@@ -51,9 +51,9 @@ public class IfEvaluator implements BasicEvaluator{
     private final BasicEvaluator condition;
     private final BasicEvaluator valueExpression;
 
-    public IfCond(IfCondition c, ExprVisitor<BasicEvaluator> evalBuilder){
-      this.condition = c.condition.accept(evalBuilder);
-      this.valueExpression = c.expression.accept(evalBuilder);
+    public IfCond(IfCondition c, ExprVisitor<BasicEvaluator, Void, RuntimeException> evalBuilder){
+      this.condition = c.condition.accept(evalBuilder, null);
+      this.valueExpression = c.expression.accept(evalBuilder, null);
     }
     
     public boolean matches(RecordPointer r){

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/sandbox/prototype/exec/ref/src/main/java/org/apache/drill/exec/ref/eval/SimpleEvaluationVisitor.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/ref/src/main/java/org/apache/drill/exec/ref/eval/SimpleEvaluationVisitor.java b/sandbox/prototype/exec/ref/src/main/java/org/apache/drill/exec/ref/eval/SimpleEvaluationVisitor.java
index f03648b..5b2c781 100644
--- a/sandbox/prototype/exec/ref/src/main/java/org/apache/drill/exec/ref/eval/SimpleEvaluationVisitor.java
+++ b/sandbox/prototype/exec/ref/src/main/java/org/apache/drill/exec/ref/eval/SimpleEvaluationVisitor.java
@@ -22,6 +22,7 @@ import java.util.List;
 
 import org.antlr.runtime.ANTLRStringStream;
 import org.antlr.runtime.CommonTokenStream;
+import org.apache.drill.common.expression.ExpressionPosition;
 import org.apache.drill.common.expression.FunctionCall;
 import org.apache.drill.common.expression.IfExpression;
 import org.apache.drill.common.expression.LogicalExpression;
@@ -34,7 +35,7 @@ import org.apache.drill.common.expression.parser.ExprLexer;
 import org.apache.drill.common.expression.parser.ExprParser;
 import org.apache.drill.common.expression.visitors.AggregateChecker;
 import org.apache.drill.common.expression.visitors.ConstantChecker;
-import org.apache.drill.common.expression.visitors.ExprVisitor;
+import org.apache.drill.common.expression.visitors.SimpleExprVisitor;
 import org.apache.drill.exec.ref.RecordPointer;
 import org.apache.drill.exec.ref.UnbackedRecord;
 import org.apache.drill.exec.ref.eval.EvaluatorTypes.AggregatingEvaluator;
@@ -49,7 +50,7 @@ import org.apache.drill.exec.ref.values.ScalarValues.IntegerScalar;
 import org.apache.drill.exec.ref.values.ScalarValues.LongScalar;
 import org.apache.drill.exec.ref.values.ScalarValues.StringScalar;
 
-public class SimpleEvaluationVisitor implements ExprVisitor<BasicEvaluator>{
+public class SimpleEvaluationVisitor extends SimpleExprVisitor<BasicEvaluator>{
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(SimpleEvaluationVisitor.class);
 
   private RecordPointer record;
@@ -72,7 +73,7 @@ public class SimpleEvaluationVisitor implements ExprVisitor<BasicEvaluator>{
     for(LogicalExpression e : call){
       if(AggregateChecker.isAggregating(e)) includesAggregates = true;
       if(!ConstantChecker.onlyIncludesConstants(e)) onlyConstants = false;
-      evals.add(e.accept(this));
+      evals.add(e.accept(this, null));
     }
     FunctionArguments args = new FunctionArguments(onlyConstants, includesAggregates, evals, call);
 
@@ -102,26 +103,31 @@ public class SimpleEvaluationVisitor implements ExprVisitor<BasicEvaluator>{
   }
 
   @Override
-  public BasicEvaluator visitLongExpression(LongExpression longExpr) {
+  public BasicEvaluator visitLongConstant(LongExpression longExpr) {
     return new LongScalar(longExpr.getLong());
   }
 
   @Override
-  public BasicEvaluator visitDoubleExpression(DoubleExpression dExpr) {
+  public BasicEvaluator visitDoubleConstant(DoubleExpression dExpr) {
     return new DoubleScalar(dExpr.getDouble());
   }
 
   @Override
-  public BasicEvaluator visitBoolean(BooleanExpression e) {
+  public BasicEvaluator visitBooleanConstant(BooleanExpression e) {
     return new BooleanScalar(e.getBoolean());
   }
 
   @Override
-  public BasicEvaluator visitQuotedString(QuotedString e) {
+  public BasicEvaluator visitQuotedStringConstant(QuotedString e) {
     return new StringScalar(e.value);
   }
   
   
+  @Override
+  public BasicEvaluator visitUnknown(LogicalExpression e, Void value) throws RuntimeException {
+    throw new UnsupportedOperationException();
+  }
+
   public static void main(String[] args) throws Exception {
     String expr = "if( a == 1) then 4 else 2 end";
     ExprLexer lexer = new ExprLexer(new ANTLRStringStream(expr));
@@ -129,9 +135,9 @@ public class SimpleEvaluationVisitor implements ExprVisitor<BasicEvaluator>{
     ExprParser parser = new ExprParser(tokens);
     LogicalExpression e = parser.parse().e;
     RecordPointer r = new UnbackedRecord();
-    r.addField(new SchemaPath("a"), new IntegerScalar(3));
+    r.addField(new SchemaPath("a", ExpressionPosition.UNKNOWN), new IntegerScalar(3));
     SimpleEvaluationVisitor builder = new SimpleEvaluationVisitor(r);
-    BasicEvaluator eval = e.accept(builder);
+    BasicEvaluator eval = e.accept(builder, null);
     DataValue v = eval.eval();
     System.out.println(v);
   }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/sandbox/prototype/exec/ref/src/main/java/org/apache/drill/exec/ref/eval/fn/MathEvaluators.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/ref/src/main/java/org/apache/drill/exec/ref/eval/fn/MathEvaluators.java b/sandbox/prototype/exec/ref/src/main/java/org/apache/drill/exec/ref/eval/fn/MathEvaluators.java
index bdf9de7..d080cd3 100644
--- a/sandbox/prototype/exec/ref/src/main/java/org/apache/drill/exec/ref/eval/fn/MathEvaluators.java
+++ b/sandbox/prototype/exec/ref/src/main/java/org/apache/drill/exec/ref/eval/fn/MathEvaluators.java
@@ -17,6 +17,7 @@
  ******************************************************************************/
 package org.apache.drill.exec.ref.eval.fn;
 
+import org.apache.drill.common.types.Types;
 import org.apache.drill.exec.ref.RecordPointer;
 import org.apache.drill.exec.ref.eval.BaseBasicEvaluator;
 import org.apache.drill.exec.ref.eval.EvaluatorTypes.BasicEvaluator;
@@ -43,7 +44,7 @@ public class MathEvaluators {
       NumericValue[] values = new NumericValue[args.length];
       for(int i =0; i < values.length; i++){
         DataValue v = args[i].eval();
-        if(v.getDataType().isNumericType()){
+        if(Types.isNumericType(v.getDataType())){
           values[i] = v.getAsNumeric();
         }
       }
@@ -71,7 +72,7 @@ public class MathEvaluators {
       for(int i =0; i < args.length; i++){
         final DataValue v = args[i].eval();
 //        logger.debug("DataValue {}", v);
-        if(v.getDataType().isNumericType()){
+        if(Types.isNumericType(v.getDataType())){
           NumericValue n = v.getAsNumeric();
           NumericType nt = n.getNumericType();
 //          logger.debug("Numeric Type: {}", nt);

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/sandbox/prototype/exec/ref/src/main/java/org/apache/drill/exec/ref/rops/CollapsingAggregateROP.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/ref/src/main/java/org/apache/drill/exec/ref/rops/CollapsingAggregateROP.java b/sandbox/prototype/exec/ref/src/main/java/org/apache/drill/exec/ref/rops/CollapsingAggregateROP.java
index 4861bc6..e2160b5 100644
--- a/sandbox/prototype/exec/ref/src/main/java/org/apache/drill/exec/ref/rops/CollapsingAggregateROP.java
+++ b/sandbox/prototype/exec/ref/src/main/java/org/apache/drill/exec/ref/rops/CollapsingAggregateROP.java
@@ -19,8 +19,8 @@ package org.apache.drill.exec.ref.rops;
 
 import org.apache.drill.common.expression.FieldReference;
 import org.apache.drill.common.expression.SchemaPath;
-import org.apache.drill.common.expression.types.DataType;
 import org.apache.drill.common.logical.data.CollapsingAggregate;
+import org.apache.drill.common.types.TypeProtos.MinorType;
 import org.apache.drill.exec.ref.RecordIterator;
 import org.apache.drill.exec.ref.RecordPointer;
 import org.apache.drill.exec.ref.UnbackedRecord;
@@ -159,7 +159,7 @@ public class CollapsingAggregateROP extends SingleInputROPBase<CollapsingAggrega
       // if we're in target mode and this row matches the target criteria, we're going to copy carry over values and mark foundTarget = true.
       if(targetMode){
         DataValue v = targetEvaluator.eval();
-        if(v.getDataType() == DataType.BOOLEAN && v.getAsBooleanValue().getBoolean()){
+        if(v.getDataType().getMinorType() == MinorType.BOOLEAN && v.getAsBooleanValue().getBoolean()){
           foundTarget = true;
           for(int i =0 ; i < carryovers.length; i++){
             carryoverValues[i] = carryovers[i].eval();

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/sandbox/prototype/exec/ref/src/main/java/org/apache/drill/exec/ref/rops/ConstantROP.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/ref/src/main/java/org/apache/drill/exec/ref/rops/ConstantROP.java b/sandbox/prototype/exec/ref/src/main/java/org/apache/drill/exec/ref/rops/ConstantROP.java
index ffc6cc8..5d6f91f 100644
--- a/sandbox/prototype/exec/ref/src/main/java/org/apache/drill/exec/ref/rops/ConstantROP.java
+++ b/sandbox/prototype/exec/ref/src/main/java/org/apache/drill/exec/ref/rops/ConstantROP.java
@@ -17,7 +17,6 @@
  ******************************************************************************/
 package org.apache.drill.exec.ref.rops;
 
-import java.io.IOException;
 import java.util.Iterator;
 
 import org.apache.drill.common.logical.data.Constant;
@@ -28,14 +27,12 @@ import org.apache.drill.exec.ref.RunOutcome.OutcomeType;
 import org.apache.drill.exec.ref.UnbackedRecord;
 import org.apache.drill.exec.ref.exceptions.SetupException;
 import org.apache.drill.exec.ref.rse.JSONRecordReader;
-import org.apache.drill.exec.ref.rse.RecordReader;
 
 import com.fasterxml.jackson.databind.JsonNode;
 
 public class ConstantROP extends ROPBase<Constant>{
     static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ScanROP.class);
 
-    private ConstantIterator iter;
     private UnbackedRecord record;
 
     public ConstantROP(Constant config) {

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/sandbox/prototype/exec/ref/src/main/java/org/apache/drill/exec/ref/rops/FlattenROP.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/ref/src/main/java/org/apache/drill/exec/ref/rops/FlattenROP.java b/sandbox/prototype/exec/ref/src/main/java/org/apache/drill/exec/ref/rops/FlattenROP.java
index 64eca8b..f6e6907 100644
--- a/sandbox/prototype/exec/ref/src/main/java/org/apache/drill/exec/ref/rops/FlattenROP.java
+++ b/sandbox/prototype/exec/ref/src/main/java/org/apache/drill/exec/ref/rops/FlattenROP.java
@@ -17,12 +17,9 @@
  ******************************************************************************/
 package org.apache.drill.exec.ref.rops;
 
-import org.apache.drill.common.expression.FieldReference;
-import org.apache.drill.common.expression.PathSegment;
 import org.apache.drill.common.expression.SchemaPath;
-import org.apache.drill.common.expression.types.DataType;
-import org.apache.drill.common.logical.data.NamedExpression;
 import org.apache.drill.common.logical.data.Flatten;
+import org.apache.drill.common.types.TypeProtos.DataMode;
 import org.apache.drill.exec.ref.RecordIterator;
 import org.apache.drill.exec.ref.RecordPointer;
 import org.apache.drill.exec.ref.UnbackedRecord;
@@ -109,7 +106,7 @@ public class FlattenROP extends SingleInputROPBase<Flatten> {
 
 
       if (currentOutcome != NextOutcome.NONE_LEFT) {
-        if (evaluator.eval().getDataType() == DataType.ARRAY) {
+        if (evaluator.eval().getDataType().getMode() == DataMode.REPEATED) {
           arrayValueIterator = new ArrayValueIterator(evaluator.eval().getAsContainer().getAsArray());
 
           while ((v = arrayValueIterator.next()) != null) {

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/sandbox/prototype/exec/ref/src/main/java/org/apache/drill/exec/ref/rops/WindowFrameROP.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/ref/src/main/java/org/apache/drill/exec/ref/rops/WindowFrameROP.java b/sandbox/prototype/exec/ref/src/main/java/org/apache/drill/exec/ref/rops/WindowFrameROP.java
index 4ed1642..e5b153e 100644
--- a/sandbox/prototype/exec/ref/src/main/java/org/apache/drill/exec/ref/rops/WindowFrameROP.java
+++ b/sandbox/prototype/exec/ref/src/main/java/org/apache/drill/exec/ref/rops/WindowFrameROP.java
@@ -23,6 +23,7 @@ import com.google.common.collect.Iterables;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Queues;
 import org.apache.drill.common.exceptions.DrillRuntimeException;
+import org.apache.drill.common.expression.ExpressionPosition;
 import org.apache.drill.common.expression.FieldReference;
 import org.apache.drill.common.logical.data.WindowFrame;
 import org.apache.drill.exec.ref.RecordIterator;
@@ -52,11 +53,11 @@ public class WindowFrameROP extends SingleInputROPBase<WindowFrame> {
         }
 
         if (positionRef == null) {
-            positionRef = new FieldReference("ref.position");
+            positionRef = new FieldReference("ref.position", ExpressionPosition.UNKNOWN);
         }
 
         if (segmentRef == null) {
-            segmentRef = new FieldReference("ref.segment");
+            segmentRef = new FieldReference("ref.segment", ExpressionPosition.UNKNOWN);
         }
 
         withinRef = config.getWithin();

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/sandbox/prototype/exec/ref/src/main/java/org/apache/drill/exec/ref/values/BaseArrayValue.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/ref/src/main/java/org/apache/drill/exec/ref/values/BaseArrayValue.java b/sandbox/prototype/exec/ref/src/main/java/org/apache/drill/exec/ref/values/BaseArrayValue.java
index 503c835..5831d37 100644
--- a/sandbox/prototype/exec/ref/src/main/java/org/apache/drill/exec/ref/values/BaseArrayValue.java
+++ b/sandbox/prototype/exec/ref/src/main/java/org/apache/drill/exec/ref/values/BaseArrayValue.java
@@ -18,13 +18,28 @@
 package org.apache.drill.exec.ref.values;
 
 import org.apache.drill.common.expression.PathSegment;
-import org.apache.drill.common.expression.types.DataType;
+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.tools.ant.types.DataType;
 
 
 public abstract class BaseArrayValue extends BaseDataValue implements ContainerValue{
 
+  private MajorType initiatingType;
+  private MajorType runningType = MajorType.newBuilder().setMode(DataMode.REPEATED).setMinorType(MinorType.LATE).build();
+  
   @Override
   public void addValue(PathSegment segment, DataValue v) {
+    if(initiatingType == null){
+      initiatingType = v.getDataType();
+      runningType = initiatingType.toBuilder().setMode(DataMode.REPEATED).build();
+    }else{
+      if(!v.getDataType().equals(initiatingType)){
+        throw new RuntimeException("The reference interpreter doesn't support polymorphic types.");
+      }
+    }
+    
     DataValue fullPathValue = ValueUtils.getIntermediateValues(segment.getChild(), v);
     if(segment.isArray()){ // we need to place this object in the given position.
       int index = segment.getArraySegment().getIndex();
@@ -60,8 +75,8 @@ public abstract class BaseArrayValue extends BaseDataValue implements ContainerV
   }
 
   @Override
-  public DataType getDataType() {
-    return DataType.ARRAY;
+  public MajorType getDataType() {
+    return runningType;
   }
   
   

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/sandbox/prototype/exec/ref/src/main/java/org/apache/drill/exec/ref/values/BaseMapValue.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/ref/src/main/java/org/apache/drill/exec/ref/values/BaseMapValue.java b/sandbox/prototype/exec/ref/src/main/java/org/apache/drill/exec/ref/values/BaseMapValue.java
index 4fa7a51..87bd344 100644
--- a/sandbox/prototype/exec/ref/src/main/java/org/apache/drill/exec/ref/values/BaseMapValue.java
+++ b/sandbox/prototype/exec/ref/src/main/java/org/apache/drill/exec/ref/values/BaseMapValue.java
@@ -21,8 +21,11 @@ import java.util.Map;
 
 import org.apache.drill.common.expression.PathSegment;
 import org.apache.drill.common.expression.ValueExpressions.CollisionBehavior;
-import org.apache.drill.common.expression.types.DataType;
+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.exec.ref.exceptions.RecordException;
+import org.apache.tools.ant.types.DataType;
 
 public abstract class BaseMapValue extends BaseDataValue implements ContainerValue,
     Iterable<Map.Entry<CharSequence, DataValue>> {
@@ -91,8 +94,8 @@ public abstract class BaseMapValue extends BaseDataValue implements ContainerVal
   }
 
   @Override
-  public DataType getDataType() {
-    return DataType.MAP;
+  public MajorType getDataType() {
+    return MajorType.newBuilder().setMinorType(MinorType.REPEATMAP).setMode(DataMode.REPEATED).build();
   }
 
   public void merge(BaseMapValue otherMap) {

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/sandbox/prototype/exec/ref/src/main/java/org/apache/drill/exec/ref/values/DataValue.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/ref/src/main/java/org/apache/drill/exec/ref/values/DataValue.java b/sandbox/prototype/exec/ref/src/main/java/org/apache/drill/exec/ref/values/DataValue.java
index c1a2980..fca9cbb 100644
--- a/sandbox/prototype/exec/ref/src/main/java/org/apache/drill/exec/ref/values/DataValue.java
+++ b/sandbox/prototype/exec/ref/src/main/java/org/apache/drill/exec/ref/values/DataValue.java
@@ -3,7 +3,7 @@ package org.apache.drill.exec.ref.values;
 import java.io.IOException;
 
 import org.apache.drill.common.expression.PathSegment;
-import org.apache.drill.common.expression.types.DataType;
+import org.apache.drill.common.types.TypeProtos.MajorType;
 import org.apache.drill.exec.ref.rops.DataWriter;
 
 
@@ -15,7 +15,7 @@ public interface DataValue {
   public void addValue(PathSegment segment, DataValue v);
   public void removeValue(PathSegment segment);
   public void write(DataWriter writer) throws IOException;
-  public DataType getDataType();
+  public MajorType getDataType();
   public NumericValue getAsNumeric();
   public ContainerValue getAsContainer();
   public StringValue getAsStringValue();

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/sandbox/prototype/exec/ref/src/main/java/org/apache/drill/exec/ref/values/NumericValue.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/ref/src/main/java/org/apache/drill/exec/ref/values/NumericValue.java b/sandbox/prototype/exec/ref/src/main/java/org/apache/drill/exec/ref/values/NumericValue.java
index 88efb92..45b8174 100644
--- a/sandbox/prototype/exec/ref/src/main/java/org/apache/drill/exec/ref/values/NumericValue.java
+++ b/sandbox/prototype/exec/ref/src/main/java/org/apache/drill/exec/ref/values/NumericValue.java
@@ -21,6 +21,7 @@ import java.math.BigDecimal;
 import java.math.BigInteger;
 
 import org.apache.drill.common.exceptions.DrillRuntimeException;
+import org.apache.drill.common.types.Types;
 import org.apache.drill.exec.ref.eval.EvaluatorTypes.BasicEvaluator;
 import org.apache.drill.exec.ref.values.ScalarValues.DoubleScalar;
 import org.apache.drill.exec.ref.values.ScalarValues.FloatScalar;
@@ -83,7 +84,7 @@ public abstract class NumericValue extends BaseDataValue implements ComparableVa
   @Override
   public boolean equals(DataValue v) {
     if(v == null) return false;
-    if(v.getDataType().isNumericType()){
+    if(Types.isNumericType(v.getDataType())){
       return this.compareTo(v) == 0;
     }else{
       return false;
@@ -140,7 +141,7 @@ public abstract class NumericValue extends BaseDataValue implements ComparableVa
   
   @Override
   public boolean supportsCompare(DataValue dv2) {
-    return dv2.getDataType().isNumericType();
+    return Types.isNumericType(dv2.getDataType());
   }
 
   
@@ -152,22 +153,22 @@ public abstract class NumericValue extends BaseDataValue implements ComparableVa
   }
 
   public long getAsLong(){
-    throw new DrillRuntimeException(String.format("A %s value can not be implicitly cast to a long.", this.getDataType().getName()));
+    throw new DrillRuntimeException(String.format("A %s value can not be implicitly cast to a long.", this.getDataType()));
   }
   public int getAsInt(){
-    throw new DrillRuntimeException(String.format("A %s value can not be implicitly cast to an int.", this.getDataType().getName()));
+    throw new DrillRuntimeException(String.format("A %s value can not be implicitly cast to an int.", this.getDataType()));
   }
   public float getAsFloat(){
-    throw new DrillRuntimeException(String.format("A %s value can not be implicitly cast to an float.", this.getDataType().getName()));
+    throw new DrillRuntimeException(String.format("A %s value can not be implicitly cast to an float.", this.getDataType()));
   }
   public double getAsDouble(){
-    throw new DrillRuntimeException(String.format("A %s value can not be implicitly cast to a double.", this.getDataType().getName()));
+    throw new DrillRuntimeException(String.format("A %s value can not be implicitly cast to a double.", this.getDataType()));
   }
   public BigDecimal getAsBigDecimal(){
-    throw new DrillRuntimeException(String.format("A %s value can not be implicitly cast to an big decimal.", this.getDataType().getName()));
+    throw new DrillRuntimeException(String.format("A %s value can not be implicitly cast to an big decimal.", this.getDataType()));
   }
   public BigInteger getAsBigInteger(){
-    throw new DrillRuntimeException(String.format("A %s value can not be implicitly cast to a big integer.", this.getDataType().getName()));
+    throw new DrillRuntimeException(String.format("A %s value can not be implicitly cast to a big integer.", this.getDataType()));
   }
   
 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/sandbox/prototype/exec/ref/src/main/java/org/apache/drill/exec/ref/values/ScalarValues.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/ref/src/main/java/org/apache/drill/exec/ref/values/ScalarValues.java b/sandbox/prototype/exec/ref/src/main/java/org/apache/drill/exec/ref/values/ScalarValues.java
index d401927..7a72b2c 100644
--- a/sandbox/prototype/exec/ref/src/main/java/org/apache/drill/exec/ref/values/ScalarValues.java
+++ b/sandbox/prototype/exec/ref/src/main/java/org/apache/drill/exec/ref/values/ScalarValues.java
@@ -22,10 +22,14 @@ import java.math.BigDecimal;
 import java.math.BigInteger;
 import java.util.Arrays;
 
-import org.apache.drill.common.expression.types.DataType;
+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 org.apache.drill.exec.ref.eval.EvaluatorTypes.BasicEvaluator;
 import org.apache.drill.exec.ref.rops.DataWriter;
 import org.apache.hadoop.io.BytesWritable;
+import org.apache.tools.ant.types.DataType;
 
 import com.google.common.hash.HashFunction;
 import com.google.common.hash.Hashing;
@@ -38,7 +42,8 @@ public final class ScalarValues {
   
   public static class StringScalar extends BaseDataValue implements StringValue, ComparableValue, BasicEvaluator {
     private CharSequence seq;
-
+    private MajorType type = MajorType.newBuilder().setMinorType(MinorType.VARCHAR4).setMode(DataMode.OPTIONAL).build();
+    
     public StringScalar(CharSequence seq){
       this.seq = seq;
     }
@@ -60,7 +65,7 @@ public final class ScalarValues {
 
     @Override
     public boolean supportsCompare(DataValue dv2) {
-      return dv2.getDataType() == DataType.NVARCHAR;
+      return Types.isStringScalarType(dv2.getDataType());
     }
 
     @Override
@@ -69,8 +74,8 @@ public final class ScalarValues {
     }
 
     @Override
-    public DataType getDataType() {
-      return DataType.NVARCHAR;
+    public MajorType getDataType() {
+      return type;
     }
 
     @Override
@@ -99,7 +104,7 @@ public final class ScalarValues {
 
     @Override
     public boolean equals(DataValue v) {
-      if(v.getDataType() != this.getDataType()) return false;
+      if(!v.getDataType().equals(this.getDataType())) return false;
       return seq.equals(v.getAsStringValue().getString());
     }
 
@@ -118,6 +123,8 @@ public final class ScalarValues {
   
   public static class BooleanScalar extends BaseDataValue implements BooleanValue, BasicEvaluator{
     private boolean b;
+    private MajorType type = MajorType.newBuilder().setMinorType(MinorType.BOOLEAN).setMode(DataMode.OPTIONAL).build();
+    
     public BooleanScalar(boolean b){
       this.b = b;
     }
@@ -138,8 +145,8 @@ public final class ScalarValues {
     }
 
     @Override
-    public DataType getDataType() {
-      return DataType.BOOLEAN;
+    public MajorType getDataType() {
+      return type;
     }
 
     @Override
@@ -159,7 +166,7 @@ public final class ScalarValues {
     
     @Override
     public boolean equals(DataValue v) {
-      if(v.getDataType() != this.getDataType()) return false;
+      if(v.getDataType().equals(this.getDataType())) return false;
       return b == v.getAsBooleanValue().getBoolean();
     }
 
@@ -176,6 +183,8 @@ public final class ScalarValues {
   }
   
   public static class LongScalar extends NumericValue{
+    
+    private MajorType type = MajorType.newBuilder().setMinorType(MinorType.BIGINT).setMode(DataMode.OPTIONAL).build();
     long l;
     public LongScalar(long l) {
       this.l = l;
@@ -212,8 +221,8 @@ public final class ScalarValues {
     }
 
     @Override
-    public DataType getDataType() {
-      return DataType.INT64;
+    public MajorType getDataType() {
+      return type;
     }
 
     @Override
@@ -244,6 +253,7 @@ public final class ScalarValues {
   }
   
   public static class IntegerScalar extends NumericValue{
+    private MajorType type = MajorType.newBuilder().setMinorType(MinorType.INT).setMode(DataMode.OPTIONAL).build();
     int i;
     
     public IntegerScalar(int i){
@@ -266,8 +276,8 @@ public final class ScalarValues {
     }
 
     @Override
-    public DataType getDataType() {
-      return DataType.INT32;
+    public MajorType getDataType() {
+      return type;
     }
 
     @Override
@@ -320,6 +330,8 @@ public final class ScalarValues {
 
   
   public static class FloatScalar extends NumericValue{
+    
+    private MajorType type = MajorType.newBuilder().setMinorType(MinorType.FLOAT4).setMode(DataMode.OPTIONAL).build();
     float f;
     public FloatScalar(float f){
       this.f = f;
@@ -336,8 +348,8 @@ public final class ScalarValues {
     }
 
     @Override
-    public DataType getDataType() {
-      return DataType.FLOAT32;
+    public MajorType getDataType() {
+      return type;
     }
 
     @Override
@@ -373,14 +385,16 @@ public final class ScalarValues {
  
   
   public static class DoubleScalar extends NumericValue{
+    private MajorType type = MajorType.newBuilder().setMinorType(MinorType.FLOAT8).setMode(DataMode.OPTIONAL).build();
     private double d;
+    
     public DoubleScalar(double d){
       this.d = d;
     }
 
     @Override
-    public DataType getDataType() {
-      return DataType.FLOAT64;
+    public MajorType getDataType() {
+      return type;
     }
 
     @Override
@@ -427,6 +441,7 @@ public final class ScalarValues {
   }
   
   public static class BytesScalar extends BaseDataValue implements BytesValue{
+    private MajorType type = MajorType.newBuilder().setMinorType(MinorType.VARBINARY4).setMode(DataMode.OPTIONAL).build();
     private BytesWritable.Comparator comparator = new BytesWritable.Comparator();
     private final static HashFunction HASH = Hashing.murmur3_32();
 
@@ -442,7 +457,7 @@ public final class ScalarValues {
 
     @Override
     public boolean supportsCompare(DataValue dv2) {
-      return dv2.getDataType() == DataType.BYTES;
+      return Types.isBytesScalarType(dv2.getDataType());
     }
 
 
@@ -454,8 +469,8 @@ public final class ScalarValues {
     }
 
     @Override
-    public DataType getDataType() {
-      return DataType.BYTES;
+    public MajorType getDataType() {
+      return type;
     }
 
     @Override
@@ -480,7 +495,7 @@ public final class ScalarValues {
 
     @Override
     public boolean equals(DataValue v) {
-      if(v.getDataType() != this.getDataType()) return false;
+      if(!v.getDataType().equals(this.getDataType())) return false;
       BytesValue other = v.getAsBytesValue();
       if(this.getLength() != other.getLength()) return false;
       for(int i =0; i < this.getLength(); i++){
@@ -505,15 +520,18 @@ public final class ScalarValues {
   
   
   static class NullValue extends BaseDataValue{
-
+    
+    // not sure what to do here... 
+    MajorType type = MajorType.newBuilder().setMode(DataMode.OPTIONAL).setMinorType(MinorType.LATE).build();
+    
     @Override
     public void write(DataWriter writer) throws IOException {
       writer.writeNullValue();
     }
 
     @Override
-    public DataType getDataType() {
-      return DataType.NULL;
+    public MajorType getDataType() {
+      return type;
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/sandbox/prototype/exec/ref/src/main/java/org/apache/drill/exec/ref/values/SimpleArrayValue.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/ref/src/main/java/org/apache/drill/exec/ref/values/SimpleArrayValue.java b/sandbox/prototype/exec/ref/src/main/java/org/apache/drill/exec/ref/values/SimpleArrayValue.java
index a8c11de..b87a4db 100644
--- a/sandbox/prototype/exec/ref/src/main/java/org/apache/drill/exec/ref/values/SimpleArrayValue.java
+++ b/sandbox/prototype/exec/ref/src/main/java/org/apache/drill/exec/ref/values/SimpleArrayValue.java
@@ -21,7 +21,7 @@ import java.io.IOException;
 import java.util.Arrays;
 import java.util.Objects;
 
-import org.apache.drill.common.expression.types.DataType;
+import org.apache.drill.common.types.TypeProtos.MinorType;
 import org.apache.drill.exec.ref.rops.DataWriter;
 
 
@@ -119,7 +119,7 @@ public class SimpleArrayValue extends BaseArrayValue{
 
   @Override
   public boolean equals(DataValue v) {
-    if(v.getDataType() != DataType.MAP) return false;
+    if(v.getDataType().getMinorType() == MinorType.REPEATMAP) return false;
     BaseArrayValue other = v.getAsContainer().getAsArray();
     if(this.size() != other.size()) return false;
     for(int i =0; i < this.size(); i++){

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/sandbox/prototype/exec/ref/src/main/java/org/apache/drill/exec/ref/values/SimpleMapValue.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/ref/src/main/java/org/apache/drill/exec/ref/values/SimpleMapValue.java b/sandbox/prototype/exec/ref/src/main/java/org/apache/drill/exec/ref/values/SimpleMapValue.java
index 1c170f2..ab231ff 100644
--- a/sandbox/prototype/exec/ref/src/main/java/org/apache/drill/exec/ref/values/SimpleMapValue.java
+++ b/sandbox/prototype/exec/ref/src/main/java/org/apache/drill/exec/ref/values/SimpleMapValue.java
@@ -24,9 +24,10 @@ import java.util.Iterator;
 import java.util.Map;
 import java.util.Map.Entry;
 
-import org.apache.drill.common.expression.types.DataType;
+import org.apache.drill.common.types.TypeProtos.DataMode;
 import org.apache.drill.exec.ref.exceptions.RecordException;
 import org.apache.drill.exec.ref.rops.DataWriter;
+import org.apache.tools.ant.types.DataType;
 
 public class SimpleMapValue extends BaseMapValue{
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(SimpleMapValue.class);
@@ -86,7 +87,7 @@ public class SimpleMapValue extends BaseMapValue{
   @Override
   public boolean equals(DataValue v) {
     if(v == null) return false;
-    if(v.getDataType() != DataType.MAP) return false;
+    if(v.getDataType().getMode() != DataMode.REPEATED) return false;
     BaseMapValue other = v.getAsContainer().getAsMap();
     for(Entry<CharSequence, DataValue> e : this){
       DataValue v2 = other.getByName(e.getKey());

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/sandbox/prototype/exec/ref/src/main/java/org/apache/drill/exec/ref/values/ValueReader.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/ref/src/main/java/org/apache/drill/exec/ref/values/ValueReader.java b/sandbox/prototype/exec/ref/src/main/java/org/apache/drill/exec/ref/values/ValueReader.java
index 9adcca0..3a6be51 100644
--- a/sandbox/prototype/exec/ref/src/main/java/org/apache/drill/exec/ref/values/ValueReader.java
+++ b/sandbox/prototype/exec/ref/src/main/java/org/apache/drill/exec/ref/values/ValueReader.java
@@ -18,35 +18,37 @@
 package org.apache.drill.exec.ref.values;
 
 import org.apache.drill.common.exceptions.DrillRuntimeException;
-import org.apache.drill.common.expression.types.DataType;
+import org.apache.drill.common.types.TypeProtos.DataMode;
+import org.apache.drill.common.types.TypeProtos.MinorType;
+import org.apache.drill.common.types.Types;
 
 public class ValueReader {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ValueReader.class);
   
   public static boolean getBoolean(DataValue v){
-    if(v.getDataType() == DataType.BOOLEAN){
+    if(v.getDataType().getMinorType() == MinorType.BOOLEAN && v.getDataType().getMode() != DataMode.REPEATED){
       return v.getAsBooleanValue().getBoolean();
     }else{
-      throw new DrillRuntimeException(String.format("Unable to get boolean.  Type os a %s", v.getClass().getCanonicalName()));
+      throw new DrillRuntimeException(String.format("Unable to get boolean.  Type was a %s", v.getClass().getCanonicalName()));
     }
   }
   
   public static long getLong(DataValue v){
-    if(v.getDataType().isNumericType()){
+    if(Types.isNumericType(v.getDataType())){
       return v.getAsNumeric().getAsLong();
     }else{
       throw new DrillRuntimeException(String.format("Unable to get value.  %s is not a numeric type.", v.getClass().getCanonicalName()));
     }
   }
   public static double getDouble(DataValue v){
-    if(v.getDataType().isNumericType()){
+    if(Types.isNumericType(v.getDataType())){
       return v.getAsNumeric().getAsDouble();
     }else{
       throw new DrillRuntimeException(String.format("Unable to get value.  %s is not a numeric type.", v.getClass().getCanonicalName()));
     }
   }
   public static CharSequence getChars(DataValue v){
-    if(v.getDataType() == DataType.NVARCHAR){
+    if(Types.isStringScalarType(v.getDataType())){
       return v.getAsStringValue().getString();
     }else{
       throw new DrillRuntimeException(String.format("Unable to get value.  %s is not a StringValue type.", v.getClass().getCanonicalName()));

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/sandbox/prototype/exec/ref/src/main/java/org/apache/drill/exec/ref/values/ValueUtils.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/ref/src/main/java/org/apache/drill/exec/ref/values/ValueUtils.java b/sandbox/prototype/exec/ref/src/main/java/org/apache/drill/exec/ref/values/ValueUtils.java
index ac51d3b..dff8b11 100644
--- a/sandbox/prototype/exec/ref/src/main/java/org/apache/drill/exec/ref/values/ValueUtils.java
+++ b/sandbox/prototype/exec/ref/src/main/java/org/apache/drill/exec/ref/values/ValueUtils.java
@@ -19,8 +19,11 @@ package org.apache.drill.exec.ref.values;
 
 import org.apache.drill.common.expression.PathSegment;
 import org.apache.drill.common.expression.ValueExpressions.CollisionBehavior;
-import org.apache.drill.common.expression.types.DataType;
+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.exec.ref.exceptions.RecordException;
+import org.apache.tools.ant.types.DataType;
 
 public class ValueUtils {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ValueUtils.class);
@@ -47,17 +50,17 @@ public class ValueUtils {
       a.addToArray(1, newValue);
       return a;
     case MERGE_OVERRIDE:
-      DataType oldT = oldValue.getDataType();
-      DataType newT = oldValue.getDataType();
-      if(oldT == DataType.MAP && newT == DataType.MAP){
+      MajorType oldT = oldValue.getDataType();
+      MajorType newT = newValue.getDataType();
+      if(oldT.getMinorType() == MinorType.REPEATMAP && newT.getMinorType() == MinorType.REPEATMAP){
         oldValue.getAsContainer().getAsMap().merge(newValue.getAsContainer().getAsMap());
         return oldValue;
-      }else if(oldT == DataType.ARRAY && newT == DataType.ARRAY){
+      }else if(oldT.getMode() == DataMode.REPEATED && newT.getMode() == DataMode.REPEATED){
         logger.debug("Merging two arrays. {} and {}", oldValue, newValue);
         oldValue.getAsContainer().getAsArray().append(newValue.getAsContainer().getAsArray());
         return oldValue;
-      }else if(oldT == DataType.ARRAY || newT == DataType.ARRAY || oldT == DataType.MAP || newT == DataType.MAP){
-        throw new RecordException(String.format("Failure while doing query.  You requested a merge of values that were incompatibile.  Examples include merging an array and a map or merging a map/array with a scalar.  Merge Types were %s and %s.", oldT.getName(), newT.getName()), null);
+      }else if(oldT.getMode() == DataMode.REPEATED || newT.getMode() == DataMode.REPEATED || oldT.getMinorType() == MinorType.REPEATMAP || newT.getMinorType() == MinorType.REPEATMAP){
+        throw new RecordException(String.format("Failure while doing query.  You requested a merge of values that were incompatibile.  Examples include merging an array and a map or merging a map/array with a scalar.  Merge Types were %s and %s.", oldT, newT), null);
       }else{
         // scalar type, just override the value.
         return newValue;

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/sandbox/prototype/exec/ref/src/test/java/org/apache/drill/exec/ref/RunSimplePlan.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/ref/src/test/java/org/apache/drill/exec/ref/RunSimplePlan.java b/sandbox/prototype/exec/ref/src/test/java/org/apache/drill/exec/ref/RunSimplePlan.java
index 110e655..5b1841b 100644
--- a/sandbox/prototype/exec/ref/src/test/java/org/apache/drill/exec/ref/RunSimplePlan.java
+++ b/sandbox/prototype/exec/ref/src/test/java/org/apache/drill/exec/ref/RunSimplePlan.java
@@ -24,6 +24,7 @@ import java.util.Collection;
 import org.apache.drill.common.config.DrillConfig;
 import org.apache.drill.common.logical.LogicalPlan;
 import org.apache.drill.common.util.FileUtils;
+import org.apache.drill.exec.ref.RunOutcome.OutcomeType;
 import org.apache.drill.exec.ref.eval.BasicEvaluatorFactory;
 import org.apache.drill.exec.ref.rse.RSERegistry;
 import org.junit.Test;
@@ -36,7 +37,7 @@ public class RunSimplePlan{
   
   
   @Test
-  public void parseSimplePlan() throws Exception{
+  public void parseSimplePlan() throws Throwable{
     DrillConfig config = DrillConfig.create();
     LogicalPlan plan = LogicalPlan.parse(config, Files.toString(FileUtils.getResourceAsFile("/simple_plan.json"), Charsets.UTF_8));
     IteratorRegistry ir = new IteratorRegistry();
@@ -48,7 +49,7 @@ public class RunSimplePlan{
   }
   
   @Test
-  public void joinPlan() throws Exception{
+  public void joinPlan() throws Throwable{
     DrillConfig config = DrillConfig.create();
     LogicalPlan plan = LogicalPlan.parse(config, Files.toString(FileUtils.getResourceAsFile("/simple_join.json"), Charsets.UTF_8));
     IteratorRegistry ir = new IteratorRegistry();
@@ -56,11 +57,12 @@ public class RunSimplePlan{
     i.setup();
     Collection<RunOutcome> outcomes = i.run();
     assertEquals(outcomes.size(), 1);
-    assertEquals(outcomes.iterator().next().outcome, RunOutcome.OutcomeType.SUCCESS);
+    RunOutcome out = outcomes.iterator().next();
+    if(out.outcome != OutcomeType.FAILED && out.exception != null) logger.error("Failure while running {}", out.exception);
   }
   
   @Test
-  public void flattenPlan() throws Exception{
+  public void flattenPlan() throws Throwable{
     DrillConfig config = DrillConfig.create();
     LogicalPlan plan = LogicalPlan.parse(config, Files.toString(FileUtils.getResourceAsFile("/simple_plan_flattened.json"), Charsets.UTF_8));
     IteratorRegistry ir = new IteratorRegistry();
@@ -68,6 +70,8 @@ public class RunSimplePlan{
     i.setup();
     Collection<RunOutcome> outcomes = i.run();
     assertEquals(outcomes.size(), 1);
-    assertEquals(outcomes.iterator().next().outcome, RunOutcome.OutcomeType.SUCCESS);
+    RunOutcome out = outcomes.iterator().next();
+    if(out.outcome != OutcomeType.FAILED && out.exception != null) logger.error("Failure while running {}", out.exception);
+    assertEquals(out.outcome, RunOutcome.OutcomeType.SUCCESS);
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/sandbox/prototype/exec/ref/src/test/java/org/apache/drill/exec/ref/TestUtils.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/ref/src/test/java/org/apache/drill/exec/ref/TestUtils.java b/sandbox/prototype/exec/ref/src/test/java/org/apache/drill/exec/ref/TestUtils.java
index d7cc690..c340e7e 100644
--- a/sandbox/prototype/exec/ref/src/test/java/org/apache/drill/exec/ref/TestUtils.java
+++ b/sandbox/prototype/exec/ref/src/test/java/org/apache/drill/exec/ref/TestUtils.java
@@ -12,6 +12,7 @@ import java.util.concurrent.ArrayBlockingQueue;
 import java.util.concurrent.BlockingQueue;
 
 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.util.FileUtils;
@@ -27,7 +28,7 @@ import com.google.common.io.Files;
 public class TestUtils {
   public static RecordIterator jsonToRecordIterator(String schemaPath, String j) throws IOException {
     InputStream is = new ByteArrayInputStream(j.getBytes());
-    JSONRecordReader reader = new JSONRecordReader(new SchemaPath(schemaPath), DrillConfig.create(), is, null);
+    JSONRecordReader reader = new JSONRecordReader(new SchemaPath(schemaPath, ExpressionPosition.UNKNOWN), DrillConfig.create(), is, null);
     return reader.getIterator();
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/sandbox/prototype/exec/ref/src/test/java/org/apache/drill/exec/ref/rops/CollapsingAggregateTest.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/ref/src/test/java/org/apache/drill/exec/ref/rops/CollapsingAggregateTest.java b/sandbox/prototype/exec/ref/src/test/java/org/apache/drill/exec/ref/rops/CollapsingAggregateTest.java
index edc92a4..34a500c 100644
--- a/sandbox/prototype/exec/ref/src/test/java/org/apache/drill/exec/ref/rops/CollapsingAggregateTest.java
+++ b/sandbox/prototype/exec/ref/src/test/java/org/apache/drill/exec/ref/rops/CollapsingAggregateTest.java
@@ -21,6 +21,7 @@ import static org.junit.Assert.assertEquals;
 
 import java.util.List;
 
+import org.apache.drill.common.expression.ExpressionPosition;
 import org.apache.drill.common.expression.SchemaPath;
 import org.apache.drill.exec.ref.TestUtils;
 import org.apache.drill.exec.ref.UnbackedRecord;
@@ -38,8 +39,8 @@ public class CollapsingAggregateTest {
 
     DataValue[] depts = {DataValue.NULL_VALUE, new LongScalar(31), new LongScalar(33), new LongScalar(34)};
     DataValue[] cnts = {new LongScalar(1), new LongScalar(1), new LongScalar(2), new LongScalar(2)};
-    SchemaPath typeCount = new SchemaPath("typeCount");
-    SchemaPath dept = new SchemaPath("deptId");
+    SchemaPath typeCount = new SchemaPath("typeCount", ExpressionPosition.UNKNOWN);
+    SchemaPath dept = new SchemaPath("deptId", ExpressionPosition.UNKNOWN);
     for(int i =0; i < depts.length; i++){
       UnbackedRecord r = records.get(i);
       assertEquals(String.format("Invalid dept value for record %d.", i), depts[i], r.getField(dept));

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/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 499b335..9aea930 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
@@ -4,6 +4,7 @@ import com.fasterxml.jackson.databind.ObjectMapper;
 import com.google.common.base.Charsets;
 import com.google.common.io.Files;
 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;
@@ -50,9 +51,9 @@ public class ConstantROPTest {
             while (iter.next() != RecordIterator.NextOutcome.NONE_LEFT){
                 System.out.println(ptr);
                 org.junit.Assert.assertEquals("Integer value in record " + i + " is incorrect.",
-                        ptr.getField(new SchemaPath("c1")), new ScalarValues.IntegerScalar(i));
+                        ptr.getField(new SchemaPath("c1", ExpressionPosition.UNKNOWN)), new ScalarValues.IntegerScalar(i));
                 org.junit.Assert.assertEquals("String value in record " + i + " is incorrect.",
-                        ptr.getField(new SchemaPath("c2")), new ScalarValues.StringScalar("string " + i));
+                        ptr.getField(new SchemaPath("c2", ExpressionPosition.UNKNOWN)), new ScalarValues.StringScalar("string " + i));
                 i++;
             }
             org.junit.Assert.assertEquals("Incorrect number of records returned by 'constant' record iterator.", 3, i - 1);

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/sandbox/prototype/exec/ref/src/test/java/org/apache/drill/exec/ref/rops/OrderROPTest.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/ref/src/test/java/org/apache/drill/exec/ref/rops/OrderROPTest.java b/sandbox/prototype/exec/ref/src/test/java/org/apache/drill/exec/ref/rops/OrderROPTest.java
index 23d0fe6..8a49040 100644
--- a/sandbox/prototype/exec/ref/src/test/java/org/apache/drill/exec/ref/rops/OrderROPTest.java
+++ b/sandbox/prototype/exec/ref/src/test/java/org/apache/drill/exec/ref/rops/OrderROPTest.java
@@ -21,6 +21,7 @@ import static org.junit.Assert.assertEquals;
 
 import java.util.List;
 
+import org.apache.drill.common.expression.ExpressionPosition;
 import org.apache.drill.common.expression.SchemaPath;
 import org.apache.drill.exec.ref.TestUtils;
 import org.apache.drill.exec.ref.UnbackedRecord;
@@ -37,7 +38,7 @@ public class OrderROPTest {
     List<UnbackedRecord> records = TestUtils.getResultAsUnbackedRecords("/order/nulls-first.json");
 
     DataValue[] depts = {DataValue.NULL_VALUE, new LongScalar(31), new LongScalar(33), new LongScalar(34)};
-    SchemaPath dept = new SchemaPath("deptId");
+    SchemaPath dept = new SchemaPath("deptId", ExpressionPosition.UNKNOWN);
     for(int i =0; i < depts.length; i++){
       UnbackedRecord r = records.get(i);
       assertEquals(String.format("Invalid dept value for record %d.", i), depts[i], r.getField(dept));
@@ -50,7 +51,7 @@ public class OrderROPTest {
     List<UnbackedRecord> records = TestUtils.getResultAsUnbackedRecords("/order/nulls-last.json");
 
     DataValue[] depts = {new LongScalar(31), new LongScalar(33), new LongScalar(34), DataValue.NULL_VALUE};
-    SchemaPath dept = new SchemaPath("deptId");
+    SchemaPath dept = new SchemaPath("deptId", ExpressionPosition.UNKNOWN);
     for(int i =0; i < depts.length; i++){
       UnbackedRecord r = records.get(i);
       assertEquals(String.format("Invalid dept value for record %d.", i), depts[i], r.getField(dept));

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/sandbox/prototype/exec/ref/src/test/java/org/apache/drill/exec/ref/rops/WindowFrameROPTest.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/ref/src/test/java/org/apache/drill/exec/ref/rops/WindowFrameROPTest.java b/sandbox/prototype/exec/ref/src/test/java/org/apache/drill/exec/ref/rops/WindowFrameROPTest.java
index b0aeb99..38f97b1 100644
--- a/sandbox/prototype/exec/ref/src/test/java/org/apache/drill/exec/ref/rops/WindowFrameROPTest.java
+++ b/sandbox/prototype/exec/ref/src/test/java/org/apache/drill/exec/ref/rops/WindowFrameROPTest.java
@@ -1,6 +1,8 @@
 package org.apache.drill.exec.ref.rops;
 
 import com.google.common.collect.Lists;
+
+import org.apache.drill.common.expression.ExpressionPosition;
 import org.apache.drill.common.expression.FieldReference;
 import org.apache.drill.common.expression.SchemaPath;
 import org.apache.drill.common.logical.data.WindowFrame;
@@ -112,7 +114,7 @@ public class WindowFrameROPTest {
                 "{id: 2, v: 2}" +
                 "{id: 3, v: 3}" +
                 "{id: 4, v: 4}";
-        WindowFrameROP rop = new WindowFrameROP(new WindowFrame(new FieldReference("test.v"), null, -2L, 2L));
+        WindowFrameROP rop = new WindowFrameROP(new WindowFrame(new FieldReference("test.v", ExpressionPosition.UNKNOWN), null, -2L, 2L));
         RecordIterator incoming = TestUtils.jsonToRecordIterator("test", withinInput);
         rop.setInput(incoming);
         RecordIterator out = rop.getOutput();
@@ -136,7 +138,7 @@ public class WindowFrameROPTest {
                 "{id: 2, v: 1}" +
                 "{id: 3, v: 1}" +
                 "{id: 4, v: 2}";
-        WindowFrameROP rop = new WindowFrameROP(new WindowFrame(new FieldReference("test.v"), null, -1L, 2L));
+        WindowFrameROP rop = new WindowFrameROP(new WindowFrame(new FieldReference("test.v", ExpressionPosition.UNKNOWN), null, -1L, 2L));
         RecordIterator incoming = TestUtils.jsonToRecordIterator("test", withinInput);
         rop.setInput(incoming);
         RecordIterator out = rop.getOutput();
@@ -166,7 +168,7 @@ public class WindowFrameROPTest {
                 "{id: 4, v: 1}" +
                 "{id: 5, v: 1}" +
                 "{id: 6, v: 2}";
-        WindowFrameROP rop = new WindowFrameROP(new WindowFrame(new FieldReference("test.v"), null, -1L, 3L));
+        WindowFrameROP rop = new WindowFrameROP(new WindowFrame(new FieldReference("test.v", ExpressionPosition.UNKNOWN), null, -1L, 3L));
         RecordIterator incoming = TestUtils.jsonToRecordIterator("test", withinInput);
         rop.setInput(incoming);
         RecordIterator out = rop.getOutput();
@@ -196,14 +198,14 @@ public class WindowFrameROPTest {
     }
 
     private void verifyWindowOrder(List<WindowObj> expectedIds, RecordIterator out) {
-        verifyWindowOrder(expectedIds, out, new SchemaPath("ref.segment"), new SchemaPath("ref.position"));
+        verifyWindowOrder(expectedIds, out, new SchemaPath("ref.segment", ExpressionPosition.UNKNOWN), new SchemaPath("ref.position", ExpressionPosition.UNKNOWN));
     }
 
     private void verifyWindowOrder(List<WindowObj> expectedIds, RecordIterator out, SchemaPath segment, SchemaPath position) {
         RecordIterator.NextOutcome outcome = out.next();
         RecordPointer pointer = out.getRecordPointer();
         int count = 0;
-        SchemaPath id = new SchemaPath("test.id");
+        SchemaPath id = new SchemaPath("test.id", ExpressionPosition.UNKNOWN);
         int expectedSize = expectedIds.size();
         while (outcome != RecordIterator.NextOutcome.NONE_LEFT) {
             count += 1;


Re: [34/53] [abbrv] git commit: Updated value vectors inheritance model. Moved Mutables to separate Mutator subclasses. Broke VVs into separate files rather than one large class.

Posted by Ted Dunning <te...@gmail.com>.
On Fri, Jul 19, 2013 at 6:57 PM, <ja...@apache.org> wrote:

> Updated value vectors inheritance model.
> Moved Mutables to separate Mutator subclasses.
> Broke VVs into separate files rather than one large class.
>
>
> Project: http://git-wip-us.apache.org/repos/asf/incubator-drill/repo
> Commit:
> http://git-wip-us.apache.org/repos/asf/incubator-drill/commit/36793bb2
> Tree: http://git-wip-us.apache.org/repos/asf/incubator-drill/tree/36793bb2
> Diff: http://git-wip-us.apache.org/repos/asf/incubator-drill/diff/36793bb2
>
> Branch: refs/heads/master
> Commit: 36793bb2f22cd427c8b70d2f5dfe4a4d8a3a6894
> Parents: 7075cca
> Author: Jacques Nadeau <ja...@apache.org>
> Authored: Sat Jul 13 21:28:12 2013 -0700
> Committer: Jacques Nadeau <ja...@apache.org>
> Committed: Mon Jul 15 11:36:32 2013 -0700
>
> ----------------------------------------------------------------------
>  sandbox/prototype/exec/java-exec/pom.xml        |   2 +-
>  .../templates/FixedValueVectors.java            | 163 ++++
>  .../templates/NullableValueVectors.java         | 151 ++++
>  .../templates/RepeatedValueVectors.java         | 158 ++++
>  .../ValueVectors/templates/TypeHelper.java      |  28 +-
>  .../ValueVectors/templates/ValueVector.java     | 768 -------------------
>  .../templates/VariableLengthVectors.java        | 152 ++++
>  .../exec/physical/config/MockRecordReader.java  |  17 +-
>  .../physical/config/MockScanBatchCreator.java   |   3 -
>  .../drill/exec/physical/config/MockScanPOP.java |   2 +-
>  .../exec/physical/impl/FilterRecordBatch.java   |   4 +-
>  .../drill/exec/physical/impl/OutputMutator.java |   5 +-
>  .../drill/exec/physical/impl/ScanBatch.java     |  24 +-
>  .../exec/physical/impl/WireRecordBatch.java     |   7 +-
>  .../apache/drill/exec/record/BatchSchema.java   |   2 -
>  .../apache/drill/exec/record/RecordBatch.java   |   4 +-
>  .../drill/exec/record/RecordBatchLoader.java    |  22 +-
>  .../apache/drill/exec/record/WritableBatch.java |  10 +-
>  .../exec/record/vector/SelectionVector.java     |  14 +-
>  .../drill/exec/store/JSONRecordReader.java      |  87 ++-
>  .../apache/drill/exec/store/VectorHolder.java   |  11 +-
>  .../org/apache/drill/exec/vector/BitVector.java | 123 +++
>  .../apache/drill/exec/vector/ValueVector.java   | 192 +++++
>  .../physical/impl/TestSimpleFragmentRun.java    |   6 +-
>  .../exec/record/vector/TestValueVector.java     | 201 +++--
>  .../drill/exec/store/JSONRecordReaderTest.java  |  40 +-
>  26 files changed, 1212 insertions(+), 984 deletions(-)
> ----------------------------------------------------------------------
>
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/36793bb2/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 4e4df95..1b6dac0 100644
> --- a/sandbox/prototype/exec/java-exec/pom.xml
> +++ b/sandbox/prototype/exec/java-exec/pom.xml
> @@ -173,7 +173,7 @@
>                                 <version>1.0</version>
>                                 <configuration>
>
> <cfgFile>src/main/codegen/ValueVectors/config.fmpp</cfgFile>
> -
> <outputDirectory>target/generated-sources/org/apache/drill/exec/record/vector</outputDirectory>
> +
> <outputDirectory>target/generated-sources/org/apache/drill/exec/vector</outputDirectory>
>
> <templateDirectory>src/main/codegen/ValueVectors/templates</templateDirectory>
>                                 </configuration>
>                                 <executions>
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/36793bb2/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/FixedValueVectors.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/FixedValueVectors.java
> b/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/FixedValueVectors.java
> new file mode 100644
> index 0000000..09dd5d8
> --- /dev/null
> +++
> b/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/FixedValueVectors.java
> @@ -0,0 +1,163 @@
> +<@pp.dropOutputFile />
> +<#list types as type>
> +<#list type.minor as minor>
> +
> +<#if type.major == "Fixed">
> +<@pp.changeOutputFile name="${minor.class}Vector.java" />
> +package org.apache.drill.exec.vector;
> +
> +import io.netty.buffer.ByteBuf;
> +
> +import org.apache.drill.exec.memory.BufferAllocator;
> +import org.apache.drill.exec.proto.UserBitShared.FieldMetadata;
> +import org.apache.drill.exec.record.DeadBuf;
> +import org.apache.drill.exec.record.MaterializedField;
> +import org.apache.drill.exec.vector.MsgPack2Vector;
> +
> +import java.util.Random;
> +
> +/**
> + * ${minor.class} implements a vector of fixed width values.  Elements in
> the vector are accessed
> + * by position, starting from the logical start of the vector.  Values
> should be pushed onto the
> + * vector sequentially, but may be randomly accessed.
> + *   The width of each element is ${type.width} byte(s)
> + *   The equivalent Java primitive is '${minor.javaType!type.javaType}'
> + *
> + * NB: this class is automatically generated from ValueVectorTypes.tdd
> using FreeMarker.
> + */
> +@SuppressWarnings("unused")
> +public final class ${minor.class}Vector extends ValueVector {
> +  static final org.slf4j.Logger logger =
> org.slf4j.LoggerFactory.getLogger(${minor.class}Vector.class);
> +
> +  public ${minor.class}Vector(MaterializedField field, BufferAllocator
> allocator) {
> +    super(field, allocator);
> +  }
> +
> +  /**
> +   * Allocate a new memory space for this vector.  Must be called prior
> to using the ValueVector.
> +   *
> +   * @param valueCount
> +   *          The number of values which can be contained within this
> vector.
> +   */
> +  public void allocateNew(int valueCount) {
> +    totalBytes = valueCount * ${type.width};
> +    allocateNew(totalBytes, allocator.buffer(totalBytes), valueCount);
> +  }
> +
> +  @Override
> +  public int getAllocatedSize() {
> +    return (int) Math.ceil(totalBytes);
> +  }
> +
> +  /**
> +   * Get the size requirement (in bytes) for the given number of values.
>  Only accurate
> +   * for fixed width value vectors.
> +   */
> +  @Override
> +  public int getSizeFromCount(int valueCount) {
> +    return valueCount * ${type.width};
> +  }
> +
> +  public Mutator getMutator() {
> +    return new Mutator();
> +  }
> +
> + <#if (type.width > 8)>
> +
> +  public ${minor.javaType!type.javaType} get(int index) {
> +    ByteBuf dst = allocator.buffer(${type.width});
> +    data.getBytes(index * ${type.width}, dst, 0, ${type.width});
> +    return dst;
> +  }
> +
> +  @Override
> +  public Object getObject(int index) {
> +    ByteBuf dst = allocator.buffer(${type.width});
> +    data.getBytes(index, dst, 0, ${type.width});
> +    return dst;
> +  }
> +
> +
> + <#else> <#-- type.width <= 8 -->
> +
> +  public ${minor.javaType!type.javaType} get(int index) {
> +    return data.get${(minor.javaType!type.javaType)?cap_first}(index *
> ${type.width});
> +  }
> +
> +  public Object getObject(int index) {
> +    return get(index);
> +  }
> +
> +
> + </#if> <#-- type.width -->
> +
> +
> + /**
> +  * ${minor.class}.Mutator implements a mutable vector of fixed width
> values.  Elements in the
> +  * vector are accessed by position from the logical start of the vector.
>  Values should be pushed
> +  * onto the vector sequentially, but may be randomly accessed.
> +  *   The width of each element is ${type.width} byte(s)
> +  *   The equivalent Java primitive is '${minor.javaType!type.javaType}'
> +  *
> +  * NB: this class is automatically generated from ValueVectorTypes.tdd
> using FreeMarker.
> +  */
> +  public class Mutator implements ValueVector.Mutator{
> +
> +    private Mutator(){};
> +   /**
> +    * Set the element at the given index to the given value.  Note that
> widths smaller than
> +    * 32 bits are handled by the ByteBuf interface.
> +    *
> +    * @param index   position of the bit to set
> +    * @param value   value to set
> +    */
> +  <#if (type.width > 8)>
> +   public void set(int index, <#if (type.width >
> 4)>${minor.javaType!type.javaType}<#else>int</#if> value) {
> +     data.setBytes(index * ${type.width}, value);
> +   }
> +
> +   @Override
> +   public void randomizeData() {
> +     if (data != DeadBuf.DEAD_BUFFER) {
> +       Random r = new Random();
> +       for(int i =0; i < data.capacity()-${type.width}; i +=
> ${type.width}){
> +         byte[] bytes = new byte[${type.width}];
> +         r.nextBytes(bytes);
> +         data.setByte(i, bytes[0]);
> +       }
> +     }
> +   }
> +
> +  <#else> <#-- type.width <= 8 -->
> +   public void set(int index, <#if (type.width >=
> 4)>${minor.javaType!type.javaType}<#else>int</#if> value) {
> +     data.set${(minor.javaType!type.javaType)?cap_first}(index *
> ${type.width}, value);
> +   }
> +
> +   @Override
> +   public void randomizeData() {
> +     if (data != DeadBuf.DEAD_BUFFER) {
> +       Random r = new Random();
> +       for(int i =0; i < data.capacity()-${type.width}; i +=
> ${type.width}){
> +         data.set${(minor.javaType!type.javaType)?cap_first}(i,
> +             r.next<#if (type.width >=
> 4)>${(minor.javaType!type.javaType)?cap_first}
> +                   <#else>Int
> +                   </#if>());
> +       }
> +     }
> +   }
> +  </#if> <#-- type.width -->
> +
> +   @Override
> +   public void setRecordCount(int recordCount) {
> +     ${minor.class}Vector.this.setRecordCount(recordCount);
> +   }
> +
> +
> +
> +
> + }
> +}
> +
> +</#if> <#-- type.major -->
> +</#list>
> +</#list>
> \ No newline at end of file
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/36793bb2/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/NullableValueVectors.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/NullableValueVectors.java
> b/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/NullableValueVectors.java
> new file mode 100644
> index 0000000..c7de73f
> --- /dev/null
> +++
> b/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/NullableValueVectors.java
> @@ -0,0 +1,151 @@
> +<@pp.dropOutputFile />
> +<#list types as type>
> +<#list type.minor as minor>
> +<@pp.changeOutputFile name="Nullable${minor.class}Vector.java" />
> +package org.apache.drill.exec.vector;
> +
> +import static com.google.common.base.Preconditions.checkArgument;
> +import static com.google.common.base.Preconditions.checkState;
> +import io.netty.buffer.ByteBuf;
> +
> +import java.io.Closeable;
> +import java.util.Random;
> +
> +import org.apache.drill.exec.memory.BufferAllocator;
> +import org.apache.drill.exec.proto.SchemaDefProtos;
> +import org.apache.drill.exec.proto.UserBitShared.FieldMetadata;
> +import org.apache.drill.exec.record.DeadBuf;
> +import org.apache.drill.exec.record.MaterializedField;
> +import org.apache.drill.exec.vector.UInt2Vector;
> +import org.apache.drill.exec.vector.UInt4Vector;
> +
> +/**
> + * Nullable${minor.class} implements a vector of values which could be
> null.  Elements in the vector
> + * are first checked against a fixed length vector of boolean values.
>  Then the element is retrieved
> + * from the base class (if not null).
> + *
> + * NB: this class is automatically generated from ValueVectorTypes.tdd
> using FreeMarker.
> + */
> +@SuppressWarnings("unused")
> +public final class Nullable${minor.class}Vector extends ValueVector {
> +
> +  private final BitVector bits;
> +  private final ${minor.class}Vector values;
> +
> +  public Nullable${minor.class}Vector(MaterializedField field,
> BufferAllocator allocator) {
> +    super(field, allocator);
> +    bits = new BitVector(null, allocator);
> +    values = new ${minor.class}Vector(null, allocator);
> +  }
> +
> +  /**
> +   * Get the element at the specified position.
> +   *
> +   * @param   index   position of the value
> +   * @return  value of the element, if not null
> +   * @throws  NullValueException if the value is null
> +   */
> +  public <#if type.major ==
> "VarLen">byte[]<#else>${minor.javaType!type.javaType}</#if> get(int index) {
> +    assert !isNull(index);
> +    return values.get(index);
> +  }
> +
> +
> +  public boolean isNull(int index) {
> +    return bits.get(index) == 0;
> +  }
> +
> +  public int isSet(int index){
> +    return bits.get(index);
> +  }
> +
> +  /**
> +   * Allocate a new memory space for this vector.  Must be called prior
> to using the ValueVector.
> +   *
> +   * @param valueCount   The number of values which may be contained by
> this vector.
> +   */
> +  public void allocateNew(int totalBytes, ByteBuf sourceBuffer, int
> valueCount) {
> +    values.allocateNew(totalBytes, sourceBuffer, valueCount);
> +    bits.allocateNew(valueCount);
> +  }
> +
> +  @Override
> +  public int getAllocatedSize() {
> +    return bits.getAllocatedSize() + values.getAllocatedSize();
> +  }
> +
> +  /**
> +   * Get the size requirement (in bytes) for the given number of values.
>  Only accurate
> +   * for fixed width value vectors.
> +   */
> +  public int getTotalSizeFromCount(int valueCount) {
> +    return values.getSizeFromCount(valueCount) +
> bits.getSizeFromCount(valueCount);
> +  }
> +
> +  public int getSizeFromCount(int valueCount){
> +    return getTotalSizeFromCount(valueCount);
> +  }
> +
> +  @Override
> +  public MaterializedField getField() {
> +    return field;
> +  }
> +
> +  @Override
> +  public ByteBuf[] getBuffers() {
> +    return new ByteBuf[]{bits.data, values.data};
> +  }
> +
> +
> +  @Override
> +  public Object getObject(int index) {
> +    return isNull(index) ? null : values.getObject(index);
> +  }
> +
> +  public Mutator getMutator(){
> +    return new Mutator();
> +  }
> +
> +  public class Mutator implements ValueVector.Mutator{
> +
> +    private final BitVector.Mutator bitMutator;
> +    private final ${minor.class}Vector.Mutator valueMutator;
> +
> +    private Mutator(){
> +      bitMutator = bits.getMutator();
> +      valueMutator = values.getMutator();
> +    }
> +
> +    /**
> +     * Set the variable length element at the specified index to the
> supplied byte array.
> +     *
> +     * @param index   position of the bit to set
> +     * @param bytes   array of bytes to write
> +     */
> +    public void set(int index, <#if type.major == "VarLen">byte[]<#elseif
> (type.width < 4)>int<#else>${minor.javaType!type.javaType}</#if> value) {
> +      setNotNull(index);
> +      valueMutator.set(index, value);
> +    }
> +
> +    public void setNull(int index) {
> +      bitMutator.set(index, 0);
> +    }
> +
> +    private void setNotNull(int index) {
> +      bitMutator.set(index, 1);
> +    }
> +
> +    @Override
> +    public void setRecordCount(int recordCount) {
> +      Nullable${minor.class}Vector.this.setRecordCount(recordCount);
> +      bits.setRecordCount(recordCount);
> +    }
> +
> +    public void randomizeData(){
> +      throw new UnsupportedOperationException();
> +    }
> +
> +  }
> +}
> +</#list>
> +</#list>
> \ No newline at end of file
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/36793bb2/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/RepeatedValueVectors.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/RepeatedValueVectors.java
> b/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/RepeatedValueVectors.java
> new file mode 100644
> index 0000000..4acc4cc
> --- /dev/null
> +++
> b/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/RepeatedValueVectors.java
> @@ -0,0 +1,158 @@
> +import org.apache.drill.exec.vector.UInt2Vector;
> +import org.apache.drill.exec.vector.UInt4Vector;
> +
> +<@pp.dropOutputFile />
> +<#list types as type>
> +<#list type.minor as minor>
> +<@pp.changeOutputFile name="Repeated${minor.class}Vector.java" />
> +package org.apache.drill.exec.vector;
> +
> +import static com.google.common.base.Preconditions.checkArgument;
> +import static com.google.common.base.Preconditions.checkState;
> +import io.netty.buffer.ByteBuf;
> +
> +import java.io.Closeable;
> +import java.util.Random;
> +
> +import org.apache.drill.exec.memory.BufferAllocator;
> +import org.apache.drill.exec.proto.SchemaDefProtos;
> +import org.apache.drill.exec.proto.UserBitShared.FieldMetadata;
> +import org.apache.drill.exec.record.DeadBuf;
> +import org.apache.drill.exec.record.MaterializedField;
> +
> +@SuppressWarnings("unused")
> +/**
> + * Repeated${minor.class} implements a vector with multple values per row
> (e.g. JSON array or
> + * repeated protobuf field).  The implementation uses two additional
> value vectors; one to convert
> + * the index offset to the underlying element offset, and another to
> store the number of values
> + * in the vector.
> + *
> + * NB: this class is automatically generated from ValueVectorTypes.tdd
> using FreeMarker.
> + */
> +
> + public final class Repeated${minor.class}Vector extends ValueVector {
> +
> +  private final UInt2Vector countVector;    // number of repeated
> elements in each record
> +  private final UInt4Vector offsetVector;   // offsets to start of each
> record
> +  private final ${minor.class}Vector valuesVector;
> +
> +  public Repeated${minor.class}Vector(MaterializedField field,
> BufferAllocator allocator) {
> +    super(field, allocator);
> +    this.countVector = new UInt2Vector(null, allocator);
> +    this.offsetVector = new UInt4Vector(null, allocator);
> +    this.valuesVector = new ${minor.class}Vector(null, allocator);
> +  }
> +
> +  public void allocateNew(int totalBytes, ByteBuf sourceBuffer, int
> valueCount) {
> +    super.allocateNew(totalBytes, sourceBuffer, valueCount);
> +    countVector.allocateNew(valueCount);
> +    offsetVector.allocateNew(valueCount);
> +  }
> +
> +
> +  /**
> +   * Get a value for the given record.  Each element in the repeated
> field is accessed by
> +   * the positionIndex param.
> +   *
> +   * @param  index           record containing the repeated field
> +   * @param  positionIndex   position within the repeated field
> +   * @return element at the given position in the given record
> +   */
> +  public <#if type.major == "VarLen">byte[]
> +         <#else>${minor.javaType!type.javaType}
> +         </#if> get(int index, int positionIndex) {
> +
> +    assert positionIndex < countVector.get(index);
> +    return valuesVector.get(offsetVector.get(index) + positionIndex);
> +  }
> +
> +  public MaterializedField getField() {
> +    return field;
> +  }
> +
> +  /**
> +   * Get the size requirement (in bytes) for the given number of values.
>  Only accurate
> +   * for fixed width value vectors.
> +   */
> +  public int getTotalSizeFromCount(int valueCount) {
> +    return valuesVector.getSizeFromCount(valueCount) +
> +           countVector.getSizeFromCount(valueCount) +
> +           offsetVector.getSizeFromCount(valueCount);
> +  }
> +
> +  public int getSizeFromCount(int valueCount){
> +    return getTotalSizeFromCount(valueCount);
> +  }
> +
> +  /**
> +   * Get the explicitly specified size of the allocated buffer, if
> available.  Otherwise
> +   * calculate the size based on width and record count.
> +   */
> +  public int getAllocatedSize() {
> +    return valuesVector.getAllocatedSize() +
> +           countVector.getAllocatedSize() +
> +           offsetVector.getAllocatedSize();
> +  }
> +
> +  /**
> +   * Get the elements at the given index.
> +   */
> +  public int getCount(int index) {
> +    return countVector.get(index);
> +  }
> +
> +  public ByteBuf[] getBuffers() {
> +    return new ByteBuf[]{countVector.data, offsetVector.data, data};
> +  }
> +
> +  public Object getObject(int index) {
> +    return data.slice(index, getSizeFromCount(countVector.get(index)));
> +  }
> +
> +  public Mutator getMutator(){
> +    return new Mutator();
> +  }
> +
> +  public class Mutator implements ValueVector.Mutator{
> +
> +
> +    private final UInt2Vector.Mutator countMutator;
> +    private final ${minor.class}Vector.Mutator valuesMutator;
> +    private final UInt4Vector.Mutator offsetMutator;
> +
> +    private Mutator(){
> +      this.countMutator = countVector.getMutator();
> +      this.offsetMutator = offsetVector.getMutator();
> +      this.valuesMutator = valuesVector.getMutator();
> +    }
> +
> +    /**
> +     * Add an element to the given record index.  This is similar to the
> set() method in other
> +     * value vectors, except that it permits setting multiple values for
> a single record.
> +     *
> +     * @param index   record of the element to add
> +     * @param value   value to add to the given row
> +     */
> +    public void add(int index, <#if (type.width > 4)>
> ${minor.javaType!type.javaType}
> +                               <#elseif type.major == "VarLen"> byte[]
> +                               <#else> int
> +                               </#if> value) {
> +      countMutator.set(index, countVector.get(index) + 1);
> +      offsetMutator.set(index, offsetVector.get(index - 1) +
> countVector.get(index-1));
> +      valuesMutator.set(offsetVector.get(index), value);
> +    }
> +
> +    public void setRecordCount(int recordCount) {
> +      valuesMutator.setRecordCount(recordCount);
> +      offsetMutator.setRecordCount(recordCount);
> +      countMutator.setRecordCount(recordCount);
> +    }
> +
> +    public void randomizeData(){
> +      throw new UnsupportedOperationException();
> +    }
> +
> +  }
> +}
> +</#list>
> +</#list>
> \ No newline at end of file
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/36793bb2/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/TypeHelper.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/TypeHelper.java
> b/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/TypeHelper.java
> index 8dfd3af..b03b842 100644
> ---
> a/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/TypeHelper.java
> +++
> b/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/TypeHelper.java
> @@ -15,7 +15,7 @@
>   * See the License for the specific language governing permissions and
>   * limitations under the License.
>
> ******************************************************************************/
> -package org.apache.drill.exec.record.vector;
> +package org.apache.drill.exec.vector;
>
>  import org.apache.drill.exec.memory.BufferAllocator;
>  import org.apache.drill.exec.proto.SchemaDefProtos.DataMode;
> @@ -57,21 +57,21 @@ public class TypeHelper {
>        case BOOLEAN:
>          switch (mode) {
>            case REQUIRED:
> -            return ValueVector.${minor.class}.class;
> +            return ${minor.class}Vector.class;
>            case OPTIONAL:
> -            return ValueVector.Nullable${minor.class}.class;
> +            return Nullable${minor.class}Vector.class;
>            case REPEATED:
> -            return ValueVector.Repeated${minor.class}.class;
> +            return Repeated${minor.class}Vector.class;
>          }
>      <#else>
>        case ${minor.class?upper_case}:
>          switch (mode) {
>            case REQUIRED:
> -            return ValueVector.${minor.class}.class;
> +            return ${minor.class}Vector.class;
>            case OPTIONAL:
> -            return ValueVector.Nullable${minor.class}.class;
> +            return Nullable${minor.class}Vector.class;
>            case REPEATED:
> -            return ValueVector.Repeated${minor.class}.class;
> +            return Repeated${minor.class}Vector.class;
>          }
>      </#if>
>    </#list>
> @@ -83,7 +83,7 @@ public class TypeHelper {
>    }
>
>
> -  public static ValueVector.Base getNewVector(MaterializedField field,
> BufferAllocator allocator){
> +  public static ValueVector getNewVector(MaterializedField field,
> BufferAllocator allocator){
>      MajorType type = field.getType();
>
>      switch (type.getMinorType()) {
> @@ -93,11 +93,11 @@ public class TypeHelper {
>        case ${minor.class?upper_case}:
>          switch (type.getMode()) {
>            case REQUIRED:
> -            return new ValueVector.${minor.class}(field, allocator);
> +            return new ${minor.class}Vector(field, allocator);
>            case OPTIONAL:
> -            return new ValueVector.Nullable${minor.class}(field,
> allocator);
> +            return new Nullable${minor.class}Vector(field, allocator);
>            case REPEATED:
> -            return new ValueVector.Repeated${minor.class}(field,
> allocator);
> +            return new Repeated${minor.class}Vector(field, allocator);
>          }
>      </#if>
>    </#list>
> @@ -105,11 +105,11 @@ public class TypeHelper {
>        case BOOLEAN:
>          switch (type.getMode()) {
>            case REQUIRED:
> -            return new ValueVector.Bit(field, allocator);
> +            return new BitVector(field, allocator);
>            case OPTIONAL:
> -            return new ValueVector.NullableBit(field, allocator);
> +            return new NullableBitVector(field, allocator);
>            case REPEATED:
> -            return new ValueVector.RepeatedBit(field, allocator);
> +            return new RepeatedBitVector(field, allocator);
>          }
>      }
>      // All ValueVector types have been handled.
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/36793bb2/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/ValueVector.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/ValueVector.java
> b/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/ValueVector.java
> deleted file mode 100644
> index e0e895b..0000000
> ---
> a/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/ValueVector.java
> +++ /dev/null
> @@ -1,768 +0,0 @@
>
> -/*******************************************************************************
> - * Licensed to the Apache Software Foundation (ASF) under one
> - * or more contributor license agreements.  See the NOTICE file
> - * distributed with this work for additional information
> - * regarding copyright ownership.  The ASF licenses this file
> - * to you under the Apache License, Version 2.0 (the
> - * "License"); you may not use this file except in compliance
> - * with the License.  You may obtain a copy of the License at
> - *
> - * http://www.apache.org/licenses/LICENSE-2.0
> - *
> - * Unless required by applicable law or agreed to in writing, software
> - * distributed under the License is distributed on an "AS IS" BASIS,
> - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
> implied.
> - * See the License for the specific language governing permissions and
> - * limitations under the License.
> -
> ******************************************************************************/
> -package org.apache.drill.exec.record.vector;
> -
> -import static com.google.common.base.Preconditions.checkArgument;
> -import static com.google.common.base.Preconditions.checkState;
> -import io.netty.buffer.ByteBuf;
> -import java.io.Closeable;
> -import java.util.Random;
> -import org.apache.drill.exec.memory.BufferAllocator;
> -import org.apache.drill.exec.proto.SchemaDefProtos;
> -import org.apache.drill.exec.proto.UserBitShared.FieldMetadata;
> -import org.apache.drill.exec.record.DeadBuf;
> -import org.apache.drill.exec.record.MaterializedField;
> -
> -/**
> - * ValueVectorTypes defines a set of template-generated classes which
> implement type-specific
> - * value vectors.  The template approach was chosen due to the lack of
> multiple inheritence.  It
> - * is also important that all related logic be as efficient as possible.
> - */
> -public class ValueVector {
> -
> -  /**
> -   * ValueVector.Base implements common logic for all immutable value
> vectors.
> -   */
> -  public abstract static class Base implements Closeable {
> -    static final org.slf4j.Logger logger =
> org.slf4j.LoggerFactory.getLogger(Base.class);
> -
> -    protected final BufferAllocator allocator;
> -    protected ByteBuf data = DeadBuf.DEAD_BUFFER;
> -    protected MaterializedField field;
> -    protected int recordCount;
> -    protected int totalBytes;
> -
> -    public Base(MaterializedField field, BufferAllocator allocator) {
> -      this.allocator = allocator;
> -      this.field = field;
> -    }
> -
> -    /**
> -     * Get the explicitly specified size of the allocated buffer, if
> available.  Otherwise
> -     * calculate the size based on width and record count.
> -     */
> -    public abstract int getAllocatedSize();
> -
> -    /**
> -     * Get the size requirement (in bytes) for the given number of
> values.  Takes derived
> -     * type specs into account.
> -     */
> -    public abstract int getSizeFromCount(int valueCount);
> -
> -    /**
> -     * Get the Java Object representation of the element at the specified
> position
> -     *
> -     * @param index   Index of the value to get
> -     */
> -    public abstract Object getObject(int index);
> -
> -    /**
> -     * Return the underlying buffers associated with this vector. Note
> that this doesn't impact the
> -     * reference counts for this buffer so it only should be used for
> in-context access. Also note
> -     * that this buffer changes regularly thus external classes shouldn't
> hold a reference to
> -     * it (unless they change it).
> -     *
> -     * @return The underlying ByteBuf.
> -     */
> -    public ByteBuf[] getBuffers() {
> -      return new ByteBuf[]{data};
> -    }
> -
> -    /**
> -     * Returns the maximum number of values contained within this vector.
> -     * @return Vector size
> -     */
> -    public int capacity() {
> -      return getRecordCount();
> -    }
> -
> -    /**
> -     * Release supporting resources.
> -     */
> -    @Override
> -    public void close() {
> -      clear();
> -    }
> -
> -    /**
> -     * Get information about how this field is materialized.
> -     * @return
> -     */
> -    public MaterializedField getField() {
> -      return field;
> -    }
> -
> -    /**
> -     * Get the number of records allocated for this value vector.
> -     * @return number of allocated records
> -     */
> -    public int getRecordCount() {
> -      return recordCount;
> -    }
> -
> -    /**
> -     * Get the metadata for this field.
> -     * @return
> -     */
> -    public FieldMetadata getMetadata() {
> -      int len = 0;
> -      for(ByteBuf b : getBuffers()){
> -        len += b.writerIndex();
> -      }
> -      return FieldMetadata.newBuilder()
> -               .setDef(getField().getDef())
> -               .setValueCount(getRecordCount())
> -               .setBufferLength(len)
> -               .build();
> -    }
> -
> -    /**
> -     * Allocate a new memory space for this vector.  Must be called prior
> to using the ValueVector.
> -     *
> -     * @param totalBytes   Optional desired size of the underlying
> buffer.  Specifying 0 will
> -     *                     estimate the size based on valueCount.
> -     * @param sourceBuffer Optional ByteBuf to use for storage (null will
> allocate automatically).
> -     * @param valueCount   Number of values in the vector.
> -     */
> -    public void allocateNew(int totalBytes, ByteBuf sourceBuffer, int
> valueCount) {
> -      clear();
> -      this.recordCount = valueCount;
> -      this.totalBytes = totalBytes > 0 ? totalBytes :
> getSizeFromCount(valueCount);
> -      this.data = (sourceBuffer != null) ? sourceBuffer :
> allocator.buffer(this.totalBytes);
> -      this.data.retain();
> -      data.readerIndex(0);
> -    }
> -
> -    /**
> -     * Allocate a new memory space for this vector.  Must be called prior
> to using the ValueVector.
> -     *
> -     * @param valueCount
> -     *          The number of elements which can be contained within this
> vector.
> -     */
> -    public void allocateNew(int valueCount) {
> -      allocateNew(0, null, valueCount);
> -    }
> -
> -    /**
> -     * Release the underlying ByteBuf and reset the ValueVector
> -     */
> -    protected void clear() {
> -      if (data != DeadBuf.DEAD_BUFFER) {
> -        data.release();
> -        data = DeadBuf.DEAD_BUFFER;
> -        recordCount = 0;
> -        totalBytes = 0;
> -      }
> -    }
> -
> -    /**
> -     * Define the number of records that are in this value vector.
> -     * @param recordCount Number of records active in this vector.
> -     */
> -    public void setRecordCount(int recordCount) {
> -      data.writerIndex(getSizeFromCount(recordCount));
> -      this.recordCount = recordCount;
> -    }
> -
> -    /**
> -     * For testing only -- randomize the buffer contents
> -     */
> -    public void randomizeData() { }
> -
> -  }
> -
> -  /**
> -   * Bit implements a vector of bit-width values.  Elements in the vector
> are accessed
> -   * by position from the logical start of the vector.
> -   *   The width of each element is 1 bit.
> -   *   The equivilent Java primitive is an int containing the value '0'
> or '1'.
> -   *
> -   * NB: this class is automatically generated from ValueVectorTypes.tdd
> using FreeMarker.
> -   */
> -  public static class Bit extends Base {
> -    static final org.slf4j.Logger logger =
> org.slf4j.LoggerFactory.getLogger(Bit.class);
> -
> -    public Bit(MaterializedField field, BufferAllocator allocator) {
> -      super(field, allocator);
> -    }
> -
> -    /**
> -     * Get the byte holding the desired bit, then mask all other bits.
>  Iff the result is 0, the
> -     * bit was not set.
> -     *
> -     * @param  index   position of the bit in the vector
> -     * @return 1 if set, otherwise 0
> -     */
> -    public int get(int index) {
> -      // logger.debug("BIT GET: index: {}, byte: {}, mask: {}, masked
> byte: {}",
> -      //             index,
> -      //             data.getByte((int)Math.floor(index/8)),
> -      //             (int)Math.pow(2, (index % 8)),
> -      //             data.getByte((int)Math.floor(index/8)) &
> (int)Math.pow(2, (index % 8)));
> -      return ((data.getByte((int)Math.floor(index/8)) & (int)Math.pow(2,
> (index % 8))) == 0) ? 0 : 1;
> -    }
> -
> -    @Override
> -    public Object getObject(int index) {
> -      return new Boolean(get(index) != 0);
> -    }
> -
> -    /**
> -     * Get the size requirement (in bytes) for the given number of values.
> -     */
> -    @Override
> -    public int getSizeFromCount(int valueCount) {
> -      return (int) Math.ceil(valueCount / 8);
> -    }
> -
> -    @Override
> -    public int getAllocatedSize() {
> -      return totalBytes;
> -    }
> -
> -    public MutableBit getMutable() {
> -      return (MutableBit)this;
> -    }
> -
> -    /**
> -     * Allocate a new memory space for this vector.  Must be called prior
> to using the ValueVector.
> -     *
> -     * @param valueCount  The number of values which can be contained
> within this vector.
> -     */
> -    @Override
> -    public void allocateNew(int valueCount) {
> -      allocateNew(getSizeFromCount(valueCount), null, valueCount);
> -      for (int i = 0; i < getSizeFromCount(valueCount); i++) {
> -        data.setByte(i, 0);
> -      }
> -    }
> -
> -  }
> -
> -  /**
> -   * MutableBit implements a vector of bit-width values.  Elements in the
> vector are accessed
> -   * by position from the logical start of the vector.  Values should be
> pushed onto the vector
> -   * sequentially, but may be randomly accessed.
> -   *
> -   * NB: this class is automatically generated from ValueVectorTypes.tdd
> using FreeMarker.
> -   */
> -  public static class MutableBit extends Bit {
> -
> -    public MutableBit(MaterializedField field, BufferAllocator allocator)
> {
> -      super(field, allocator);
> -    }
> -
> -    /**
> -     * Set the bit at the given index to the specified value.
> -     *
> -     * @param index   position of the bit to set
> -     * @param value   value to set (either 1 or 0)
> -     */
> -    public void set(int index, int value) {
> -      byte currentByte = data.getByte((int)Math.floor(index/8));
> -      if (value != 0) {
> -        // true
> -        currentByte |= (byte) Math.pow(2, (index % 8));
> -      }
> -      else if ((currentByte & (byte) Math.pow(2, (index % 8))) == (byte)
> Math.pow(2, (index % 8))) {
> -        // false, and bit was previously set
> -        currentByte -= (byte) Math.pow(2, (index % 8));
> -      }
> -      data.setByte((int) Math.floor(index/8), currentByte);
> -    }
> -
> -    @Override
> -    public void randomizeData() {
> -      if (this.data != DeadBuf.DEAD_BUFFER) {
> -        Random r = new Random();
> -        for (int i = 0; i < data.capacity() - 1; i++) {
> -          byte[] bytes = new byte[1];
> -          r.nextBytes(bytes);
> -          data.setByte(i, bytes[0]);
> -        }
> -      }
> -    }
> -  }
> -
> -<#list types as type>
> - <#list type.minor as minor>
> -  <#if type.major == "Fixed">
> -
> -  /**
> -   * ${minor.class} implements a vector of fixed width values.  Elements
> in the vector are accessed
> -   * by position, starting from the logical start of the vector.  Values
> should be pushed onto the
> -   * vector sequentially, but may be randomly accessed.
> -   *   The width of each element is ${type.width} byte(s)
> -   *   The equivilent Java primitive is '${minor.javaType!type.javaType}'
> -   *
> -   * NB: this class is automatically generated from ValueVectorTypes.tdd
> using FreeMarker.
> -   */
> -  public static class ${minor.class} extends Base {
> -    static final org.slf4j.Logger logger =
> org.slf4j.LoggerFactory.getLogger(${minor.class}.class);
> -
> -    public ${minor.class}(MaterializedField field, BufferAllocator
> allocator) {
> -      super(field, allocator);
> -    }
> -
> -    /**
> -     * Allocate a new memory space for this vector.  Must be called prior
> to using the ValueVector.
> -     *
> -     * @param valueCount
> -     *          The number of values which can be contained within this
> vector.
> -     */
> -    public void allocateNew(int valueCount) {
> -      totalBytes = valueCount * ${type.width};
> -      allocateNew(totalBytes, allocator.buffer(totalBytes), valueCount);
> -    }
> -
> -    @Override
> -    public int getAllocatedSize() {
> -      return (int) Math.ceil(totalBytes);
> -    }
> -
> -    /**
> -     * Get the size requirement (in bytes) for the given number of
> values.  Only accurate
> -     * for fixed width value vectors.
> -     */
> -    @Override
> -    public int getSizeFromCount(int valueCount) {
> -      return valueCount * ${type.width};
> -    }
> -
> -    public Mutable${minor.class} getMutable() {
> -      return (Mutable${minor.class})this;
> -    }
> -
> -   <#if (type.width > 8)>
> -
> -    public ${minor.javaType!type.javaType} get(int index) {
> -      ByteBuf dst = allocator.buffer(${type.width});
> -      data.getBytes(index * ${type.width}, dst, 0, ${type.width});
> -      return dst;
> -    }
> -
> -    @Override
> -    public Object getObject(int index) {
> -      ByteBuf dst = allocator.buffer(${type.width});
> -      data.getBytes(index, dst, 0, ${type.width});
> -      return dst;
> -    }
> -
> -    @Override
> -    public void randomizeData() {
> -      if (this.data != DeadBuf.DEAD_BUFFER) {
> -        Random r = new Random();
> -        for(int i =0; i < data.capacity()-${type.width}; i +=
> ${type.width}){
> -          byte[] bytes = new byte[${type.width}];
> -          r.nextBytes(bytes);
> -          data.setByte(i, bytes[0]);
> -        }
> -      }
> -    }
> -
> -   <#else> <#-- type.width <= 8 -->
> -
> -    public ${minor.javaType!type.javaType} get(int index) {
> -      return data.get${(minor.javaType!type.javaType)?cap_first}(index *
> ${type.width});
> -    }
> -
> -    public Object getObject(int index) {
> -      return get(index);
> -    }
> -
> -    @Override
> -    public void randomizeData() {
> -      if (this.data != DeadBuf.DEAD_BUFFER) {
> -        Random r = new Random();
> -        for(int i =0; i < data.capacity()-${type.width}; i +=
> ${type.width}){
> -          data.set${(minor.javaType!type.javaType)?cap_first}(i,
> -              r.next<#if (type.width >=
> 4)>${(minor.javaType!type.javaType)?cap_first}
> -                    <#else>Int
> -                    </#if>());
> -        }
> -      }
> -    }
> -
> -   </#if> <#-- type.width -->
> -  }
> -
> -  /**
> -   * Mutable${minor.class} implements a mutable vector of fixed width
> values.  Elements in the
> -   * vector are accessed by position from the logical start of the
> vector.  Values should be pushed
> -   * onto the vector sequentially, but may be randomly accessed.
> -   *   The width of each element is ${type.width} byte(s)
> -   *   The equivilent Java primitive is '${minor.javaType!type.javaType}'
> -   *
> -   * NB: this class is automatically generated from ValueVectorTypes.tdd
> using FreeMarker.
> -   */
> -   public static class Mutable${minor.class} extends ${minor.class} {
> -
> -    public Mutable${minor.class}(MaterializedField field, BufferAllocator
> allocator) {
> -      super(field, allocator);
> -    }
> -
> -    /**
> -     * Set the element at the given index to the given value.  Note that
> widths smaller than
> -     * 32 bits are handled by the ByteBuf interface.
> -     *
> -     * @param index   position of the bit to set
> -     * @param value   value to set
> -     */
> -   <#if (type.width > 8)>
> -    public void set(int index, <#if (type.width >
> 4)>${minor.javaType!type.javaType}<#else>int</#if> value) {
> -      data.setBytes(index * ${type.width}, value);
> -   <#else> <#-- type.width <= 8 -->
> -    public void set(int index, <#if (type.width >=
> 4)>${minor.javaType!type.javaType}<#else>int</#if> value) {
> -      data.set${(minor.javaType!type.javaType)?cap_first}(index *
> ${type.width}, value);
> -   </#if> <#-- type.width -->
> -    }
> -  }
> -
> -  <#elseif type.major == "VarLen">
> -
> -  /**
> -   * ${minor.class} implements a vector of variable width values.
>  Elements in the vector
> -   * are accessed by position from the logical start of the vector.  A
> fixed width lengthVector
> -   * is used to convert an element's position to it's offset from the
> start of the (0-based)
> -   * ByteBuf.  Size is inferred by adjacent elements.
> -   *   The width of each element is ${type.width} byte(s)
> -   *   The equivilent Java primitive is '${minor.javaType!type.javaType}'
> -   *
> -   * NB: this class is automatically generated from ValueVectorTypes.tdd
> using FreeMarker.
> -   */
> -  public static class ${minor.class} extends Base {
> -    static final org.slf4j.Logger logger =
> org.slf4j.LoggerFactory.getLogger(${minor.class}.class);
> -
> -    protected final MutableUInt${type.width} lengthVector;
> -
> -    public ${minor.class}(MaterializedField field, BufferAllocator
> allocator) {
> -      super(field, allocator);
> -      this.lengthVector = new MutableUInt${type.width}(null, allocator);
> -    }
> -
> -    public byte[] get(int index) {
> -      checkArgument(index >= 0);
> -      int startIdx = 0;
> -      int size = 0;
> -      if (index == 0) {
> -        size = lengthVector.get(1);
> -      } else {
> -        startIdx = lengthVector.get(index);
> -        size = lengthVector.get(index + 1) - startIdx;
> -      }
> -      checkState(size >= 0);
> -      byte[] dst = new byte[size];
> -      data.getBytes(startIdx, dst, 0, size);
> -      return dst;
> -    }
> -
> -    @Override
> -    public int getAllocatedSize() {
> -      return lengthVector.getAllocatedSize() + totalBytes;
> -    }
> -
> -    /**
> -     * Get the size requirement (in bytes) for the given number of
> values.  Only accurate
> -     * for fixed width value vectors.
> -     */
> -    public int getSizeFromCount(int valueCount) {
> -      return valueCount * ${type.width};
> -    }
> -
> -    @Override
> -    protected void clear() {
> -      super.clear();
> -      lengthVector.clear();
> -    }
> -
> -    /**
> -     * Allocate a new memory space for this vector.  Must be called prior
> to using the ValueVector.
> -     *
> -     * @param valueCount
> -     *          The number of values which can be contained within this
> vector.
> -     */
> -    public void allocateNew(int totalBytes, ByteBuf sourceBuffer, int
> valueCount) {
> -      super.allocateNew(totalBytes, sourceBuffer, valueCount);
> -      lengthVector.allocateNew(valueCount);
> -    }
> -
> -    @Override
> -    public ByteBuf[] getBuffers() {
> -      return new ByteBuf[]{lengthVector.data, data};
> -    }
> -
> -    public Object getObject(int index) {
> -      return get(index);
> -    }
> -
> -    public Mutable${minor.class} getMutable() {
> -      return (Mutable${minor.class})this;
> -    }
> -  }
> -
> -  /**
> -   * Mutable${minor.class} implements a vector of variable width values.
>  Elements in the vector
> -   * are accessed by position from the logical start of the vector.  A
> fixed width lengthVector
> -   * is used to convert an element's position to it's offset from the
> start of the (0-based)
> -   * ByteBuf.  Size is inferred by adjacent elements.
> -   *   The width of each element is ${type.width} byte(s)
> -   *   The equivilent Java primitive is '${minor.javaType!type.javaType}'
> -   *
> -   * NB: this class is automatically generated from ValueVectorTypes.tdd
> using FreeMarker.
> -   */
> -  public static class Mutable${minor.class} extends ${minor.class} {
> -
> -    public Mutable${minor.class}(MaterializedField field, BufferAllocator
> allocator) {
> -      super(field, allocator);
> -    }
> -
> -    /**
> -     * Set the variable length element at the specified index to the
> supplied byte array.
> -     *
> -     * @param index   position of the bit to set
> -     * @param bytes   array of bytes to write
> -     */
> -    public void set(int index, byte[] bytes) {
> -      checkArgument(index >= 0);
> -      if (index == 0) {
> -        lengthVector.set(0, 0);
> -        lengthVector.set(1, bytes.length);
> -        data.setBytes(0, bytes);
> -      }
> -      else {
> -        int currentOffset = lengthVector.get(index);
> -        // set the end offset of the buffer
> -        lengthVector.set(index + 1, currentOffset + bytes.length);
> -        data.setBytes(currentOffset, bytes);
> -      }
> -    }
> -
> -    @Override
> -    public void setRecordCount(int recordCount) {
> -      super.setRecordCount(recordCount);
> -      lengthVector.setRecordCount(recordCount);
> -    }
> -
> -  }
> -
> -  </#if> <#-- type.major -->
> -
> -  /**
> -   * Nullable${minor.class} implements a vector of values which could be
> null.  Elements in the vector
> -   * are first checked against a fixed length vector of boolean values.
>  Then the element is retrieved
> -   * from the base class (if not null).
> -   *
> -   * NB: this class is automatically generated from ValueVectorTypes.tdd
> using FreeMarker.
> -   */
> -  public static class Nullable${minor.class} extends
> Mutable${minor.class} {
> -
> -    protected MutableBit bits;
> -
> -    public Nullable${minor.class}(MaterializedField field,
> BufferAllocator allocator) {
> -      super(field, allocator);
> -      bits = new MutableBit(null, allocator);
> -    }
> -
> -    /**
> -     * Set the variable length element at the specified index to the
> supplied byte array.
> -     *
> -     * @param index   position of the bit to set
> -     * @param bytes   array of bytes to write
> -     */
> -    public void set(int index, <#if type.major == "VarLen">byte[]<#elseif
> (type.width < 4)>int<#else>${minor.javaType!type.javaType}</#if> value) {
> -      setNotNull(index);
> -      super.set(index, value);
> -    }
> -
> -    /**
> -     * Get the element at the specified position.
> -     *
> -     * @param   index   position of the value
> -     * @return  value of the element, if not null
> -     * @throws  NullValueException if the value is null
> -     */
> -    public <#if type.major ==
> "VarLen">byte[]<#else>${minor.javaType!type.javaType}</#if> get(int index) {
> -      if (isNull(index))
> -        throw new NullValueException(index);
> -      return super.get(index);
> -    }
> -
> -    public void setNull(int index) {
> -      bits.set(index, 0);
> -    }
> -
> -    private void setNotNull(int index) {
> -      bits.set(index, 1);
> -    }
> -
> -    public boolean isNull(int index) {
> -      return bits.get(index) == 0;
> -    }
> -
> -    /**
> -     * Allocate a new memory space for this vector.  Must be called prior
> to using the ValueVector.
> -     *
> -     * @param valueCount   The number of values which may be contained by
> this vector.
> -     */
> -    public void allocateNew(int totalBytes, ByteBuf sourceBuffer, int
> valueCount) {
> -      super.allocateNew(totalBytes, sourceBuffer, valueCount);
> -      bits.allocateNew(valueCount);
> -    }
> -
> -    @Override
> -    public int getAllocatedSize() {
> -      return bits.getAllocatedSize() + super.getAllocatedSize();
> -    }
> -
> -    /**
> -     * Get the size requirement (in bytes) for the given number of
> values.  Only accurate
> -     * for fixed width value vectors.
> -     */
> -    public int getTotalSizeFromCount(int valueCount) {
> -      return getSizeFromCount(valueCount) +
> bits.getSizeFromCount(valueCount);
> -    }
> -
> -    @Override
> -    public MaterializedField getField() {
> -      return field;
> -    }
> -
> -    @Override
> -    public ByteBuf[] getBuffers() {
> -      return new ByteBuf[]{bits.data, super.data};
> -    }
> -
> -    @Override
> -    public void setRecordCount(int recordCount) {
> -      super.setRecordCount(recordCount);
> -      bits.setRecordCount(recordCount);
> -    }
> -
> -    @Override
> -    public Object getObject(int index) {
> -      return isNull(index) ? null : super.getObject(index);
> -    }
> -  }
> -
> -  /**
> -   * Repeated${minor.class} implements a vector with multple values per
> row (e.g. JSON array or
> -   * repeated protobuf field).  The implementation uses two additional
> value vectors; one to convert
> -   * the index offset to the underlying element offset, and another to
> store the number of values
> -   * in the vector.
> -   *
> -   * NB: this class is automatically generated from ValueVectorTypes.tdd
> using FreeMarker.
> -   */
> -   public static class Repeated${minor.class} extends
> Mutable${minor.class} {
> -
> -    private MutableUInt4 countVector;    // number of repeated elements
> in each record
> -    private MutableUInt4 offsetVector;   // offsets to start of each
> record
> -
> -    public Repeated${minor.class}(MaterializedField field,
> BufferAllocator allocator) {
> -      super(field, allocator);
> -      countVector = new MutableUInt4(null, allocator);
> -      offsetVector = new MutableUInt4(null, allocator);
> -    }
> -
> -    public void allocateNew(int totalBytes, ByteBuf sourceBuffer, int
> valueCount) {
> -      super.allocateNew(totalBytes, sourceBuffer, valueCount);
> -      countVector.allocateNew(valueCount);
> -      offsetVector.allocateNew(valueCount);
> -    }
> -
> -    /**
> -     * Add an element to the given record index.  This is similar to the
> set() method in other
> -     * value vectors, except that it permits setting multiple values for
> a single record.
> -     *
> -     * @param index   record of the element to add
> -     * @param value   value to add to the given row
> -     */
> -    public void add(int index, <#if (type.width > 4)>
> ${minor.javaType!type.javaType}
> -                               <#elseif type.major == "VarLen"> byte[]
> -                               <#else> int
> -                               </#if> value) {
> -      countVector.set(index, countVector.get(index) + 1);
> -      offsetVector.set(index, offsetVector.get(index - 1) +
> countVector.get(index-1));
> -      super.set(offsetVector.get(index), value);
> -    }
> -
> -    /**
> -     * Get a value for the given record.  Each element in the repeated
> field is accessed by
> -     * the positionIndex param.
> -     *
> -     * @param  index           record containing the repeated field
> -     * @param  positionIndex   position within the repeated field
> -     * @return element at the given position in the given record
> -     */
> -    public <#if type.major == "VarLen">byte[]
> -           <#else>${minor.javaType!type.javaType}
> -           </#if> get(int index, int positionIndex) {
> -
> -      assert positionIndex < countVector.get(index);
> -      return super.get(offsetVector.get(index) + positionIndex);
> -    }
> -
> -    public MaterializedField getField() {
> -      return field;
> -    }
> -
> -    /**
> -     * Get the size requirement (in bytes) for the given number of
> values.  Only accurate
> -     * for fixed width value vectors.
> -     */
> -    public int getTotalSizeFromCount(int valueCount) {
> -      return getSizeFromCount(valueCount) +
> -             countVector.getSizeFromCount(valueCount) +
> -             offsetVector.getSizeFromCount(valueCount);
> -    }
> -
> -    /**
> -     * Get the explicitly specified size of the allocated buffer, if
> available.  Otherwise
> -     * calculate the size based on width and record count.
> -     */
> -    public int getAllocatedSize() {
> -      return super.getAllocatedSize() +
> -             countVector.getAllocatedSize() +
> -             offsetVector.getAllocatedSize();
> -    }
> -
> -    /**
> -     * Get the elements at the given index.
> -     */
> -    public int getCount(int index) {
> -      return countVector.get(index);
> -    }
> -
> -    public void setRecordCount(int recordCount) {
> -      super.setRecordCount(recordCount);
> -      offsetVector.setRecordCount(recordCount);
> -      countVector.setRecordCount(recordCount);
> -    }
> -
> -    public ByteBuf[] getBuffers() {
> -      return new ByteBuf[]{countVector.data, offsetVector.data, data};
> -    }
> -
> -    public Object getObject(int index) {
> -      return data.slice(index, getSizeFromCount(countVector.get(index)));
> -    }
> -
> -  }
> - </#list>
> -</#list>
> -}
> -
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/36793bb2/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/VariableLengthVectors.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/VariableLengthVectors.java
> b/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/VariableLengthVectors.java
> new file mode 100644
> index 0000000..954836a
> --- /dev/null
> +++
> b/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/VariableLengthVectors.java
> @@ -0,0 +1,152 @@
> +<@pp.dropOutputFile />
> +<#list types as type>
> +<#list type.minor as minor>
> +
> +<#if type.major == "VarLen">
> +<@pp.changeOutputFile name="${minor.class}Vector.java" />
> +package org.apache.drill.exec.vector;
> +
> +import static com.google.common.base.Preconditions.checkArgument;
> +import static com.google.common.base.Preconditions.checkState;
> +import io.netty.buffer.ByteBuf;
> +
> +import java.io.Closeable;
> +import java.util.Random;
> +
> +import org.apache.drill.exec.memory.BufferAllocator;
> +import org.apache.drill.exec.proto.SchemaDefProtos;
> +import org.apache.drill.exec.proto.UserBitShared.FieldMetadata;
> +import org.apache.drill.exec.record.DeadBuf;
> +import org.apache.drill.exec.record.MaterializedField;
> +
> +/**
> + * ${minor.class}Vector implements a vector of variable width values.
>  Elements in the vector
> + * are accessed by position from the logical start of the vector.  A
> fixed width lengthVector
> + * is used to convert an element's position to it's offset from the start
> of the (0-based)
> + * ByteBuf.  Size is inferred by adjacent elements.
> + *   The width of each element is ${type.width} byte(s)
> + *   The equivalent Java primitive is '${minor.javaType!type.javaType}'
> + *
> + * NB: this class is automatically generated from ValueVectorTypes.tdd
> using FreeMarker.
> + */
> +@SuppressWarnings("unused")
> +public final class ${minor.class}Vector extends ValueVector {
> +  static final org.slf4j.Logger logger =
> org.slf4j.LoggerFactory.getLogger(${minor.class}Vector.class);
> +
> +  private final UInt${type.width}Vector lengthVector;
> +  private final UInt${type.width}Vector.Mutator lengthVectorMutator;
> +
> +  public ${minor.class}Vector(MaterializedField field, BufferAllocator
> allocator) {
> +    super(field, allocator);
> +    this.lengthVector = new UInt${type.width}Vector(null, allocator);
> +    this.lengthVectorMutator = lengthVector.getMutator();
> +  }
> +
> +  public byte[] get(int index) {
> +    checkArgument(index >= 0);
> +    int startIdx = 0;
> +    int size = 0;
> +    if (index == 0) {
> +      size = lengthVector.get(1);
> +    } else {
> +      startIdx = lengthVector.get(index);
> +      size = lengthVector.get(index + 1) - startIdx;
> +    }
> +    checkState(size >= 0);
> +    byte[] dst = new byte[size];
> +    data.getBytes(startIdx, dst, 0, size);
> +    return dst;
> +  }
> +
> +  @Override
> +  public int getAllocatedSize() {
> +    return lengthVector.getAllocatedSize() + totalBytes;
> +  }
> +
> +  /**
> +   * Get the size requirement (in bytes) for the given number of values.
>  Only accurate
> +   * for fixed width value vectors.
> +   */
> +  public int getSizeFromCount(int valueCount) {
> +    return valueCount * ${type.width};
> +  }
> +
> +  @Override
> +  protected void clear() {
> +    super.clear();
> +    lengthVector.clear();
> +  }
> +
> +  /**
> +   * Allocate a new memory space for this vector.  Must be called prior
> to using the ValueVector.
> +   *
> +   * @param valueCount
> +   *          The number of values which can be contained within this
> vector.
> +   */
> +  public void allocateNew(int totalBytes, ByteBuf sourceBuffer, int
> valueCount) {
> +    super.allocateNew(totalBytes, sourceBuffer, valueCount);
> +    lengthVector.allocateNew(valueCount);
> +  }
> +
> +  @Override
> +  public ByteBuf[] getBuffers() {
> +    return new ByteBuf[]{lengthVector.data, data};
> +  }
> +
> +  public Object getObject(int index) {
> +    return get(index);
> +  }
> +
> +  public Mutator getMutator() {
> +    return new Mutator();
> +  }
> +
> +
> +  /**
> +   * Mutable${minor.class} implements a vector of variable width values.
>  Elements in the vector
> +   * are accessed by position from the logical start of the vector.  A
> fixed width lengthVector
> +   * is used to convert an element's position to it's offset from the
> start of the (0-based)
> +   * ByteBuf.  Size is inferred by adjacent elements.
> +   *   The width of each element is ${type.width} byte(s)
> +   *   The equivalent Java primitive is '${minor.javaType!type.javaType}'
> +   *
> +   * NB: this class is automatically generated from ValueVectorTypes.tdd
> using FreeMarker.
> +   */
> +  public class Mutator implements ValueVector.Mutator{
> +
> +    /**
> +     * Set the variable length element at the specified index to the
> supplied byte array.
> +     *
> +     * @param index   position of the bit to set
> +     * @param bytes   array of bytes to write
> +     */
> +    public void set(int index, byte[] bytes) {
> +      checkArgument(index >= 0);
> +      if (index == 0) {
> +        lengthVectorMutator.set(0, 0);
> +        lengthVectorMutator.set(1, bytes.length);
> +        data.setBytes(0, bytes);
> +      } else {
> +        int currentOffset = lengthVector.get(index);
> +        // set the end offset of the buffer
> +        lengthVectorMutator.set(index + 1, currentOffset + bytes.length);
> +        data.setBytes(currentOffset, bytes);
> +      }
> +    }
> +
> +    @Override
> +    public void setRecordCount(int recordCount) {
> +      ${minor.class}Vector.this.setRecordCount(recordCount);
> +      lengthVector.setRecordCount(recordCount);
> +    }
> +
> +    @Override
> +    public void randomizeData(){}
> +  }
> +
> +}
> +
> +
> +</#if> <#-- type.major -->
> +</#list>
> +</#list>
> \ No newline at end of file
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/36793bb2/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockRecordReader.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockRecordReader.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockRecordReader.java
> index 65584db..cd3371d 100644
> ---
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockRecordReader.java
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockRecordReader.java
> @@ -27,9 +27,9 @@ import org.apache.drill.exec.physical.impl.OutputMutator;
>  import org.apache.drill.exec.proto.SchemaDefProtos.DataMode;
>  import org.apache.drill.exec.proto.SchemaDefProtos.MajorType;
>  import org.apache.drill.exec.record.MaterializedField;
> -import org.apache.drill.exec.record.vector.TypeHelper;
> -import org.apache.drill.exec.record.vector.ValueVector;
>  import org.apache.drill.exec.store.RecordReader;
> +import org.apache.drill.exec.vector.TypeHelper;
> +import org.apache.drill.exec.vector.ValueVector;
>
>  public class MockRecordReader implements RecordReader {
>    static final org.slf4j.Logger logger =
> org.slf4j.LoggerFactory.getLogger(MockRecordReader.class);
> @@ -37,7 +37,7 @@ public class MockRecordReader implements RecordReader {
>    private OutputMutator output;
>    private MockScanEntry config;
>    private FragmentContext context;
> -  private ValueVector.Base[] valueVectors;
> +  private ValueVector[] valueVectors;
>    private int recordsRead;
>
>    public MockRecordReader(FragmentContext context, MockScanEntry config) {
> @@ -53,12 +53,12 @@ public class MockRecordReader implements RecordReader {
>      return x;
>    }
>
> -  private ValueVector.Base getVector(int fieldId, String name, MajorType
> type, int length) {
> +  private ValueVector getVector(int fieldId, String name, MajorType type,
> int length) {
>      assert context != null : "Context shouldn't be null.";
>      if(type.getMode() != DataMode.REQUIRED) throw new
> UnsupportedOperationException();
>
>      MaterializedField f = MaterializedField.create(new SchemaPath(name),
> fieldId, 0, type);
> -    ValueVector.Base v;
> +    ValueVector v;
>      v = TypeHelper.getNewVector(f, context.getAllocator());
>      v.allocateNew(length);
>      return v;
> @@ -70,7 +70,7 @@ public class MockRecordReader implements RecordReader {
>      try {
>        this.output = output;
>        int estimateRowSize = getEstimatedRecordSize(config.getTypes());
> -      valueVectors = new ValueVector.Base[config.getTypes().length];
> +      valueVectors = new ValueVector[config.getTypes().length];
>        int batchRecordCount = 250000 / estimateRowSize;
>
>        for (int i = 0; i < config.getTypes().length; i++) {
> @@ -88,10 +88,11 @@ public class MockRecordReader implements RecordReader {
>    public int next() {
>      int recordSetSize = Math.min(valueVectors[0].capacity(),
> this.config.getRecords()- recordsRead);
>      recordsRead += recordSetSize;
> -    for(ValueVector.Base v : valueVectors){
> +    for(ValueVector v : valueVectors){
>        logger.debug("MockRecordReader:  Generating random data for VV of
> type " + v.getClass().getName());
>        v.randomizeData();
> -      v.setRecordCount(recordSetSize);
> +
> +      v.getMutator().setRecordCount(recordSetSize);
>      }
>      return recordSetSize;
>    }
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/36793bb2/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockScanBatchCreator.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockScanBatchCreator.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockScanBatchCreator.java
> index b821d6e..bfc19af 100644
> ---
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockScanBatchCreator.java
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockScanBatchCreator.java
> @@ -24,10 +24,7 @@ import org.apache.drill.exec.ops.FragmentContext;
>  import org.apache.drill.exec.physical.config.MockScanPOP.MockScanEntry;
>  import org.apache.drill.exec.physical.impl.BatchCreator;
>  import org.apache.drill.exec.physical.impl.ScanBatch;
> -import org.apache.drill.exec.record.BatchSchema;
> -import org.apache.drill.exec.record.InvalidValueAccessor;
>  import org.apache.drill.exec.record.RecordBatch;
> -import org.apache.drill.exec.record.vector.ValueVector;
>  import org.apache.drill.exec.store.RecordReader;
>
>  import com.google.common.base.Preconditions;
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/36793bb2/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockScanPOP.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockScanPOP.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockScanPOP.java
> index 4a3a606..40227e5 100644
> ---
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockScanPOP.java
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockScanPOP.java
> @@ -32,7 +32,7 @@ import
> org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
>  import org.apache.drill.exec.proto.SchemaDefProtos.DataMode;
>  import org.apache.drill.exec.proto.SchemaDefProtos.MajorType;
>  import org.apache.drill.exec.proto.SchemaDefProtos.MinorType;
> -import org.apache.drill.exec.record.vector.TypeHelper;
> +import org.apache.drill.exec.vector.TypeHelper;
>
>  import com.fasterxml.jackson.annotation.JsonCreator;
>  import com.fasterxml.jackson.annotation.JsonIgnore;
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/36793bb2/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/FilterRecordBatch.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/FilterRecordBatch.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/FilterRecordBatch.java
> index b7b7d93..6440d98 100644
> ---
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/FilterRecordBatch.java
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/FilterRecordBatch.java
> @@ -22,7 +22,7 @@ import org.apache.drill.exec.record.BatchSchema;
>  import org.apache.drill.exec.record.InvalidValueAccessor;
>  import org.apache.drill.exec.record.RecordBatch;
>  import org.apache.drill.exec.record.vector.SelectionVector;
> -import org.apache.drill.exec.record.vector.ValueVector;
> +import org.apache.drill.exec.vector.ValueVector;
>
>  public abstract class FilterRecordBatch implements RecordBatch {
>    static final org.slf4j.Logger logger =
> org.slf4j.LoggerFactory.getLogger(FilterRecordBatch.class);
> @@ -58,7 +58,7 @@ public abstract class FilterRecordBatch implements
> RecordBatch {
>    }
>
>    @Override
> -  public <T extends ValueVector.Base> T getValueVector(int fieldId,
> Class<T> clazz) throws InvalidValueAccessor {
> +  public <T extends ValueVector> T getValueVector(int fieldId, Class<T>
> clazz) throws InvalidValueAccessor {
>      return null;
>    }
>
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/36793bb2/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/OutputMutator.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/OutputMutator.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/OutputMutator.java
> index e96c1be..b3b9f5f 100644
> ---
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/OutputMutator.java
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/OutputMutator.java
> @@ -18,11 +18,10 @@
>  package org.apache.drill.exec.physical.impl;
>
>  import org.apache.drill.exec.exception.SchemaChangeException;
> -import org.apache.drill.exec.record.BatchSchema;
> -import org.apache.drill.exec.record.vector.ValueVector;
> +import org.apache.drill.exec.vector.ValueVector;
>
>  public interface OutputMutator {
>    public void removeField(int fieldId) throws SchemaChangeException;
> -  public void addField(int fieldId, ValueVector.Base vector) throws
> SchemaChangeException ;
> +  public void addField(int fieldId, ValueVector vector) throws
> SchemaChangeException ;
>    public void setNewSchema() throws SchemaChangeException ;
>  }
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/36793bb2/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScanBatch.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScanBatch.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScanBatch.java
> index 822d828..2f3e1fe 100644
> ---
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScanBatch.java
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScanBatch.java
> @@ -17,27 +17,21 @@
>
> ******************************************************************************/
>  package org.apache.drill.exec.physical.impl;
>
> -import io.netty.buffer.ByteBuf;
> -
>  import java.util.Iterator;
> -import java.util.List;
>
>  import org.apache.drill.common.exceptions.ExecutionSetupException;
>  import org.apache.drill.exec.exception.SchemaChangeException;
>  import org.apache.drill.exec.ops.FragmentContext;
> -import org.apache.drill.exec.proto.UserBitShared.FieldMetadata;
> -import org.apache.drill.exec.proto.UserBitShared.RecordBatchDef;
>  import org.apache.drill.exec.record.BatchSchema;
>  import org.apache.drill.exec.record.InvalidValueAccessor;
>  import org.apache.drill.exec.record.RecordBatch;
>  import org.apache.drill.exec.record.SchemaBuilder;
>  import org.apache.drill.exec.record.WritableBatch;
> -import org.apache.drill.exec.record.vector.ValueVector;
>  import org.apache.drill.exec.store.RecordReader;
> +import org.apache.drill.exec.vector.ValueVector;
>
>  import com.carrotsearch.hppc.IntObjectOpenHashMap;
>  import com.carrotsearch.hppc.procedures.IntObjectProcedure;
> -import com.google.common.collect.Lists;
>
>  /**
>   * Record batch used for a particular scan. Operators against one or more
> @@ -45,7 +39,7 @@ import com.google.common.collect.Lists;
>  public class ScanBatch implements RecordBatch {
>    static final org.slf4j.Logger logger =
> org.slf4j.LoggerFactory.getLogger(ScanBatch.class);
>
> -  private IntObjectOpenHashMap<ValueVector.Base> fields = new
> IntObjectOpenHashMap<ValueVector.Base>();
> +  private IntObjectOpenHashMap<ValueVector> fields = new
> IntObjectOpenHashMap<ValueVector>();
>    private BatchSchema schema;
>    private int recordCount;
>    private boolean schemaChanged = true;
> @@ -89,9 +83,9 @@ public class ScanBatch implements RecordBatch {
>    }
>
>    private void releaseAssets() {
> -    fields.forEach(new IntObjectProcedure<ValueVector.Base>() {
> +    fields.forEach(new IntObjectProcedure<ValueVector>() {
>        @Override
> -      public void apply(int key, ValueVector.Base value) {
> +      public void apply(int key, ValueVector value) {
>          value.close();
>        }
>      });
> @@ -99,9 +93,9 @@ public class ScanBatch implements RecordBatch {
>
>    @SuppressWarnings("unchecked")
>    @Override
> -  public <T extends ValueVector.Base> T getValueVector(int fieldId,
> Class<T> clazz) throws InvalidValueAccessor {
> +  public <T extends ValueVector> T getValueVector(int fieldId, Class<T>
> clazz) throws InvalidValueAccessor {
>      if (fields.containsKey(fieldId)) throw new
> InvalidValueAccessor(String.format("Unknown value accesor for field id
> %d."));
> -    ValueVector.Base vector = this.fields.lget();
> +    ValueVector vector = this.fields.lget();
>      if (vector.getClass().isAssignableFrom(clazz)) {
>        return (T) vector;
>      } else {
> @@ -143,14 +137,14 @@ public class ScanBatch implements RecordBatch {
>
>      public void removeField(int fieldId) throws SchemaChangeException {
>        schemaChanged();
> -      ValueVector.Base v = fields.remove(fieldId);
> +      ValueVector v = fields.remove(fieldId);
>        if (v == null) throw new SchemaChangeException("Failure attempting
> to remove an unknown field.");
>        v.close();
>      }
>
> -    public void addField(int fieldId, ValueVector.Base vector) {
> +    public void addField(int fieldId, ValueVector vector) {
>        schemaChanged();
> -      ValueVector.Base v = fields.put(fieldId, vector);
> +      ValueVector v = fields.put(fieldId, vector);
>        vector.getField();
>        builder.addField(vector.getField());
>        if (v != null) v.close();
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/36793bb2/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/WireRecordBatch.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/WireRecordBatch.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/WireRecordBatch.java
> index be32d1f..fcbd272 100644
> ---
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/WireRecordBatch.java
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/WireRecordBatch.java
> @@ -17,8 +17,6 @@
>
> ******************************************************************************/
>  package org.apache.drill.exec.physical.impl;
>
> -import java.util.Iterator;
> -
>  import org.apache.drill.exec.exception.SchemaChangeException;
>  import org.apache.drill.exec.ops.FragmentContext;
>  import org.apache.drill.exec.proto.UserBitShared.RecordBatchDef;
> @@ -29,8 +27,7 @@ import
> org.apache.drill.exec.record.RawFragmentBatchProvider;
>  import org.apache.drill.exec.record.RecordBatch;
>  import org.apache.drill.exec.record.RecordBatchLoader;
>  import org.apache.drill.exec.record.WritableBatch;
> -import org.apache.drill.exec.record.RecordBatch.IterOutcome;
> -import org.apache.drill.exec.record.vector.ValueVector;
> +import org.apache.drill.exec.vector.ValueVector;
>
>  public class WireRecordBatch implements RecordBatch{
>    static final org.slf4j.Logger logger =
> org.slf4j.LoggerFactory.getLogger(WireRecordBatch.class);
> @@ -68,7 +65,7 @@ public class WireRecordBatch implements RecordBatch{
>    }
>
>    @Override
> -  public <T extends ValueVector.Base> T getValueVector(int fieldId,
> Class<T> clazz) throws InvalidValueAccessor {
> +  public <T extends ValueVector> T getValueVector(int fieldId, Class<T>
> clazz) throws InvalidValueAccessor {
>      return batchLoader.getValueVector(fieldId, clazz);
>    }
>
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/36793bb2/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/BatchSchema.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/BatchSchema.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/BatchSchema.java
> index 05b1cc7..b26e742 100644
> ---
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/BatchSchema.java
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/BatchSchema.java
> @@ -20,8 +20,6 @@ package org.apache.drill.exec.record;
>  import java.util.Iterator;
>  import java.util.List;
>
> -import org.apache.drill.exec.record.vector.ValueVector;
> -
>
>  public class BatchSchema implements Iterable<MaterializedField> {
>    static final org.slf4j.Logger logger =
> org.slf4j.LoggerFactory.getLogger(BatchSchema.class);
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/36793bb2/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatch.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatch.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatch.java
> index 042c40c..c6b7888 100644
> ---
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatch.java
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatch.java
> @@ -18,7 +18,7 @@
>  package org.apache.drill.exec.record;
>
>  import org.apache.drill.exec.ops.FragmentContext;
> -import org.apache.drill.exec.record.vector.ValueVector;
> +import org.apache.drill.exec.vector.ValueVector;
>
>  /**
>   * A record batch contains a set of field values for a particular range
> of records. In the case of a record batch
> @@ -68,7 +68,7 @@ public interface RecordBatch {
>    public void kill();
>
>
> -  public abstract <T extends ValueVector.Base> T getValueVector(int
> fieldId, Class<T> clazz) throws InvalidValueAccessor;
> +  public abstract <T extends ValueVector> T getValueVector(int fieldId,
> Class<T> clazz) throws InvalidValueAccessor;
>
>  //  public abstract void getDictReader(int fieldId, Class<?> clazz)
> throws InvalidValueAccessor;
>  //
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/36793bb2/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatchLoader.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatchLoader.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatchLoader.java
> index ea1de73..be43026 100644
> ---
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatchLoader.java
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatchLoader.java
> @@ -27,16 +27,16 @@ import org.apache.drill.exec.memory.BufferAllocator;
>  import org.apache.drill.exec.proto.SchemaDefProtos.FieldDef;
>  import org.apache.drill.exec.proto.UserBitShared.FieldMetadata;
>  import org.apache.drill.exec.proto.UserBitShared.RecordBatchDef;
> -import org.apache.drill.exec.record.vector.TypeHelper;
> -import org.apache.drill.exec.record.vector.ValueVector;
> +import org.apache.drill.exec.vector.TypeHelper;
> +import org.apache.drill.exec.vector.ValueVector;
>
>  import com.carrotsearch.hppc.IntObjectOpenHashMap;
>  import com.carrotsearch.hppc.cursors.IntObjectCursor;
>
> -public class RecordBatchLoader implements
> Iterable<IntObjectCursor<ValueVector.Base>>{
> +public class RecordBatchLoader implements
> Iterable<IntObjectCursor<ValueVector>>{
>    static final org.slf4j.Logger logger =
> org.slf4j.LoggerFactory.getLogger(RecordBatchLoader.class);
>
> -  private IntObjectOpenHashMap<ValueVector.Base> vectors = new
> IntObjectOpenHashMap<ValueVector.Base>();
> +  private IntObjectOpenHashMap<ValueVector> vectors = new
> IntObjectOpenHashMap<ValueVector>();
>    private final BufferAllocator allocator;
>    private int recordCount;
>    private BatchSchema schema;
> @@ -61,14 +61,14 @@ public class RecordBatchLoader implements
> Iterable<IntObjectCursor<ValueVector.B
>      this.recordCount = def.getRecordCount();
>      boolean schemaChanged = false;
>
> -    IntObjectOpenHashMap<ValueVector.Base> newVectors = new
> IntObjectOpenHashMap<ValueVector.Base>();
> +    IntObjectOpenHashMap<ValueVector> newVectors = new
> IntObjectOpenHashMap<ValueVector>();
>
>      List<FieldMetadata> fields = def.getFieldList();
>
>      int bufOffset = 0;
>      for (FieldMetadata fmd : fields) {
>        FieldDef fieldDef = fmd.getDef();
> -      ValueVector.Base v = vectors.remove(fieldDef.getFieldId());
> +      ValueVector v = vectors.remove(fieldDef.getFieldId());
>        if (v != null) {
>          if (v.getField().getDef().equals(fieldDef)) {
>            v.allocateNew(fmd.getBufferLength(), buf.slice(bufOffset,
> fmd.getBufferLength()), recordCount);
> @@ -89,7 +89,7 @@ public class RecordBatchLoader implements
> Iterable<IntObjectCursor<ValueVector.B
>
>      if(!vectors.isEmpty()){
>        schemaChanged = true;
> -      for(IntObjectCursor<ValueVector.Base> cursor : newVectors){
> +      for(IntObjectCursor<ValueVector> cursor : newVectors){
>          cursor.value.close();
>        }
>
> @@ -98,7 +98,7 @@ public class RecordBatchLoader implements
> Iterable<IntObjectCursor<ValueVector.B
>      if(schemaChanged){
>        // rebuild the schema.
>        SchemaBuilder b = BatchSchema.newBuilder();
> -      for(IntObjectCursor<ValueVector.Base> cursor : newVectors){
> +      for(IntObjectCursor<ValueVector> cursor : newVectors){
>          b.addField(cursor.value.getField());
>        }
>        b.setSelectionVector(false);
> @@ -110,8 +110,8 @@ public class RecordBatchLoader implements
> Iterable<IntObjectCursor<ValueVector.B
>    }
>
>    @SuppressWarnings("unchecked")
> -  public <T extends ValueVector.Base> T getValueVector(int fieldId,
> Class<T> clazz) throws InvalidValueAccessor {
> -    ValueVector.Base v = vectors.get(fieldId);
> +  public <T extends ValueVector> T getValueVector(int fieldId, Class<T>
> clazz) throws InvalidValueAccessor {
> +    ValueVector v = vectors.get(fieldId);
>      assert v != null;
>      if (v.getClass() != clazz)
>        throw new InvalidValueAccessor(String.format(
> @@ -130,7 +130,7 @@ public class RecordBatchLoader implements
> Iterable<IntObjectCursor<ValueVector.B
>    }
>
>    @Override
> -  public Iterator<IntObjectCursor<ValueVector.Base>> iterator() {
> +  public Iterator<IntObjectCursor<ValueVector>> iterator() {
>      return this.vectors.iterator();
>    }
>
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/36793bb2/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/WritableBatch.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/WritableBatch.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/WritableBatch.java
> index 67c6cb9..a367b6d 100644
> ---
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/WritableBatch.java
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/WritableBatch.java
> @@ -23,7 +23,7 @@ import java.util.List;
>
>  import org.apache.drill.exec.proto.UserBitShared.FieldMetadata;
>  import org.apache.drill.exec.proto.UserBitShared.RecordBatchDef;
> -import org.apache.drill.exec.record.vector.ValueVector;
> +import org.apache.drill.exec.vector.ValueVector;
>
>  import com.carrotsearch.hppc.IntObjectOpenHashMap;
>  import com.carrotsearch.hppc.procedures.IntObjectProcedure;
> @@ -58,7 +58,7 @@ public class WritableBatch {
>      return buffers;
>    }
>
> -//  public static WritableBatch get(ValueVector.Base[] vectors){
> +//  public static WritableBatch get(ValueVector[] vectors){
>  //    WritableCreator c = new WritableCreator();
>  //    for(int i =0; i < vectors.length; i++){
>  //      c.apply(i, vectors[i]);
> @@ -67,14 +67,14 @@ public class WritableBatch {
>  //  }
>  //
>
> -  public static WritableBatch get(int recordCount,
> IntObjectOpenHashMap<ValueVector.Base> fields){
> +  public static WritableBatch get(int recordCount,
> IntObjectOpenHashMap<ValueVector> fields){
>      WritableCreator creator = new WritableCreator(recordCount);
>      fields.forEach(creator);
>      return creator.get();
>
>    }
>
> -  private static class WritableCreator implements
> IntObjectProcedure<ValueVector.Base>{
> +  private static class WritableCreator implements
> IntObjectProcedure<ValueVector>{
>
>      List<ByteBuf> buffers = Lists.newArrayList();
>      List<FieldMetadata> metadata = Lists.newArrayList();
> @@ -87,7 +87,7 @@ public class WritableBatch {
>      }
>
>      @Override
> -    public void apply(int key, ValueVector.Base value) {
> +    public void apply(int key, ValueVector value) {
>        metadata.add(value.getMetadata());
>        for(ByteBuf b : value.getBuffers()){
>          buffers.add(b);
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/36793bb2/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/SelectionVector.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/SelectionVector.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/SelectionVector.java
> index e79a525..02b75ce 100644
> ---
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/SelectionVector.java
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/SelectionVector.java
> @@ -17,19 +17,25 @@
>
> ******************************************************************************/
>  package org.apache.drill.exec.record.vector;
>
> -import io.netty.buffer.ByteBufAllocator;
> -
>  import org.apache.drill.exec.memory.BufferAllocator;
>  import org.apache.drill.exec.record.MaterializedField;
>
>  /**
>   * Convenience/Clarification Fixed2 wrapper.
>   */
> -public class SelectionVector extends ValueVector.UInt2 {
> +public class SelectionVector {
>    static final org.slf4j.Logger logger =
> org.slf4j.LoggerFactory.getLogger(SelectionVector.class);
>
>    public SelectionVector(MaterializedField field, BufferAllocator
> allocator) {
> -    super(field, allocator);
> +
> +  }
> +
> +  public int capacity() {
> +    return -1;
> +  }
> +
> +  public void allocateNew(int count) {
> +
>    }
>
>  }
>
>

[34/53] [abbrv] git commit: Updated value vectors inheritance model. Moved Mutables to separate Mutator subclasses. Broke VVs into separate files rather than one large class.

Posted by ja...@apache.org.
Updated value vectors inheritance model.
Moved Mutables to separate Mutator subclasses.
Broke VVs into separate files rather than one large class.


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

Branch: refs/heads/master
Commit: 36793bb2f22cd427c8b70d2f5dfe4a4d8a3a6894
Parents: 7075cca
Author: Jacques Nadeau <ja...@apache.org>
Authored: Sat Jul 13 21:28:12 2013 -0700
Committer: Jacques Nadeau <ja...@apache.org>
Committed: Mon Jul 15 11:36:32 2013 -0700

----------------------------------------------------------------------
 sandbox/prototype/exec/java-exec/pom.xml        |   2 +-
 .../templates/FixedValueVectors.java            | 163 ++++
 .../templates/NullableValueVectors.java         | 151 ++++
 .../templates/RepeatedValueVectors.java         | 158 ++++
 .../ValueVectors/templates/TypeHelper.java      |  28 +-
 .../ValueVectors/templates/ValueVector.java     | 768 -------------------
 .../templates/VariableLengthVectors.java        | 152 ++++
 .../exec/physical/config/MockRecordReader.java  |  17 +-
 .../physical/config/MockScanBatchCreator.java   |   3 -
 .../drill/exec/physical/config/MockScanPOP.java |   2 +-
 .../exec/physical/impl/FilterRecordBatch.java   |   4 +-
 .../drill/exec/physical/impl/OutputMutator.java |   5 +-
 .../drill/exec/physical/impl/ScanBatch.java     |  24 +-
 .../exec/physical/impl/WireRecordBatch.java     |   7 +-
 .../apache/drill/exec/record/BatchSchema.java   |   2 -
 .../apache/drill/exec/record/RecordBatch.java   |   4 +-
 .../drill/exec/record/RecordBatchLoader.java    |  22 +-
 .../apache/drill/exec/record/WritableBatch.java |  10 +-
 .../exec/record/vector/SelectionVector.java     |  14 +-
 .../drill/exec/store/JSONRecordReader.java      |  87 ++-
 .../apache/drill/exec/store/VectorHolder.java   |  11 +-
 .../org/apache/drill/exec/vector/BitVector.java | 123 +++
 .../apache/drill/exec/vector/ValueVector.java   | 192 +++++
 .../physical/impl/TestSimpleFragmentRun.java    |   6 +-
 .../exec/record/vector/TestValueVector.java     | 201 +++--
 .../drill/exec/store/JSONRecordReaderTest.java  |  40 +-
 26 files changed, 1212 insertions(+), 984 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/36793bb2/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 4e4df95..1b6dac0 100644
--- a/sandbox/prototype/exec/java-exec/pom.xml
+++ b/sandbox/prototype/exec/java-exec/pom.xml
@@ -173,7 +173,7 @@
 				<version>1.0</version>
 				<configuration>
 					<cfgFile>src/main/codegen/ValueVectors/config.fmpp</cfgFile>
-					<outputDirectory>target/generated-sources/org/apache/drill/exec/record/vector</outputDirectory>
+					<outputDirectory>target/generated-sources/org/apache/drill/exec/vector</outputDirectory>
 					<templateDirectory>src/main/codegen/ValueVectors/templates</templateDirectory>
 				</configuration>
 				<executions>

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/36793bb2/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/FixedValueVectors.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/FixedValueVectors.java b/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/FixedValueVectors.java
new file mode 100644
index 0000000..09dd5d8
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/FixedValueVectors.java
@@ -0,0 +1,163 @@
+<@pp.dropOutputFile />
+<#list types as type>
+<#list type.minor as minor>
+
+<#if type.major == "Fixed">
+<@pp.changeOutputFile name="${minor.class}Vector.java" />
+package org.apache.drill.exec.vector;
+
+import io.netty.buffer.ByteBuf;
+
+import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.proto.UserBitShared.FieldMetadata;
+import org.apache.drill.exec.record.DeadBuf;
+import org.apache.drill.exec.record.MaterializedField;
+import org.apache.drill.exec.vector.MsgPack2Vector;
+
+import java.util.Random;
+
+/**
+ * ${minor.class} implements a vector of fixed width values.  Elements in the vector are accessed
+ * by position, starting from the logical start of the vector.  Values should be pushed onto the
+ * vector sequentially, but may be randomly accessed.
+ *   The width of each element is ${type.width} byte(s)
+ *   The equivalent Java primitive is '${minor.javaType!type.javaType}'
+ *
+ * NB: this class is automatically generated from ValueVectorTypes.tdd using FreeMarker.
+ */
+@SuppressWarnings("unused")
+public final class ${minor.class}Vector extends ValueVector {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(${minor.class}Vector.class);
+
+  public ${minor.class}Vector(MaterializedField field, BufferAllocator allocator) {
+    super(field, allocator);
+  }
+
+  /**
+   * Allocate a new memory space for this vector.  Must be called prior to using the ValueVector.
+   *
+   * @param valueCount
+   *          The number of values which can be contained within this vector.
+   */
+  public void allocateNew(int valueCount) {
+    totalBytes = valueCount * ${type.width};
+    allocateNew(totalBytes, allocator.buffer(totalBytes), valueCount);
+  }
+
+  @Override
+  public int getAllocatedSize() {
+    return (int) Math.ceil(totalBytes);
+  }
+
+  /**
+   * Get the size requirement (in bytes) for the given number of values.  Only accurate
+   * for fixed width value vectors.
+   */
+  @Override
+  public int getSizeFromCount(int valueCount) {
+    return valueCount * ${type.width};
+  }
+
+  public Mutator getMutator() {
+    return new Mutator();
+  }
+
+ <#if (type.width > 8)>
+
+  public ${minor.javaType!type.javaType} get(int index) {
+    ByteBuf dst = allocator.buffer(${type.width});
+    data.getBytes(index * ${type.width}, dst, 0, ${type.width});
+    return dst;
+  }
+
+  @Override
+  public Object getObject(int index) {
+    ByteBuf dst = allocator.buffer(${type.width});
+    data.getBytes(index, dst, 0, ${type.width});
+    return dst;
+  }
+
+
+ <#else> <#-- type.width <= 8 -->
+
+  public ${minor.javaType!type.javaType} get(int index) {
+    return data.get${(minor.javaType!type.javaType)?cap_first}(index * ${type.width});
+  }
+
+  public Object getObject(int index) {
+    return get(index);
+  }
+
+
+ </#if> <#-- type.width -->
+ 
+ 
+ /**
+  * ${minor.class}.Mutator implements a mutable vector of fixed width values.  Elements in the
+  * vector are accessed by position from the logical start of the vector.  Values should be pushed
+  * onto the vector sequentially, but may be randomly accessed.
+  *   The width of each element is ${type.width} byte(s)
+  *   The equivalent Java primitive is '${minor.javaType!type.javaType}'
+  *
+  * NB: this class is automatically generated from ValueVectorTypes.tdd using FreeMarker.
+  */
+  public class Mutator implements ValueVector.Mutator{
+
+    private Mutator(){};
+   /**
+    * Set the element at the given index to the given value.  Note that widths smaller than
+    * 32 bits are handled by the ByteBuf interface.
+    *
+    * @param index   position of the bit to set
+    * @param value   value to set
+    */
+  <#if (type.width > 8)>
+   public void set(int index, <#if (type.width > 4)>${minor.javaType!type.javaType}<#else>int</#if> value) {
+     data.setBytes(index * ${type.width}, value);
+   }
+   
+   @Override
+   public void randomizeData() {
+     if (data != DeadBuf.DEAD_BUFFER) {
+       Random r = new Random();
+       for(int i =0; i < data.capacity()-${type.width}; i += ${type.width}){
+         byte[] bytes = new byte[${type.width}];
+         r.nextBytes(bytes);
+         data.setByte(i, bytes[0]);
+       }
+     }
+   }
+   
+  <#else> <#-- type.width <= 8 -->
+   public void set(int index, <#if (type.width >= 4)>${minor.javaType!type.javaType}<#else>int</#if> value) {
+     data.set${(minor.javaType!type.javaType)?cap_first}(index * ${type.width}, value);
+   }
+   
+   @Override
+   public void randomizeData() {
+     if (data != DeadBuf.DEAD_BUFFER) {
+       Random r = new Random();
+       for(int i =0; i < data.capacity()-${type.width}; i += ${type.width}){
+         data.set${(minor.javaType!type.javaType)?cap_first}(i,
+             r.next<#if (type.width >= 4)>${(minor.javaType!type.javaType)?cap_first}
+                   <#else>Int
+                   </#if>());
+       }
+     }
+   }
+  </#if> <#-- type.width -->
+  
+   @Override
+   public void setRecordCount(int recordCount) {
+     ${minor.class}Vector.this.setRecordCount(recordCount);
+   }
+
+
+
+  
+ }
+}
+
+</#if> <#-- type.major -->
+</#list>
+</#list>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/36793bb2/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/NullableValueVectors.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/NullableValueVectors.java b/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/NullableValueVectors.java
new file mode 100644
index 0000000..c7de73f
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/NullableValueVectors.java
@@ -0,0 +1,151 @@
+<@pp.dropOutputFile />
+<#list types as type>
+<#list type.minor as minor>
+<@pp.changeOutputFile name="Nullable${minor.class}Vector.java" />
+package org.apache.drill.exec.vector;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static com.google.common.base.Preconditions.checkState;
+import io.netty.buffer.ByteBuf;
+
+import java.io.Closeable;
+import java.util.Random;
+
+import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.proto.SchemaDefProtos;
+import org.apache.drill.exec.proto.UserBitShared.FieldMetadata;
+import org.apache.drill.exec.record.DeadBuf;
+import org.apache.drill.exec.record.MaterializedField;
+import org.apache.drill.exec.vector.UInt2Vector;
+import org.apache.drill.exec.vector.UInt4Vector;
+
+/**
+ * Nullable${minor.class} implements a vector of values which could be null.  Elements in the vector
+ * are first checked against a fixed length vector of boolean values.  Then the element is retrieved
+ * from the base class (if not null).
+ *
+ * NB: this class is automatically generated from ValueVectorTypes.tdd using FreeMarker.
+ */
+@SuppressWarnings("unused")
+public final class Nullable${minor.class}Vector extends ValueVector {
+
+  private final BitVector bits;
+  private final ${minor.class}Vector values;
+
+  public Nullable${minor.class}Vector(MaterializedField field, BufferAllocator allocator) {
+    super(field, allocator);
+    bits = new BitVector(null, allocator);
+    values = new ${minor.class}Vector(null, allocator);
+  }
+
+  /**
+   * Get the element at the specified position.
+   *
+   * @param   index   position of the value
+   * @return  value of the element, if not null
+   * @throws  NullValueException if the value is null
+   */
+  public <#if type.major == "VarLen">byte[]<#else>${minor.javaType!type.javaType}</#if> get(int index) {
+    assert !isNull(index);
+    return values.get(index);
+  }
+
+
+  public boolean isNull(int index) {
+    return bits.get(index) == 0;
+  }
+
+  public int isSet(int index){
+    return bits.get(index);
+  }
+  
+  /**
+   * Allocate a new memory space for this vector.  Must be called prior to using the ValueVector.
+   *
+   * @param valueCount   The number of values which may be contained by this vector.
+   */
+  public void allocateNew(int totalBytes, ByteBuf sourceBuffer, int valueCount) {
+    values.allocateNew(totalBytes, sourceBuffer, valueCount);
+    bits.allocateNew(valueCount);
+  }
+
+  @Override
+  public int getAllocatedSize() {
+    return bits.getAllocatedSize() + values.getAllocatedSize();
+  }
+
+  /**
+   * Get the size requirement (in bytes) for the given number of values.  Only accurate
+   * for fixed width value vectors.
+   */
+  public int getTotalSizeFromCount(int valueCount) {
+    return values.getSizeFromCount(valueCount) + bits.getSizeFromCount(valueCount);
+  }
+  
+  public int getSizeFromCount(int valueCount){
+    return getTotalSizeFromCount(valueCount);
+  }
+
+  @Override
+  public MaterializedField getField() {
+    return field;
+  }
+
+  @Override
+  public ByteBuf[] getBuffers() {
+    return new ByteBuf[]{bits.data, values.data};
+  }
+
+
+  @Override
+  public Object getObject(int index) {
+    return isNull(index) ? null : values.getObject(index);
+  }
+  
+  public Mutator getMutator(){
+    return new Mutator();
+  }
+  
+  public class Mutator implements ValueVector.Mutator{
+
+    private final BitVector.Mutator bitMutator;
+    private final ${minor.class}Vector.Mutator valueMutator;
+    
+    private Mutator(){
+      bitMutator = bits.getMutator();
+      valueMutator = values.getMutator();
+    }
+
+    /**
+     * Set the variable length element at the specified index to the supplied byte array.
+     *
+     * @param index   position of the bit to set
+     * @param bytes   array of bytes to write
+     */
+    public void set(int index, <#if type.major == "VarLen">byte[]<#elseif (type.width < 4)>int<#else>${minor.javaType!type.javaType}</#if> value) {
+      setNotNull(index);
+      valueMutator.set(index, value);
+    }
+
+    public void setNull(int index) {
+      bitMutator.set(index, 0);
+    }
+
+    private void setNotNull(int index) {
+      bitMutator.set(index, 1);
+    }
+    
+    @Override
+    public void setRecordCount(int recordCount) {
+      Nullable${minor.class}Vector.this.setRecordCount(recordCount);
+      bits.setRecordCount(recordCount);
+    }
+    
+    public void randomizeData(){
+      throw new UnsupportedOperationException();
+    }
+    
+  }
+}
+</#list>
+</#list>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/36793bb2/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/RepeatedValueVectors.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/RepeatedValueVectors.java b/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/RepeatedValueVectors.java
new file mode 100644
index 0000000..4acc4cc
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/RepeatedValueVectors.java
@@ -0,0 +1,158 @@
+import org.apache.drill.exec.vector.UInt2Vector;
+import org.apache.drill.exec.vector.UInt4Vector;
+
+<@pp.dropOutputFile />
+<#list types as type>
+<#list type.minor as minor>
+<@pp.changeOutputFile name="Repeated${minor.class}Vector.java" />
+package org.apache.drill.exec.vector;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static com.google.common.base.Preconditions.checkState;
+import io.netty.buffer.ByteBuf;
+
+import java.io.Closeable;
+import java.util.Random;
+
+import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.proto.SchemaDefProtos;
+import org.apache.drill.exec.proto.UserBitShared.FieldMetadata;
+import org.apache.drill.exec.record.DeadBuf;
+import org.apache.drill.exec.record.MaterializedField;
+
+@SuppressWarnings("unused")
+/**
+ * Repeated${minor.class} implements a vector with multple values per row (e.g. JSON array or
+ * repeated protobuf field).  The implementation uses two additional value vectors; one to convert
+ * the index offset to the underlying element offset, and another to store the number of values
+ * in the vector.
+ *
+ * NB: this class is automatically generated from ValueVectorTypes.tdd using FreeMarker.
+ */
+
+ public final class Repeated${minor.class}Vector extends ValueVector {
+
+  private final UInt2Vector countVector;    // number of repeated elements in each record
+  private final UInt4Vector offsetVector;   // offsets to start of each record
+  private final ${minor.class}Vector valuesVector;
+
+  public Repeated${minor.class}Vector(MaterializedField field, BufferAllocator allocator) {
+    super(field, allocator);
+    this.countVector = new UInt2Vector(null, allocator);
+    this.offsetVector = new UInt4Vector(null, allocator);
+    this.valuesVector = new ${minor.class}Vector(null, allocator);
+  }
+
+  public void allocateNew(int totalBytes, ByteBuf sourceBuffer, int valueCount) {
+    super.allocateNew(totalBytes, sourceBuffer, valueCount);
+    countVector.allocateNew(valueCount);
+    offsetVector.allocateNew(valueCount);
+  }
+
+
+  /**
+   * Get a value for the given record.  Each element in the repeated field is accessed by
+   * the positionIndex param.
+   *
+   * @param  index           record containing the repeated field
+   * @param  positionIndex   position within the repeated field
+   * @return element at the given position in the given record
+   */
+  public <#if type.major == "VarLen">byte[]
+         <#else>${minor.javaType!type.javaType}
+         </#if> get(int index, int positionIndex) {
+
+    assert positionIndex < countVector.get(index);
+    return valuesVector.get(offsetVector.get(index) + positionIndex);
+  }
+
+  public MaterializedField getField() {
+    return field;
+  }
+
+  /**
+   * Get the size requirement (in bytes) for the given number of values.  Only accurate
+   * for fixed width value vectors.
+   */
+  public int getTotalSizeFromCount(int valueCount) {
+    return valuesVector.getSizeFromCount(valueCount) +
+           countVector.getSizeFromCount(valueCount) +
+           offsetVector.getSizeFromCount(valueCount);
+  }
+  
+  public int getSizeFromCount(int valueCount){
+    return getTotalSizeFromCount(valueCount);
+  }
+
+  /**
+   * Get the explicitly specified size of the allocated buffer, if available.  Otherwise
+   * calculate the size based on width and record count.
+   */
+  public int getAllocatedSize() {
+    return valuesVector.getAllocatedSize() +
+           countVector.getAllocatedSize() +
+           offsetVector.getAllocatedSize();
+  }
+
+  /**
+   * Get the elements at the given index.
+   */
+  public int getCount(int index) {
+    return countVector.get(index);
+  }
+
+  public ByteBuf[] getBuffers() {
+    return new ByteBuf[]{countVector.data, offsetVector.data, data};
+  }
+
+  public Object getObject(int index) {
+    return data.slice(index, getSizeFromCount(countVector.get(index)));
+  }
+
+  public Mutator getMutator(){
+    return new Mutator();
+  }
+  
+  public class Mutator implements ValueVector.Mutator{
+
+    
+    private final UInt2Vector.Mutator countMutator;
+    private final ${minor.class}Vector.Mutator valuesMutator;
+    private final UInt4Vector.Mutator offsetMutator;
+    
+    private Mutator(){
+      this.countMutator = countVector.getMutator();
+      this.offsetMutator = offsetVector.getMutator();
+      this.valuesMutator = valuesVector.getMutator();
+    }
+
+    /**
+     * Add an element to the given record index.  This is similar to the set() method in other
+     * value vectors, except that it permits setting multiple values for a single record.
+     *
+     * @param index   record of the element to add
+     * @param value   value to add to the given row
+     */
+    public void add(int index, <#if (type.width > 4)> ${minor.javaType!type.javaType}
+                               <#elseif type.major == "VarLen"> byte[]
+                               <#else> int
+                               </#if> value) {
+      countMutator.set(index, countVector.get(index) + 1);
+      offsetMutator.set(index, offsetVector.get(index - 1) + countVector.get(index-1));
+      valuesMutator.set(offsetVector.get(index), value);
+    }
+    
+    public void setRecordCount(int recordCount) {
+      valuesMutator.setRecordCount(recordCount);
+      offsetMutator.setRecordCount(recordCount);
+      countMutator.setRecordCount(recordCount);
+    }
+    
+    public void randomizeData(){
+      throw new UnsupportedOperationException();
+    }
+    
+  }
+}
+</#list>
+</#list>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/36793bb2/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/TypeHelper.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/TypeHelper.java b/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/TypeHelper.java
index 8dfd3af..b03b842 100644
--- a/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/TypeHelper.java
+++ b/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/TypeHelper.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  ******************************************************************************/
-package org.apache.drill.exec.record.vector;
+package org.apache.drill.exec.vector;
 
 import org.apache.drill.exec.memory.BufferAllocator;
 import org.apache.drill.exec.proto.SchemaDefProtos.DataMode;
@@ -57,21 +57,21 @@ public class TypeHelper {
       case BOOLEAN:
         switch (mode) {
           case REQUIRED:
-            return ValueVector.${minor.class}.class;
+            return ${minor.class}Vector.class;
           case OPTIONAL:
-            return ValueVector.Nullable${minor.class}.class;
+            return Nullable${minor.class}Vector.class;
           case REPEATED:
-            return ValueVector.Repeated${minor.class}.class;
+            return Repeated${minor.class}Vector.class;
         }
     <#else>
       case ${minor.class?upper_case}:
         switch (mode) {
           case REQUIRED:
-            return ValueVector.${minor.class}.class;
+            return ${minor.class}Vector.class;
           case OPTIONAL:
-            return ValueVector.Nullable${minor.class}.class;
+            return Nullable${minor.class}Vector.class;
           case REPEATED:
-            return ValueVector.Repeated${minor.class}.class;
+            return Repeated${minor.class}Vector.class;
         }
     </#if>
   </#list>
@@ -83,7 +83,7 @@ public class TypeHelper {
   }
 
 
-  public static ValueVector.Base getNewVector(MaterializedField field, BufferAllocator allocator){
+  public static ValueVector getNewVector(MaterializedField field, BufferAllocator allocator){
     MajorType type = field.getType();
 
     switch (type.getMinorType()) {
@@ -93,11 +93,11 @@ public class TypeHelper {
       case ${minor.class?upper_case}:
         switch (type.getMode()) {
           case REQUIRED:
-            return new ValueVector.${minor.class}(field, allocator);
+            return new ${minor.class}Vector(field, allocator);
           case OPTIONAL:
-            return new ValueVector.Nullable${minor.class}(field, allocator);
+            return new Nullable${minor.class}Vector(field, allocator);
           case REPEATED:
-            return new ValueVector.Repeated${minor.class}(field, allocator);
+            return new Repeated${minor.class}Vector(field, allocator);
         }
     </#if>
   </#list>
@@ -105,11 +105,11 @@ public class TypeHelper {
       case BOOLEAN:
         switch (type.getMode()) {
           case REQUIRED:
-            return new ValueVector.Bit(field, allocator);
+            return new BitVector(field, allocator);
           case OPTIONAL:
-            return new ValueVector.NullableBit(field, allocator);
+            return new NullableBitVector(field, allocator);
           case REPEATED:
-            return new ValueVector.RepeatedBit(field, allocator);
+            return new RepeatedBitVector(field, allocator);
         }
     }
     // All ValueVector types have been handled.

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/36793bb2/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/ValueVector.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/ValueVector.java b/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/ValueVector.java
deleted file mode 100644
index e0e895b..0000000
--- a/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/ValueVector.java
+++ /dev/null
@@ -1,768 +0,0 @@
-/*******************************************************************************
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- ******************************************************************************/
-package org.apache.drill.exec.record.vector;
-
-import static com.google.common.base.Preconditions.checkArgument;
-import static com.google.common.base.Preconditions.checkState;
-import io.netty.buffer.ByteBuf;
-import java.io.Closeable;
-import java.util.Random;
-import org.apache.drill.exec.memory.BufferAllocator;
-import org.apache.drill.exec.proto.SchemaDefProtos;
-import org.apache.drill.exec.proto.UserBitShared.FieldMetadata;
-import org.apache.drill.exec.record.DeadBuf;
-import org.apache.drill.exec.record.MaterializedField;
-
-/**
- * ValueVectorTypes defines a set of template-generated classes which implement type-specific
- * value vectors.  The template approach was chosen due to the lack of multiple inheritence.  It
- * is also important that all related logic be as efficient as possible.
- */
-public class ValueVector {
-
-  /**
-   * ValueVector.Base implements common logic for all immutable value vectors.
-   */
-  public abstract static class Base implements Closeable {
-    static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(Base.class);
-
-    protected final BufferAllocator allocator;
-    protected ByteBuf data = DeadBuf.DEAD_BUFFER;
-    protected MaterializedField field;
-    protected int recordCount;
-    protected int totalBytes;
-
-    public Base(MaterializedField field, BufferAllocator allocator) {
-      this.allocator = allocator;
-      this.field = field;
-    }
-
-    /**
-     * Get the explicitly specified size of the allocated buffer, if available.  Otherwise
-     * calculate the size based on width and record count.
-     */
-    public abstract int getAllocatedSize();
-
-    /**
-     * Get the size requirement (in bytes) for the given number of values.  Takes derived
-     * type specs into account.
-     */
-    public abstract int getSizeFromCount(int valueCount);
-
-    /**
-     * Get the Java Object representation of the element at the specified position
-     *
-     * @param index   Index of the value to get
-     */
-    public abstract Object getObject(int index);
-
-    /**
-     * Return the underlying buffers associated with this vector. Note that this doesn't impact the
-     * reference counts for this buffer so it only should be used for in-context access. Also note
-     * that this buffer changes regularly thus external classes shouldn't hold a reference to
-     * it (unless they change it).
-     *
-     * @return The underlying ByteBuf.
-     */
-    public ByteBuf[] getBuffers() {
-      return new ByteBuf[]{data};
-    }
-
-    /**
-     * Returns the maximum number of values contained within this vector.
-     * @return Vector size
-     */
-    public int capacity() {
-      return getRecordCount();
-    }
-
-    /**
-     * Release supporting resources.
-     */
-    @Override
-    public void close() {
-      clear();
-    }
-
-    /**
-     * Get information about how this field is materialized.
-     * @return
-     */
-    public MaterializedField getField() {
-      return field;
-    }
-
-    /**
-     * Get the number of records allocated for this value vector.
-     * @return number of allocated records
-     */
-    public int getRecordCount() {
-      return recordCount;
-    }
-
-    /**
-     * Get the metadata for this field.
-     * @return
-     */
-    public FieldMetadata getMetadata() {
-      int len = 0;
-      for(ByteBuf b : getBuffers()){
-        len += b.writerIndex();
-      }
-      return FieldMetadata.newBuilder()
-               .setDef(getField().getDef())
-               .setValueCount(getRecordCount())
-               .setBufferLength(len)
-               .build();
-    }
-
-    /**
-     * Allocate a new memory space for this vector.  Must be called prior to using the ValueVector.
-     *
-     * @param totalBytes   Optional desired size of the underlying buffer.  Specifying 0 will
-     *                     estimate the size based on valueCount.
-     * @param sourceBuffer Optional ByteBuf to use for storage (null will allocate automatically).
-     * @param valueCount   Number of values in the vector.
-     */
-    public void allocateNew(int totalBytes, ByteBuf sourceBuffer, int valueCount) {
-      clear();
-      this.recordCount = valueCount;
-      this.totalBytes = totalBytes > 0 ? totalBytes : getSizeFromCount(valueCount);
-      this.data = (sourceBuffer != null) ? sourceBuffer : allocator.buffer(this.totalBytes);
-      this.data.retain();
-      data.readerIndex(0);
-    }
-
-    /**
-     * Allocate a new memory space for this vector.  Must be called prior to using the ValueVector.
-     *
-     * @param valueCount
-     *          The number of elements which can be contained within this vector.
-     */
-    public void allocateNew(int valueCount) {
-      allocateNew(0, null, valueCount);
-    }
-
-    /**
-     * Release the underlying ByteBuf and reset the ValueVector
-     */
-    protected void clear() {
-      if (data != DeadBuf.DEAD_BUFFER) {
-        data.release();
-        data = DeadBuf.DEAD_BUFFER;
-        recordCount = 0;
-        totalBytes = 0;
-      }
-    }
-
-    /**
-     * Define the number of records that are in this value vector.
-     * @param recordCount Number of records active in this vector.
-     */
-    public void setRecordCount(int recordCount) {
-      data.writerIndex(getSizeFromCount(recordCount));
-      this.recordCount = recordCount;
-    }
-
-    /**
-     * For testing only -- randomize the buffer contents
-     */
-    public void randomizeData() { }
-
-  }
-
-  /**
-   * Bit implements a vector of bit-width values.  Elements in the vector are accessed
-   * by position from the logical start of the vector.
-   *   The width of each element is 1 bit.
-   *   The equivilent Java primitive is an int containing the value '0' or '1'.
-   *
-   * NB: this class is automatically generated from ValueVectorTypes.tdd using FreeMarker.
-   */
-  public static class Bit extends Base {
-    static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(Bit.class);
-
-    public Bit(MaterializedField field, BufferAllocator allocator) {
-      super(field, allocator);
-    }
-
-    /**
-     * Get the byte holding the desired bit, then mask all other bits.  Iff the result is 0, the
-     * bit was not set.
-     *
-     * @param  index   position of the bit in the vector
-     * @return 1 if set, otherwise 0
-     */
-    public int get(int index) {
-      // logger.debug("BIT GET: index: {}, byte: {}, mask: {}, masked byte: {}",
-      //             index,
-      //             data.getByte((int)Math.floor(index/8)),
-      //             (int)Math.pow(2, (index % 8)),
-      //             data.getByte((int)Math.floor(index/8)) & (int)Math.pow(2, (index % 8)));
-      return ((data.getByte((int)Math.floor(index/8)) & (int)Math.pow(2, (index % 8))) == 0) ? 0 : 1;
-    }
-
-    @Override
-    public Object getObject(int index) {
-      return new Boolean(get(index) != 0);
-    }
-
-    /**
-     * Get the size requirement (in bytes) for the given number of values.
-     */
-    @Override
-    public int getSizeFromCount(int valueCount) {
-      return (int) Math.ceil(valueCount / 8);
-    }
-
-    @Override
-    public int getAllocatedSize() {
-      return totalBytes;
-    }
-
-    public MutableBit getMutable() {
-      return (MutableBit)this;
-    }
-
-    /**
-     * Allocate a new memory space for this vector.  Must be called prior to using the ValueVector.
-     *
-     * @param valueCount  The number of values which can be contained within this vector.
-     */
-    @Override
-    public void allocateNew(int valueCount) {
-      allocateNew(getSizeFromCount(valueCount), null, valueCount);
-      for (int i = 0; i < getSizeFromCount(valueCount); i++) {
-        data.setByte(i, 0);
-      }
-    }
-
-  }
-
-  /**
-   * MutableBit implements a vector of bit-width values.  Elements in the vector are accessed
-   * by position from the logical start of the vector.  Values should be pushed onto the vector
-   * sequentially, but may be randomly accessed.
-   *
-   * NB: this class is automatically generated from ValueVectorTypes.tdd using FreeMarker.
-   */
-  public static class MutableBit extends Bit {
-
-    public MutableBit(MaterializedField field, BufferAllocator allocator) {
-      super(field, allocator);
-    }
-
-    /**
-     * Set the bit at the given index to the specified value.
-     *
-     * @param index   position of the bit to set
-     * @param value   value to set (either 1 or 0)
-     */
-    public void set(int index, int value) {
-      byte currentByte = data.getByte((int)Math.floor(index/8));
-      if (value != 0) {
-        // true
-        currentByte |= (byte) Math.pow(2, (index % 8));
-      }
-      else if ((currentByte & (byte) Math.pow(2, (index % 8))) == (byte) Math.pow(2, (index % 8))) {
-        // false, and bit was previously set
-        currentByte -= (byte) Math.pow(2, (index % 8));
-      }
-      data.setByte((int) Math.floor(index/8), currentByte);
-    }
-
-    @Override
-    public void randomizeData() {
-      if (this.data != DeadBuf.DEAD_BUFFER) {
-        Random r = new Random();
-        for (int i = 0; i < data.capacity() - 1; i++) {
-          byte[] bytes = new byte[1];
-          r.nextBytes(bytes);
-          data.setByte(i, bytes[0]);
-        }
-      }
-    }
-  }
-
-<#list types as type>
- <#list type.minor as minor>
-  <#if type.major == "Fixed">
-
-  /**
-   * ${minor.class} implements a vector of fixed width values.  Elements in the vector are accessed
-   * by position, starting from the logical start of the vector.  Values should be pushed onto the
-   * vector sequentially, but may be randomly accessed.
-   *   The width of each element is ${type.width} byte(s)
-   *   The equivilent Java primitive is '${minor.javaType!type.javaType}'
-   *
-   * NB: this class is automatically generated from ValueVectorTypes.tdd using FreeMarker.
-   */
-  public static class ${minor.class} extends Base {
-    static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(${minor.class}.class);
-
-    public ${minor.class}(MaterializedField field, BufferAllocator allocator) {
-      super(field, allocator);
-    }
-
-    /**
-     * Allocate a new memory space for this vector.  Must be called prior to using the ValueVector.
-     *
-     * @param valueCount
-     *          The number of values which can be contained within this vector.
-     */
-    public void allocateNew(int valueCount) {
-      totalBytes = valueCount * ${type.width};
-      allocateNew(totalBytes, allocator.buffer(totalBytes), valueCount);
-    }
-
-    @Override
-    public int getAllocatedSize() {
-      return (int) Math.ceil(totalBytes);
-    }
-
-    /**
-     * Get the size requirement (in bytes) for the given number of values.  Only accurate
-     * for fixed width value vectors.
-     */
-    @Override
-    public int getSizeFromCount(int valueCount) {
-      return valueCount * ${type.width};
-    }
-
-    public Mutable${minor.class} getMutable() {
-      return (Mutable${minor.class})this;
-    }
-
-   <#if (type.width > 8)>
-
-    public ${minor.javaType!type.javaType} get(int index) {
-      ByteBuf dst = allocator.buffer(${type.width});
-      data.getBytes(index * ${type.width}, dst, 0, ${type.width});
-      return dst;
-    }
-
-    @Override
-    public Object getObject(int index) {
-      ByteBuf dst = allocator.buffer(${type.width});
-      data.getBytes(index, dst, 0, ${type.width});
-      return dst;
-    }
-
-    @Override
-    public void randomizeData() {
-      if (this.data != DeadBuf.DEAD_BUFFER) {
-        Random r = new Random();
-        for(int i =0; i < data.capacity()-${type.width}; i += ${type.width}){
-          byte[] bytes = new byte[${type.width}];
-          r.nextBytes(bytes);
-          data.setByte(i, bytes[0]);
-        }
-      }
-    }
-
-   <#else> <#-- type.width <= 8 -->
-
-    public ${minor.javaType!type.javaType} get(int index) {
-      return data.get${(minor.javaType!type.javaType)?cap_first}(index * ${type.width});
-    }
-
-    public Object getObject(int index) {
-      return get(index);
-    }
-
-    @Override
-    public void randomizeData() {
-      if (this.data != DeadBuf.DEAD_BUFFER) {
-        Random r = new Random();
-        for(int i =0; i < data.capacity()-${type.width}; i += ${type.width}){
-          data.set${(minor.javaType!type.javaType)?cap_first}(i,
-              r.next<#if (type.width >= 4)>${(minor.javaType!type.javaType)?cap_first}
-                    <#else>Int
-                    </#if>());
-        }
-      }
-    }
-
-   </#if> <#-- type.width -->
-  }
-
-  /**
-   * Mutable${minor.class} implements a mutable vector of fixed width values.  Elements in the
-   * vector are accessed by position from the logical start of the vector.  Values should be pushed
-   * onto the vector sequentially, but may be randomly accessed.
-   *   The width of each element is ${type.width} byte(s)
-   *   The equivilent Java primitive is '${minor.javaType!type.javaType}'
-   *
-   * NB: this class is automatically generated from ValueVectorTypes.tdd using FreeMarker.
-   */
-   public static class Mutable${minor.class} extends ${minor.class} {
-
-    public Mutable${minor.class}(MaterializedField field, BufferAllocator allocator) {
-      super(field, allocator);
-    }
-
-    /**
-     * Set the element at the given index to the given value.  Note that widths smaller than
-     * 32 bits are handled by the ByteBuf interface.
-     *
-     * @param index   position of the bit to set
-     * @param value   value to set
-     */
-   <#if (type.width > 8)>
-    public void set(int index, <#if (type.width > 4)>${minor.javaType!type.javaType}<#else>int</#if> value) {
-      data.setBytes(index * ${type.width}, value);
-   <#else> <#-- type.width <= 8 -->
-    public void set(int index, <#if (type.width >= 4)>${minor.javaType!type.javaType}<#else>int</#if> value) {
-      data.set${(minor.javaType!type.javaType)?cap_first}(index * ${type.width}, value);
-   </#if> <#-- type.width -->
-    }
-  }
-
-  <#elseif type.major == "VarLen">
-
-  /**
-   * ${minor.class} implements a vector of variable width values.  Elements in the vector
-   * are accessed by position from the logical start of the vector.  A fixed width lengthVector
-   * is used to convert an element's position to it's offset from the start of the (0-based)
-   * ByteBuf.  Size is inferred by adjacent elements.
-   *   The width of each element is ${type.width} byte(s)
-   *   The equivilent Java primitive is '${minor.javaType!type.javaType}'
-   *
-   * NB: this class is automatically generated from ValueVectorTypes.tdd using FreeMarker.
-   */
-  public static class ${minor.class} extends Base {
-    static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(${minor.class}.class);
-
-    protected final MutableUInt${type.width} lengthVector;
-
-    public ${minor.class}(MaterializedField field, BufferAllocator allocator) {
-      super(field, allocator);
-      this.lengthVector = new MutableUInt${type.width}(null, allocator);
-    }
-
-    public byte[] get(int index) {
-      checkArgument(index >= 0);
-      int startIdx = 0;
-      int size = 0;
-      if (index == 0) {
-        size = lengthVector.get(1);
-      } else {
-        startIdx = lengthVector.get(index);
-        size = lengthVector.get(index + 1) - startIdx;
-      }
-      checkState(size >= 0);
-      byte[] dst = new byte[size];
-      data.getBytes(startIdx, dst, 0, size);
-      return dst;
-    }
-
-    @Override
-    public int getAllocatedSize() {
-      return lengthVector.getAllocatedSize() + totalBytes;
-    }
-
-    /**
-     * Get the size requirement (in bytes) for the given number of values.  Only accurate
-     * for fixed width value vectors.
-     */
-    public int getSizeFromCount(int valueCount) {
-      return valueCount * ${type.width};
-    }
-
-    @Override
-    protected void clear() {
-      super.clear();
-      lengthVector.clear();
-    }
-
-    /**
-     * Allocate a new memory space for this vector.  Must be called prior to using the ValueVector.
-     *
-     * @param valueCount
-     *          The number of values which can be contained within this vector.
-     */
-    public void allocateNew(int totalBytes, ByteBuf sourceBuffer, int valueCount) {
-      super.allocateNew(totalBytes, sourceBuffer, valueCount);
-      lengthVector.allocateNew(valueCount);
-    }
-
-    @Override
-    public ByteBuf[] getBuffers() {
-      return new ByteBuf[]{lengthVector.data, data};
-    }
-
-    public Object getObject(int index) {
-      return get(index);
-    }
-
-    public Mutable${minor.class} getMutable() {
-      return (Mutable${minor.class})this;
-    }
-  }
-
-  /**
-   * Mutable${minor.class} implements a vector of variable width values.  Elements in the vector
-   * are accessed by position from the logical start of the vector.  A fixed width lengthVector
-   * is used to convert an element's position to it's offset from the start of the (0-based)
-   * ByteBuf.  Size is inferred by adjacent elements.
-   *   The width of each element is ${type.width} byte(s)
-   *   The equivilent Java primitive is '${minor.javaType!type.javaType}'
-   *
-   * NB: this class is automatically generated from ValueVectorTypes.tdd using FreeMarker.
-   */
-  public static class Mutable${minor.class} extends ${minor.class} {
-
-    public Mutable${minor.class}(MaterializedField field, BufferAllocator allocator) {
-      super(field, allocator);
-    }
-
-    /**
-     * Set the variable length element at the specified index to the supplied byte array.
-     *
-     * @param index   position of the bit to set
-     * @param bytes   array of bytes to write
-     */
-    public void set(int index, byte[] bytes) {
-      checkArgument(index >= 0);
-      if (index == 0) {
-        lengthVector.set(0, 0);
-        lengthVector.set(1, bytes.length);
-        data.setBytes(0, bytes);
-      }
-      else {
-        int currentOffset = lengthVector.get(index);
-        // set the end offset of the buffer
-        lengthVector.set(index + 1, currentOffset + bytes.length);
-        data.setBytes(currentOffset, bytes);
-      }
-    }
-
-    @Override
-    public void setRecordCount(int recordCount) {
-      super.setRecordCount(recordCount);
-      lengthVector.setRecordCount(recordCount);
-    }
-
-  }
-
-  </#if> <#-- type.major -->
-
-  /**
-   * Nullable${minor.class} implements a vector of values which could be null.  Elements in the vector
-   * are first checked against a fixed length vector of boolean values.  Then the element is retrieved
-   * from the base class (if not null).
-   *
-   * NB: this class is automatically generated from ValueVectorTypes.tdd using FreeMarker.
-   */
-  public static class Nullable${minor.class} extends Mutable${minor.class} {
-
-    protected MutableBit bits;
-
-    public Nullable${minor.class}(MaterializedField field, BufferAllocator allocator) {
-      super(field, allocator);
-      bits = new MutableBit(null, allocator);
-    }
-
-    /**
-     * Set the variable length element at the specified index to the supplied byte array.
-     *
-     * @param index   position of the bit to set
-     * @param bytes   array of bytes to write
-     */
-    public void set(int index, <#if type.major == "VarLen">byte[]<#elseif (type.width < 4)>int<#else>${minor.javaType!type.javaType}</#if> value) {
-      setNotNull(index);
-      super.set(index, value);
-    }
-
-    /**
-     * Get the element at the specified position.
-     *
-     * @param   index   position of the value
-     * @return  value of the element, if not null
-     * @throws  NullValueException if the value is null
-     */
-    public <#if type.major == "VarLen">byte[]<#else>${minor.javaType!type.javaType}</#if> get(int index) {
-      if (isNull(index))
-        throw new NullValueException(index);
-      return super.get(index);
-    }
-
-    public void setNull(int index) {
-      bits.set(index, 0);
-    }
-
-    private void setNotNull(int index) {
-      bits.set(index, 1);
-    }
-
-    public boolean isNull(int index) {
-      return bits.get(index) == 0;
-    }
-
-    /**
-     * Allocate a new memory space for this vector.  Must be called prior to using the ValueVector.
-     *
-     * @param valueCount   The number of values which may be contained by this vector.
-     */
-    public void allocateNew(int totalBytes, ByteBuf sourceBuffer, int valueCount) {
-      super.allocateNew(totalBytes, sourceBuffer, valueCount);
-      bits.allocateNew(valueCount);
-    }
-
-    @Override
-    public int getAllocatedSize() {
-      return bits.getAllocatedSize() + super.getAllocatedSize();
-    }
-
-    /**
-     * Get the size requirement (in bytes) for the given number of values.  Only accurate
-     * for fixed width value vectors.
-     */
-    public int getTotalSizeFromCount(int valueCount) {
-      return getSizeFromCount(valueCount) + bits.getSizeFromCount(valueCount);
-    }
-
-    @Override
-    public MaterializedField getField() {
-      return field;
-    }
-
-    @Override
-    public ByteBuf[] getBuffers() {
-      return new ByteBuf[]{bits.data, super.data};
-    }
-
-    @Override
-    public void setRecordCount(int recordCount) {
-      super.setRecordCount(recordCount);
-      bits.setRecordCount(recordCount);
-    }
-
-    @Override
-    public Object getObject(int index) {
-      return isNull(index) ? null : super.getObject(index);
-    }
-  }
-
-  /**
-   * Repeated${minor.class} implements a vector with multple values per row (e.g. JSON array or
-   * repeated protobuf field).  The implementation uses two additional value vectors; one to convert
-   * the index offset to the underlying element offset, and another to store the number of values
-   * in the vector.
-   *
-   * NB: this class is automatically generated from ValueVectorTypes.tdd using FreeMarker.
-   */
-   public static class Repeated${minor.class} extends Mutable${minor.class} {
-
-    private MutableUInt4 countVector;    // number of repeated elements in each record
-    private MutableUInt4 offsetVector;   // offsets to start of each record
-
-    public Repeated${minor.class}(MaterializedField field, BufferAllocator allocator) {
-      super(field, allocator);
-      countVector = new MutableUInt4(null, allocator);
-      offsetVector = new MutableUInt4(null, allocator);
-    }
-
-    public void allocateNew(int totalBytes, ByteBuf sourceBuffer, int valueCount) {
-      super.allocateNew(totalBytes, sourceBuffer, valueCount);
-      countVector.allocateNew(valueCount);
-      offsetVector.allocateNew(valueCount);
-    }
-
-    /**
-     * Add an element to the given record index.  This is similar to the set() method in other
-     * value vectors, except that it permits setting multiple values for a single record.
-     *
-     * @param index   record of the element to add
-     * @param value   value to add to the given row
-     */
-    public void add(int index, <#if (type.width > 4)> ${minor.javaType!type.javaType}
-                               <#elseif type.major == "VarLen"> byte[]
-                               <#else> int
-                               </#if> value) {
-      countVector.set(index, countVector.get(index) + 1);
-      offsetVector.set(index, offsetVector.get(index - 1) + countVector.get(index-1));
-      super.set(offsetVector.get(index), value);
-    }
-
-    /**
-     * Get a value for the given record.  Each element in the repeated field is accessed by
-     * the positionIndex param.
-     *
-     * @param  index           record containing the repeated field
-     * @param  positionIndex   position within the repeated field
-     * @return element at the given position in the given record
-     */
-    public <#if type.major == "VarLen">byte[]
-           <#else>${minor.javaType!type.javaType}
-           </#if> get(int index, int positionIndex) {
-
-      assert positionIndex < countVector.get(index);
-      return super.get(offsetVector.get(index) + positionIndex);
-    }
-
-    public MaterializedField getField() {
-      return field;
-    }
-
-    /**
-     * Get the size requirement (in bytes) for the given number of values.  Only accurate
-     * for fixed width value vectors.
-     */
-    public int getTotalSizeFromCount(int valueCount) {
-      return getSizeFromCount(valueCount) +
-             countVector.getSizeFromCount(valueCount) +
-             offsetVector.getSizeFromCount(valueCount);
-    }
-
-    /**
-     * Get the explicitly specified size of the allocated buffer, if available.  Otherwise
-     * calculate the size based on width and record count.
-     */
-    public int getAllocatedSize() {
-      return super.getAllocatedSize() +
-             countVector.getAllocatedSize() +
-             offsetVector.getAllocatedSize();
-    }
-
-    /**
-     * Get the elements at the given index.
-     */
-    public int getCount(int index) {
-      return countVector.get(index);
-    }
-
-    public void setRecordCount(int recordCount) {
-      super.setRecordCount(recordCount);
-      offsetVector.setRecordCount(recordCount);
-      countVector.setRecordCount(recordCount);
-    }
-
-    public ByteBuf[] getBuffers() {
-      return new ByteBuf[]{countVector.data, offsetVector.data, data};
-    }
-
-    public Object getObject(int index) {
-      return data.slice(index, getSizeFromCount(countVector.get(index)));
-    }
-
-  }
- </#list>
-</#list>
-}
-

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/36793bb2/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/VariableLengthVectors.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/VariableLengthVectors.java b/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/VariableLengthVectors.java
new file mode 100644
index 0000000..954836a
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/VariableLengthVectors.java
@@ -0,0 +1,152 @@
+<@pp.dropOutputFile />
+<#list types as type>
+<#list type.minor as minor>
+
+<#if type.major == "VarLen">
+<@pp.changeOutputFile name="${minor.class}Vector.java" />
+package org.apache.drill.exec.vector;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static com.google.common.base.Preconditions.checkState;
+import io.netty.buffer.ByteBuf;
+
+import java.io.Closeable;
+import java.util.Random;
+
+import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.proto.SchemaDefProtos;
+import org.apache.drill.exec.proto.UserBitShared.FieldMetadata;
+import org.apache.drill.exec.record.DeadBuf;
+import org.apache.drill.exec.record.MaterializedField;
+
+/**
+ * ${minor.class}Vector implements a vector of variable width values.  Elements in the vector
+ * are accessed by position from the logical start of the vector.  A fixed width lengthVector
+ * is used to convert an element's position to it's offset from the start of the (0-based)
+ * ByteBuf.  Size is inferred by adjacent elements.
+ *   The width of each element is ${type.width} byte(s)
+ *   The equivalent Java primitive is '${minor.javaType!type.javaType}'
+ *
+ * NB: this class is automatically generated from ValueVectorTypes.tdd using FreeMarker.
+ */
+@SuppressWarnings("unused")
+public final class ${minor.class}Vector extends ValueVector {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(${minor.class}Vector.class);
+
+  private final UInt${type.width}Vector lengthVector;
+  private final UInt${type.width}Vector.Mutator lengthVectorMutator;
+
+  public ${minor.class}Vector(MaterializedField field, BufferAllocator allocator) {
+    super(field, allocator);
+    this.lengthVector = new UInt${type.width}Vector(null, allocator);
+    this.lengthVectorMutator = lengthVector.getMutator();
+  }
+
+  public byte[] get(int index) {
+    checkArgument(index >= 0);
+    int startIdx = 0;
+    int size = 0;
+    if (index == 0) {
+      size = lengthVector.get(1);
+    } else {
+      startIdx = lengthVector.get(index);
+      size = lengthVector.get(index + 1) - startIdx;
+    }
+    checkState(size >= 0);
+    byte[] dst = new byte[size];
+    data.getBytes(startIdx, dst, 0, size);
+    return dst;
+  }
+
+  @Override
+  public int getAllocatedSize() {
+    return lengthVector.getAllocatedSize() + totalBytes;
+  }
+
+  /**
+   * Get the size requirement (in bytes) for the given number of values.  Only accurate
+   * for fixed width value vectors.
+   */
+  public int getSizeFromCount(int valueCount) {
+    return valueCount * ${type.width};
+  }
+
+  @Override
+  protected void clear() {
+    super.clear();
+    lengthVector.clear();
+  }
+
+  /**
+   * Allocate a new memory space for this vector.  Must be called prior to using the ValueVector.
+   *
+   * @param valueCount
+   *          The number of values which can be contained within this vector.
+   */
+  public void allocateNew(int totalBytes, ByteBuf sourceBuffer, int valueCount) {
+    super.allocateNew(totalBytes, sourceBuffer, valueCount);
+    lengthVector.allocateNew(valueCount);
+  }
+
+  @Override
+  public ByteBuf[] getBuffers() {
+    return new ByteBuf[]{lengthVector.data, data};
+  }
+
+  public Object getObject(int index) {
+    return get(index);
+  }
+
+  public Mutator getMutator() {
+    return new Mutator();
+  }
+  
+  
+  /**
+   * Mutable${minor.class} implements a vector of variable width values.  Elements in the vector
+   * are accessed by position from the logical start of the vector.  A fixed width lengthVector
+   * is used to convert an element's position to it's offset from the start of the (0-based)
+   * ByteBuf.  Size is inferred by adjacent elements.
+   *   The width of each element is ${type.width} byte(s)
+   *   The equivalent Java primitive is '${minor.javaType!type.javaType}'
+   *
+   * NB: this class is automatically generated from ValueVectorTypes.tdd using FreeMarker.
+   */
+  public class Mutator implements ValueVector.Mutator{
+
+    /**
+     * Set the variable length element at the specified index to the supplied byte array.
+     *
+     * @param index   position of the bit to set
+     * @param bytes   array of bytes to write
+     */
+    public void set(int index, byte[] bytes) {
+      checkArgument(index >= 0);
+      if (index == 0) {
+        lengthVectorMutator.set(0, 0);
+        lengthVectorMutator.set(1, bytes.length);
+        data.setBytes(0, bytes);
+      } else {
+        int currentOffset = lengthVector.get(index);
+        // set the end offset of the buffer
+        lengthVectorMutator.set(index + 1, currentOffset + bytes.length);
+        data.setBytes(currentOffset, bytes);
+      }
+    }
+
+    @Override
+    public void setRecordCount(int recordCount) {
+      ${minor.class}Vector.this.setRecordCount(recordCount);
+      lengthVector.setRecordCount(recordCount);
+    }
+
+    @Override
+    public void randomizeData(){}
+  }
+  
+}
+
+
+</#if> <#-- type.major -->
+</#list>
+</#list>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/36793bb2/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockRecordReader.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockRecordReader.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockRecordReader.java
index 65584db..cd3371d 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockRecordReader.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockRecordReader.java
@@ -27,9 +27,9 @@ import org.apache.drill.exec.physical.impl.OutputMutator;
 import org.apache.drill.exec.proto.SchemaDefProtos.DataMode;
 import org.apache.drill.exec.proto.SchemaDefProtos.MajorType;
 import org.apache.drill.exec.record.MaterializedField;
-import org.apache.drill.exec.record.vector.TypeHelper;
-import org.apache.drill.exec.record.vector.ValueVector;
 import org.apache.drill.exec.store.RecordReader;
+import org.apache.drill.exec.vector.TypeHelper;
+import org.apache.drill.exec.vector.ValueVector;
 
 public class MockRecordReader implements RecordReader {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(MockRecordReader.class);
@@ -37,7 +37,7 @@ public class MockRecordReader implements RecordReader {
   private OutputMutator output;
   private MockScanEntry config;
   private FragmentContext context;
-  private ValueVector.Base[] valueVectors;
+  private ValueVector[] valueVectors;
   private int recordsRead;
 
   public MockRecordReader(FragmentContext context, MockScanEntry config) {
@@ -53,12 +53,12 @@ public class MockRecordReader implements RecordReader {
     return x;
   }
 
-  private ValueVector.Base getVector(int fieldId, String name, MajorType type, int length) {
+  private ValueVector getVector(int fieldId, String name, MajorType type, int length) {
     assert context != null : "Context shouldn't be null.";
     if(type.getMode() != DataMode.REQUIRED) throw new UnsupportedOperationException();
     
     MaterializedField f = MaterializedField.create(new SchemaPath(name), fieldId, 0, type);
-    ValueVector.Base v;
+    ValueVector v;
     v = TypeHelper.getNewVector(f, context.getAllocator());
     v.allocateNew(length);
     return v;
@@ -70,7 +70,7 @@ public class MockRecordReader implements RecordReader {
     try {
       this.output = output;
       int estimateRowSize = getEstimatedRecordSize(config.getTypes());
-      valueVectors = new ValueVector.Base[config.getTypes().length];
+      valueVectors = new ValueVector[config.getTypes().length];
       int batchRecordCount = 250000 / estimateRowSize;
 
       for (int i = 0; i < config.getTypes().length; i++) {
@@ -88,10 +88,11 @@ public class MockRecordReader implements RecordReader {
   public int next() {
     int recordSetSize = Math.min(valueVectors[0].capacity(), this.config.getRecords()- recordsRead);
     recordsRead += recordSetSize;
-    for(ValueVector.Base v : valueVectors){
+    for(ValueVector v : valueVectors){
       logger.debug("MockRecordReader:  Generating random data for VV of type " + v.getClass().getName());
       v.randomizeData();
-      v.setRecordCount(recordSetSize);
+      
+      v.getMutator().setRecordCount(recordSetSize);
     }
     return recordSetSize;
   }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/36793bb2/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockScanBatchCreator.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockScanBatchCreator.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockScanBatchCreator.java
index b821d6e..bfc19af 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockScanBatchCreator.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockScanBatchCreator.java
@@ -24,10 +24,7 @@ import org.apache.drill.exec.ops.FragmentContext;
 import org.apache.drill.exec.physical.config.MockScanPOP.MockScanEntry;
 import org.apache.drill.exec.physical.impl.BatchCreator;
 import org.apache.drill.exec.physical.impl.ScanBatch;
-import org.apache.drill.exec.record.BatchSchema;
-import org.apache.drill.exec.record.InvalidValueAccessor;
 import org.apache.drill.exec.record.RecordBatch;
-import org.apache.drill.exec.record.vector.ValueVector;
 import org.apache.drill.exec.store.RecordReader;
 
 import com.google.common.base.Preconditions;

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/36793bb2/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockScanPOP.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockScanPOP.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockScanPOP.java
index 4a3a606..40227e5 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockScanPOP.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockScanPOP.java
@@ -32,7 +32,7 @@ import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
 import org.apache.drill.exec.proto.SchemaDefProtos.DataMode;
 import org.apache.drill.exec.proto.SchemaDefProtos.MajorType;
 import org.apache.drill.exec.proto.SchemaDefProtos.MinorType;
-import org.apache.drill.exec.record.vector.TypeHelper;
+import org.apache.drill.exec.vector.TypeHelper;
 
 import com.fasterxml.jackson.annotation.JsonCreator;
 import com.fasterxml.jackson.annotation.JsonIgnore;

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/36793bb2/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/FilterRecordBatch.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/FilterRecordBatch.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/FilterRecordBatch.java
index b7b7d93..6440d98 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/FilterRecordBatch.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/FilterRecordBatch.java
@@ -22,7 +22,7 @@ import org.apache.drill.exec.record.BatchSchema;
 import org.apache.drill.exec.record.InvalidValueAccessor;
 import org.apache.drill.exec.record.RecordBatch;
 import org.apache.drill.exec.record.vector.SelectionVector;
-import org.apache.drill.exec.record.vector.ValueVector;
+import org.apache.drill.exec.vector.ValueVector;
 
 public abstract class FilterRecordBatch implements RecordBatch {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(FilterRecordBatch.class);
@@ -58,7 +58,7 @@ public abstract class FilterRecordBatch implements RecordBatch {
   }
 
   @Override
-  public <T extends ValueVector.Base> T getValueVector(int fieldId, Class<T> clazz) throws InvalidValueAccessor {
+  public <T extends ValueVector> T getValueVector(int fieldId, Class<T> clazz) throws InvalidValueAccessor {
     return null;
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/36793bb2/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/OutputMutator.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/OutputMutator.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/OutputMutator.java
index e96c1be..b3b9f5f 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/OutputMutator.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/OutputMutator.java
@@ -18,11 +18,10 @@
 package org.apache.drill.exec.physical.impl;
 
 import org.apache.drill.exec.exception.SchemaChangeException;
-import org.apache.drill.exec.record.BatchSchema;
-import org.apache.drill.exec.record.vector.ValueVector;
+import org.apache.drill.exec.vector.ValueVector;
 
 public interface OutputMutator {
   public void removeField(int fieldId) throws SchemaChangeException;
-  public void addField(int fieldId, ValueVector.Base vector) throws SchemaChangeException ;
+  public void addField(int fieldId, ValueVector vector) throws SchemaChangeException ;
   public void setNewSchema() throws SchemaChangeException ;
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/36793bb2/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScanBatch.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScanBatch.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScanBatch.java
index 822d828..2f3e1fe 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScanBatch.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScanBatch.java
@@ -17,27 +17,21 @@
  ******************************************************************************/
 package org.apache.drill.exec.physical.impl;
 
-import io.netty.buffer.ByteBuf;
-
 import java.util.Iterator;
-import java.util.List;
 
 import org.apache.drill.common.exceptions.ExecutionSetupException;
 import org.apache.drill.exec.exception.SchemaChangeException;
 import org.apache.drill.exec.ops.FragmentContext;
-import org.apache.drill.exec.proto.UserBitShared.FieldMetadata;
-import org.apache.drill.exec.proto.UserBitShared.RecordBatchDef;
 import org.apache.drill.exec.record.BatchSchema;
 import org.apache.drill.exec.record.InvalidValueAccessor;
 import org.apache.drill.exec.record.RecordBatch;
 import org.apache.drill.exec.record.SchemaBuilder;
 import org.apache.drill.exec.record.WritableBatch;
-import org.apache.drill.exec.record.vector.ValueVector;
 import org.apache.drill.exec.store.RecordReader;
+import org.apache.drill.exec.vector.ValueVector;
 
 import com.carrotsearch.hppc.IntObjectOpenHashMap;
 import com.carrotsearch.hppc.procedures.IntObjectProcedure;
-import com.google.common.collect.Lists;
 
 /**
  * Record batch used for a particular scan. Operators against one or more
@@ -45,7 +39,7 @@ import com.google.common.collect.Lists;
 public class ScanBatch implements RecordBatch {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ScanBatch.class);
 
-  private IntObjectOpenHashMap<ValueVector.Base> fields = new IntObjectOpenHashMap<ValueVector.Base>();
+  private IntObjectOpenHashMap<ValueVector> fields = new IntObjectOpenHashMap<ValueVector>();
   private BatchSchema schema;
   private int recordCount;
   private boolean schemaChanged = true;
@@ -89,9 +83,9 @@ public class ScanBatch implements RecordBatch {
   }
 
   private void releaseAssets() {
-    fields.forEach(new IntObjectProcedure<ValueVector.Base>() {
+    fields.forEach(new IntObjectProcedure<ValueVector>() {
       @Override
-      public void apply(int key, ValueVector.Base value) {
+      public void apply(int key, ValueVector value) {
         value.close();
       }
     });
@@ -99,9 +93,9 @@ public class ScanBatch implements RecordBatch {
 
   @SuppressWarnings("unchecked")
   @Override
-  public <T extends ValueVector.Base> T getValueVector(int fieldId, Class<T> clazz) throws InvalidValueAccessor {
+  public <T extends ValueVector> T getValueVector(int fieldId, Class<T> clazz) throws InvalidValueAccessor {
     if (fields.containsKey(fieldId)) throw new InvalidValueAccessor(String.format("Unknown value accesor for field id %d."));
-    ValueVector.Base vector = this.fields.lget();
+    ValueVector vector = this.fields.lget();
     if (vector.getClass().isAssignableFrom(clazz)) {
       return (T) vector;
     } else {
@@ -143,14 +137,14 @@ public class ScanBatch implements RecordBatch {
     
     public void removeField(int fieldId) throws SchemaChangeException {
       schemaChanged();
-      ValueVector.Base v = fields.remove(fieldId);
+      ValueVector v = fields.remove(fieldId);
       if (v == null) throw new SchemaChangeException("Failure attempting to remove an unknown field.");
       v.close();
     }
 
-    public void addField(int fieldId, ValueVector.Base vector) {
+    public void addField(int fieldId, ValueVector vector) {
       schemaChanged();
-      ValueVector.Base v = fields.put(fieldId, vector);
+      ValueVector v = fields.put(fieldId, vector);
       vector.getField();
       builder.addField(vector.getField());
       if (v != null) v.close();

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/36793bb2/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/WireRecordBatch.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/WireRecordBatch.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/WireRecordBatch.java
index be32d1f..fcbd272 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/WireRecordBatch.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/WireRecordBatch.java
@@ -17,8 +17,6 @@
  ******************************************************************************/
 package org.apache.drill.exec.physical.impl;
 
-import java.util.Iterator;
-
 import org.apache.drill.exec.exception.SchemaChangeException;
 import org.apache.drill.exec.ops.FragmentContext;
 import org.apache.drill.exec.proto.UserBitShared.RecordBatchDef;
@@ -29,8 +27,7 @@ import org.apache.drill.exec.record.RawFragmentBatchProvider;
 import org.apache.drill.exec.record.RecordBatch;
 import org.apache.drill.exec.record.RecordBatchLoader;
 import org.apache.drill.exec.record.WritableBatch;
-import org.apache.drill.exec.record.RecordBatch.IterOutcome;
-import org.apache.drill.exec.record.vector.ValueVector;
+import org.apache.drill.exec.vector.ValueVector;
 
 public class WireRecordBatch implements RecordBatch{
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(WireRecordBatch.class);
@@ -68,7 +65,7 @@ public class WireRecordBatch implements RecordBatch{
   }
 
   @Override
-  public <T extends ValueVector.Base> T getValueVector(int fieldId, Class<T> clazz) throws InvalidValueAccessor {
+  public <T extends ValueVector> T getValueVector(int fieldId, Class<T> clazz) throws InvalidValueAccessor {
     return batchLoader.getValueVector(fieldId, clazz);
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/36793bb2/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/BatchSchema.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/BatchSchema.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/BatchSchema.java
index 05b1cc7..b26e742 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/BatchSchema.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/BatchSchema.java
@@ -20,8 +20,6 @@ package org.apache.drill.exec.record;
 import java.util.Iterator;
 import java.util.List;
 
-import org.apache.drill.exec.record.vector.ValueVector;
-
 
 public class BatchSchema implements Iterable<MaterializedField> {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(BatchSchema.class);

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/36793bb2/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatch.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatch.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatch.java
index 042c40c..c6b7888 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatch.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatch.java
@@ -18,7 +18,7 @@
 package org.apache.drill.exec.record;
 
 import org.apache.drill.exec.ops.FragmentContext;
-import org.apache.drill.exec.record.vector.ValueVector;
+import org.apache.drill.exec.vector.ValueVector;
 
 /**
  * A record batch contains a set of field values for a particular range of records. In the case of a record batch
@@ -68,7 +68,7 @@ public interface RecordBatch {
   public void kill();
 
 
-  public abstract <T extends ValueVector.Base> T getValueVector(int fieldId, Class<T> clazz) throws InvalidValueAccessor;
+  public abstract <T extends ValueVector> T getValueVector(int fieldId, Class<T> clazz) throws InvalidValueAccessor;
 
 //  public abstract void getDictReader(int fieldId, Class<?> clazz) throws InvalidValueAccessor;
 //

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/36793bb2/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatchLoader.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatchLoader.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatchLoader.java
index ea1de73..be43026 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatchLoader.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatchLoader.java
@@ -27,16 +27,16 @@ import org.apache.drill.exec.memory.BufferAllocator;
 import org.apache.drill.exec.proto.SchemaDefProtos.FieldDef;
 import org.apache.drill.exec.proto.UserBitShared.FieldMetadata;
 import org.apache.drill.exec.proto.UserBitShared.RecordBatchDef;
-import org.apache.drill.exec.record.vector.TypeHelper;
-import org.apache.drill.exec.record.vector.ValueVector;
+import org.apache.drill.exec.vector.TypeHelper;
+import org.apache.drill.exec.vector.ValueVector;
 
 import com.carrotsearch.hppc.IntObjectOpenHashMap;
 import com.carrotsearch.hppc.cursors.IntObjectCursor;
 
-public class RecordBatchLoader implements Iterable<IntObjectCursor<ValueVector.Base>>{
+public class RecordBatchLoader implements Iterable<IntObjectCursor<ValueVector>>{
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(RecordBatchLoader.class);
 
-  private IntObjectOpenHashMap<ValueVector.Base> vectors = new IntObjectOpenHashMap<ValueVector.Base>();
+  private IntObjectOpenHashMap<ValueVector> vectors = new IntObjectOpenHashMap<ValueVector>();
   private final BufferAllocator allocator;
   private int recordCount; 
   private BatchSchema schema;
@@ -61,14 +61,14 @@ public class RecordBatchLoader implements Iterable<IntObjectCursor<ValueVector.B
     this.recordCount = def.getRecordCount();
     boolean schemaChanged = false;
     
-    IntObjectOpenHashMap<ValueVector.Base> newVectors = new IntObjectOpenHashMap<ValueVector.Base>();
+    IntObjectOpenHashMap<ValueVector> newVectors = new IntObjectOpenHashMap<ValueVector>();
 
     List<FieldMetadata> fields = def.getFieldList();
     
     int bufOffset = 0;
     for (FieldMetadata fmd : fields) {
       FieldDef fieldDef = fmd.getDef();
-      ValueVector.Base v = vectors.remove(fieldDef.getFieldId());
+      ValueVector v = vectors.remove(fieldDef.getFieldId());
       if (v != null) {
         if (v.getField().getDef().equals(fieldDef)) {
           v.allocateNew(fmd.getBufferLength(), buf.slice(bufOffset, fmd.getBufferLength()), recordCount);
@@ -89,7 +89,7 @@ public class RecordBatchLoader implements Iterable<IntObjectCursor<ValueVector.B
     
     if(!vectors.isEmpty()){
       schemaChanged = true;
-      for(IntObjectCursor<ValueVector.Base> cursor : newVectors){
+      for(IntObjectCursor<ValueVector> cursor : newVectors){
         cursor.value.close();
       }
       
@@ -98,7 +98,7 @@ public class RecordBatchLoader implements Iterable<IntObjectCursor<ValueVector.B
     if(schemaChanged){
       // rebuild the schema.
       SchemaBuilder b = BatchSchema.newBuilder();
-      for(IntObjectCursor<ValueVector.Base> cursor : newVectors){
+      for(IntObjectCursor<ValueVector> cursor : newVectors){
         b.addField(cursor.value.getField());
       }
       b.setSelectionVector(false);
@@ -110,8 +110,8 @@ public class RecordBatchLoader implements Iterable<IntObjectCursor<ValueVector.B
   }
 
   @SuppressWarnings("unchecked")
-  public <T extends ValueVector.Base> T getValueVector(int fieldId, Class<T> clazz) throws InvalidValueAccessor {
-    ValueVector.Base v = vectors.get(fieldId);
+  public <T extends ValueVector> T getValueVector(int fieldId, Class<T> clazz) throws InvalidValueAccessor {
+    ValueVector v = vectors.get(fieldId);
     assert v != null;
     if (v.getClass() != clazz)
       throw new InvalidValueAccessor(String.format(
@@ -130,7 +130,7 @@ public class RecordBatchLoader implements Iterable<IntObjectCursor<ValueVector.B
   }
 
   @Override
-  public Iterator<IntObjectCursor<ValueVector.Base>> iterator() {
+  public Iterator<IntObjectCursor<ValueVector>> iterator() {
     return this.vectors.iterator();
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/36793bb2/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/WritableBatch.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/WritableBatch.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/WritableBatch.java
index 67c6cb9..a367b6d 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/WritableBatch.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/WritableBatch.java
@@ -23,7 +23,7 @@ import java.util.List;
 
 import org.apache.drill.exec.proto.UserBitShared.FieldMetadata;
 import org.apache.drill.exec.proto.UserBitShared.RecordBatchDef;
-import org.apache.drill.exec.record.vector.ValueVector;
+import org.apache.drill.exec.vector.ValueVector;
 
 import com.carrotsearch.hppc.IntObjectOpenHashMap;
 import com.carrotsearch.hppc.procedures.IntObjectProcedure;
@@ -58,7 +58,7 @@ public class WritableBatch {
     return buffers;
   }
   
-//  public static WritableBatch get(ValueVector.Base[] vectors){
+//  public static WritableBatch get(ValueVector[] vectors){
 //    WritableCreator c = new WritableCreator();
 //    for(int i =0; i < vectors.length; i++){
 //      c.apply(i, vectors[i]);
@@ -67,14 +67,14 @@ public class WritableBatch {
 //  }
 //  
   
-  public static WritableBatch get(int recordCount, IntObjectOpenHashMap<ValueVector.Base> fields){
+  public static WritableBatch get(int recordCount, IntObjectOpenHashMap<ValueVector> fields){
     WritableCreator creator = new WritableCreator(recordCount);
     fields.forEach(creator);
     return creator.get();
     
   }
   
-  private static class WritableCreator implements IntObjectProcedure<ValueVector.Base>{
+  private static class WritableCreator implements IntObjectProcedure<ValueVector>{
     
     List<ByteBuf> buffers = Lists.newArrayList();
     List<FieldMetadata> metadata = Lists.newArrayList();
@@ -87,7 +87,7 @@ public class WritableBatch {
     }
     
     @Override
-    public void apply(int key, ValueVector.Base value) {
+    public void apply(int key, ValueVector value) {
       metadata.add(value.getMetadata());
       for(ByteBuf b : value.getBuffers()){
         buffers.add(b);

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/36793bb2/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/SelectionVector.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/SelectionVector.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/SelectionVector.java
index e79a525..02b75ce 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/SelectionVector.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/SelectionVector.java
@@ -17,19 +17,25 @@
  ******************************************************************************/
 package org.apache.drill.exec.record.vector;
 
-import io.netty.buffer.ByteBufAllocator;
-
 import org.apache.drill.exec.memory.BufferAllocator;
 import org.apache.drill.exec.record.MaterializedField;
 
 /**
  * Convenience/Clarification Fixed2 wrapper.
  */
-public class SelectionVector extends ValueVector.UInt2 {
+public class SelectionVector {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(SelectionVector.class);
 
   public SelectionVector(MaterializedField field, BufferAllocator allocator) {
-    super(field, allocator);
+
+  }
+
+  public int capacity() {
+    return -1;
+  }
+
+  public void allocateNew(int count) {
+
   }
 
 }


[16/53] [abbrv] Update typing system. Update RPC system. Add Fragmenting Implementation. Working single node. Distributed failing due to threading issues.

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractExchange.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractExchange.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractExchange.java
new file mode 100644
index 0000000..42a15ae
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractExchange.java
@@ -0,0 +1,90 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.physical.base;
+
+import java.util.List;
+
+import org.apache.drill.common.exceptions.PhysicalOperatorSetupException;
+import org.apache.drill.exec.physical.OperatorCost;
+import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
+
+import com.fasterxml.jackson.annotation.JsonIgnore;
+import com.fasterxml.jackson.annotation.JsonProperty;
+
+public abstract class AbstractExchange extends AbstractSingle implements Exchange {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(AbstractExchange.class);
+
+  protected int senderMajorFragmentId;
+  protected int receiverMajorFragmentId;
+
+  public AbstractExchange(PhysicalOperator child) {
+    super(child);
+  }
+
+  /**
+   * Exchanges are not executable. The Execution layer first has to set their parallelization and convert them into
+   * something executable
+   */
+  @Override
+  public boolean isExecutable() {
+    return false;
+  }
+
+  protected abstract void setupSenders(List<DrillbitEndpoint> senderLocations) throws PhysicalOperatorSetupException ;
+  protected abstract void setupReceivers(List<DrillbitEndpoint> senderLocations) throws PhysicalOperatorSetupException ;
+  
+  @Override
+  public final void setupSenders(int majorFragmentId, List<DrillbitEndpoint> senderLocations) throws PhysicalOperatorSetupException {
+    this.senderMajorFragmentId = majorFragmentId;
+    setupSenders(senderLocations);
+  }
+  
+
+  @Override
+  public final void setupReceivers(int majorFragmentId, List<DrillbitEndpoint> receiverLocations) throws PhysicalOperatorSetupException {
+    this.receiverMajorFragmentId = majorFragmentId;
+    setupReceivers(receiverLocations);
+  }
+  
+  @Override
+  public OperatorCost getAggregateSendCost() {
+    return getExchangeCost().getSendCost();
+  }
+
+  @Override
+  public OperatorCost getAggregateReceiveCost() {
+    return getExchangeCost().getReceiveCost();
+  }
+
+  @Override
+  public final <T, X, E extends Throwable> T accept(PhysicalVisitor<T, X, E> physicalVisitor, X value) throws E {
+    return physicalVisitor.visitExchange(this, value);
+  }
+
+  @Override
+  public ExchangeCost getExchangeCost(){
+    return ExchangeCost.getSimpleEstimate(getSize());
+  }
+
+  @JsonIgnore
+  @Override
+  public OperatorCost getCost() {
+    return getExchangeCost().getCombinedCost();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/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
new file mode 100644
index 0000000..f782325
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractPhysicalVisitor.java
@@ -0,0 +1,124 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.physical.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;
+
+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);
+
+  @Override
+  public T visitExchange(Exchange exchange, X value) throws E{
+    return visitOp(exchange, value);
+  }
+
+  @Override
+  public T visitFilter(Filter filter, X value) throws E{
+    return visitOp(filter, value);
+  }
+
+  @Override
+  public T visitProject(Project project, X value) throws E{
+    return visitOp(project, value);
+  }
+
+  @Override
+  public T visitSort(Sort sort, X value) throws E{
+    return visitOp(sort, value);
+  }
+
+  @Override
+  public T visitSender(Sender sender, X value) throws E {
+    return visitOp(sender, value);
+  }
+
+  @Override
+  public T visitReceiver(Receiver receiver, X value) throws E {
+    return visitOp(receiver, value);
+  }
+
+  @Override
+  public T visitScan(Scan<?> scan, X value) throws E{
+    return visitOp(scan, value);
+  }
+
+  @Override
+  public T visitStore(Store store, X value) throws E{
+    return visitOp(store, value);
+  }
+
+  
+  public T visitChildren(PhysicalOperator op, X value) throws E{
+    for(PhysicalOperator child : op){
+      child.accept(this, value);
+    }
+    return null;
+  }
+  
+  @Override
+  public T visitHashPartitionSender(HashPartitionSender op, X value) throws E {
+    return visitSender(op, value);
+  }
+
+  @Override
+  public T visitRandomReceiver(RandomReceiver op, X value) throws E {
+    return visitReceiver(op, value);
+  }
+
+  @Override
+  public T visitHashPartitionSender(HashToRandomExchange op, X value) throws E {
+    return visitExchange(op, value);
+  }
+
+  @Override
+  public T visitRangeSender(RangeSender op, X value) throws E {
+    return visitSender(op, value);
+  }
+
+  @Override
+  public T visitScreen(Screen op, X value) throws E {
+    return visitStore(op, value);
+  }
+
+  @Override
+  public T visitSingleSender(SingleSender op, X value) throws E {
+    return visitSender(op, value);
+  }
+
+  @Override
+  public T visitUnionExchange(UnionExchange op, X value) throws E {
+    return visitExchange(op, value);
+  }
+
+  @Override
+  public T visitOp(PhysicalOperator op, X value) throws E{
+    throw new UnsupportedOperationException(String.format(
+        "The PhysicalVisitor of type %s does not currently support visiting the PhysicalOperator type %s.", this
+            .getClass().getCanonicalName(), op.getClass().getCanonicalName()));
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractReceiver.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractReceiver.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractReceiver.java
new file mode 100644
index 0000000..e8ba19c
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractReceiver.java
@@ -0,0 +1,63 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.physical.base;
+
+import java.util.Iterator;
+import java.util.List;
+
+import org.apache.drill.exec.physical.OperatorCost;
+
+import com.fasterxml.jackson.annotation.JsonIgnore;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Iterators;
+
+public abstract class AbstractReceiver extends AbstractBase implements Receiver{
+
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(AbstractReceiver.class);
+
+  private final int oppositeMajorFragmentId; 
+  
+  public AbstractReceiver(int oppositeMajorFragmentId){
+    this.oppositeMajorFragmentId = oppositeMajorFragmentId;
+  }
+  
+  @Override
+  public Iterator<PhysicalOperator> iterator() {
+    return Iterators.emptyIterator();
+  }
+
+  @Override
+  public <T, X, E extends Throwable> T accept(PhysicalVisitor<T, X, E> physicalVisitor, X value) throws E {
+    return physicalVisitor.visitReceiver(this, value);
+  }
+
+  @Override
+  public final PhysicalOperator getNewWithChildren(List<PhysicalOperator> children) {
+    Preconditions.checkArgument(children.isEmpty());
+    //rewriting is unnecessary since the inputs haven't changed.
+    return this;
+  }
+
+  @JsonProperty("sender-major-fragment")
+  public int getOppositeMajorFragmentId() {
+    return oppositeMajorFragmentId;
+  }
+
+}
+

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractScan.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractScan.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractScan.java
new file mode 100644
index 0000000..dbde9c5
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractScan.java
@@ -0,0 +1,84 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.physical.base;
+
+import java.util.Iterator;
+import java.util.List;
+
+import org.apache.drill.exec.physical.OperatorCost;
+import org.apache.drill.exec.physical.ReadEntry;
+
+import com.fasterxml.jackson.annotation.JsonIgnore;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.google.common.collect.Iterators;
+
+public abstract class AbstractScan<R extends ReadEntry> extends AbstractBase implements Scan<R>{
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(AbstractScan.class);
+  
+  protected final List<R> readEntries;
+  private final OperatorCost cost;
+  private final Size size;
+  
+  public AbstractScan(List<R> readEntries) {
+    this.readEntries = readEntries;
+    OperatorCost cost = new OperatorCost(0,0,0,0);
+    Size size = new Size(0,0);
+    for(R r : readEntries){
+      cost = cost.add(r.getCost());
+      size = size.add(r.getSize());
+    }
+    this.cost = cost;
+    this.size = size;
+  }
+
+  @Override
+  @JsonProperty("entries")
+  public List<R> getReadEntries() {
+    return readEntries;
+  }
+  
+  @Override
+  public Iterator<PhysicalOperator> iterator() {
+    return Iterators.emptyIterator();
+  }
+
+  @Override
+  public boolean isExecutable() {
+    return true;
+  }
+
+  @Override
+  public <T, X, E extends Throwable> T accept(PhysicalVisitor<T, X, E> physicalVisitor, X value) throws E{
+    return physicalVisitor.visitScan(this, value);
+  }
+
+  @Override
+  public OperatorCost getCost() {
+    return cost;
+  }
+
+  @Override
+  public Size getSize() {
+    return size;
+  }
+  
+  
+  
+  
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractSender.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractSender.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractSender.java
new file mode 100644
index 0000000..f8c22b3
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractSender.java
@@ -0,0 +1,53 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.physical.base;
+
+import org.apache.drill.exec.physical.OperatorCost;
+
+import com.fasterxml.jackson.annotation.JsonProperty;
+
+
+
+public abstract class AbstractSender extends AbstractSingle implements Sender {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(AbstractSender.class);
+
+  protected final int oppositeMajorFragmentId;
+  
+  public AbstractSender(int oppositeMajorFragmentId, PhysicalOperator child) {
+    super(child);
+    this.oppositeMajorFragmentId = oppositeMajorFragmentId;
+  }
+
+  @Override
+  public <T, X, E extends Throwable> T accept(PhysicalVisitor<T, X, E> physicalVisitor, X value) throws E {
+    return physicalVisitor.visitSender(this, value);
+  }
+
+  @Override
+  public OperatorCost getCost() {
+    float network = child.getSize().getAggSize();
+    return new OperatorCost(network, 0, 1000, child.getSize().getRecordCount());
+  }
+
+  @Override
+  public int getOppositeMajorFragmentId() {
+    return oppositeMajorFragmentId;
+  }
+  
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractSingle.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractSingle.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractSingle.java
new file mode 100644
index 0000000..264ee91
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractSingle.java
@@ -0,0 +1,68 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.physical.base;
+
+import java.util.Iterator;
+import java.util.List;
+
+
+import com.fasterxml.jackson.annotation.JsonIgnore;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Iterators;
+
+/**
+ * Describes an operator that expects a single child operator as its input.
+ * @param <T> The type of Exec model supported.
+ */
+public abstract class AbstractSingle extends AbstractBase{
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(AbstractSingle.class);
+  
+  protected final PhysicalOperator child;
+
+  public AbstractSingle(PhysicalOperator child) {
+    super();
+    this.child = child;
+  }
+
+  @Override
+  public Iterator<PhysicalOperator> iterator() {
+    return Iterators.singletonIterator(child);
+  }
+
+  public PhysicalOperator getChild(){
+    return child;
+  }
+  
+  @Override
+  public Size getSize() {
+    return child.getSize();
+  }
+  
+  @Override
+  @JsonIgnore
+  public final PhysicalOperator getNewWithChildren(List<PhysicalOperator> children) {
+    Preconditions.checkArgument(children.size() == 1);
+    return getNewWithChild(children.iterator().next());
+  }
+  
+  protected abstract PhysicalOperator getNewWithChild(PhysicalOperator child);
+    
+  
+  
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractStore.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractStore.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractStore.java
new file mode 100644
index 0000000..a833a4e
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractStore.java
@@ -0,0 +1,36 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.physical.base;
+
+
+
+
+public abstract class AbstractStore extends AbstractSingle implements Store{
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(AbstractStore.class);
+
+  public AbstractStore(PhysicalOperator child) {
+    super(child);
+  }
+
+  @Override
+  public <T, X, E extends Throwable> T accept(PhysicalVisitor<T, X, E> physicalVisitor, X value) throws E{
+    return physicalVisitor.visitStore(this, value);
+  }
+
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/Exchange.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/Exchange.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/Exchange.java
new file mode 100644
index 0000000..c8bc3a8
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/Exchange.java
@@ -0,0 +1,92 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.physical.base;
+
+import java.util.List;
+
+import org.apache.drill.common.exceptions.PhysicalOperatorSetupException;
+import org.apache.drill.exec.physical.OperatorCost;
+import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
+
+import com.fasterxml.jackson.annotation.JsonIgnore;
+import com.fasterxml.jackson.annotation.JsonProperty;
+
+public interface Exchange extends PhysicalOperator {
+
+  @JsonIgnore
+  public abstract OperatorCost getAggregateSendCost();
+
+  @JsonIgnore
+  public abstract OperatorCost getAggregateReceiveCost();
+
+  @JsonIgnore
+  public abstract ExchangeCost getExchangeCost();
+
+  /**
+   * Inform this Exchange node about its sender locations. This list should be index-ordered the same as the expected
+   * minorFragmentIds for each sender.
+   * 
+   * @param senderLocations
+   */
+  public abstract void setupSenders(int majorFragmentId, List<DrillbitEndpoint> senderLocations) throws PhysicalOperatorSetupException;
+
+  /**
+   * Inform this Exchange node about its receiver locations. This list should be index-ordered the same as the expected
+   * minorFragmentIds for each receiver.
+   * 
+   * @param receiverLocations
+   */
+  public abstract void setupReceivers(int majorFragmentId, List<DrillbitEndpoint> receiverLocations) throws PhysicalOperatorSetupException;
+
+  /**
+   * Get the Sender associated with the given minorFragmentId. Cannot be called until after setupSenders() and
+   * setupReceivers() have been called.
+   * 
+   * @param minorFragmentId
+   *          The minor fragment id, must be in the range [0, fragment.width).
+   * @param child
+   *          The feeding node for the requested sender.
+   * @return The materialized sender for the given arguments.
+   */
+  public abstract Sender getSender(int minorFragmentId, PhysicalOperator child) throws PhysicalOperatorSetupException;
+
+  /**
+   * Get the Receiver associated with the given minorFragmentId. Cannot be called until after setupSenders() and
+   * setupReceivers() have been called.
+   * 
+   * @param minorFragmentId
+   *          The minor fragment id, must be in the range [0, fragment.width).
+   * @return The materialized recevier for the given arguments.
+   */
+  public abstract Receiver getReceiver(int minorFragmentId);
+
+  /**
+   * The widest width this sender can send (max sending parallelization). Typically Integer.MAX_VALUE.
+   * 
+   * @return
+   */
+  @JsonIgnore
+  public abstract int getMaxSendWidth();
+
+  /**
+   * Return the feeding child of this operator node.
+   * 
+   * @return
+   */
+  public PhysicalOperator getChild();
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/ExchangeCost.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/ExchangeCost.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/ExchangeCost.java
new file mode 100644
index 0000000..9e94f06
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/ExchangeCost.java
@@ -0,0 +1,68 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.physical.base;
+
+import org.apache.drill.exec.physical.OperatorCost;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonIgnore;
+import com.fasterxml.jackson.annotation.JsonProperty;
+
+/**
+ * A container class that holds both send and receive costs for an exchange node.
+ */
+public class ExchangeCost {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ExchangeCost.class);
+  
+  private final OperatorCost send;
+  private final OperatorCost receive;
+  private final OperatorCost combined;
+  
+  @JsonCreator
+  public ExchangeCost(@JsonProperty("send") OperatorCost send, @JsonProperty("receive") OperatorCost receive) {
+    this.send = send;
+    this.receive = receive;
+    this.combined =  OperatorCost.combine(send,  receive);
+  }
+
+
+  
+  @JsonIgnore
+  public OperatorCost getCombinedCost(){
+    return combined;
+  }
+
+  @JsonProperty("send")
+  public OperatorCost getSendCost() {
+    return send;
+  }
+
+  @JsonProperty("receive")
+  public OperatorCost getReceiveCost() {
+    return receive;
+  }
+  
+  public static ExchangeCost getSimpleEstimate(Size s){
+    long cnt = s.getRecordCount();
+    int sz = s.getRecordSize();
+    OperatorCost send = new OperatorCost(cnt*sz, 0, 0, cnt);
+    OperatorCost receive = new OperatorCost(cnt*sz, 0, 0, cnt);
+    return new ExchangeCost(send, receive);
+  }
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/FragmentLeaf.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/FragmentLeaf.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/FragmentLeaf.java
new file mode 100644
index 0000000..522ef7b
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/FragmentLeaf.java
@@ -0,0 +1,25 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.physical.base;
+
+/**
+ * A Physical Operator that can be the leaf node of one particular execution fragment. Typically includes Receivers and
+ * Scans
+ */
+public interface FragmentLeaf extends PhysicalOperator {
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/FragmentRoot.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/FragmentRoot.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/FragmentRoot.java
new file mode 100644
index 0000000..66147cc
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/FragmentRoot.java
@@ -0,0 +1,25 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.physical.base;
+
+
+/**
+ * Describes the root operation within a particular Fragment. This includes things Sender nodes. 
+ */
+public interface FragmentRoot extends FragmentLeaf{
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/HasAffinity.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/HasAffinity.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/HasAffinity.java
new file mode 100644
index 0000000..1a9a3a9
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/HasAffinity.java
@@ -0,0 +1,37 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.physical.base;
+
+import java.util.List;
+
+import org.apache.drill.exec.physical.EndpointAffinity;
+
+import com.fasterxml.jackson.annotation.JsonIgnore;
+
+/**
+ * Describes a physical operator that has affinity to particular nodes. Used for assignment decisions.
+ */
+public interface HasAffinity extends PhysicalOperator {
+  
+  /**
+   * Get the list of Endpoints with associated affinities that this operator has preference for.
+   * @return List of EndpointAffinity objects.
+   */
+  @JsonIgnore
+  public List<EndpointAffinity> getOperatorAffinity();
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/Leaf.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/Leaf.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/Leaf.java
new file mode 100644
index 0000000..d4ed456
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/Leaf.java
@@ -0,0 +1,25 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.physical.base;
+
+/**
+ * An operator which specifically is a lowest level leaf node of a query plan across all possible fragments. Currently, the only operator that is a Leaf
+ * node are Scan nodes. Ultimately this could include use of Cache scans and other types of atypical data production systems.
+ */
+public interface Leaf extends FragmentLeaf {
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/PhysicalOperator.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/PhysicalOperator.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/PhysicalOperator.java
new file mode 100644
index 0000000..d412c2d
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/PhysicalOperator.java
@@ -0,0 +1,80 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.physical.base;
+
+import java.util.List;
+
+import org.apache.drill.common.graph.GraphValue;
+import org.apache.drill.exec.physical.OperatorCost;
+
+import com.fasterxml.jackson.annotation.JsonIdentityInfo;
+import com.fasterxml.jackson.annotation.JsonIgnore;
+import com.fasterxml.jackson.annotation.JsonInclude;
+import com.fasterxml.jackson.annotation.JsonInclude.Include;
+import com.fasterxml.jackson.annotation.JsonPropertyOrder;
+import com.fasterxml.jackson.annotation.JsonTypeInfo;
+import com.fasterxml.jackson.annotation.ObjectIdGenerators;
+
+@JsonInclude(Include.NON_NULL)
+@JsonPropertyOrder({ "@id" })
+@JsonIdentityInfo(generator = ObjectIdGenerators.IntSequenceGenerator.class, property = "@id")
+@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, include = JsonTypeInfo.As.PROPERTY, property = "pop")
+public interface PhysicalOperator extends GraphValue<PhysicalOperator> {
+
+  /**
+   * Get the cost of execution of this particular operator.
+   * 
+   * @return
+   */
+  @JsonIgnore
+  public OperatorCost getCost();
+  
+  /**
+   * Get the estimated size of this particular operator.
+   * @return
+   */
+  @JsonIgnore
+  public Size getSize();
+  
+  /**
+   * Describes whether or not a particular physical operator can actually be executed. Most physical operators can be
+   * executed. However, Exchange nodes cannot be executed. In order to be executed, they must be converted into their
+   * Exec sub components.
+   * 
+   * @return
+   */
+  @JsonIgnore
+  public boolean isExecutable();
+
+  /**
+   * Provides capability to build a set of output based on traversing a query graph tree.
+   * 
+   * @param physicalVisitor
+   * @return
+   */
+  public <T, X, E extends Throwable> T accept(PhysicalVisitor<T, X, E> physicalVisitor, X value) throws E;
+
+  /**
+   * Regenerate with this node with a new set of children.  This is used in the case of materialization or optimization.
+   * @param children
+   * @return
+   */
+  @JsonIgnore
+  public PhysicalOperator getNewWithChildren(List<PhysicalOperator> children);
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/PhysicalOperatorUtil.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/PhysicalOperatorUtil.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/PhysicalOperatorUtil.java
new file mode 100644
index 0000000..d6e2fc4
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/PhysicalOperatorUtil.java
@@ -0,0 +1,34 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.physical.base;
+
+import org.apache.drill.common.config.CommonConstants;
+import org.apache.drill.common.config.DrillConfig;
+import org.apache.drill.common.util.PathScanner;
+
+public class PhysicalOperatorUtil {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(PhysicalOperatorUtil.class);
+  
+  private PhysicalOperatorUtil(){}
+  
+  public synchronized static Class<?>[] getSubTypes(DrillConfig config){
+    Class<?>[] ops = PathScanner.scanForImplementationsArr(PhysicalOperator.class, config.getStringList(CommonConstants.PHYSICAL_OPERATOR_SCAN_PACKAGES));
+    logger.debug("Adding Physical Operator sub types: {}", ((Object) ops) );
+    return ops;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/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
new file mode 100644
index 0000000..f36633f
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/PhysicalVisitor.java
@@ -0,0 +1,61 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.physical.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;
+
+/**
+ * Visitor class designed to traversal of a operator tree.  Basis for a number of operator manipulations including fragmentation and materialization.
+ * @param <RETURN> The class associated with the return of each visit method.
+ * @param <EXTRA> The class object associated with additional data required for a particular operator modification.
+ * @param <EXCEP> An optional exception class that can be thrown when a portion of a modification or traversal fails.  Must extend Throwable.  In the case where the visitor does not throw any caught exception, this can be set as RuntimeException.
+ */
+public interface PhysicalVisitor<RETURN, EXTRA, EXCEP extends Throwable> {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(PhysicalVisitor.class);
+  
+  
+  public RETURN visitExchange(Exchange exchange, EXTRA value) throws EXCEP;
+  public RETURN visitScan(Scan<?> scan, EXTRA value) throws EXCEP;
+  public RETURN visitStore(Store store, EXTRA value) throws EXCEP;
+
+  public RETURN visitFilter(Filter filter, 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;
+  public RETURN visitReceiver(Receiver receiver, EXTRA value) throws EXCEP;
+  
+  public RETURN visitOp(PhysicalOperator op, EXTRA value) throws EXCEP;
+  
+  
+  public RETURN visitHashPartitionSender(HashPartitionSender op, EXTRA value) throws EXCEP;
+  public RETURN visitRandomReceiver(RandomReceiver op, EXTRA value) throws EXCEP;
+  public RETURN visitHashPartitionSender(HashToRandomExchange op, EXTRA value) throws EXCEP;
+  public RETURN visitRangeSender(RangeSender op, EXTRA value) throws EXCEP;
+  public RETURN visitScreen(Screen op, EXTRA value) throws EXCEP;
+  public RETURN visitSingleSender(SingleSender op, EXTRA value) throws EXCEP;
+  public RETURN visitUnionExchange(UnionExchange op, EXTRA value) throws EXCEP;
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/Receiver.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/Receiver.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/Receiver.java
new file mode 100644
index 0000000..cd4cb4a
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/Receiver.java
@@ -0,0 +1,51 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.physical.base;
+
+import java.util.List;
+
+import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
+
+import com.fasterxml.jackson.annotation.JsonIgnore;
+import com.fasterxml.jackson.annotation.JsonProperty;
+
+/**
+ * A receiver is one half of an exchange operator. The receiver is responsible for taking in one or more streams from
+ * corresponding Senders.  Receivers are a special type of Physical Operator that are typically only expressed within the execution plan.
+ */
+public interface Receiver extends FragmentLeaf {
+  
+  /**
+   * A receiver is expecting streams from one or more providing endpoints.  This method should return a list of the expected sending endpoints.
+   * @return List of counterpart sending DrillbitEndpoints.
+   */
+  public abstract List<DrillbitEndpoint> getProvidingEndpoints();
+
+  /**
+   * Whether or not this receive supports out of order exchange. This provides a hint for the scheduling node on whether
+   * the receiver can start work if only a subset of all sending endpoints are currently providing data. A random
+   * receiver would supports this form of operation. A NWAY receiver would not.
+   * 
+   * @return True if this receiver supports working on a streaming/out of order input.
+   */
+  @JsonIgnore
+  public abstract boolean supportsOutOfOrderExchange();
+
+  @JsonProperty("sender-major-fragment")
+  public int getOppositeMajorFragmentId();
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/Root.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/Root.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/Root.java
new file mode 100644
index 0000000..7adef63
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/Root.java
@@ -0,0 +1,24 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.physical.base;
+
+/**
+ * Marker interface describe the root of a query plan.  Currently, this is constrained to Screen.
+ */
+public interface Root extends FragmentRoot{
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/Scan.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/Scan.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/Scan.java
new file mode 100644
index 0000000..2207f79
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/Scan.java
@@ -0,0 +1,36 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.physical.base;
+
+import java.util.List;
+
+import org.apache.drill.exec.physical.ReadEntry;
+import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
+
+import com.fasterxml.jackson.annotation.JsonProperty;
+
+public interface Scan<R extends ReadEntry> extends Leaf, HasAffinity{
+
+  @JsonProperty("entries")
+  public abstract List<R> getReadEntries();
+
+  public abstract void applyAssignments(List<DrillbitEndpoint> endpoints);
+
+  public abstract Scan<?> getSpecificScan(int minorFragmentId);
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/Sender.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/Sender.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/Sender.java
new file mode 100644
index 0000000..71513c7
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/Sender.java
@@ -0,0 +1,44 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.physical.base;
+
+import java.util.List;
+
+import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
+
+import com.fasterxml.jackson.annotation.JsonProperty;
+
+/**
+ * A sender is one half of an exchange node operations. It is responsible for subdividing/cloning and sending a local
+ * record set to a set of destination locations. This is typically only utilized at the level of the execution plan.
+ */
+public interface Sender extends FragmentRoot {
+  
+  /**
+   * Get the list of destination endpoints that this Sender will be communicating with.
+   * @return List of DrillbitEndpoints.
+   */
+  public abstract List<DrillbitEndpoint> getDestinations();
+  
+  /**
+   * Get the receiver major fragment id that is opposite this sender.
+   * @return
+   */
+  @JsonProperty("receiver-major-fragment")
+  public int getOppositeMajorFragmentId();
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/Size.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/Size.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/Size.java
new file mode 100644
index 0000000..7bc6f93
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/Size.java
@@ -0,0 +1,48 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.physical.base;
+
+public class Size {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(Size.class);
+
+  private final long rowCount;
+  private final int rowSize;
+
+  public Size(long rowCount, int rowSize) {
+    super();
+    this.rowCount = rowCount;
+    this.rowSize = rowSize;
+  }
+
+  public long getRecordCount() {
+    return rowCount;
+  }
+
+  public int getRecordSize() {
+    return rowSize;
+  }
+  
+  public Size add(Size s){
+    return new Size(rowCount + s.rowCount, Math.max(rowSize, s.rowSize));
+  }
+  
+  public long getAggSize(){
+    return rowCount * rowSize;
+  }
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/Store.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/Store.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/Store.java
new file mode 100644
index 0000000..9b3a812
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/Store.java
@@ -0,0 +1,74 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.physical.base;
+
+import java.util.List;
+
+import org.apache.drill.common.exceptions.PhysicalOperatorSetupException;
+import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
+
+import com.fasterxml.jackson.annotation.JsonIgnore;
+
+/**
+ * An interface which supports storing a record stream. In contrast to the logical layer, in the physical/execution
+ * layers, a Store node is actually an outputting node (rather than a root node) that provides returns one or more
+ * records regarding the completion of the query.
+ */
+public interface Store extends HasAffinity {
+
+  /**
+   * Inform the Store node about the actual decided DrillbitEndpoint assignments desired for storage purposes. This is a
+   * precursor to the execution planner running a set of getSpecificStore() method calls for full Store node
+   * materialization.
+   * 
+   * @param endpoints
+   *          The list of endpoints that this Store node are going to be executed on.
+   * @throws PhysicalOperatorSetupException
+   */
+  public abstract void applyAssignments(List<DrillbitEndpoint> endpoints) throws PhysicalOperatorSetupException;
+
+  /**
+   * Provides full materialized Store operators for execution purposes.
+   * 
+   * @param child
+   *          The child operator that this operator will consume from.
+   * @param minorFragmentId
+   *          The particular minor fragment id associated with this particular fragment materialization.
+   * @return A materialized Store Operator.
+   * @throws PhysicalOperatorSetupException
+   */
+  public abstract Store getSpecificStore(PhysicalOperator child, int minorFragmentId)
+      throws PhysicalOperatorSetupException;
+
+  /**
+   * The maximum allowable width for the Store operation. In some cases, a store operation has a limited number of
+   * parallelizations that it can support. For example, a Screen return cannot be parallelized at all. In this case, a
+   * maxWidth value of 1 will be returned. In the case that there is no limit for parallelization, this method should
+   * return Integer.MAX_VALUE.
+   * 
+   * @return
+   */
+  @JsonIgnore
+  public abstract int getMaxWidth();
+
+  /**
+   * Get the child of this store operator as this will be needed for parallelization materialization purposes.
+   * @return
+   */
+  public abstract PhysicalOperator getChild();
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/Filter.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/Filter.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/Filter.java
new file mode 100644
index 0000000..4af4243
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/Filter.java
@@ -0,0 +1,75 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.physical.config;
+
+import org.apache.drill.common.expression.LogicalExpression;
+import org.apache.drill.exec.physical.OperatorCost;
+import org.apache.drill.exec.physical.base.AbstractSingle;
+import org.apache.drill.exec.physical.base.PhysicalOperator;
+import org.apache.drill.exec.physical.base.PhysicalVisitor;
+import org.apache.drill.exec.physical.base.Size;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.annotation.JsonTypeName;
+
+@JsonTypeName("filter")
+public class Filter extends AbstractSingle {
+
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(Filter.class);
+
+  private final LogicalExpression expr;
+  private final float selectivity;
+  
+  @JsonCreator
+  public Filter(@JsonProperty("child") PhysicalOperator child, @JsonProperty("expr") LogicalExpression expr, @JsonProperty("selectivity") float selectivity) {
+    super(child);
+    this.expr = expr;
+    this.selectivity = selectivity;
+  }
+
+  public LogicalExpression getExpr() {
+    return expr;
+  }
+
+  @Override
+  public <T, X, E extends Throwable> T accept(PhysicalVisitor<T, X, E> physicalVisitor, X value) throws E{
+    return physicalVisitor.visitFilter(this, value);
+  }
+
+  @Override
+  public OperatorCost getCost() {
+    return child.getCost();
+  }
+
+  @Override
+  protected PhysicalOperator getNewWithChild(PhysicalOperator child) {
+    return new Filter(child, expr, selectivity);
+  }
+
+  @Override
+  public Size getSize() {
+    return new Size( (long) (child.getSize().getRecordCount()*selectivity), child.getSize().getRecordSize());
+  }
+  
+  
+
+  
+  
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/HashPartitionSender.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/HashPartitionSender.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/HashPartitionSender.java
new file mode 100644
index 0000000..84994f6
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/HashPartitionSender.java
@@ -0,0 +1,58 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.physical.config;
+
+import java.util.List;
+
+import org.apache.drill.common.expression.LogicalExpression;
+import org.apache.drill.exec.physical.base.AbstractSender;
+import org.apache.drill.exec.physical.base.PhysicalOperator;
+import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.annotation.JsonTypeName;
+import com.google.common.collect.Lists;
+
+@JsonTypeName("hash-partition-sender")
+public class HashPartitionSender extends AbstractSender {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(HashPartitionSender.class);
+
+  private final List<DrillbitEndpoint> endpoints;
+  private final LogicalExpression expr;
+  
+  @JsonCreator
+  public HashPartitionSender(@JsonProperty("receiver-major-fragment") int oppositeMajorFragmentId, @JsonProperty("child") PhysicalOperator child, @JsonProperty("expr") LogicalExpression expr, @JsonProperty("destinations") List<DrillbitEndpoint> endpoints) {
+    super(oppositeMajorFragmentId, child);
+    this.expr = expr;
+    this.endpoints = endpoints;
+  }
+
+  @Override
+  public List<DrillbitEndpoint> getDestinations() {
+    return endpoints;
+  }
+
+  @Override
+  protected PhysicalOperator getNewWithChild(PhysicalOperator child) {
+    return new HashPartitionSender(oppositeMajorFragmentId, child, expr, endpoints);
+  }
+
+  
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/HashToRandomExchange.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/HashToRandomExchange.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/HashToRandomExchange.java
new file mode 100644
index 0000000..1f158ce
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/HashToRandomExchange.java
@@ -0,0 +1,86 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.physical.config;
+
+import java.beans.Transient;
+import java.util.List;
+
+import org.apache.drill.common.expression.LogicalExpression;
+import org.apache.drill.exec.physical.base.AbstractExchange;
+import org.apache.drill.exec.physical.base.PhysicalOperator;
+import org.apache.drill.exec.physical.base.Receiver;
+import org.apache.drill.exec.physical.base.Sender;
+import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.annotation.JsonTypeName;
+
+@JsonTypeName("hash-to-random-exchange")
+public class HashToRandomExchange extends AbstractExchange{
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(HashToRandomExchange.class);
+
+  
+  private final LogicalExpression expr;
+
+  //ephemeral for setup tasks.
+  private List<DrillbitEndpoint> senderLocations;
+  private List<DrillbitEndpoint> receiverLocations;
+  
+  @JsonCreator
+  public HashToRandomExchange(@JsonProperty("child") PhysicalOperator child, @JsonProperty("expr") LogicalExpression expr) {
+    super(child);
+    this.expr = expr;
+  }
+
+  @Override
+  public int getMaxSendWidth() {
+    return Integer.MAX_VALUE;
+  }
+
+
+  @Override
+  protected void setupSenders(List<DrillbitEndpoint> senderLocations) {
+    this.senderLocations = senderLocations;
+  }
+
+  @Override
+  protected void setupReceivers(List<DrillbitEndpoint> receiverLocations) {
+    this.receiverLocations = receiverLocations;
+  }
+
+  @Override
+  public Sender getSender(int minorFragmentId, PhysicalOperator child) {
+    return new HashPartitionSender(receiverMajorFragmentId, child, expr, receiverLocations);
+  }
+
+  @Override
+  public Receiver getReceiver(int minorFragmentId) {
+    return new RandomReceiver(senderMajorFragmentId, senderLocations);
+  }
+
+  @Override
+  protected PhysicalOperator getNewWithChild(PhysicalOperator child) {
+    return new HashToRandomExchange(child, expr);
+  }
+
+  
+
+  
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockRecordReader.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockRecordReader.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockRecordReader.java
new file mode 100644
index 0000000..eaaeaa3
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockRecordReader.java
@@ -0,0 +1,113 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.physical.config;
+
+import org.apache.drill.common.exceptions.ExecutionSetupException;
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.exec.exception.SchemaChangeException;
+import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.physical.config.MockScanPOP.MockColumn;
+import org.apache.drill.exec.physical.config.MockScanPOP.MockScanEntry;
+import org.apache.drill.exec.physical.impl.OutputMutator;
+import org.apache.drill.exec.proto.SchemaDefProtos.DataMode;
+import org.apache.drill.exec.proto.SchemaDefProtos.MajorType;
+import org.apache.drill.exec.record.MaterializedField;
+import org.apache.drill.exec.record.vector.TypeHelper;
+import org.apache.drill.exec.record.vector.ValueVector;
+import org.apache.drill.exec.store.RecordReader;
+
+public class MockRecordReader implements RecordReader {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(MockRecordReader.class);
+
+  private OutputMutator output;
+  private MockScanEntry config;
+  private FragmentContext context;
+  private ValueVector<?>[] valueVectors;
+  private int recordsRead;
+
+  public MockRecordReader(FragmentContext context, MockScanEntry config) {
+    this.context = context;
+    this.config = config;
+  }
+
+  private int getEstimatedRecordSize(MockColumn[] types) {
+    int x = 0;
+    for (int i = 0; i < types.length; i++) {
+      x += TypeHelper.getSize(types[i].getMajorType());
+    }
+    return x;
+  }
+
+  private ValueVector<?> getVector(int fieldId, String name, MajorType type, int length) {
+    assert context != null : "Context shouldn't be null.";
+    
+    if(type.getMode() != DataMode.REQUIRED) throw new UnsupportedOperationException();
+    
+    MaterializedField f = MaterializedField.create(new SchemaPath(name), fieldId, 0, type);
+    ValueVector<?> v;
+    v = TypeHelper.getNewVector(f, context.getAllocator());
+    v.allocateNew(length);
+    
+    return v;
+
+  }
+
+  @Override
+  public void setup(OutputMutator output) throws ExecutionSetupException {
+    try {
+      this.output = output;
+      int estimateRowSize = getEstimatedRecordSize(config.getTypes());
+      valueVectors = new ValueVector<?>[config.getTypes().length];
+      int batchRecordCount = 250000 / estimateRowSize;
+
+      for (int i = 0; i < config.getTypes().length; i++) {
+        logger.debug("Adding field {} of type {}", i, config.getTypes()[i]);
+        valueVectors[i] = getVector(i, config.getTypes()[i].getName(), config.getTypes()[i].getMajorType(), batchRecordCount);
+        output.addField(i, valueVectors[i]);
+      }
+      output.setNewSchema();
+    } catch (SchemaChangeException e) {
+      throw new ExecutionSetupException("Failure while setting up fields", e);
+    }
+
+  }
+
+  @Override
+  public int next() {
+    int recordSetSize = Math.min(valueVectors[0].capacity(), this.config.getRecords()- recordsRead);
+    recordsRead += recordSetSize;
+    for(ValueVector<?> v : valueVectors){
+      v.randomizeData();
+      v.setRecordCount(recordSetSize);
+    }
+    return recordSetSize;
+  }
+
+  @Override
+  public void cleanup() {
+    for (int i = 0; i < valueVectors.length; i++) {
+      try {
+        output.removeField(valueVectors[i].getField().getFieldId());
+      } catch (SchemaChangeException e) {
+        logger.warn("Failure while trying tremove field.", e);
+      }
+      valueVectors[i].close();
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockScanBatchCreator.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockScanBatchCreator.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockScanBatchCreator.java
new file mode 100644
index 0000000..b821d6e
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockScanBatchCreator.java
@@ -0,0 +1,49 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.physical.config;
+
+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.MockScanPOP.MockScanEntry;
+import org.apache.drill.exec.physical.impl.BatchCreator;
+import org.apache.drill.exec.physical.impl.ScanBatch;
+import org.apache.drill.exec.record.BatchSchema;
+import org.apache.drill.exec.record.InvalidValueAccessor;
+import org.apache.drill.exec.record.RecordBatch;
+import org.apache.drill.exec.record.vector.ValueVector;
+import org.apache.drill.exec.store.RecordReader;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+
+public class MockScanBatchCreator implements BatchCreator<MockScanPOP>{
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(MockScanBatchCreator.class);
+
+  @Override
+  public RecordBatch getBatch(FragmentContext context, MockScanPOP config, List<RecordBatch> children) throws ExecutionSetupException {
+    Preconditions.checkArgument(children.isEmpty());
+    List<MockScanEntry> entries = config.getReadEntries();
+    List<RecordReader> readers = Lists.newArrayList();
+    for(MockScanEntry e : entries){
+      readers.add(new MockRecordReader(context, e));
+    }
+    return new ScanBatch(context, readers.iterator());
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockScanPOP.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockScanPOP.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockScanPOP.java
new file mode 100644
index 0000000..4a3a606
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockScanPOP.java
@@ -0,0 +1,193 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.physical.config;
+
+import java.util.Collections;
+import java.util.LinkedList;
+import java.util.List;
+
+import org.apache.drill.exec.physical.EndpointAffinity;
+import org.apache.drill.exec.physical.OperatorCost;
+import org.apache.drill.exec.physical.ReadEntry;
+import org.apache.drill.exec.physical.base.AbstractScan;
+import org.apache.drill.exec.physical.base.PhysicalOperator;
+import org.apache.drill.exec.physical.base.Scan;
+import org.apache.drill.exec.physical.base.Size;
+import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
+import org.apache.drill.exec.proto.SchemaDefProtos.DataMode;
+import org.apache.drill.exec.proto.SchemaDefProtos.MajorType;
+import org.apache.drill.exec.proto.SchemaDefProtos.MinorType;
+import org.apache.drill.exec.record.vector.TypeHelper;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonIgnore;
+import com.fasterxml.jackson.annotation.JsonInclude;
+import com.fasterxml.jackson.annotation.JsonInclude.Include;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.annotation.JsonTypeName;
+import com.google.common.base.Preconditions;
+
+@JsonTypeName("mock-scan")
+public class MockScanPOP extends AbstractScan<MockScanPOP.MockScanEntry> {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(MockScanPOP.class);
+
+  private final String url;
+  private  LinkedList<MockScanEntry>[] mappings;
+
+  @JsonCreator
+  public MockScanPOP(@JsonProperty("url") String url, @JsonProperty("entries") List<MockScanEntry> readEntries) {
+    super(readEntries);
+    this.url = url;
+  }
+
+  public String getUrl() {
+    return url;
+  }
+
+  
+  public static class MockScanEntry implements ReadEntry {
+
+    private final int records;
+    private final MockColumn[] types;
+    private final int recordSize;
+    
+
+    @JsonCreator
+    public MockScanEntry(@JsonProperty("records") int records, @JsonProperty("types") MockColumn[] types) {
+      this.records = records;
+      this.types = types;
+      int size = 0;
+      for(MockColumn dt : types){
+        size += TypeHelper.getSize(dt.getMajorType());
+      }
+      this.recordSize = size;
+    }
+
+    @Override
+    public OperatorCost getCost() {
+      return new OperatorCost(1, 2, 1, 1);
+    }
+
+    
+    public int getRecords() {
+      return records;
+    }
+
+    public MockColumn[] getTypes() {
+      return types;
+    }
+
+    @Override
+    public Size getSize() {
+      return new Size(records, recordSize);
+    }
+  }
+  
+  @JsonInclude(Include.NON_NULL)
+  public static class MockColumn{
+    @JsonProperty("type") public MinorType minorType;
+    public String name;
+    public DataMode mode;
+    public Integer width;
+    public Integer precision;
+    public Integer scale;
+    
+    
+    @JsonCreator
+    public MockColumn(@JsonProperty("name") String name, @JsonProperty("type") MinorType minorType, @JsonProperty("mode") DataMode mode, @JsonProperty("width") Integer width, @JsonProperty("precision") Integer precision, @JsonProperty("scale") Integer scale) {
+      this.name = name;
+      this.minorType = minorType;
+      this.mode = mode;
+      this.width = width;
+      this.precision = precision;
+      this.scale = scale;
+    }
+    
+    @JsonProperty("type")
+    public MinorType getMinorType() {
+      return minorType;
+    }
+    public String getName() {
+      return name;
+    }
+    public DataMode getMode() {
+      return mode;
+    }
+    public Integer getWidth() {
+      return width;
+    }
+    public Integer getPrecision() {
+      return precision;
+    }
+    public Integer getScale() {
+      return scale;
+    }
+    
+    @JsonIgnore
+    public MajorType getMajorType(){
+      MajorType.Builder b = MajorType.newBuilder();
+      b.setMode(mode);
+      b.setMinorType(minorType);
+      if(precision != null) b.setPrecision(precision);
+      if(width != null) b.setWidth(width);
+      if(scale != null) b.setScale(scale);
+      return b.build();
+    }
+    
+  }
+
+  @Override
+  public List<EndpointAffinity> getOperatorAffinity() {
+    return Collections.emptyList();
+  }
+
+  @SuppressWarnings("unchecked")
+  @Override
+  public void applyAssignments(List<DrillbitEndpoint> endpoints) {
+    Preconditions.checkArgument(endpoints.size() <= getReadEntries().size());
+    
+    mappings = new LinkedList[endpoints.size()];
+
+    int i =0;
+    for(MockScanEntry e : this.getReadEntries()){
+      if(i == endpoints.size()) i -= endpoints.size();
+      LinkedList<MockScanEntry> entries = mappings[i];
+      if(entries == null){
+        entries = new LinkedList<MockScanEntry>();
+        mappings[i] = entries;
+      }
+      entries.add(e);
+      i++;
+    }
+  }
+
+  @Override
+  public Scan<?> 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);
+    return new MockScanPOP(url, mappings[minorFragmentId]);
+  }
+
+  @Override
+  @JsonIgnore
+  public PhysicalOperator getNewWithChildren(List<PhysicalOperator> children) {
+    Preconditions.checkArgument(children.isEmpty());
+    return new MockScanPOP(url, readEntries);
+
+  }
+
+}


[21/53] [abbrv] Clean up threading of client/server. Utilize command pattern for BitCom stuff to abstract away connection failures. Works on one bit single exchange remote query now. Next up, two bit single exchange query.

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/b8db98ad/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/BitClient.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/BitClient.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/BitClient.java
index 4ba99a1..82a6aa6 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/BitClient.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/BitClient.java
@@ -22,57 +22,54 @@ import io.netty.channel.Channel;
 import io.netty.channel.ChannelFuture;
 import io.netty.util.concurrent.GenericFutureListener;
 
-import java.util.concurrent.ConcurrentMap;
-
 import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
 import org.apache.drill.exec.proto.ExecProtos.BitHandshake;
 import org.apache.drill.exec.proto.ExecProtos.RpcType;
 import org.apache.drill.exec.rpc.BasicClient;
 import org.apache.drill.exec.rpc.Response;
 import org.apache.drill.exec.rpc.RpcBus;
+import org.apache.drill.exec.rpc.RpcConnectionHandler;
 import org.apache.drill.exec.rpc.RpcException;
+import org.apache.drill.exec.rpc.bit.BitConnectionManager.CloseHandlerCreator;
 import org.apache.drill.exec.server.BootStrapContext;
 import org.apache.drill.exec.work.batch.BitComHandler;
 
 import com.google.protobuf.MessageLite;
 
-public class BitClient  extends BasicClient<RpcType, BitConnection>{
+public class BitClient  extends BasicClient<RpcType, BitConnection, BitHandshake, BitHandshake>{
 
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(BitClient.class);
 
   private final BitComHandler handler;
-  private final DrillbitEndpoint endpoint;
-  private BitConnection connection;
-  private final AvailabilityListener openListener;
-  private final ConcurrentMap<DrillbitEndpoint, BitConnection> registry;
+  private final DrillbitEndpoint remoteEndpoint;
+  private volatile BitConnection connection;
   private final ListenerPool listeners;
+  private final CloseHandlerCreator closeHandlerFactory;
+  private final DrillbitEndpoint localIdentity;
   
-  public BitClient(DrillbitEndpoint endpoint, AvailabilityListener openListener, BitComHandler handler, BootStrapContext context, ConcurrentMap<DrillbitEndpoint, BitConnection> registry, ListenerPool listeners) {
-    super(BitRpcConfig.MAPPING, context.getAllocator().getUnderlyingAllocator(), context.getBitLoopGroup());
-
-    this.endpoint = endpoint;
+  public BitClient(DrillbitEndpoint remoteEndpoint, DrillbitEndpoint localEndpoint, BitComHandler handler, BootStrapContext context, CloseHandlerCreator closeHandlerFactory, ListenerPool listeners) {
+    super(BitRpcConfig.MAPPING, context.getAllocator().getUnderlyingAllocator(), context.getBitLoopGroup(), RpcType.HANDSHAKE, BitHandshake.class, BitHandshake.PARSER);
+    this.localIdentity = localEndpoint;
+    this.remoteEndpoint = remoteEndpoint;
     this.handler = handler;
-    this.openListener = openListener;
-    this.registry = registry;
     this.listeners = listeners;
+    this.closeHandlerFactory = closeHandlerFactory;
   }
   
-  public BitHandshake connect() throws RpcException, InterruptedException{
-    BitHandshake bs = connectAsClient(RpcType.HANDSHAKE, BitHandshake.newBuilder().setRpcVersion(BitRpcConfig.RPC_VERSION).build(), endpoint.getAddress(), endpoint.getBitPort(), BitHandshake.class);
-    connection.setEndpoint(endpoint);
-    return bs;
+  public void connect(RpcConnectionHandler<BitConnection> connectionHandler) {
+    connectAsClient(connectionHandler, BitHandshake.newBuilder().setRpcVersion(BitRpcConfig.RPC_VERSION).setEndpoint(localIdentity).build(), remoteEndpoint.getAddress(), remoteEndpoint.getBitPort());
   }
 
   @SuppressWarnings("unchecked")
   @Override
   public BitConnection initRemoteConnection(Channel channel) {
-    this.connection = new BitConnection(openListener, channel, (RpcBus<RpcType, BitConnection>) (RpcBus<?, ?>) this, registry, listeners);
+    this.connection = new BitConnection(channel, (RpcBus<RpcType, BitConnection>) (RpcBus<?, ?>) this, listeners);
     return connection;
   }
 
   @Override
   protected GenericFutureListener<ChannelFuture> getCloseHandler(BitConnection clientConnection) {
-    return clientConnection.getCloseHandler(super.getCloseHandler(clientConnection));
+    return closeHandlerFactory.getHandler(clientConnection, super.getCloseHandler(clientConnection));
   }
 
   @Override
@@ -86,18 +83,15 @@ public class BitClient  extends BasicClient<RpcType, BitConnection>{
   }
 
   @Override
-  protected ClientHandshakeHandler<BitHandshake> getHandshakeHandler() {
-    return new ClientHandshakeHandler<BitHandshake>(RpcType.HANDSHAKE, BitHandshake.class, BitHandshake.PARSER){
-
-      @Override
-      protected void validateHandshake(BitHandshake inbound) throws Exception {
-        logger.debug("Handling handshake from bit server to bit client. {}", inbound);
-        if(inbound.getRpcVersion() != BitRpcConfig.RPC_VERSION) throw new RpcException(String.format("Invalid rpc version.  Expected %d, actual %d.", inbound.getRpcVersion(), BitRpcConfig.RPC_VERSION));
-      }
+  protected void validateHandshake(BitHandshake handshake) throws RpcException {
+    if(handshake.getRpcVersion() != BitRpcConfig.RPC_VERSION) throw new RpcException(String.format("Invalid rpc version.  Expected %d, actual %d.", handshake.getRpcVersion(), BitRpcConfig.RPC_VERSION));
+  }
 
-    };
+  @Override
+  protected void finalizeConnection(BitHandshake handshake, BitConnection connection) {
+    connection.setEndpoint(handshake.getEndpoint());
   }
-  
+
   public BitConnection getConnection(){
     return this.connection;
   }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/b8db98ad/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/BitCom.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/BitCom.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/BitCom.java
index c60d36b..f7f508e 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/BitCom.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/BitCom.java
@@ -40,11 +40,17 @@ public interface BitCom extends Closeable {
    */
   public BitTunnel getTunnel(DrillbitEndpoint node) ;
 
-  public int start() throws InterruptedException, DrillbitStartupException;
+  public DrillbitEndpoint start(DrillbitEndpoint partialEndpoint) throws InterruptedException, DrillbitStartupException;
 
   /**
    * Register an incoming batch handler for a local foreman.  
    * @param handler
    */
   public void registerIncomingBatchHandler(IncomingFragmentHandler handler);
+  
+  /**
+   * Get ListenerPool
+   * @return
+   */
+  public ListenerPool getListeners();
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/b8db98ad/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/BitComImpl.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/BitComImpl.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/BitComImpl.java
index c98be44..d1cadc7 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/BitComImpl.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/BitComImpl.java
@@ -18,157 +18,68 @@
 package org.apache.drill.exec.rpc.bit;
 
 import java.util.concurrent.ConcurrentMap;
-import java.util.concurrent.ExecutionException;
-import java.util.concurrent.Executor;
-import java.util.concurrent.Executors;
 
 import org.apache.drill.exec.ExecConstants;
 import org.apache.drill.exec.exception.DrillbitStartupException;
-import org.apache.drill.exec.exception.FragmentSetupException;
 import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
-import org.apache.drill.exec.proto.ExecProtos.BitHandshake;
-import org.apache.drill.exec.proto.ExecProtos.RpcType;
-import org.apache.drill.exec.rpc.NamedThreadFactory;
-import org.apache.drill.exec.rpc.RpcException;
 import org.apache.drill.exec.server.BootStrapContext;
 import org.apache.drill.exec.work.batch.BitComHandler;
 import org.apache.drill.exec.work.fragment.IncomingFragmentHandler;
 
 import com.google.common.collect.Maps;
 import com.google.common.io.Closeables;
-import com.google.common.util.concurrent.AbstractCheckedFuture;
-import com.google.common.util.concurrent.CheckedFuture;
-import com.google.common.util.concurrent.ListenableFuture;
-import com.google.common.util.concurrent.SettableFuture;
 
 /**
- * Manages communication tunnels between nodes.   
+ * Manages communication tunnels between nodes.
  */
 public class BitComImpl implements BitCom {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(BitComImpl.class);
 
-  private final ConcurrentMap<DrillbitEndpoint, BitConnection> registry = Maps.newConcurrentMap();
   private final ListenerPool listeners;
   private volatile BitServer server;
   private final BitComHandler handler;
   private final BootStrapContext context;
-  
-  // TODO: this executor should be removed.
-  private final Executor exec = Executors.newCachedThreadPool(new NamedThreadFactory("BitComImpl execution pool: "));
+  private final ConnectionManagerRegistry connectionRegistry;
 
   public BitComImpl(BootStrapContext context, BitComHandler handler) {
     super();
     this.handler = handler;
     this.context = context;
     this.listeners = new ListenerPool(8);
+    this.connectionRegistry = new ConnectionManagerRegistry(handler, context, listeners);
   }
 
-  public int start() throws InterruptedException, DrillbitStartupException {
-    server = new BitServer(handler, context, registry, listeners);
+  @Override
+  public DrillbitEndpoint start(DrillbitEndpoint partialEndpoint) throws InterruptedException, DrillbitStartupException {
+    server = new BitServer(handler, context, connectionRegistry, listeners);
     int port = context.getConfig().getInt(ExecConstants.INITIAL_BIT_PORT);
-    return server.bind(port);
-  }
-
-  private CheckedFuture<BitConnection, RpcException> getNode(final DrillbitEndpoint endpoint, boolean check) {
-    
-    
-    SettableFuture<BitConnection> future = SettableFuture.create();
-    BitComFuture<BitConnection> checkedFuture = new BitComFuture<BitConnection>(future);
-    BitConnection t = null;
-
-    if (check) {
-      t = registry.get(endpoint);
-
-      if (t != null) {
-        future.set(t);
-        return checkedFuture;
-      }
-    }
-    
-    try {
-      AvailWatcher watcher = new AvailWatcher(future);
-      BitClient c = new BitClient(endpoint, watcher, handler, context, registry, listeners);
-      c.connect();
-      return checkedFuture;
-    } catch (InterruptedException | RpcException e) {
-      future.setException(new FragmentSetupException("Unable to open connection"));
-      return checkedFuture;
-    }
-
+    port = server.bind(port);
+    DrillbitEndpoint completeEndpoint = partialEndpoint.toBuilder().setBitPort(port).build();
+    connectionRegistry.setEndpoint(completeEndpoint);
+    return completeEndpoint;
   }
 
-  private class AvailWatcher implements AvailabilityListener{
-    final SettableFuture<BitConnection> future;
-    
-    public AvailWatcher(SettableFuture<BitConnection> future) {
-      super();
-      this.future = future;
-    }
-
-    @Override
-    public void isAvailable(BitConnection connection) {
-      future.set(connection);
-    }
-    
-  }
   
-  BitConnection getConnection(DrillbitEndpoint endpoint) throws RpcException {
-    BitConnection t = registry.get(endpoint);
-    if(t != null) return t;
-    return this.getNode(endpoint, false).checkedGet();
+   
+  public ListenerPool getListeners() {
+    return listeners;
   }
 
-  
-  CheckedFuture<BitConnection, RpcException> getConnectionAsync(DrillbitEndpoint endpoint) {
-    return this.getNode(endpoint, true);
-  }
-
-  
   @Override
-  public BitTunnel getTunnel(DrillbitEndpoint endpoint){
-    BitConnection t = registry.get(endpoint);
-    if(t == null){
-      return new BitTunnel(exec, endpoint, this, t);
-    }else{
-      return new BitTunnel(exec, endpoint, this,  this.getNode(endpoint, false));
-    }
+  public BitTunnel getTunnel(DrillbitEndpoint endpoint) {
+    return new BitTunnel(endpoint, connectionRegistry.getConnectionManager(endpoint));
   }
 
-
-  /**
-   * A future which remaps exceptions to a BitComException.
-   * @param <T>
-   */
-  private class BitComFuture<T> extends AbstractCheckedFuture<T, RpcException>{
-
-    protected BitComFuture(ListenableFuture<T> delegate) {
-      super(delegate);
-    }
-
-    @Override
-    protected RpcException mapException(Exception e) {
-      Throwable t = e;
-      if(e instanceof ExecutionException){
-        t = e.getCause();
-      }
-      
-      if(t instanceof RpcException) return (RpcException) t;
-      return new RpcException(t);
-    }
+  @Override
+  public void registerIncomingBatchHandler(IncomingFragmentHandler handler) {
+    this.handler.registerIncomingFragmentHandler(handler);
   }
 
   public void close() {
     Closeables.closeQuietly(server);
-    for (BitConnection bt : registry.values()) {
-      bt.shutdownIfClient();
+    for (BitConnectionManager bt : connectionRegistry) {
+      bt.close();
     }
   }
 
-  @Override
-  public void registerIncomingBatchHandler(IncomingFragmentHandler handler) {
-    this.handler.registerIncomingFragmentHandler(handler);
-  }
-  
-  
-
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/b8db98ad/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/BitCommand.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/BitCommand.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/BitCommand.java
new file mode 100644
index 0000000..692c63e
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/BitCommand.java
@@ -0,0 +1,28 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.rpc.bit;
+
+import org.apache.drill.exec.rpc.RpcConnectionHandler;
+
+import com.google.protobuf.MessageLite;
+
+public interface BitCommand<T extends MessageLite> extends RpcConnectionHandler<BitConnection>{
+
+  public abstract void connectionAvailable(BitConnection connection);
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/b8db98ad/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/BitConnection.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/BitConnection.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/BitConnection.java
index 73980f9..f85ea74 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/BitConnection.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/BitConnection.java
@@ -17,6 +17,7 @@
  ******************************************************************************/
 package org.apache.drill.exec.rpc.bit;
 
+import io.netty.buffer.ByteBuf;
 import io.netty.channel.Channel;
 import io.netty.channel.ChannelFuture;
 import io.netty.util.concurrent.GenericFutureListener;
@@ -35,31 +36,35 @@ import org.apache.drill.exec.record.FragmentWritableBatch;
 import org.apache.drill.exec.rpc.DrillRpcFuture;
 import org.apache.drill.exec.rpc.RemoteConnection;
 import org.apache.drill.exec.rpc.RpcBus;
+import org.apache.drill.exec.rpc.RpcConnectionHandler;
+import org.apache.drill.exec.rpc.RpcOutcomeListener;
 
 import com.google.common.base.Preconditions;
 import com.google.common.io.Closeables;
+import com.google.protobuf.MessageLite;
 
 public class BitConnection extends RemoteConnection{
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(BitConnection.class); 
   
   private final RpcBus<RpcType, BitConnection> bus;
-  private final ConcurrentMap<DrillbitEndpoint, BitConnection> registry;
   private final ListenerPool listeners;
-
-  private final AvailabilityListener listener;
   private volatile DrillbitEndpoint endpoint;
   private volatile boolean active = false;
   private final UUID id;
   
-  public BitConnection(AvailabilityListener listener, Channel channel, RpcBus<RpcType, BitConnection> bus, ConcurrentMap<DrillbitEndpoint, BitConnection> registry, ListenerPool listeners){
+  public BitConnection(Channel channel, RpcBus<RpcType, BitConnection> bus, ListenerPool listeners){
     super(channel);
     this.bus = bus;
-    this.registry = registry;
     // we use a local listener pool unless a global one is provided.
     this.listeners = listeners != null ? listeners : new ListenerPool(2);
-    this.listener = listener;
     this.id = UUID.randomUUID();
   }
+  
+  void setEndpoint(DrillbitEndpoint endpoint){
+    assert this.endpoint == null : "Endpoint should only be set once (only in the case in incoming server requests).";
+    this.endpoint = endpoint;
+    active = true;
+  }
 
   protected DrillbitEndpoint getEndpoint() {
     return endpoint;
@@ -69,48 +74,12 @@ public class BitConnection extends RemoteConnection{
     return listeners;
   }
   
-  protected void setEndpoint(DrillbitEndpoint endpoint) {
-    Preconditions.checkNotNull(endpoint);
-    Preconditions.checkArgument(this.endpoint == null);
-    
-    this.endpoint = endpoint;
-    BitServer.logger.debug("Adding new endpoint to available BitServer connections.  Endpoint: {}.", endpoint);
-    synchronized(this){
-      BitConnection c = registry.putIfAbsent(endpoint, this);
-      
-      if(c != null){ // the registry already has a connection like this
-        
-        // give the awaiting future an alternative connection.
-        if(listener != null){
-          listener.isAvailable(c);
-        }
-        
-        // shut this down if this is a client as it won't be available in the registry.
-        // otherwise we'll leave as, possibly allowing to bit coms to use different tunnels to talk to each other.  This shouldn't cause a problem.
-        logger.debug("Shutting down connection to {} since the registry already has an active connection that endpoint.", endpoint);
-        shutdownIfClient();
-        
-      }
-      active = true;
-      if(listener != null) listener.isAvailable(this);
-    }
-  }
-
-  public DrillRpcFuture<Ack> sendRecordBatch(FragmentContext context, FragmentWritableBatch batch){
-    return bus.send(this, RpcType.REQ_RECORD_BATCH, batch.getHeader(), Ack.class, batch.getBuffers());
-  }
-  
-  public DrillRpcFuture<Ack> sendFragment(PlanFragment fragment){
-    return bus.send(this, RpcType.REQ_INIATILIZE_FRAGMENT, fragment, Ack.class);
-  }
   
-  public DrillRpcFuture<Ack> cancelFragment(FragmentHandle handle){
-    return bus.send(this,  RpcType.REQ_CANCEL_FRAGMENT, handle, Ack.class);
+  public <SEND extends MessageLite, RECEIVE extends MessageLite> void send(RpcOutcomeListener<RECEIVE> outcomeListener, RpcType rpcType,
+      SEND protobufBody, Class<RECEIVE> clazz, ByteBuf... dataBodies){
+    bus.send(outcomeListener, this, rpcType, protobufBody, clazz, dataBodies);
   }
   
-  public DrillRpcFuture<Ack> sendFragmentStatus(FragmentStatus status){
-    return bus.send(this,  RpcType.REQ_FRAGMENT_STATUS, status, Ack.class);
-  }
 
   public void disable(){
     active = false;
@@ -140,27 +109,7 @@ public class BitConnection extends RemoteConnection{
     return true;
   }
 
-  public GenericFutureListener<ChannelFuture> getCloseHandler(GenericFutureListener<ChannelFuture> parent){
-    return new CloseHandler(this, parent);
-  }
-  
-  private class CloseHandler implements GenericFutureListener<ChannelFuture>{
-    private BitConnection connection;
-    private GenericFutureListener<ChannelFuture> parent;
-    
-    public CloseHandler(BitConnection connection, GenericFutureListener<ChannelFuture> parent) {
-      super();
-      this.connection = connection;
-      this.parent = parent;
-    }
 
-    @Override
-    public void operationComplete(ChannelFuture future) throws Exception {
-      if(connection.getEndpoint() != null) registry.remove(connection.getEndpoint(), connection);
-      parent.operationComplete(future);
-    }
-    
-  }
   
   public void shutdownIfClient(){
     if(bus.isClient()) Closeables.closeQuietly(bus);

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/b8db98ad/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/BitConnectionManager.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/BitConnectionManager.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/BitConnectionManager.java
index 0160d24..d99bb22 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/BitConnectionManager.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/BitConnectionManager.java
@@ -17,58 +17,152 @@
  ******************************************************************************/
 package org.apache.drill.exec.rpc.bit;
 
+import io.netty.channel.ChannelFuture;
+import io.netty.util.concurrent.GenericFutureListener;
+
+import java.io.Closeable;
 import java.util.concurrent.atomic.AtomicReference;
 
 import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
+import org.apache.drill.exec.rpc.RpcConnectionHandler;
 import org.apache.drill.exec.rpc.RpcException;
+import org.apache.drill.exec.server.BootStrapContext;
+import org.apache.drill.exec.work.batch.BitComHandler;
 
-import com.google.common.util.concurrent.CheckedFuture;
+import com.google.protobuf.MessageLite;
 
-public class BitConnectionManager {
+/**
+ * Manager all connections between two particular bits.
+ */
+public class BitConnectionManager implements Closeable{
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(BitConnectionManager.class);
   
-  private final int maxAttempts;
-  private final BitComImpl com;
   private final DrillbitEndpoint endpoint;
-  private final AtomicReference<BitConnection> connection;
-  private final AtomicReference<CheckedFuture<BitConnection, RpcException>> future;
+  private final AtomicReference<BitConnection> connectionHolder;
+  private final BitComHandler handler;
+  private final BootStrapContext context;
+  private final ListenerPool listenerPool;
+  private final DrillbitEndpoint localIdentity;
+  
+  public BitConnectionManager(DrillbitEndpoint remoteEndpoint, DrillbitEndpoint localIdentity, BitComHandler handler, BootStrapContext context, ListenerPool listenerPool) {
+    assert remoteEndpoint != null : "Endpoint cannot be null.";
+    assert remoteEndpoint.getAddress() != null && !remoteEndpoint.getAddress().isEmpty(): "Endpoint address cannot be null.";
+    assert remoteEndpoint.getBitPort() > 0 : String.format("Bit Port must be set to a port between 1 and 65k.  Was set to %d.", remoteEndpoint.getBitPort());
+    
+    this.connectionHolder =  new AtomicReference<BitConnection>();
+    this.endpoint = remoteEndpoint;
+    this.localIdentity = localIdentity;
+    this.handler = handler;
+    this.context = context;
+    this.listenerPool = listenerPool;
+  }
+  
+  public <R extends MessageLite> BitCommand<R> runCommand(BitCommand<R> cmd){
+    logger.debug("Running command {}", cmd);
+    BitConnection connection = connectionHolder.get();
+    if(connection != null){
+      if(connection.isActive()){
+        cmd.connectionAvailable(connection);
+        return cmd;
+      }else{
+        // remove the old connection. (don't worry if we fail since someone else should have done it.
+        connectionHolder.compareAndSet(connection, null);
+      }
+    }
+    
+    /** We've arrived here without a connection, let's make sure only one of us makes a connection. (fyi, another endpoint could create a reverse connection **/
+    synchronized(this){
+      connection = connectionHolder.get();
+      if(connection != null){
+        cmd.connectionAvailable(connection);
+      }else{
+        BitClient client = new BitClient(endpoint, localIdentity, handler, context, new CloseHandlerCreator(), listenerPool);
+        
+        client.connect(new ConnectionListeningDecorator(cmd, !endpoint.equals(localIdentity)));
+      }
+      return cmd;
+      
+    }
+  }
+  
+  CloseHandlerCreator getCloseHandlerCreator(){
+    return new CloseHandlerCreator();
+  }
 
-  BitConnectionManager(DrillbitEndpoint endpoint, BitComImpl com, BitConnection connection, CheckedFuture<BitConnection, RpcException> future, int maxAttempts) {
-    assert endpoint != null && endpoint.getAddress() != null && endpoint.getBitPort() > 0;
-    this.com = com;
-    this.connection =  new AtomicReference<BitConnection>(connection);
-    this.future = new AtomicReference<CheckedFuture<BitConnection, RpcException>>(future);
-    this.endpoint = endpoint;
-    this.maxAttempts = maxAttempts;
+  /** Factory for close handlers **/
+  class CloseHandlerCreator{
+    public GenericFutureListener<ChannelFuture> getHandler(BitConnection connection, GenericFutureListener<ChannelFuture> parent){
+      return new CloseHandler(connection, parent);
+    }
   }
   
-  BitConnection getConnection(int attempt) throws RpcException{
-    BitConnection con = connection.get();
+  
+  
+  /**
+   * Listens for connection closes and clears connection holder.
+   */
+  private class CloseHandler implements GenericFutureListener<ChannelFuture>{
+    private BitConnection connection;
+    private GenericFutureListener<ChannelFuture> parent;
     
-    if(con != null){
-      if(con.isActive()) return con;
-      connection.compareAndSet(con, null);
+    public CloseHandler(BitConnection connection, GenericFutureListener<ChannelFuture> parent) {
+      super();
+      this.connection = connection;
+      this.parent = parent;
+    }
+
+    @Override
+    public void operationComplete(ChannelFuture future) throws Exception {
+      connectionHolder.compareAndSet(connection, null);
+      parent.operationComplete(future);
     }
     
-    CheckedFuture<BitConnection, RpcException> fut = future.get();
+  } 
+  
+  /**
+   * Decorate a connection creation so that we capture a success and keep it available for future requests.  If we have raced and another is already available... we return that one and close things down on this one.
+   */
+  private class ConnectionListeningDecorator implements RpcConnectionHandler<BitConnection>{
+
+    private final RpcConnectionHandler<BitConnection> delegate;
+    private final boolean closeOnDupe;
+    
+    public ConnectionListeningDecorator(RpcConnectionHandler<BitConnection> delegate,  boolean closeOnDupe) {
+      this.delegate = delegate;
+      this.closeOnDupe = closeOnDupe;
+    }
 
-    if(fut != null){
-      try{
-        return fut.checkedGet();
-      }catch(RpcException ex){
-        future.compareAndSet(fut, null);
-        if(attempt < maxAttempts){
-          return getConnection(attempt + 1);
-        }else{
-          throw ex;
+    @Override
+    public void connectionSucceeded(BitConnection incoming) {
+      BitConnection connection = connectionHolder.get();
+      while(true){
+        boolean setted = connectionHolder.compareAndSet(null, incoming);
+        if(setted){
+          connection = incoming;
+          break;
         }
+        connection = connectionHolder.get();
+        if(connection != null) break; 
+      }
+      
+      
+      if(connection == incoming){
+        delegate.connectionSucceeded(connection);
+      }else{
+
+        if(closeOnDupe){
+          // close the incoming because another channel was created in the mean time (unless this is a self connection).
+          logger.debug("Closing incoming connection because a connection was already set.");
+          incoming.getChannel().close();
+        }
+        delegate.connectionSucceeded(connection);
       }
     }
-    
-    // no checked future, let's make one.
-    fut = com.getConnectionAsync(endpoint);
-    future.compareAndSet(null, fut);
-    return getConnection(attempt);
+
+    @Override
+    public void connectionFailed(org.apache.drill.exec.rpc.RpcConnectionHandler.FailureType type, Throwable t) {
+      delegate.connectionFailed(type, t);
+    }
     
   }
 
@@ -76,5 +170,20 @@ public class BitConnectionManager {
     return endpoint;
   }
   
+  public void addServerConnection(BitConnection connection){
+    // if the connection holder is not set, set it to this incoming connection.
+    logger.debug("Setting server connection.");
+    this.connectionHolder.compareAndSet(null, connection);
+  }
+
+  @Override
+  public void close() {
+    BitConnection c = connectionHolder.getAndSet(null);
+    if(c != null){
+      c.getChannel().close();
+    }
+  }
+  
+  
   
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/b8db98ad/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/BitServer.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/BitServer.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/BitServer.java
index 88ac6cc..d4665a8 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/BitServer.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/BitServer.java
@@ -22,18 +22,13 @@ import io.netty.channel.Channel;
 import io.netty.channel.ChannelFuture;
 import io.netty.util.concurrent.GenericFutureListener;
 
-import java.util.concurrent.ConcurrentMap;
-
 import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
 import org.apache.drill.exec.proto.ExecProtos.BitHandshake;
-import org.apache.drill.exec.proto.ExecProtos.BitStatus;
-import org.apache.drill.exec.proto.ExecProtos.FragmentHandle;
-import org.apache.drill.exec.proto.ExecProtos.FragmentStatus;
 import org.apache.drill.exec.proto.ExecProtos.RpcType;
-import org.apache.drill.exec.proto.GeneralRPCProtos.Ack;
 import org.apache.drill.exec.rpc.BasicServer;
 import org.apache.drill.exec.rpc.Response;
 import org.apache.drill.exec.rpc.RpcException;
+import org.apache.drill.exec.rpc.bit.BitConnectionManager.CloseHandlerCreator;
 import org.apache.drill.exec.server.BootStrapContext;
 import org.apache.drill.exec.work.batch.BitComHandler;
 
@@ -43,13 +38,14 @@ public class BitServer extends BasicServer<RpcType, BitConnection>{
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(BitServer.class);
   
   private final BitComHandler handler;
-  private final ConcurrentMap<DrillbitEndpoint, BitConnection> registry;
   private final ListenerPool listeners;
+  private final ConnectionManagerRegistry connectionRegistry;
+  private volatile ProxyCloseHandler proxyCloseHandler;
   
-  public BitServer(BitComHandler handler, BootStrapContext context, ConcurrentMap<DrillbitEndpoint, BitConnection> registry, ListenerPool listeners) {
+  public BitServer(BitComHandler handler, BootStrapContext context, ConnectionManagerRegistry connectionRegistry, ListenerPool listeners) {
     super(BitRpcConfig.MAPPING, context.getAllocator().getUnderlyingAllocator(), context.getBitLoopGroup());
     this.handler = handler;
-    this.registry = registry;
+    this.connectionRegistry = connectionRegistry;
     this.listeners = listeners;
   }
   
@@ -65,23 +61,36 @@ public class BitServer extends BasicServer<RpcType, BitConnection>{
 
   @Override
   protected GenericFutureListener<ChannelFuture> getCloseHandler(BitConnection connection) {
-    return connection.getCloseHandler(super.getCloseHandler(connection));
+    this.proxyCloseHandler = new ProxyCloseHandler(super.getCloseHandler(connection));
+    return proxyCloseHandler;
   }
 
   @Override
   public BitConnection initRemoteConnection(Channel channel) {
-    return new BitConnection(null, channel, this, registry, listeners);
+    return new BitConnection(channel, this, listeners);
   }
   
   
   @Override
-  protected ServerHandshakeHandler<BitHandshake> getHandshakeHandler() {
+  protected ServerHandshakeHandler<BitHandshake> getHandshakeHandler(final BitConnection connection) {
     return new ServerHandshakeHandler<BitHandshake>(RpcType.HANDSHAKE, BitHandshake.PARSER){
       
       @Override
       public MessageLite getHandshakeResponse(BitHandshake inbound) throws Exception {
-        logger.debug("Handling handshake from other bit. {}", inbound);
+//        logger.debug("Handling handshake from other bit. {}", inbound);
         if(inbound.getRpcVersion() != BitRpcConfig.RPC_VERSION) throw new RpcException(String.format("Invalid rpc version.  Expected %d, actual %d.", inbound.getRpcVersion(), BitRpcConfig.RPC_VERSION));
+        if(!inbound.hasEndpoint() || inbound.getEndpoint().getAddress().isEmpty() || inbound.getEndpoint().getBitPort() < 1) throw new RpcException(String.format("RPC didn't provide valid counter endpoint information.  Received %s.", inbound.getEndpoint()));
+        connection.setEndpoint(inbound.getEndpoint());
+
+        // add the 
+        BitConnectionManager manager = connectionRegistry.getConnectionManager(inbound.getEndpoint());
+        
+        // update the close handler.
+        proxyCloseHandler.setHandler(manager.getCloseHandlerCreator().getHandler(connection, proxyCloseHandler.getHandler()));
+        
+        // add to the connection manager. 
+        manager.addServerConnection(connection);
+
         return BitHandshake.newBuilder().setRpcVersion(BitRpcConfig.RPC_VERSION).build();
       }
 
@@ -89,5 +98,30 @@ public class BitServer extends BasicServer<RpcType, BitConnection>{
   }
 
 
+  private class ProxyCloseHandler implements GenericFutureListener<ChannelFuture> {
+
+    private volatile GenericFutureListener<ChannelFuture>  handler;
+    
+    public ProxyCloseHandler(GenericFutureListener<ChannelFuture> handler) {
+      super();
+      this.handler = handler;
+    }
+
+
+    public GenericFutureListener<ChannelFuture> getHandler() {
+      return handler;
+    }
+
+
+    public void setHandler(GenericFutureListener<ChannelFuture> handler) {
+      this.handler = handler;
+    }
+
+    @Override
+    public void operationComplete(ChannelFuture future) throws Exception {
+      handler.operationComplete(future);
+    }
+    
+  }
   
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/b8db98ad/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/BitTunnel.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/BitTunnel.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/BitTunnel.java
index 652fa52..83b7959 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/BitTunnel.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/BitTunnel.java
@@ -17,95 +17,79 @@
  ******************************************************************************/
 package org.apache.drill.exec.rpc.bit;
 
-import java.util.concurrent.ExecutionException;
-import java.util.concurrent.Executor;
-
 import org.apache.drill.exec.ops.FragmentContext;
 import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
 import org.apache.drill.exec.proto.ExecProtos.FragmentHandle;
 import org.apache.drill.exec.proto.ExecProtos.FragmentStatus;
 import org.apache.drill.exec.proto.ExecProtos.PlanFragment;
+import org.apache.drill.exec.proto.ExecProtos.RpcType;
 import org.apache.drill.exec.proto.GeneralRPCProtos.Ack;
 import org.apache.drill.exec.record.FragmentWritableBatch;
-import org.apache.drill.exec.record.WritableBatch;
 import org.apache.drill.exec.rpc.DrillRpcFuture;
-import org.apache.drill.exec.rpc.RpcException;
 import org.apache.drill.exec.rpc.RpcOutcomeListener;
 
-import com.google.common.util.concurrent.AbstractCheckedFuture;
-import com.google.common.util.concurrent.CheckedFuture;
-import com.google.common.util.concurrent.MoreExecutors;
-import com.google.common.util.concurrent.SettableFuture;
 
-/**
- * Interface provided for communication between two bits. Underlying connection may be server or client based. Resilient
- * to connection loss. Right now, this has to jump through some hoops and bridge futures between the connection creation
- * and action. A better approach should be done.
- */
 public class BitTunnel {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(BitTunnel.class);
 
-  private static final int MAX_ATTEMPTS = 3;
-
   private final BitConnectionManager manager;
-  private final Executor exec;
-  
+  private final DrillbitEndpoint endpoint;
 
-  public BitTunnel(Executor exec, DrillbitEndpoint endpoint, BitComImpl com, BitConnection connection) {
-    this.manager = new BitConnectionManager(endpoint, com, connection, null, MAX_ATTEMPTS);
-    this.exec = exec;
-  }
-
-  public BitTunnel(Executor exec, DrillbitEndpoint endpoint, BitComImpl com,
-      CheckedFuture<BitConnection, RpcException> future) {
-    this.manager = new BitConnectionManager(endpoint, com, (BitConnection) null, future, MAX_ATTEMPTS);
-    this.exec = exec;
+  public BitTunnel(DrillbitEndpoint endpoint, BitConnectionManager manager) {
+    this.manager = manager;
+    this.endpoint = endpoint;
   }
   
   public DrillbitEndpoint getEndpoint(){
     return manager.getEndpoint();
   }
 
-  private <T> DrillRpcFuture<T> submit(BitCommand<T> command) {
-    exec.execute(command);
-    return command;
-  }
-
-  public DrillRpcFuture<Ack> sendRecordBatch(FragmentContext context, FragmentWritableBatch batch) {
-    return submit(new SendBatch(batch, context));
+  public void sendRecordBatch(RpcOutcomeListener<Ack> outcomeListener, FragmentContext context, FragmentWritableBatch batch) {
+    SendBatch b = new SendBatch(outcomeListener, batch, context);
+    manager.runCommand(b);
   }
 
-  public DrillRpcFuture<Ack> sendFragment(PlanFragment fragment) {
-    return submit(new SendFragment(fragment));
+  public void sendFragment(RpcOutcomeListener<Ack> outcomeListener, PlanFragment fragment){
+    SendFragment b = new SendFragment(outcomeListener, fragment);
+    manager.runCommand(b);
   }
-
-  public DrillRpcFuture<Ack> cancelFragment(FragmentHandle handle) {
-    return submit(new CancelFragment(handle));
+  
+  public DrillRpcFuture<Ack> cancelFragment(FragmentHandle handle){
+    CancelFragment b = new CancelFragment(handle);
+    manager.runCommand(b);
+    return b.getFuture();
   }
-
+  
   public DrillRpcFuture<Ack> sendFragmentStatus(FragmentStatus status){
-    return submit(new SendFragmentStatus(status));
+    SendFragmentStatus b = new SendFragmentStatus(status);
+    manager.runCommand(b);
+    return b.getFuture();
   }
 
-  public class SendBatch extends BitCommand<Ack> {
+  public static class SendBatch extends ListeningBitCommand<Ack> {
     final FragmentWritableBatch batch;
     final FragmentContext context;
 
-    public SendBatch(FragmentWritableBatch batch, FragmentContext context) {
-      super();
+    public SendBatch(RpcOutcomeListener<Ack> listener, FragmentWritableBatch batch, FragmentContext context) {
+      super(listener);
       this.batch = batch;
       this.context = context;
     }
 
     @Override
-    public CheckedFuture<Ack, RpcException> doRpcCall(BitConnection connection) {
-      logger.debug("Sending record batch. {}", batch);
-      return connection.sendRecordBatch(context, batch);
+    public void doRpcCall(RpcOutcomeListener<Ack> outcomeListener, BitConnection connection) {
+      connection.send(outcomeListener, RpcType.REQ_RECORD_BATCH, batch.getHeader(), Ack.class, batch.getBuffers());
     }
 
+    @Override
+    public String toString() {
+      return "SendBatch [batch.header=" + batch.getHeader() + "]";
+    }
+    
+    
   }
 
-  public class SendFragmentStatus extends BitCommand<Ack> {
+  public static class SendFragmentStatus extends FutureBitCommand<Ack> {
     final FragmentStatus status;
 
     public SendFragmentStatus(FragmentStatus status) {
@@ -114,12 +98,13 @@ public class BitTunnel {
     }
 
     @Override
-    public CheckedFuture<Ack, RpcException> doRpcCall(BitConnection connection) {
-      return connection.sendFragmentStatus(status);
+    public void doRpcCall(RpcOutcomeListener<Ack> outcomeListener, BitConnection connection) {
+      connection.send(outcomeListener, RpcType.REQ_FRAGMENT_STATUS, status, Ack.class);
     }
+
   }
 
-  public class CancelFragment extends BitCommand<Ack> {
+  public static class CancelFragment extends FutureBitCommand<Ack> {
     final FragmentHandle handle;
 
     public CancelFragment(FragmentHandle handle) {
@@ -128,109 +113,23 @@ public class BitTunnel {
     }
 
     @Override
-    public CheckedFuture<Ack, RpcException> doRpcCall(BitConnection connection) {
-      return connection.cancelFragment(handle);
+    public void doRpcCall(RpcOutcomeListener<Ack> outcomeListener, BitConnection connection) {
+      connection.send(outcomeListener, RpcType.REQ_CANCEL_FRAGMENT, handle,  Ack.class);
     }
 
   }
 
-  public class SendFragment extends BitCommand<Ack> {
+  public static class SendFragment extends ListeningBitCommand<Ack> {
     final PlanFragment fragment;
 
-    public SendFragment(PlanFragment fragment) {
-      super();
+    public SendFragment(RpcOutcomeListener<Ack> listener, PlanFragment fragment) {
+      super(listener);
       this.fragment = fragment;
     }
 
     @Override
-    public CheckedFuture<Ack, RpcException> doRpcCall(BitConnection connection) {
-      return connection.sendFragment(fragment);
-    }
-
-  }
-
-
-  
-
-  private abstract class BitCommand<T> extends AbstractCheckedFuture<T, RpcException> implements Runnable, DrillRpcFuture<T> {
-
-    public void addLightListener(RpcOutcomeListener<T> outcomeListener){
-      this.addListener(new RpcOutcomeListenerWrapper(outcomeListener), MoreExecutors.sameThreadExecutor());
-    }
-
-    public BitCommand() {
-      super(SettableFuture.<T> create());
-    }
-
-    public abstract CheckedFuture<T, RpcException> doRpcCall(BitConnection connection);
-
-    public final void run() {
-      
-      try {
-        
-        BitConnection connection = manager.getConnection(0);
-        assert connection != null : "The connection manager should never return a null connection.  Worse case, it should throw an exception.";
-        CheckedFuture<T, RpcException> rpc = doRpcCall(connection);
-        rpc.addListener(new FutureBridge<T>((SettableFuture<T>) delegate(), rpc), MoreExecutors.sameThreadExecutor());
-      } catch (RpcException ex) {
-        ((SettableFuture<T>) delegate()).setException(ex);
-      }
-
-    }
-
-    @Override
-    protected RpcException mapException(Exception e) {
-      Throwable t = e;
-      if (e instanceof ExecutionException) {
-        t = e.getCause();
-      }
-      if (t instanceof RpcException) return (RpcException) t;
-      return new RpcException(t);
-    }
-
-    public class RpcOutcomeListenerWrapper implements Runnable{
-      final RpcOutcomeListener<T> inner;
-      
-      public RpcOutcomeListenerWrapper(RpcOutcomeListener<T> inner) {
-        this.inner = inner;
-      }
-
-      @Override
-      public void run() {
-        try{
-          inner.success(BitCommand.this.checkedGet());
-        }catch(RpcException e){
-          inner.failed(e);
-        }
-      }
-    }
-
-    @Override
-    public String toString() {
-      return "BitCommand ["+this.getClass().getSimpleName()+"]";
-    }
-    
-    
-    
-  }
-
-  private class FutureBridge<T> implements Runnable {
-    final SettableFuture<T> out;
-    final CheckedFuture<T, RpcException> in;
-
-    public FutureBridge(SettableFuture<T> out, CheckedFuture<T, RpcException> in) {
-      super();
-      this.out = out;
-      this.in = in;
-    }
-
-    @Override
-    public void run() {
-      try {
-        out.set(in.checkedGet());
-      } catch (RpcException ex) {
-        out.setException(ex);
-      }
+    public void doRpcCall(RpcOutcomeListener<Ack> outcomeListener, BitConnection connection) {
+      connection.send(outcomeListener, RpcType.REQ_INIATILIZE_FRAGMENT, fragment, Ack.class);
     }
 
   }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/b8db98ad/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/ConnectionManagerRegistry.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/ConnectionManagerRegistry.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/ConnectionManagerRegistry.java
new file mode 100644
index 0000000..8afbc33
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/ConnectionManagerRegistry.java
@@ -0,0 +1,73 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.rpc.bit;
+
+import io.netty.channel.Channel;
+
+import java.util.Iterator;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.atomic.AtomicReference;
+
+import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
+import org.apache.drill.exec.rpc.RpcException;
+import org.apache.drill.exec.server.BootStrapContext;
+import org.apache.drill.exec.work.batch.BitComHandler;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Maps;
+import com.google.common.util.concurrent.CheckedFuture;
+
+public class ConnectionManagerRegistry implements Iterable<BitConnectionManager>{
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ConnectionManagerRegistry.class);
+  
+  private final ConcurrentMap<DrillbitEndpoint, BitConnectionManager> registry = Maps.newConcurrentMap();
+  
+  private final BitComHandler handler;
+  private final BootStrapContext context;
+  private final ListenerPool listenerPool;
+  private volatile DrillbitEndpoint localEndpoint;
+  
+  public ConnectionManagerRegistry(BitComHandler handler, BootStrapContext context, ListenerPool listenerPool) {
+    super();
+    this.handler = handler;
+    this.context = context;
+    this.listenerPool = listenerPool;
+  }
+
+  public BitConnectionManager getConnectionManager(DrillbitEndpoint endpoint){
+    assert localEndpoint != null : "DrillbitEndpoint must be set before a connection manager can be retrieved";
+    BitConnectionManager m = registry.get(endpoint);
+    if(m == null){
+      m = new BitConnectionManager(endpoint, localEndpoint, handler, context, listenerPool);
+      BitConnectionManager m2 = registry.putIfAbsent(endpoint, m);
+      if(m2 != null) m = m2;
+    }
+    
+    return m;
+  }
+
+  @Override
+  public Iterator<BitConnectionManager> iterator() {
+    return registry.values().iterator();
+  }
+  
+  public void setEndpoint(DrillbitEndpoint endpoint){
+    this.localEndpoint = endpoint;
+  }
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/b8db98ad/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/FutureBitCommand.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/FutureBitCommand.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/FutureBitCommand.java
new file mode 100644
index 0000000..fa3b518
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/FutureBitCommand.java
@@ -0,0 +1,78 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.rpc.bit;
+
+import org.apache.drill.exec.rpc.DrillRpcFuture;
+import org.apache.drill.exec.rpc.RpcCheckedFuture;
+import org.apache.drill.exec.rpc.RpcConnectionHandler;
+import org.apache.drill.exec.rpc.RpcException;
+import org.apache.drill.exec.rpc.RpcOutcomeListener;
+
+import com.google.common.util.concurrent.CheckedFuture;
+import com.google.common.util.concurrent.SettableFuture;
+import com.google.protobuf.MessageLite;
+
+public abstract class FutureBitCommand<T extends MessageLite> implements BitCommand<T> {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(FutureBitCommand.class);
+
+  protected final SettableFuture<T> settableFuture;
+  private final RpcCheckedFuture<T> parentFuture;
+
+  public FutureBitCommand() {
+    this.settableFuture = SettableFuture.create();
+    this.parentFuture = new RpcCheckedFuture<T>(settableFuture);
+  }
+
+  public abstract void doRpcCall(RpcOutcomeListener<T> outcomeListener, BitConnection connection);
+
+  @Override
+  public void connectionAvailable(BitConnection connection) {
+    
+    doRpcCall(new DeferredRpcOutcome(), connection);
+  }
+
+  @Override
+  public void connectionSucceeded(BitConnection connection) {
+    connectionAvailable(connection);
+  }
+
+  private class DeferredRpcOutcome implements RpcOutcomeListener<T> {
+
+    @Override
+    public void failed(RpcException ex) {
+      settableFuture.setException(ex);
+    }
+
+    @Override
+    public void success(T value) {
+      settableFuture.set(value);
+    }
+
+  }
+
+  public DrillRpcFuture<T> getFuture() {
+    return parentFuture;
+  }
+
+  @Override
+  public void connectionFailed(FailureType type, Throwable t) {
+    settableFuture.setException(RpcException.mapException(
+        String.format("Command failed while establishing connection.  Failure type %s.", type), t));
+  }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/b8db98ad/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/ListenerPool.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/ListenerPool.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/ListenerPool.java
index 8f299d2..84dba85 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/ListenerPool.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/ListenerPool.java
@@ -22,32 +22,35 @@ import java.util.concurrent.ConcurrentMap;
 
 import org.apache.drill.exec.proto.ExecProtos.FragmentHandle;
 import org.apache.drill.exec.proto.ExecProtos.FragmentStatus;
+import org.apache.drill.exec.proto.UserBitShared.QueryId;
 import org.apache.drill.exec.rpc.RpcException;
 import org.apache.drill.exec.work.foreman.FragmentStatusListener;
-import org.apache.drill.exec.work.fragment.RemoteFragmentHandler;
 
 public class ListenerPool {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ListenerPool.class);
   
-  private final ConcurrentMap<FragmentHandle, FragmentStatusListener> listeners;
+  private final ConcurrentMap<QueryId, FragmentStatusListener> listeners;
   
   public ListenerPool(int par){
-    listeners = new ConcurrentHashMap<FragmentHandle, FragmentStatusListener>(16, 0.75f, par);
+    listeners = new ConcurrentHashMap<QueryId, FragmentStatusListener>(16, 0.75f, par);
   }
   
   public void removeFragmentStatusListener(FragmentHandle handle) throws RpcException{
+    logger.debug("Removing framgent status listener for handle {}.", handle);
     listeners.remove(handle);
   }
   
   public void addFragmentStatusListener(FragmentHandle handle, FragmentStatusListener listener) throws RpcException{
-    FragmentStatusListener old = listeners.putIfAbsent(handle, listener);
+    logger.debug("Adding framgent status listener for handle {}.", handle);
+    FragmentStatusListener old = listeners.putIfAbsent(handle.getQueryId(), listener);
     if(old != null) throw new RpcException("Failure.  The provided handle already exists in the listener pool.  You need to remove one listener before adding another.");
   }
   
   public void status(FragmentStatus status){
-    FragmentStatusListener l = listeners.get(status.getHandle());
+    FragmentStatusListener l = listeners.get(status.getHandle().getQueryId());
     if(l == null){
-      logger.info("A fragment message arrived but there was no registered listener for that message.");
+      
+      logger.error("A fragment message arrived but there was no registered listener for that message for handle {}.", status.getHandle());
       return;
     }else{
       l.statusUpdate(status);

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/b8db98ad/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/ListeningBitCommand.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/ListeningBitCommand.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/ListeningBitCommand.java
new file mode 100644
index 0000000..90db6a6
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/ListeningBitCommand.java
@@ -0,0 +1,73 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.rpc.bit;
+
+import org.apache.drill.exec.rpc.DrillRpcFuture;
+import org.apache.drill.exec.rpc.RpcCheckedFuture;
+import org.apache.drill.exec.rpc.RpcConnectionHandler;
+import org.apache.drill.exec.rpc.RpcException;
+import org.apache.drill.exec.rpc.RpcOutcomeListener;
+
+import com.google.common.util.concurrent.CheckedFuture;
+import com.google.common.util.concurrent.SettableFuture;
+import com.google.protobuf.MessageLite;
+
+public abstract class ListeningBitCommand<T extends MessageLite> implements BitCommand<T> {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ListeningBitCommand.class);
+
+  private final RpcOutcomeListener<T> listener;
+
+  public ListeningBitCommand(RpcOutcomeListener<T> listener) {
+    this.listener = listener;
+  }
+
+  public abstract void doRpcCall(RpcOutcomeListener<T> outcomeListener, BitConnection connection);
+
+  @Override
+  public void connectionAvailable(BitConnection connection) {
+    
+    doRpcCall(new DeferredRpcOutcome(), connection);
+  }
+
+  @Override
+  public void connectionSucceeded(BitConnection connection) {
+    connectionAvailable(connection);
+  }
+
+  private class DeferredRpcOutcome implements RpcOutcomeListener<T> {
+
+    @Override
+    public void failed(RpcException ex) {
+      listener.failed(ex);
+    }
+
+    @Override
+    public void success(T value) {
+      listener.success(value);
+    }
+
+  }
+
+
+  @Override
+  public void connectionFailed(FailureType type, Throwable t) {
+    listener.failed(RpcException.mapException(
+        String.format("Command failed while establishing connection.  Failure type %s.", type), t));
+  }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/b8db98ad/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/QueryResultBatch.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/QueryResultBatch.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/QueryResultBatch.java
index 3df88b7..779085c 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/QueryResultBatch.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/QueryResultBatch.java
@@ -45,5 +45,12 @@ public class QueryResultBatch {
   public boolean hasData(){
     return data != null;
   }
+
+  @Override
+  public String toString() {
+    return "QueryResultBatch [header=" + header + ", data=" + data + "]";
+  }
+  
+  
   
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/b8db98ad/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
new file mode 100644
index 0000000..0aa7c86
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/QueryResultHandler.java
@@ -0,0 +1,153 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.rpc.user;
+
+import io.netty.buffer.ByteBuf;
+
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ConcurrentMap;
+
+import org.apache.drill.exec.proto.UserBitShared.QueryId;
+import org.apache.drill.exec.proto.UserProtos.QueryResult;
+import org.apache.drill.exec.rpc.BaseRpcOutcomeListener;
+import org.apache.drill.exec.rpc.RpcBus;
+import org.apache.drill.exec.rpc.RpcException;
+import org.apache.drill.exec.rpc.RpcOutcomeListener;
+
+import com.google.common.collect.Maps;
+import com.google.common.collect.Queues;
+
+/**
+ * Encapsulates the future management of query submissions. This entails a potential race condition. Normal ordering is:
+ * 1. Submit query to be executed. 2. Receive QueryHandle for buffer management 3. Start receiving results batches for
+ * query.
+ * 
+ * However, 3 could potentially occur before 2. As such, we need to handle this case and then do a switcheroo.
+ * 
+ */
+public class QueryResultHandler {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(QueryResultHandler.class);
+
+  private ConcurrentMap<QueryId, UserResultsListener> resultsListener = Maps.newConcurrentMap();
+
+  
+  public RpcOutcomeListener<QueryId> getWrappedListener(UserResultsListener listener){
+    return new SubmissionListener(listener);
+  }
+  
+  public void batchArrived(ByteBuf pBody, ByteBuf dBody) throws RpcException {
+    final QueryResult result = RpcBus.get(pBody, QueryResult.PARSER);
+    final QueryResultBatch batch = new QueryResultBatch(result, dBody);
+    UserResultsListener l = resultsListener.get(result.getQueryId());
+    // logger.debug("For QueryId [{}], retrieved result listener {}", result.getQueryId(), l);
+    if (l != null) {
+      // logger.debug("Results listener available, using existing.");
+      l.resultArrived(batch);
+      if (result.getIsLastChunk()) {
+        resultsListener.remove(result.getQueryId(), l);
+      }
+    } else {
+      logger.debug("Results listener not available, creating a buffering listener.");
+      // manage race condition where we start getting results before we receive the queryid back.
+      BufferingListener bl = new BufferingListener();
+      l = resultsListener.putIfAbsent(result.getQueryId(), bl);
+      if (l != null) {
+        l.resultArrived(batch);
+      } else {
+        bl.resultArrived(batch);
+      }
+    }
+  }
+
+  private class BufferingListener implements UserResultsListener {
+
+    private ConcurrentLinkedQueue<QueryResultBatch> results = Queues.newConcurrentLinkedQueue();
+    private volatile UserResultsListener output;
+
+    public boolean transferTo(UserResultsListener l) {
+      synchronized (this) {
+        output = l;
+        boolean last = false;
+        for (QueryResultBatch r : results) {
+          l.resultArrived(r);
+          last = r.getHeader().getIsLastChunk();
+        }
+        return last;
+      }
+    }
+
+    @Override
+    public void resultArrived(QueryResultBatch result) {
+      synchronized (this) {
+        if (output == null) {
+          this.results.add(result);
+        } else {
+          output.resultArrived(result);
+        }
+      }
+    }
+
+    @Override
+    public void submissionFailed(RpcException ex) {
+      throw new UnsupportedOperationException("You cannot report failed submissions to a buffering listener.");
+    }
+
+  }
+
+  private class SubmissionListener extends BaseRpcOutcomeListener<QueryId> {
+    private UserResultsListener listener;
+
+    public SubmissionListener(UserResultsListener listener) {
+      super();
+      this.listener = listener;
+    }
+
+    @Override
+    public void failed(RpcException ex) {
+      listener.submissionFailed(ex);
+    }
+
+    @Override
+    public void success(QueryId queryId) {
+      logger.debug("Received QueryId {} succesfully.  Adding listener {}", queryId, listener);
+      UserResultsListener oldListener = resultsListener.putIfAbsent(queryId, listener);
+
+      // we need to deal with the situation where we already received results by the time we got the query id back. In
+      // that case, we'll need to transfer the buffering listener over, grabbing a lock against reception of additional
+      // results during the transition
+      if (oldListener != null) {
+        logger.debug("Unable to place user results listener, buffering listener was already in place.");
+        if (oldListener instanceof BufferingListener) {
+          resultsListener.remove(oldListener);
+          boolean all = ((BufferingListener) oldListener).transferTo(this.listener);
+          // simply remove the buffering listener if we already have the last response.
+          if (all) {
+            resultsListener.remove(oldListener);
+          } else {
+            boolean replaced = resultsListener.replace(queryId, oldListener, listener);
+            if (!replaced) throw new IllegalStateException();
+          }
+        } else {
+          throw new IllegalStateException("Trying to replace a non-buffering User Results listener.");
+        }
+      }
+
+    }
+
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/b8db98ad/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserClient.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserClient.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserClient.java
index 5d2e799..ad44ff2 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserClient.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserClient.java
@@ -21,11 +21,6 @@ import io.netty.buffer.ByteBuf;
 import io.netty.buffer.ByteBufAllocator;
 import io.netty.channel.EventLoopGroup;
 
-import java.util.concurrent.ConcurrentLinkedQueue;
-import java.util.concurrent.ConcurrentMap;
-import java.util.concurrent.Future;
-import java.util.concurrent.locks.ReentrantReadWriteLock;
-
 import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
 import org.apache.drill.exec.proto.GeneralRPCProtos.Ack;
 import org.apache.drill.exec.proto.UserBitShared.QueryId;
@@ -36,115 +31,27 @@ import org.apache.drill.exec.proto.UserProtos.RunQuery;
 import org.apache.drill.exec.proto.UserProtos.UserToBitHandshake;
 import org.apache.drill.exec.rpc.BasicClientWithConnection;
 import org.apache.drill.exec.rpc.Response;
+import org.apache.drill.exec.rpc.RpcConnectionHandler;
 import org.apache.drill.exec.rpc.RpcException;
-import org.apache.drill.exec.rpc.RpcOutcomeListener;
 
-import com.google.common.collect.Maps;
-import com.google.common.collect.Queues;
 import com.google.protobuf.MessageLite;
 
-public class UserClient extends BasicClientWithConnection<RpcType> {
+public class UserClient extends BasicClientWithConnection<RpcType, UserToBitHandshake, BitToUserHandshake> {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(UserClient.class);
 
-  private ConcurrentMap<QueryId, UserResultsListener> resultsListener = Maps.newConcurrentMap();
+  private final QueryResultHandler queryResultHandler = new QueryResultHandler();
 
   public UserClient(ByteBufAllocator alloc, EventLoopGroup eventLoopGroup) {
-    super(UserRpcConfig.MAPPING, alloc, eventLoopGroup);
-  }
-
-  public Future<Void> submitQuery(RunQuery query, UserResultsListener resultsListener) throws RpcException {
-    this.send(RpcType.RUN_QUERY, query, QueryId.class).addLightListener(new SubmissionListener(resultsListener));
-    return resultsListener.getFuture();
+    super(UserRpcConfig.MAPPING, alloc, eventLoopGroup, RpcType.HANDSHAKE, BitToUserHandshake.class, BitToUserHandshake.PARSER);
   }
 
-  public BitToUserHandshake connect(DrillbitEndpoint endpoint) throws RpcException, InterruptedException{
-    return this.connectAsClient(RpcType.HANDSHAKE, UserToBitHandshake.newBuilder().setRpcVersion(UserRpcConfig.RPC_VERSION).build(), endpoint.getAddress(), endpoint.getUserPort(), BitToUserHandshake.class);
-  }
-  
-  private class BufferingListener extends UserResultsListener {
-
-    private ConcurrentLinkedQueue<QueryResultBatch> results = Queues.newConcurrentLinkedQueue();
-    private ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
-    private volatile UserResultsListener output;
-
-    public boolean transferTo(UserResultsListener l) {
-      lock.writeLock().lock();
-      output = l;
-      boolean last = false;
-      for (QueryResultBatch r : results) {
-        l.resultArrived(r);
-        last = r.getHeader().getIsLastChunk();
-      }
-      if (future.isDone()) {
-        l.set();
-      }
-      return last;
-    }
-
-    @Override
-    public void resultArrived(QueryResultBatch result) {
-      logger.debug("Result arrvied.");
-      lock.readLock().lock();
-      try {
-        if (output == null) {
-          this.results.add(result);
-        } else {
-          output.resultArrived(result);
-        }
-
-      } finally {
-        lock.readLock().unlock();
-      }
-
-    }
-
-    @Override
-    public void submissionFailed(RpcException ex) {
-      throw new UnsupportedOperationException("You cannot report failed submissions to a buffering listener.");
-    }
-
+  public void submitQuery(UserResultsListener resultsListener, RunQuery query) throws RpcException {
+    send(queryResultHandler.getWrappedListener(resultsListener), RpcType.RUN_QUERY, query, QueryId.class);
   }
 
-  private class SubmissionListener extends RpcOutcomeListener<QueryId> {
-    private UserResultsListener listener;
-
-    public SubmissionListener(UserResultsListener listener) {
-      super();
-      this.listener = listener;
-    }
-
-    @Override
-    public void failed(RpcException ex) {
-      listener.submissionFailed(ex);
-    }
-
-    @Override
-    public void success(QueryId queryId) {
-      logger.debug("Received QueryId {} succesfully.  Adding listener {}", queryId, listener);
-      UserResultsListener oldListener = resultsListener.putIfAbsent(queryId, listener);
-
-      // we need to deal with the situation where we already received results by the time we got the query id back. In
-      // that case, we'll need to transfer the buffering listener over, grabbing a lock against reception of additional
-      // results during the transition
-      if (oldListener != null) {
-        logger.debug("Unable to place user results listener, buffering listener was already in place.");
-        if (oldListener instanceof BufferingListener) {
-          resultsListener.remove(oldListener);
-          boolean all = ((BufferingListener) oldListener).transferTo(this.listener);
-          // simply remove the buffering listener if we already have the last response.
-          if (all) {
-            resultsListener.remove(oldListener);
-          } else {
-            boolean replaced = resultsListener.replace(queryId, oldListener, listener);
-            if (!replaced) throw new IllegalStateException();
-          }
-        } else {
-          throw new IllegalStateException("Trying to replace a non-buffering User Results listener.");
-        }
-      }
-
-    }
-
+  public void connect(RpcConnectionHandler<ServerConnection> handler, DrillbitEndpoint endpoint) throws RpcException, InterruptedException {
+    UserToBitHandshake hs = UserToBitHandshake.newBuilder().setRpcVersion(UserRpcConfig.RPC_VERSION).setSupportListening(true).build();
+    this.connectAsClient(handler, hs, endpoint.getAddress(), endpoint.getUserPort());
   }
 
   @Override
@@ -165,29 +72,7 @@ public class UserClient extends BasicClientWithConnection<RpcType> {
   protected Response handle(int rpcType, ByteBuf pBody, ByteBuf dBody) throws RpcException {
     switch (rpcType) {
     case RpcType.QUERY_RESULT_VALUE:
-      final QueryResult result = get(pBody, QueryResult.PARSER);
-      final QueryResultBatch batch = new QueryResultBatch(result, dBody);
-      UserResultsListener l = resultsListener.get(result.getQueryId());
-//      logger.debug("For QueryId [{}], retrieved result listener {}", result.getQueryId(), l);
-      if (l != null) {
-//        logger.debug("Results listener available, using existing.");
-        l.resultArrived(batch);
-        if (result.getIsLastChunk()) {
-          resultsListener.remove(result.getQueryId(), l);
-          l.set();
-        }
-      } else {
-        logger.debug("Results listener not available, creating a buffering listener.");
-        // manage race condition where we start getting results before we receive the queryid back.
-        BufferingListener bl = new BufferingListener();
-        l = resultsListener.putIfAbsent(result.getQueryId(), bl);
-        if (l != null) {
-          l.resultArrived(batch);
-        } else {
-          bl.resultArrived(batch);
-        }
-      }
-
+      queryResultHandler.batchArrived(pBody, dBody);
       return new Response(RpcType.ACK, Ack.getDefaultInstance());
     default:
       throw new RpcException(String.format("Unknown Rpc Type %d. ", rpcType));
@@ -196,18 +81,16 @@ public class UserClient extends BasicClientWithConnection<RpcType> {
   }
 
   @Override
-  protected ClientHandshakeHandler<BitToUserHandshake> getHandshakeHandler() {
-    return new ClientHandshakeHandler<BitToUserHandshake>(RpcType.HANDSHAKE, BitToUserHandshake.class, BitToUserHandshake.PARSER) {
+  protected void validateHandshake(BitToUserHandshake inbound) throws RpcException {
+    logger.debug("Handling handshake from bit to user. {}", inbound);
+    if (inbound.getRpcVersion() != UserRpcConfig.RPC_VERSION)
+      throw new RpcException(String.format("Invalid rpc version.  Expected %d, actual %d.", inbound.getRpcVersion(),
+          UserRpcConfig.RPC_VERSION));
 
-      @Override
-      protected void validateHandshake(BitToUserHandshake inbound) throws Exception {
-        logger.debug("Handling handshake from bit to user. {}", inbound);
-        if (inbound.getRpcVersion() != UserRpcConfig.RPC_VERSION)
-          throw new RpcException(String.format("Invalid rpc version.  Expected %d, actual %d.",
-              inbound.getRpcVersion(), UserRpcConfig.RPC_VERSION));
-      }
+  }
 
-    };
+  @Override
+  protected void finalizeConnection(BitToUserHandshake handshake, BasicClientWithConnection.ServerConnection connection) {
   }
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/b8db98ad/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserResultsListener.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserResultsListener.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserResultsListener.java
index 3ce14f0..b1dbfe8 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserResultsListener.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserResultsListener.java
@@ -24,17 +24,8 @@ import org.apache.drill.exec.rpc.RpcException;
 
 import com.google.common.util.concurrent.SettableFuture;
 
-public abstract class UserResultsListener {
-  SettableFuture<Void> future = SettableFuture.create();
+public interface UserResultsListener {
   
-  final void set(){
-    future.set(null);
-  }
-  
-  Future<Void> getFuture(){
-    return future;
-  }
-
   public abstract void submissionFailed(RpcException ex); 
   public abstract void resultArrived(QueryResultBatch result);
 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/b8db98ad/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserServer.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserServer.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserServer.java
index 406afc4..908af61 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserServer.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserServer.java
@@ -26,16 +26,15 @@ import io.netty.channel.EventLoopGroup;
 import org.apache.drill.exec.physical.impl.materialize.QueryWritableBatch;
 import org.apache.drill.exec.proto.GeneralRPCProtos.Ack;
 import org.apache.drill.exec.proto.UserProtos.BitToUserHandshake;
-import org.apache.drill.exec.proto.UserProtos.QueryResult;
 import org.apache.drill.exec.proto.UserProtos.RequestResults;
 import org.apache.drill.exec.proto.UserProtos.RpcType;
 import org.apache.drill.exec.proto.UserProtos.RunQuery;
 import org.apache.drill.exec.proto.UserProtos.UserToBitHandshake;
 import org.apache.drill.exec.rpc.BasicServer;
-import org.apache.drill.exec.rpc.DrillRpcFuture;
 import org.apache.drill.exec.rpc.RemoteConnection;
 import org.apache.drill.exec.rpc.Response;
 import org.apache.drill.exec.rpc.RpcException;
+import org.apache.drill.exec.rpc.RpcOutcomeListener;
 import org.apache.drill.exec.work.user.UserWorker;
 
 import com.google.protobuf.InvalidProtocolBufferException;
@@ -100,8 +99,9 @@ public class UserServer extends BasicServer<RpcType, UserServer.UserClientConnec
       super(channel);
     }
 
-    public DrillRpcFuture<Ack> sendResult(QueryWritableBatch result){
-      return send(this, RpcType.QUERY_RESULT, result.getHeader(), Ack.class, result.getBuffers());
+    public void sendResult(RpcOutcomeListener<Ack> listener, QueryWritableBatch result){
+      logger.debug("Sending result to client with {}", result);
+      send(listener, this, RpcType.QUERY_RESULT, result.getHeader(), Ack.class, result.getBuffers());
     }
 
   }
@@ -112,7 +112,7 @@ public class UserServer extends BasicServer<RpcType, UserServer.UserClientConnec
   }
   
   @Override
-  protected ServerHandshakeHandler<UserToBitHandshake> getHandshakeHandler() {
+  protected ServerHandshakeHandler<UserToBitHandshake> getHandshakeHandler(UserClientConnection connection) {
     return new ServerHandshakeHandler<UserToBitHandshake>(RpcType.HANDSHAKE, UserToBitHandshake.PARSER){
 
       @Override

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/b8db98ad/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/server/BootStrapContext.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/server/BootStrapContext.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/server/BootStrapContext.java
index 3c4d9af..ed13748 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/server/BootStrapContext.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/server/BootStrapContext.java
@@ -39,7 +39,7 @@ public class BootStrapContext implements Closeable{
   public BootStrapContext(DrillConfig config) {
     super();
     this.config = config;
-    this.loop = new NioEventLoopGroup(4, new NamedThreadFactory("BitServer-"));
+    this.loop = new NioEventLoopGroup(1, new NamedThreadFactory("BitServer-"));
     this.metrics = new MetricRegistry(config.getString(ExecConstants.METRICS_CONTEXT_NAME));
     this.allocator = BufferAllocator.getAllocator(config);
   }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/b8db98ad/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/server/RemoteServiceSet.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/server/RemoteServiceSet.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/server/RemoteServiceSet.java
index 0337a68..199768f 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/server/RemoteServiceSet.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/server/RemoteServiceSet.java
@@ -24,6 +24,7 @@ import org.apache.drill.exec.cache.DistributedCache;
 import org.apache.drill.exec.cache.LocalCache;
 import org.apache.drill.exec.coord.ClusterCoordinator;
 import org.apache.drill.exec.coord.LocalClusterCoordinator;
+import org.apache.drill.exec.exception.DrillbitStartupException;
 
 public class RemoteServiceSet implements Closeable{
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(RemoteServiceSet.class);
@@ -37,6 +38,7 @@ public class RemoteServiceSet implements Closeable{
     this.coordinator = coordinator;
   }
 
+
   public DistributedCache getCache() {
     return cache;
   }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/b8db98ad/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/service/ServiceEngine.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/service/ServiceEngine.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/service/ServiceEngine.java
index d6d3b9c..b07f274 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/service/ServiceEngine.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/service/ServiceEngine.java
@@ -53,12 +53,12 @@ public class ServiceEngine implements Closeable{
   
   public DrillbitEndpoint start() throws DrillbitStartupException, InterruptedException, UnknownHostException{
     int userPort = userServer.bind(config.getInt(ExecConstants.INITIAL_USER_PORT));
-    int bitPort = bitCom.start();
-    return DrillbitEndpoint.newBuilder()
+    DrillbitEndpoint partialEndpoint = DrillbitEndpoint.newBuilder()
         .setAddress(InetAddress.getLocalHost().getHostAddress())
-        .setBitPort(bitPort)
         .setUserPort(userPort)
         .build();
+
+    return bitCom.start(partialEndpoint);
   }
 
   public BitCom getBitCom(){

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/b8db98ad/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/EndpointListener.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/EndpointListener.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/EndpointListener.java
index f6a9786..9a72845 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/EndpointListener.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/EndpointListener.java
@@ -18,10 +18,9 @@
 package org.apache.drill.exec.work;
 
 import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
-import org.apache.drill.exec.rpc.RpcException;
-import org.apache.drill.exec.rpc.RpcOutcomeListener;
+import org.apache.drill.exec.rpc.BaseRpcOutcomeListener;
 
-public abstract class EndpointListener<RET, V> extends RpcOutcomeListener<RET>{
+public abstract class EndpointListener<RET, V> extends BaseRpcOutcomeListener<RET>{
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(EndpointListener.class);
 
   protected final DrillbitEndpoint endpoint;

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/b8db98ad/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/FragmentRunner.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/FragmentRunner.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/FragmentRunner.java
index 2900d99..554b398 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/FragmentRunner.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/FragmentRunner.java
@@ -65,6 +65,7 @@ public class FragmentRunner implements Runnable, CancelableQuery, StatusProvider
   
   @Override
   public void run() {
+    logger.debug("Starting fragment runner. {}:{}", context.getHandle().getMajorFragmentId(), context.getHandle().getMinorFragmentId());
     if(!updateState(FragmentState.AWAITING_ALLOCATION, FragmentState.RUNNING, false)){
       internalFail(new RuntimeException(String.format("Run was called when fragment was in %s state.  FragmentRunnables should only be started when they are currently in awaiting allocation state.", FragmentState.valueOf(state.get()))));
       return;
@@ -76,7 +77,12 @@ public class FragmentRunner implements Runnable, CancelableQuery, StatusProvider
     try{
       while(state.get() == FragmentState.RUNNING_VALUE){
         if(!root.next()){
-          updateState(FragmentState.RUNNING, FragmentState.FINISHED, false);
+          if(context.isFailed()){
+            updateState(FragmentState.RUNNING, FragmentState.FAILED, false);  
+          }else{
+            updateState(FragmentState.RUNNING, FragmentState.FINISHED, false);
+          }
+          
         }
       }
       
@@ -90,7 +96,7 @@ public class FragmentRunner implements Runnable, CancelableQuery, StatusProvider
     }finally{
       t.stop();
     }
-    
+    logger.debug("Fragment runner complete. {}:{}", context.getHandle().getMajorFragmentId(), context.getHandle().getMinorFragmentId());
   }
   
   private void internalFail(Throwable excep){


[32/53] [abbrv] git commit: Create new generated value vectors utilizing fmpp. Includes: - First pass; integrate build system and some cursory implementations - starting to split common logic into base class - implement most of varlen value vector funct

Posted by ja...@apache.org.
Create new generated value vectors utilizing fmpp.  Includes:
- First pass; integrate build system and some cursory implementations
- starting to split common logic into base class
- implement most of varlen value vector functionality, minor cleanup of tdd tags
- added nullable derived class
- Merge changes from JA, minor format cleanup.
- minor fix and cleanup
- added bit vector, removed widthInBits which also allowed removal of FixedBase ctor
- apply TC's fix for resetAllocation()
- added repeated value vectors
- Hooked up templated ValueVectors to codebase.  Removed old ValueVector classes.  Cleanup.
- fix repeated get() and add()
- added some value vector tests.  fixed bugs in VV and some call sites.  generated TypeHelper from FMPP template.  removed unused VV methods
- made base immutable, some debugging
- split mutable/immutable basic VV types. minor refactoring
- fix several allocation bugs
- fix various bugs, only JSONRecordReader test is failing
- fix nullable bit value vector
- make bit vectors use ints to represent the bit value
- remove superfluous logging
- fix value vector getters and setter
- comments and cleanup
- temp disable repeated map JSONReader test
- formatting
- whitespace cleanups


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

Branch: refs/heads/master
Commit: 7075cca1f1be45b876ef846762f13d0780627c3a
Parents: 5a5d07f
Author: Ben Becker <be...@gmail.com>
Authored: Tue Jun 18 17:36:11 2013 -0700
Committer: Jacques Nadeau <ja...@apache.org>
Committed: Mon Jul 15 11:36:32 2013 -0700

----------------------------------------------------------------------
 sandbox/prototype/exec/java-exec/pom.xml        |  20 +
 .../src/main/codegen/ValueVectors/config.fmpp   |   3 +
 .../ValueVectors/data/ValueVectorTypes.tdd      | 106 +++
 .../ValueVectors/templates/TypeHelper.java      | 119 +++
 .../ValueVectors/templates/ValueVector.java     | 768 +++++++++++++++++++
 .../exec/physical/config/MockRecordReader.java  |  15 +-
 .../exec/physical/impl/FilterRecordBatch.java   |   2 +-
 .../drill/exec/physical/impl/OutputMutator.java |   2 +-
 .../drill/exec/physical/impl/ScanBatch.java     |  16 +-
 .../drill/exec/physical/impl/ScreenCreator.java |   2 +-
 .../exec/physical/impl/SingleSenderCreator.java |   2 +-
 .../exec/physical/impl/WireRecordBatch.java     |   4 +-
 .../drill/exec/record/MaterializedField.java    |   5 -
 .../apache/drill/exec/record/RecordBatch.java   |   2 +-
 .../drill/exec/record/RecordBatchLoader.java    |  22 +-
 .../apache/drill/exec/record/WritableBatch.java |   9 +-
 .../record/vector/AbstractFixedValueVector.java |  77 --
 .../drill/exec/record/vector/AnyVector.java     |  30 -
 .../exec/record/vector/BaseValueVector.java     | 157 ----
 .../apache/drill/exec/record/vector/Bit.java    | 168 ----
 .../apache/drill/exec/record/vector/Fixed1.java |  43 --
 .../drill/exec/record/vector/Fixed12.java       |  35 -
 .../drill/exec/record/vector/Fixed16.java       |  37 -
 .../apache/drill/exec/record/vector/Fixed2.java |  53 --
 .../apache/drill/exec/record/vector/Fixed4.java |  59 --
 .../apache/drill/exec/record/vector/Fixed8.java |  58 --
 .../drill/exec/record/vector/FixedLen.java      |  45 --
 .../exec/record/vector/NullValueException.java  |   9 +
 .../drill/exec/record/vector/NullableBit.java   |  20 -
 .../exec/record/vector/NullableFixed4.java      |  44 --
 .../exec/record/vector/NullableValueVector.java | 108 ---
 .../exec/record/vector/NullableVarLen4.java     |  21 -
 .../drill/exec/record/vector/RepeatMap.java     |  57 --
 .../exec/record/vector/SelectionVector.java     |   2 +-
 .../drill/exec/record/vector/TypeHelper.java    | 259 -------
 .../drill/exec/record/vector/ValueVector.java   | 118 ---
 .../drill/exec/record/vector/VarLen1.java       |  36 -
 .../drill/exec/record/vector/VarLen2.java       |  36 -
 .../drill/exec/record/vector/VarLen4.java       |  48 --
 .../exec/record/vector/VariableVector.java      | 102 ---
 .../java/org/apache/drill/exec/rpc/RpcBus.java  |   2 +-
 .../drill/exec/store/JSONRecordReader.java      |  30 +-
 .../apache/drill/exec/store/VectorHolder.java   |   6 +-
 .../work/AbstractFragmentRunnerListener.java    |   3 +-
 .../apache/drill/exec/work/FragmentRunner.java  |   3 +-
 .../physical/impl/TestSimpleFragmentRun.java    |  90 ++-
 .../exec/record/vector/TestValueVector.java     | 252 ++++++
 .../drill/exec/store/JSONRecordReaderTest.java  |  36 +-
 48 files changed, 1400 insertions(+), 1741 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/7075cca1/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 f5ece33..4e4df95 100644
--- a/sandbox/prototype/exec/java-exec/pom.xml
+++ b/sandbox/prototype/exec/java-exec/pom.xml
@@ -167,6 +167,26 @@
 					</execution>
 				</executions>
 			</plugin>
+			<plugin>
+				<groupId>com.googlecode.fmpp-maven-plugin</groupId>
+				<artifactId>fmpp-maven-plugin</artifactId>
+				<version>1.0</version>
+				<configuration>
+					<cfgFile>src/main/codegen/ValueVectors/config.fmpp</cfgFile>
+					<outputDirectory>target/generated-sources/org/apache/drill/exec/record/vector</outputDirectory>
+					<templateDirectory>src/main/codegen/ValueVectors/templates</templateDirectory>
+				</configuration>
+				<executions>
+					<execution>
+						<id>generate-sources</id>
+						<phase>generate-sources</phase>
+						<goals>
+							<goal>generate</goal>
+						</goals>
+					</execution>
+				</executions>
+			</plugin>
+
 			<!-- <plugin> -->
 			<!-- <groupId>com.github.igor-petruk.protobuf</groupId> -->
 			<!-- <artifactId>protobuf-maven-plugin</artifactId> -->

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/7075cca1/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/config.fmpp
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/config.fmpp b/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/config.fmpp
new file mode 100644
index 0000000..da05e2d
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/config.fmpp
@@ -0,0 +1,3 @@
+data: {
+    tdd(../data/ValueVectorTypes.tdd)
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/7075cca1/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/data/ValueVectorTypes.tdd
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/data/ValueVectorTypes.tdd b/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/data/ValueVectorTypes.tdd
new file mode 100644
index 0000000..42153b6
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/data/ValueVectorTypes.tdd
@@ -0,0 +1,106 @@
+{
+  types: [
+    {
+      major: "Fixed",
+      width: 1,
+      javaType: "byte",
+      minor: [
+        { class: "TinyInt" },
+        { class: "UInt1" }
+      ]
+    },
+    {
+      major: "Fixed",
+      width: 2,
+      javaType: "short",
+      minor: [
+        { class: "SmallInt" },
+        { class: "UInt2" }
+      ]
+    },
+    {
+      major: "Fixed",
+      width: 4,
+      javaType: "int",
+      minor: [
+        { class: "Int" },
+        { class: "UInt4" },
+        { class: "Decimal4", maxPrecisionDigits: 8, scale: 1, javaType: "float"},
+        { class: "Float4", javaType: "float" },
+        { class: "Date" }
+      ]
+    },
+    {
+      major: "Fixed",
+      width: 8,
+      javaType: "long",
+      minor: [
+        { class: "BigInt" },
+        { class: "UInt8" },
+        { class: "Decimal8", maxPrecisionDigits: 18, scale: 1, javaType: "double"},
+        { class: "Float8", javaType: "double" },
+        { class: "Money", maxPrecisionDigits: 2, scale: 1, javaType: "double" }
+        { class: "Time" },
+        { class: "TimeStamp" },
+        { class: "DateTime" }
+      ]
+    },
+    {
+      major: "Fixed",
+      width: 12,
+      javaType: "ByteBuf",
+      minor: [
+        { class: "Decimal12", maxPrecisionDigits: 28, scale: 1},
+        { class: "TimeTZ" },
+        { class: "Interval" }
+      ]
+    },
+    {
+      major: "Fixed",
+      width: 16,
+      javaType: "ByteBuf",
+      minor: [
+        { class: "Decimal16", maxPrecisionDigits: 37, scale: 1}
+      ]
+    },
+    {
+      major: "VarLen",
+      width: 1,
+      javaType: "byte",
+      minor: [
+        { class: "VarBinary1" },
+        { class: "VarChar1" }
+      ]
+    },
+    {
+      major: "VarLen",
+      width: 2,
+      javaType: "short",
+      minor: [
+        { class: "VarBinary2" },
+        { class: "VarChar2" },
+        { class: "Proto2" },
+        { class: "MsgPack2" }
+      ]
+    },
+    {
+      major: "VarLen",
+      width: 4,
+      javaType: "int",
+      minor: [
+        { class: "VarBinary4" },
+        { class: "VarChar4" }
+        { class: "Proto4" },
+        { class: "MsgPack4" }
+      ]
+    },
+    {
+      major: "Bit",
+      width: 1,
+      javaType: "int",
+      minor: [
+        { class: "Bit" }
+      ]
+    }
+  ]
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/7075cca1/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/TypeHelper.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/TypeHelper.java b/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/TypeHelper.java
new file mode 100644
index 0000000..8dfd3af
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/TypeHelper.java
@@ -0,0 +1,119 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.record.vector;
+
+import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.proto.SchemaDefProtos.DataMode;
+import org.apache.drill.exec.proto.SchemaDefProtos.MajorType;
+import org.apache.drill.exec.proto.SchemaDefProtos.MinorType;
+import org.apache.drill.exec.record.MaterializedField;
+
+public class TypeHelper {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TypeHelper.class);
+
+  private static final int WIDTH_ESTIMATE_1 = 10;
+  private static final int WIDTH_ESTIMATE_2 = 50000;
+  private static final int WIDTH_ESTIMATE_4 = 1024*1024;
+
+  public static int getSize(MajorType major) {
+    switch (major.getMinorType()) {
+<#list types as type>
+  <#list type.minor as minor>
+    <#if minor.class != "Bit">
+      case ${minor.class?upper_case}:
+        return ${type.width}<#if minor.class?substring(0, 3) == "Var" ||
+                                 minor.class?substring(0, 3) == "PRO" ||
+                                 minor.class?substring(0, 3) == "MSG"> + WIDTH_ESTIMATE_${type.width}</#if>;
+    </#if>
+  </#list>
+</#list>
+      case BOOLEAN: return 1;
+      case FIXEDCHAR: return major.getWidth();
+      case FIXEDBINARY: return major.getWidth();
+    }
+    throw new UnsupportedOperationException();
+  }
+
+  public static Class<?> getValueVectorClass(MinorType type, DataMode mode){
+    switch (type) {
+<#list types as type>
+  <#list type.minor as minor>
+    <#if minor.class == "Bit">
+      case BOOLEAN:
+        switch (mode) {
+          case REQUIRED:
+            return ValueVector.${minor.class}.class;
+          case OPTIONAL:
+            return ValueVector.Nullable${minor.class}.class;
+          case REPEATED:
+            return ValueVector.Repeated${minor.class}.class;
+        }
+    <#else>
+      case ${minor.class?upper_case}:
+        switch (mode) {
+          case REQUIRED:
+            return ValueVector.${minor.class}.class;
+          case OPTIONAL:
+            return ValueVector.Nullable${minor.class}.class;
+          case REPEATED:
+            return ValueVector.Repeated${minor.class}.class;
+        }
+    </#if>
+  </#list>
+</#list>
+    default:
+      break;
+    }
+    throw new UnsupportedOperationException();
+  }
+
+
+  public static ValueVector.Base getNewVector(MaterializedField field, BufferAllocator allocator){
+    MajorType type = field.getType();
+
+    switch (type.getMinorType()) {
+<#list types as type>
+  <#list type.minor as minor>
+    <#if minor.class != "Bit">
+      case ${minor.class?upper_case}:
+        switch (type.getMode()) {
+          case REQUIRED:
+            return new ValueVector.${minor.class}(field, allocator);
+          case OPTIONAL:
+            return new ValueVector.Nullable${minor.class}(field, allocator);
+          case REPEATED:
+            return new ValueVector.Repeated${minor.class}(field, allocator);
+        }
+    </#if>
+  </#list>
+</#list>
+      case BOOLEAN:
+        switch (type.getMode()) {
+          case REQUIRED:
+            return new ValueVector.Bit(field, allocator);
+          case OPTIONAL:
+            return new ValueVector.NullableBit(field, allocator);
+          case REPEATED:
+            return new ValueVector.RepeatedBit(field, allocator);
+        }
+    }
+    // All ValueVector types have been handled.
+    throw new UnsupportedOperationException(type.getMinorType() + " type is not supported. Mode: " + type.getMode());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/7075cca1/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/ValueVector.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/ValueVector.java b/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/ValueVector.java
new file mode 100644
index 0000000..e0e895b
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/ValueVector.java
@@ -0,0 +1,768 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.record.vector;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static com.google.common.base.Preconditions.checkState;
+import io.netty.buffer.ByteBuf;
+import java.io.Closeable;
+import java.util.Random;
+import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.proto.SchemaDefProtos;
+import org.apache.drill.exec.proto.UserBitShared.FieldMetadata;
+import org.apache.drill.exec.record.DeadBuf;
+import org.apache.drill.exec.record.MaterializedField;
+
+/**
+ * ValueVectorTypes defines a set of template-generated classes which implement type-specific
+ * value vectors.  The template approach was chosen due to the lack of multiple inheritence.  It
+ * is also important that all related logic be as efficient as possible.
+ */
+public class ValueVector {
+
+  /**
+   * ValueVector.Base implements common logic for all immutable value vectors.
+   */
+  public abstract static class Base implements Closeable {
+    static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(Base.class);
+
+    protected final BufferAllocator allocator;
+    protected ByteBuf data = DeadBuf.DEAD_BUFFER;
+    protected MaterializedField field;
+    protected int recordCount;
+    protected int totalBytes;
+
+    public Base(MaterializedField field, BufferAllocator allocator) {
+      this.allocator = allocator;
+      this.field = field;
+    }
+
+    /**
+     * Get the explicitly specified size of the allocated buffer, if available.  Otherwise
+     * calculate the size based on width and record count.
+     */
+    public abstract int getAllocatedSize();
+
+    /**
+     * Get the size requirement (in bytes) for the given number of values.  Takes derived
+     * type specs into account.
+     */
+    public abstract int getSizeFromCount(int valueCount);
+
+    /**
+     * Get the Java Object representation of the element at the specified position
+     *
+     * @param index   Index of the value to get
+     */
+    public abstract Object getObject(int index);
+
+    /**
+     * Return the underlying buffers associated with this vector. Note that this doesn't impact the
+     * reference counts for this buffer so it only should be used for in-context access. Also note
+     * that this buffer changes regularly thus external classes shouldn't hold a reference to
+     * it (unless they change it).
+     *
+     * @return The underlying ByteBuf.
+     */
+    public ByteBuf[] getBuffers() {
+      return new ByteBuf[]{data};
+    }
+
+    /**
+     * Returns the maximum number of values contained within this vector.
+     * @return Vector size
+     */
+    public int capacity() {
+      return getRecordCount();
+    }
+
+    /**
+     * Release supporting resources.
+     */
+    @Override
+    public void close() {
+      clear();
+    }
+
+    /**
+     * Get information about how this field is materialized.
+     * @return
+     */
+    public MaterializedField getField() {
+      return field;
+    }
+
+    /**
+     * Get the number of records allocated for this value vector.
+     * @return number of allocated records
+     */
+    public int getRecordCount() {
+      return recordCount;
+    }
+
+    /**
+     * Get the metadata for this field.
+     * @return
+     */
+    public FieldMetadata getMetadata() {
+      int len = 0;
+      for(ByteBuf b : getBuffers()){
+        len += b.writerIndex();
+      }
+      return FieldMetadata.newBuilder()
+               .setDef(getField().getDef())
+               .setValueCount(getRecordCount())
+               .setBufferLength(len)
+               .build();
+    }
+
+    /**
+     * Allocate a new memory space for this vector.  Must be called prior to using the ValueVector.
+     *
+     * @param totalBytes   Optional desired size of the underlying buffer.  Specifying 0 will
+     *                     estimate the size based on valueCount.
+     * @param sourceBuffer Optional ByteBuf to use for storage (null will allocate automatically).
+     * @param valueCount   Number of values in the vector.
+     */
+    public void allocateNew(int totalBytes, ByteBuf sourceBuffer, int valueCount) {
+      clear();
+      this.recordCount = valueCount;
+      this.totalBytes = totalBytes > 0 ? totalBytes : getSizeFromCount(valueCount);
+      this.data = (sourceBuffer != null) ? sourceBuffer : allocator.buffer(this.totalBytes);
+      this.data.retain();
+      data.readerIndex(0);
+    }
+
+    /**
+     * Allocate a new memory space for this vector.  Must be called prior to using the ValueVector.
+     *
+     * @param valueCount
+     *          The number of elements which can be contained within this vector.
+     */
+    public void allocateNew(int valueCount) {
+      allocateNew(0, null, valueCount);
+    }
+
+    /**
+     * Release the underlying ByteBuf and reset the ValueVector
+     */
+    protected void clear() {
+      if (data != DeadBuf.DEAD_BUFFER) {
+        data.release();
+        data = DeadBuf.DEAD_BUFFER;
+        recordCount = 0;
+        totalBytes = 0;
+      }
+    }
+
+    /**
+     * Define the number of records that are in this value vector.
+     * @param recordCount Number of records active in this vector.
+     */
+    public void setRecordCount(int recordCount) {
+      data.writerIndex(getSizeFromCount(recordCount));
+      this.recordCount = recordCount;
+    }
+
+    /**
+     * For testing only -- randomize the buffer contents
+     */
+    public void randomizeData() { }
+
+  }
+
+  /**
+   * Bit implements a vector of bit-width values.  Elements in the vector are accessed
+   * by position from the logical start of the vector.
+   *   The width of each element is 1 bit.
+   *   The equivilent Java primitive is an int containing the value '0' or '1'.
+   *
+   * NB: this class is automatically generated from ValueVectorTypes.tdd using FreeMarker.
+   */
+  public static class Bit extends Base {
+    static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(Bit.class);
+
+    public Bit(MaterializedField field, BufferAllocator allocator) {
+      super(field, allocator);
+    }
+
+    /**
+     * Get the byte holding the desired bit, then mask all other bits.  Iff the result is 0, the
+     * bit was not set.
+     *
+     * @param  index   position of the bit in the vector
+     * @return 1 if set, otherwise 0
+     */
+    public int get(int index) {
+      // logger.debug("BIT GET: index: {}, byte: {}, mask: {}, masked byte: {}",
+      //             index,
+      //             data.getByte((int)Math.floor(index/8)),
+      //             (int)Math.pow(2, (index % 8)),
+      //             data.getByte((int)Math.floor(index/8)) & (int)Math.pow(2, (index % 8)));
+      return ((data.getByte((int)Math.floor(index/8)) & (int)Math.pow(2, (index % 8))) == 0) ? 0 : 1;
+    }
+
+    @Override
+    public Object getObject(int index) {
+      return new Boolean(get(index) != 0);
+    }
+
+    /**
+     * Get the size requirement (in bytes) for the given number of values.
+     */
+    @Override
+    public int getSizeFromCount(int valueCount) {
+      return (int) Math.ceil(valueCount / 8);
+    }
+
+    @Override
+    public int getAllocatedSize() {
+      return totalBytes;
+    }
+
+    public MutableBit getMutable() {
+      return (MutableBit)this;
+    }
+
+    /**
+     * Allocate a new memory space for this vector.  Must be called prior to using the ValueVector.
+     *
+     * @param valueCount  The number of values which can be contained within this vector.
+     */
+    @Override
+    public void allocateNew(int valueCount) {
+      allocateNew(getSizeFromCount(valueCount), null, valueCount);
+      for (int i = 0; i < getSizeFromCount(valueCount); i++) {
+        data.setByte(i, 0);
+      }
+    }
+
+  }
+
+  /**
+   * MutableBit implements a vector of bit-width values.  Elements in the vector are accessed
+   * by position from the logical start of the vector.  Values should be pushed onto the vector
+   * sequentially, but may be randomly accessed.
+   *
+   * NB: this class is automatically generated from ValueVectorTypes.tdd using FreeMarker.
+   */
+  public static class MutableBit extends Bit {
+
+    public MutableBit(MaterializedField field, BufferAllocator allocator) {
+      super(field, allocator);
+    }
+
+    /**
+     * Set the bit at the given index to the specified value.
+     *
+     * @param index   position of the bit to set
+     * @param value   value to set (either 1 or 0)
+     */
+    public void set(int index, int value) {
+      byte currentByte = data.getByte((int)Math.floor(index/8));
+      if (value != 0) {
+        // true
+        currentByte |= (byte) Math.pow(2, (index % 8));
+      }
+      else if ((currentByte & (byte) Math.pow(2, (index % 8))) == (byte) Math.pow(2, (index % 8))) {
+        // false, and bit was previously set
+        currentByte -= (byte) Math.pow(2, (index % 8));
+      }
+      data.setByte((int) Math.floor(index/8), currentByte);
+    }
+
+    @Override
+    public void randomizeData() {
+      if (this.data != DeadBuf.DEAD_BUFFER) {
+        Random r = new Random();
+        for (int i = 0; i < data.capacity() - 1; i++) {
+          byte[] bytes = new byte[1];
+          r.nextBytes(bytes);
+          data.setByte(i, bytes[0]);
+        }
+      }
+    }
+  }
+
+<#list types as type>
+ <#list type.minor as minor>
+  <#if type.major == "Fixed">
+
+  /**
+   * ${minor.class} implements a vector of fixed width values.  Elements in the vector are accessed
+   * by position, starting from the logical start of the vector.  Values should be pushed onto the
+   * vector sequentially, but may be randomly accessed.
+   *   The width of each element is ${type.width} byte(s)
+   *   The equivilent Java primitive is '${minor.javaType!type.javaType}'
+   *
+   * NB: this class is automatically generated from ValueVectorTypes.tdd using FreeMarker.
+   */
+  public static class ${minor.class} extends Base {
+    static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(${minor.class}.class);
+
+    public ${minor.class}(MaterializedField field, BufferAllocator allocator) {
+      super(field, allocator);
+    }
+
+    /**
+     * Allocate a new memory space for this vector.  Must be called prior to using the ValueVector.
+     *
+     * @param valueCount
+     *          The number of values which can be contained within this vector.
+     */
+    public void allocateNew(int valueCount) {
+      totalBytes = valueCount * ${type.width};
+      allocateNew(totalBytes, allocator.buffer(totalBytes), valueCount);
+    }
+
+    @Override
+    public int getAllocatedSize() {
+      return (int) Math.ceil(totalBytes);
+    }
+
+    /**
+     * Get the size requirement (in bytes) for the given number of values.  Only accurate
+     * for fixed width value vectors.
+     */
+    @Override
+    public int getSizeFromCount(int valueCount) {
+      return valueCount * ${type.width};
+    }
+
+    public Mutable${minor.class} getMutable() {
+      return (Mutable${minor.class})this;
+    }
+
+   <#if (type.width > 8)>
+
+    public ${minor.javaType!type.javaType} get(int index) {
+      ByteBuf dst = allocator.buffer(${type.width});
+      data.getBytes(index * ${type.width}, dst, 0, ${type.width});
+      return dst;
+    }
+
+    @Override
+    public Object getObject(int index) {
+      ByteBuf dst = allocator.buffer(${type.width});
+      data.getBytes(index, dst, 0, ${type.width});
+      return dst;
+    }
+
+    @Override
+    public void randomizeData() {
+      if (this.data != DeadBuf.DEAD_BUFFER) {
+        Random r = new Random();
+        for(int i =0; i < data.capacity()-${type.width}; i += ${type.width}){
+          byte[] bytes = new byte[${type.width}];
+          r.nextBytes(bytes);
+          data.setByte(i, bytes[0]);
+        }
+      }
+    }
+
+   <#else> <#-- type.width <= 8 -->
+
+    public ${minor.javaType!type.javaType} get(int index) {
+      return data.get${(minor.javaType!type.javaType)?cap_first}(index * ${type.width});
+    }
+
+    public Object getObject(int index) {
+      return get(index);
+    }
+
+    @Override
+    public void randomizeData() {
+      if (this.data != DeadBuf.DEAD_BUFFER) {
+        Random r = new Random();
+        for(int i =0; i < data.capacity()-${type.width}; i += ${type.width}){
+          data.set${(minor.javaType!type.javaType)?cap_first}(i,
+              r.next<#if (type.width >= 4)>${(minor.javaType!type.javaType)?cap_first}
+                    <#else>Int
+                    </#if>());
+        }
+      }
+    }
+
+   </#if> <#-- type.width -->
+  }
+
+  /**
+   * Mutable${minor.class} implements a mutable vector of fixed width values.  Elements in the
+   * vector are accessed by position from the logical start of the vector.  Values should be pushed
+   * onto the vector sequentially, but may be randomly accessed.
+   *   The width of each element is ${type.width} byte(s)
+   *   The equivilent Java primitive is '${minor.javaType!type.javaType}'
+   *
+   * NB: this class is automatically generated from ValueVectorTypes.tdd using FreeMarker.
+   */
+   public static class Mutable${minor.class} extends ${minor.class} {
+
+    public Mutable${minor.class}(MaterializedField field, BufferAllocator allocator) {
+      super(field, allocator);
+    }
+
+    /**
+     * Set the element at the given index to the given value.  Note that widths smaller than
+     * 32 bits are handled by the ByteBuf interface.
+     *
+     * @param index   position of the bit to set
+     * @param value   value to set
+     */
+   <#if (type.width > 8)>
+    public void set(int index, <#if (type.width > 4)>${minor.javaType!type.javaType}<#else>int</#if> value) {
+      data.setBytes(index * ${type.width}, value);
+   <#else> <#-- type.width <= 8 -->
+    public void set(int index, <#if (type.width >= 4)>${minor.javaType!type.javaType}<#else>int</#if> value) {
+      data.set${(minor.javaType!type.javaType)?cap_first}(index * ${type.width}, value);
+   </#if> <#-- type.width -->
+    }
+  }
+
+  <#elseif type.major == "VarLen">
+
+  /**
+   * ${minor.class} implements a vector of variable width values.  Elements in the vector
+   * are accessed by position from the logical start of the vector.  A fixed width lengthVector
+   * is used to convert an element's position to it's offset from the start of the (0-based)
+   * ByteBuf.  Size is inferred by adjacent elements.
+   *   The width of each element is ${type.width} byte(s)
+   *   The equivilent Java primitive is '${minor.javaType!type.javaType}'
+   *
+   * NB: this class is automatically generated from ValueVectorTypes.tdd using FreeMarker.
+   */
+  public static class ${minor.class} extends Base {
+    static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(${minor.class}.class);
+
+    protected final MutableUInt${type.width} lengthVector;
+
+    public ${minor.class}(MaterializedField field, BufferAllocator allocator) {
+      super(field, allocator);
+      this.lengthVector = new MutableUInt${type.width}(null, allocator);
+    }
+
+    public byte[] get(int index) {
+      checkArgument(index >= 0);
+      int startIdx = 0;
+      int size = 0;
+      if (index == 0) {
+        size = lengthVector.get(1);
+      } else {
+        startIdx = lengthVector.get(index);
+        size = lengthVector.get(index + 1) - startIdx;
+      }
+      checkState(size >= 0);
+      byte[] dst = new byte[size];
+      data.getBytes(startIdx, dst, 0, size);
+      return dst;
+    }
+
+    @Override
+    public int getAllocatedSize() {
+      return lengthVector.getAllocatedSize() + totalBytes;
+    }
+
+    /**
+     * Get the size requirement (in bytes) for the given number of values.  Only accurate
+     * for fixed width value vectors.
+     */
+    public int getSizeFromCount(int valueCount) {
+      return valueCount * ${type.width};
+    }
+
+    @Override
+    protected void clear() {
+      super.clear();
+      lengthVector.clear();
+    }
+
+    /**
+     * Allocate a new memory space for this vector.  Must be called prior to using the ValueVector.
+     *
+     * @param valueCount
+     *          The number of values which can be contained within this vector.
+     */
+    public void allocateNew(int totalBytes, ByteBuf sourceBuffer, int valueCount) {
+      super.allocateNew(totalBytes, sourceBuffer, valueCount);
+      lengthVector.allocateNew(valueCount);
+    }
+
+    @Override
+    public ByteBuf[] getBuffers() {
+      return new ByteBuf[]{lengthVector.data, data};
+    }
+
+    public Object getObject(int index) {
+      return get(index);
+    }
+
+    public Mutable${minor.class} getMutable() {
+      return (Mutable${minor.class})this;
+    }
+  }
+
+  /**
+   * Mutable${minor.class} implements a vector of variable width values.  Elements in the vector
+   * are accessed by position from the logical start of the vector.  A fixed width lengthVector
+   * is used to convert an element's position to it's offset from the start of the (0-based)
+   * ByteBuf.  Size is inferred by adjacent elements.
+   *   The width of each element is ${type.width} byte(s)
+   *   The equivilent Java primitive is '${minor.javaType!type.javaType}'
+   *
+   * NB: this class is automatically generated from ValueVectorTypes.tdd using FreeMarker.
+   */
+  public static class Mutable${minor.class} extends ${minor.class} {
+
+    public Mutable${minor.class}(MaterializedField field, BufferAllocator allocator) {
+      super(field, allocator);
+    }
+
+    /**
+     * Set the variable length element at the specified index to the supplied byte array.
+     *
+     * @param index   position of the bit to set
+     * @param bytes   array of bytes to write
+     */
+    public void set(int index, byte[] bytes) {
+      checkArgument(index >= 0);
+      if (index == 0) {
+        lengthVector.set(0, 0);
+        lengthVector.set(1, bytes.length);
+        data.setBytes(0, bytes);
+      }
+      else {
+        int currentOffset = lengthVector.get(index);
+        // set the end offset of the buffer
+        lengthVector.set(index + 1, currentOffset + bytes.length);
+        data.setBytes(currentOffset, bytes);
+      }
+    }
+
+    @Override
+    public void setRecordCount(int recordCount) {
+      super.setRecordCount(recordCount);
+      lengthVector.setRecordCount(recordCount);
+    }
+
+  }
+
+  </#if> <#-- type.major -->
+
+  /**
+   * Nullable${minor.class} implements a vector of values which could be null.  Elements in the vector
+   * are first checked against a fixed length vector of boolean values.  Then the element is retrieved
+   * from the base class (if not null).
+   *
+   * NB: this class is automatically generated from ValueVectorTypes.tdd using FreeMarker.
+   */
+  public static class Nullable${minor.class} extends Mutable${minor.class} {
+
+    protected MutableBit bits;
+
+    public Nullable${minor.class}(MaterializedField field, BufferAllocator allocator) {
+      super(field, allocator);
+      bits = new MutableBit(null, allocator);
+    }
+
+    /**
+     * Set the variable length element at the specified index to the supplied byte array.
+     *
+     * @param index   position of the bit to set
+     * @param bytes   array of bytes to write
+     */
+    public void set(int index, <#if type.major == "VarLen">byte[]<#elseif (type.width < 4)>int<#else>${minor.javaType!type.javaType}</#if> value) {
+      setNotNull(index);
+      super.set(index, value);
+    }
+
+    /**
+     * Get the element at the specified position.
+     *
+     * @param   index   position of the value
+     * @return  value of the element, if not null
+     * @throws  NullValueException if the value is null
+     */
+    public <#if type.major == "VarLen">byte[]<#else>${minor.javaType!type.javaType}</#if> get(int index) {
+      if (isNull(index))
+        throw new NullValueException(index);
+      return super.get(index);
+    }
+
+    public void setNull(int index) {
+      bits.set(index, 0);
+    }
+
+    private void setNotNull(int index) {
+      bits.set(index, 1);
+    }
+
+    public boolean isNull(int index) {
+      return bits.get(index) == 0;
+    }
+
+    /**
+     * Allocate a new memory space for this vector.  Must be called prior to using the ValueVector.
+     *
+     * @param valueCount   The number of values which may be contained by this vector.
+     */
+    public void allocateNew(int totalBytes, ByteBuf sourceBuffer, int valueCount) {
+      super.allocateNew(totalBytes, sourceBuffer, valueCount);
+      bits.allocateNew(valueCount);
+    }
+
+    @Override
+    public int getAllocatedSize() {
+      return bits.getAllocatedSize() + super.getAllocatedSize();
+    }
+
+    /**
+     * Get the size requirement (in bytes) for the given number of values.  Only accurate
+     * for fixed width value vectors.
+     */
+    public int getTotalSizeFromCount(int valueCount) {
+      return getSizeFromCount(valueCount) + bits.getSizeFromCount(valueCount);
+    }
+
+    @Override
+    public MaterializedField getField() {
+      return field;
+    }
+
+    @Override
+    public ByteBuf[] getBuffers() {
+      return new ByteBuf[]{bits.data, super.data};
+    }
+
+    @Override
+    public void setRecordCount(int recordCount) {
+      super.setRecordCount(recordCount);
+      bits.setRecordCount(recordCount);
+    }
+
+    @Override
+    public Object getObject(int index) {
+      return isNull(index) ? null : super.getObject(index);
+    }
+  }
+
+  /**
+   * Repeated${minor.class} implements a vector with multple values per row (e.g. JSON array or
+   * repeated protobuf field).  The implementation uses two additional value vectors; one to convert
+   * the index offset to the underlying element offset, and another to store the number of values
+   * in the vector.
+   *
+   * NB: this class is automatically generated from ValueVectorTypes.tdd using FreeMarker.
+   */
+   public static class Repeated${minor.class} extends Mutable${minor.class} {
+
+    private MutableUInt4 countVector;    // number of repeated elements in each record
+    private MutableUInt4 offsetVector;   // offsets to start of each record
+
+    public Repeated${minor.class}(MaterializedField field, BufferAllocator allocator) {
+      super(field, allocator);
+      countVector = new MutableUInt4(null, allocator);
+      offsetVector = new MutableUInt4(null, allocator);
+    }
+
+    public void allocateNew(int totalBytes, ByteBuf sourceBuffer, int valueCount) {
+      super.allocateNew(totalBytes, sourceBuffer, valueCount);
+      countVector.allocateNew(valueCount);
+      offsetVector.allocateNew(valueCount);
+    }
+
+    /**
+     * Add an element to the given record index.  This is similar to the set() method in other
+     * value vectors, except that it permits setting multiple values for a single record.
+     *
+     * @param index   record of the element to add
+     * @param value   value to add to the given row
+     */
+    public void add(int index, <#if (type.width > 4)> ${minor.javaType!type.javaType}
+                               <#elseif type.major == "VarLen"> byte[]
+                               <#else> int
+                               </#if> value) {
+      countVector.set(index, countVector.get(index) + 1);
+      offsetVector.set(index, offsetVector.get(index - 1) + countVector.get(index-1));
+      super.set(offsetVector.get(index), value);
+    }
+
+    /**
+     * Get a value for the given record.  Each element in the repeated field is accessed by
+     * the positionIndex param.
+     *
+     * @param  index           record containing the repeated field
+     * @param  positionIndex   position within the repeated field
+     * @return element at the given position in the given record
+     */
+    public <#if type.major == "VarLen">byte[]
+           <#else>${minor.javaType!type.javaType}
+           </#if> get(int index, int positionIndex) {
+
+      assert positionIndex < countVector.get(index);
+      return super.get(offsetVector.get(index) + positionIndex);
+    }
+
+    public MaterializedField getField() {
+      return field;
+    }
+
+    /**
+     * Get the size requirement (in bytes) for the given number of values.  Only accurate
+     * for fixed width value vectors.
+     */
+    public int getTotalSizeFromCount(int valueCount) {
+      return getSizeFromCount(valueCount) +
+             countVector.getSizeFromCount(valueCount) +
+             offsetVector.getSizeFromCount(valueCount);
+    }
+
+    /**
+     * Get the explicitly specified size of the allocated buffer, if available.  Otherwise
+     * calculate the size based on width and record count.
+     */
+    public int getAllocatedSize() {
+      return super.getAllocatedSize() +
+             countVector.getAllocatedSize() +
+             offsetVector.getAllocatedSize();
+    }
+
+    /**
+     * Get the elements at the given index.
+     */
+    public int getCount(int index) {
+      return countVector.get(index);
+    }
+
+    public void setRecordCount(int recordCount) {
+      super.setRecordCount(recordCount);
+      offsetVector.setRecordCount(recordCount);
+      countVector.setRecordCount(recordCount);
+    }
+
+    public ByteBuf[] getBuffers() {
+      return new ByteBuf[]{countVector.data, offsetVector.data, data};
+    }
+
+    public Object getObject(int index) {
+      return data.slice(index, getSizeFromCount(countVector.get(index)));
+    }
+
+  }
+ </#list>
+</#list>
+}
+

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/7075cca1/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockRecordReader.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockRecordReader.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockRecordReader.java
index 6a1eba4..65584db 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockRecordReader.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockRecordReader.java
@@ -37,7 +37,7 @@ public class MockRecordReader implements RecordReader {
   private OutputMutator output;
   private MockScanEntry config;
   private FragmentContext context;
-  private ValueVector<?>[] valueVectors;
+  private ValueVector.Base[] valueVectors;
   private int recordsRead;
 
   public MockRecordReader(FragmentContext context, MockScanEntry config) {
@@ -53,16 +53,14 @@ public class MockRecordReader implements RecordReader {
     return x;
   }
 
-  private ValueVector<?> getVector(int fieldId, String name, MajorType type, int length) {
+  private ValueVector.Base getVector(int fieldId, String name, MajorType type, int length) {
     assert context != null : "Context shouldn't be null.";
-    
     if(type.getMode() != DataMode.REQUIRED) throw new UnsupportedOperationException();
     
     MaterializedField f = MaterializedField.create(new SchemaPath(name), fieldId, 0, type);
-    ValueVector<?> v;
+    ValueVector.Base v;
     v = TypeHelper.getNewVector(f, context.getAllocator());
     v.allocateNew(length);
-    
     return v;
 
   }
@@ -72,7 +70,7 @@ public class MockRecordReader implements RecordReader {
     try {
       this.output = output;
       int estimateRowSize = getEstimatedRecordSize(config.getTypes());
-      valueVectors = new ValueVector<?>[config.getTypes().length];
+      valueVectors = new ValueVector.Base[config.getTypes().length];
       int batchRecordCount = 250000 / estimateRowSize;
 
       for (int i = 0; i < config.getTypes().length; i++) {
@@ -90,7 +88,8 @@ public class MockRecordReader implements RecordReader {
   public int next() {
     int recordSetSize = Math.min(valueVectors[0].capacity(), this.config.getRecords()- recordsRead);
     recordsRead += recordSetSize;
-    for(ValueVector<?> v : valueVectors){
+    for(ValueVector.Base v : valueVectors){
+      logger.debug("MockRecordReader:  Generating random data for VV of type " + v.getClass().getName());
       v.randomizeData();
       v.setRecordCount(recordSetSize);
     }
@@ -103,7 +102,7 @@ public class MockRecordReader implements RecordReader {
       try {
         output.removeField(valueVectors[i].getField().getFieldId());
       } catch (SchemaChangeException e) {
-        logger.warn("Failure while trying tremove field.", e);
+        logger.warn("Failure while trying to remove field.", e);
       }
       valueVectors[i].close();
     }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/7075cca1/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/FilterRecordBatch.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/FilterRecordBatch.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/FilterRecordBatch.java
index 6592ca1..b7b7d93 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/FilterRecordBatch.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/FilterRecordBatch.java
@@ -58,7 +58,7 @@ public abstract class FilterRecordBatch implements RecordBatch {
   }
 
   @Override
-  public <T extends ValueVector<T>> T getValueVector(int fieldId, Class<T> clazz) throws InvalidValueAccessor {
+  public <T extends ValueVector.Base> T getValueVector(int fieldId, Class<T> clazz) throws InvalidValueAccessor {
     return null;
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/7075cca1/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/OutputMutator.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/OutputMutator.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/OutputMutator.java
index ce0cf66..e96c1be 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/OutputMutator.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/OutputMutator.java
@@ -23,6 +23,6 @@ import org.apache.drill.exec.record.vector.ValueVector;
 
 public interface OutputMutator {
   public void removeField(int fieldId) throws SchemaChangeException;
-  public void addField(int fieldId, ValueVector<?> vector) throws SchemaChangeException ;
+  public void addField(int fieldId, ValueVector.Base vector) throws SchemaChangeException ;
   public void setNewSchema() throws SchemaChangeException ;
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/7075cca1/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScanBatch.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScanBatch.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScanBatch.java
index 33c1e29..822d828 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScanBatch.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScanBatch.java
@@ -45,7 +45,7 @@ import com.google.common.collect.Lists;
 public class ScanBatch implements RecordBatch {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ScanBatch.class);
 
-  private IntObjectOpenHashMap<ValueVector<?>> fields = new IntObjectOpenHashMap<ValueVector<?>>();
+  private IntObjectOpenHashMap<ValueVector.Base> fields = new IntObjectOpenHashMap<ValueVector.Base>();
   private BatchSchema schema;
   private int recordCount;
   private boolean schemaChanged = true;
@@ -89,9 +89,9 @@ public class ScanBatch implements RecordBatch {
   }
 
   private void releaseAssets() {
-    fields.forEach(new IntObjectProcedure<ValueVector<?>>() {
+    fields.forEach(new IntObjectProcedure<ValueVector.Base>() {
       @Override
-      public void apply(int key, ValueVector<?> value) {
+      public void apply(int key, ValueVector.Base value) {
         value.close();
       }
     });
@@ -99,9 +99,9 @@ public class ScanBatch implements RecordBatch {
 
   @SuppressWarnings("unchecked")
   @Override
-  public <T extends ValueVector<T>> T getValueVector(int fieldId, Class<T> clazz) throws InvalidValueAccessor {
+  public <T extends ValueVector.Base> T getValueVector(int fieldId, Class<T> clazz) throws InvalidValueAccessor {
     if (fields.containsKey(fieldId)) throw new InvalidValueAccessor(String.format("Unknown value accesor for field id %d."));
-    ValueVector<?> vector = this.fields.lget();
+    ValueVector.Base vector = this.fields.lget();
     if (vector.getClass().isAssignableFrom(clazz)) {
       return (T) vector;
     } else {
@@ -143,14 +143,14 @@ public class ScanBatch implements RecordBatch {
     
     public void removeField(int fieldId) throws SchemaChangeException {
       schemaChanged();
-      ValueVector<?> v = fields.remove(fieldId);
+      ValueVector.Base v = fields.remove(fieldId);
       if (v == null) throw new SchemaChangeException("Failure attempting to remove an unknown field.");
       v.close();
     }
 
-    public void addField(int fieldId, ValueVector<?> vector) {
+    public void addField(int fieldId, ValueVector.Base vector) {
       schemaChanged();
-      ValueVector<?> v = fields.put(fieldId, vector);
+      ValueVector.Base v = fields.put(fieldId, vector);
       vector.getField();
       builder.addField(vector.getField());
       if (v != null) v.close();

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/7075cca1/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 c20538d..3819036 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
@@ -73,7 +73,7 @@ public class ScreenCreator implements RootCreator<Screen>{
         stop();
         return false;
       }
-      
+
       IterOutcome outcome = incoming.next();
       logger.debug("Screen Outcome {}", outcome);
       switch(outcome){

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/7075cca1/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/SingleSenderCreator.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/SingleSenderCreator.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/SingleSenderCreator.java
index b7d4c7e..a2b9865 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/SingleSenderCreator.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/SingleSenderCreator.java
@@ -103,7 +103,7 @@ public class SingleSenderCreator implements RootCreator<SingleSender>{
       @Override
       public void success(Ack value) {
         if(value.getOk()) return;
-        
+
         logger.error("Downstream fragment was not accepted.  Stopping future sends.");
         // if we didn't get ack ok, we'll need to kill the query.
         context.fail(new RpcException("A downstream fragment batch wasn't accepted.  This fragment thus fails."));

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/7075cca1/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/WireRecordBatch.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/WireRecordBatch.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/WireRecordBatch.java
index b41b0cd..be32d1f 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/WireRecordBatch.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/WireRecordBatch.java
@@ -68,7 +68,7 @@ public class WireRecordBatch implements RecordBatch{
   }
 
   @Override
-  public <T extends ValueVector<T>> T getValueVector(int fieldId, Class<T> clazz) throws InvalidValueAccessor {
+  public <T extends ValueVector.Base> T getValueVector(int fieldId, Class<T> clazz) throws InvalidValueAccessor {
     return batchLoader.getValueVector(fieldId, clazz);
   }
 
@@ -76,7 +76,7 @@ public class WireRecordBatch implements RecordBatch{
   public IterOutcome next() {
     RawFragmentBatch batch = fragProvider.getNext();
     try{
-      if(batch == null) return IterOutcome.NONE;
+      if (batch == null) return IterOutcome.NONE;
 
       logger.debug("Next received batch {}", batch);
 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/7075cca1/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/MaterializedField.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/MaterializedField.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/MaterializedField.java
index 09427ef..d1858f1 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/MaterializedField.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/MaterializedField.java
@@ -28,7 +28,6 @@ import org.apache.drill.exec.proto.SchemaDefProtos.FieldDef;
 import org.apache.drill.exec.proto.SchemaDefProtos.MajorType;
 import org.apache.drill.exec.proto.SchemaDefProtos.NamePart;
 import org.apache.drill.exec.proto.SchemaDefProtos.NamePart.Type;
-import org.apache.drill.exec.record.vector.TypeHelper;
 
 public class MaterializedField implements Comparable<MaterializedField> {
   private final FieldDef def;
@@ -107,10 +106,6 @@ public class MaterializedField implements Comparable<MaterializedField> {
     return def.getMajorType().getMode();
   }
 
-  public Class<?> getValueClass() {
-    return TypeHelper.getValueVectorClass(getType().getMinorType(), getDataMode());
-  }
-
   public boolean matches(SchemaPath path) {
     Iterator<NamePart> iter = def.getNameList().iterator();
     

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/7075cca1/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatch.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatch.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatch.java
index 3e4ded2..042c40c 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatch.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatch.java
@@ -68,7 +68,7 @@ public interface RecordBatch {
   public void kill();
 
 
-  public abstract <T extends ValueVector<T>> T getValueVector(int fieldId, Class<T> clazz) throws InvalidValueAccessor;
+  public abstract <T extends ValueVector.Base> T getValueVector(int fieldId, Class<T> clazz) throws InvalidValueAccessor;
 
 //  public abstract void getDictReader(int fieldId, Class<?> clazz) throws InvalidValueAccessor;
 //

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/7075cca1/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatchLoader.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatchLoader.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatchLoader.java
index d990198..ea1de73 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatchLoader.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatchLoader.java
@@ -33,10 +33,10 @@ import org.apache.drill.exec.record.vector.ValueVector;
 import com.carrotsearch.hppc.IntObjectOpenHashMap;
 import com.carrotsearch.hppc.cursors.IntObjectCursor;
 
-public class RecordBatchLoader implements Iterable<IntObjectCursor<ValueVector<?>>>{
+public class RecordBatchLoader implements Iterable<IntObjectCursor<ValueVector.Base>>{
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(RecordBatchLoader.class);
 
-  private IntObjectOpenHashMap<ValueVector<?>> vectors = new IntObjectOpenHashMap<ValueVector<?>>();
+  private IntObjectOpenHashMap<ValueVector.Base> vectors = new IntObjectOpenHashMap<ValueVector.Base>();
   private final BufferAllocator allocator;
   private int recordCount; 
   private BatchSchema schema;
@@ -61,17 +61,17 @@ public class RecordBatchLoader implements Iterable<IntObjectCursor<ValueVector<?
     this.recordCount = def.getRecordCount();
     boolean schemaChanged = false;
     
-    IntObjectOpenHashMap<ValueVector<?>> newVectors = new IntObjectOpenHashMap<ValueVector<?>>();
+    IntObjectOpenHashMap<ValueVector.Base> newVectors = new IntObjectOpenHashMap<ValueVector.Base>();
 
     List<FieldMetadata> fields = def.getFieldList();
     
     int bufOffset = 0;
     for (FieldMetadata fmd : fields) {
       FieldDef fieldDef = fmd.getDef();
-      ValueVector<?> v = vectors.remove(fieldDef.getFieldId());
+      ValueVector.Base v = vectors.remove(fieldDef.getFieldId());
       if (v != null) {
         if (v.getField().getDef().equals(fieldDef)) {
-          v.setTo(fmd, buf.slice(bufOffset, fmd.getBufferLength()));
+          v.allocateNew(fmd.getBufferLength(), buf.slice(bufOffset, fmd.getBufferLength()), recordCount);
           newVectors.put(fieldDef.getFieldId(), v);
           continue;
         } else {
@@ -83,13 +83,13 @@ public class RecordBatchLoader implements Iterable<IntObjectCursor<ValueVector<?
       schemaChanged = true;
       MaterializedField m = new MaterializedField(fieldDef);
       v = TypeHelper.getNewVector(m, allocator);
-      v.setTo(fmd, buf.slice(bufOffset, fmd.getBufferLength()));
+      v.allocateNew(fmd.getBufferLength(), buf.slice(bufOffset, fmd.getBufferLength()), recordCount);
       newVectors.put(fieldDef.getFieldId(), v);
     }
     
     if(!vectors.isEmpty()){
       schemaChanged = true;
-      for(IntObjectCursor<ValueVector<?>> cursor : newVectors){
+      for(IntObjectCursor<ValueVector.Base> cursor : newVectors){
         cursor.value.close();
       }
       
@@ -98,7 +98,7 @@ public class RecordBatchLoader implements Iterable<IntObjectCursor<ValueVector<?
     if(schemaChanged){
       // rebuild the schema.
       SchemaBuilder b = BatchSchema.newBuilder();
-      for(IntObjectCursor<ValueVector<?>> cursor : newVectors){
+      for(IntObjectCursor<ValueVector.Base> cursor : newVectors){
         b.addField(cursor.value.getField());
       }
       b.setSelectionVector(false);
@@ -110,8 +110,8 @@ public class RecordBatchLoader implements Iterable<IntObjectCursor<ValueVector<?
   }
 
   @SuppressWarnings("unchecked")
-  public <T extends ValueVector<T>> T getValueVector(int fieldId, Class<T> clazz) throws InvalidValueAccessor {
-    ValueVector<?> v = vectors.get(fieldId);
+  public <T extends ValueVector.Base> T getValueVector(int fieldId, Class<T> clazz) throws InvalidValueAccessor {
+    ValueVector.Base v = vectors.get(fieldId);
     assert v != null;
     if (v.getClass() != clazz)
       throw new InvalidValueAccessor(String.format(
@@ -130,7 +130,7 @@ public class RecordBatchLoader implements Iterable<IntObjectCursor<ValueVector<?
   }
 
   @Override
-  public Iterator<IntObjectCursor<ValueVector<?>>> iterator() {
+  public Iterator<IntObjectCursor<ValueVector.Base>> iterator() {
     return this.vectors.iterator();
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/7075cca1/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/WritableBatch.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/WritableBatch.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/WritableBatch.java
index 788c731..67c6cb9 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/WritableBatch.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/WritableBatch.java
@@ -58,7 +58,7 @@ public class WritableBatch {
     return buffers;
   }
   
-//  public static WritableBatch get(ValueVector<?>[] vectors){
+//  public static WritableBatch get(ValueVector.Base[] vectors){
 //    WritableCreator c = new WritableCreator();
 //    for(int i =0; i < vectors.length; i++){
 //      c.apply(i, vectors[i]);
@@ -67,14 +67,14 @@ public class WritableBatch {
 //  }
 //  
   
-  public static WritableBatch get(int recordCount, IntObjectOpenHashMap<ValueVector<?>> fields){
+  public static WritableBatch get(int recordCount, IntObjectOpenHashMap<ValueVector.Base> fields){
     WritableCreator creator = new WritableCreator(recordCount);
     fields.forEach(creator);
     return creator.get();
     
   }
   
-  private static class WritableCreator implements IntObjectProcedure<ValueVector<?>>{
+  private static class WritableCreator implements IntObjectProcedure<ValueVector.Base>{
     
     List<ByteBuf> buffers = Lists.newArrayList();
     List<FieldMetadata> metadata = Lists.newArrayList();
@@ -87,7 +87,7 @@ public class WritableBatch {
     }
     
     @Override
-    public void apply(int key, ValueVector<?> value) {
+    public void apply(int key, ValueVector.Base value) {
       metadata.add(value.getMetadata());
       for(ByteBuf b : value.getBuffers()){
         buffers.add(b);
@@ -97,7 +97,6 @@ public class WritableBatch {
       value.allocateNew(value.capacity());
     }
 
-
     public WritableBatch get(){
       RecordBatchDef batchDef = RecordBatchDef.newBuilder().addAllField(metadata).setRecordCount(recordCount).build();
       WritableBatch b = new WritableBatch(batchDef, buffers);

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/7075cca1/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/AbstractFixedValueVector.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/AbstractFixedValueVector.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/AbstractFixedValueVector.java
deleted file mode 100644
index 8748285..0000000
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/AbstractFixedValueVector.java
+++ /dev/null
@@ -1,77 +0,0 @@
-/*******************************************************************************
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- ******************************************************************************/
-package org.apache.drill.exec.record.vector;
-
-import io.netty.buffer.ByteBuf;
-
-import org.apache.drill.exec.memory.BufferAllocator;
-import org.apache.drill.exec.proto.UserBitShared.FieldMetadata;
-import org.apache.drill.exec.record.MaterializedField;
-
-/**
- * Abstract class that fixed value vectors are derived from.
- */
-abstract class AbstractFixedValueVector<T extends AbstractFixedValueVector<T>> extends BaseValueVector<T> {
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(AbstractFixedValueVector.class);
-
-  protected final int widthInBits;
-
-  protected int longWords = 0;
-  
-  public AbstractFixedValueVector(MaterializedField field, BufferAllocator allocator, int widthInBits) {
-    super(field, allocator);
-    this.widthInBits = widthInBits;
-  }
-
-    public int getWidthInBits() {
-        return widthInBits;
-    }
-
-    @Override
-  protected int getAllocationSize(int valueCount) {
-    return (int) Math.ceil(valueCount*widthInBits*1.0/8);
-  }
-  
-  @Override
-  protected void childResetAllocation(int valueCount, ByteBuf buf) {
-    this.longWords = valueCount/8;
-  }
-
-  @Override
-  protected void childCloneMetadata(T other) {
-    other.longWords = this.longWords;
-  }
-
-  @Override
-  protected void childClear() {
-    longWords = 0;
-  }
-
-  @Override
-  public void setRecordCount(int recordCount) {
-    this.data.writerIndex(recordCount*(widthInBits/8));
-    super.setRecordCount(recordCount);
-  }
-
-
-
-
-
-  
-}
-

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/7075cca1/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/AnyVector.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/AnyVector.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/AnyVector.java
deleted file mode 100644
index 6becfcd..0000000
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/AnyVector.java
+++ /dev/null
@@ -1,30 +0,0 @@
-/*******************************************************************************
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- ******************************************************************************/
-package org.apache.drill.exec.record.vector;
-
-/**
- * A Vector that holds each value with type information. In the case of complex types, an any vector will inline the
- * complex type within the value space.  The complex value will be stored in 
- */
-public class AnyVector {
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(AnyVector.class);
-
-  
-   
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/7075cca1/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/BaseValueVector.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/BaseValueVector.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/BaseValueVector.java
deleted file mode 100644
index 69cd628..0000000
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/BaseValueVector.java
+++ /dev/null
@@ -1,157 +0,0 @@
-/*******************************************************************************
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- ******************************************************************************/
-package org.apache.drill.exec.record.vector;
-
-import io.netty.buffer.ByteBuf;
-
-import java.util.Random;
-
-import org.apache.drill.exec.memory.BufferAllocator;
-import org.apache.drill.exec.proto.UserBitShared.FieldMetadata;
-import org.apache.drill.exec.record.DeadBuf;
-import org.apache.drill.exec.record.MaterializedField;
-
-public abstract class BaseValueVector<T extends BaseValueVector<T>> implements ValueVector<T>{
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(BaseValueVector.class);
-  
-  protected final BufferAllocator allocator;
-  protected ByteBuf data = DeadBuf.DEAD_BUFFER;
-  protected int maxValueCount = 0;
-  protected final MaterializedField field;
-  private int recordCount;
-  
-  public BaseValueVector(MaterializedField field, BufferAllocator allocator) {
-    this.allocator = allocator;
-    this.field = field;
-  }
-
-  public final void allocateNew(int valueCount){
-    int allocationSize = getAllocationSize(valueCount);
-    ByteBuf newBuf = allocator.buffer(allocationSize);
-    resetAllocation(valueCount, newBuf);
-  }
-
-  protected abstract int getAllocationSize(int maxValueCount);
-  protected abstract void childResetAllocation(int valueCount, ByteBuf buf);
-  protected abstract void childCloneMetadata(T other);
-  protected abstract void childClear();
-  
-  /**
-   * Update the current buffer allocation utilize the provided allocation.
-   * @param maxValueCount
-   * @param buf
-   */
-  protected final void resetAllocation(int maxValueCount, ByteBuf buf){
-    clear();
-    buf.retain();
-    this.maxValueCount = maxValueCount;
-    this.data = buf;
-    childResetAllocation(maxValueCount, buf);
-  }
-  
-  public final void cloneMetadata(T other){
-    other.maxValueCount = this.maxValueCount;
-  }
-  
-  
-  @Override
-  public final void cloneInto(T vector) {
-    vector.allocateNew(maxValueCount);
-    data.writeBytes(vector.data);
-    cloneMetadata(vector);
-    childResetAllocation(maxValueCount, vector.data);
-  }
-  
-  @Override
-  public final void transferTo(T vector) {
-    vector.data = this.data;
-    cloneMetadata(vector);
-    childResetAllocation(maxValueCount, data);
-    clear();
-  }
-
-  protected final void clear(){
-    if(this.data != DeadBuf.DEAD_BUFFER){
-      this.data.release();
-      this.data = DeadBuf.DEAD_BUFFER;
-      this.maxValueCount = 0;
-    }
-    childClear();
-  }
-  
-  /**
-   * Give the length of the value vector in bytes.
-   * 
-   * @return
-   */
-  public int capacity() {
-    return maxValueCount;
-  }
-  
-  @Override
-  public void close() {
-    clear();
-  }
-
-  @Override
-  public ByteBuf[] getBuffers() {
-    return new ByteBuf[]{data};
-  }
-  
-  public MaterializedField getField(){
-    return field;
-  }
-  
-  
-  public int getRecordCount() {
-    return recordCount;
-  }
-
-  public void setRecordCount(int recordCount) {
-    this.recordCount = recordCount;
-  }
-
-  @Override
-  public FieldMetadata getMetadata() {
-    int len = 0;
-    for(ByteBuf b : getBuffers()){
-      len += b.writerIndex();
-    }
-    return FieldMetadata.newBuilder().setDef(getField().getDef()).setValueCount(getRecordCount()).setBufferLength(len).build();
-  }
-  
-  @Override
-  public void setTo(FieldMetadata metadata, ByteBuf data) {
-//    logger.debug("Updating value vector to {}, {}", metadata, data);
-    clear();
-    resetAllocation(metadata.getValueCount(), data);
-  }
-
-  @Override
-  public void randomizeData() {
-    if(this.data != DeadBuf.DEAD_BUFFER){
-      Random r = new Random();
-      for(int i =0; i < data.capacity()-8; i+=8){
-        data.setLong(i, r.nextLong());
-      }
-    }
-    
-  }
-  
-  
-}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/7075cca1/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/Bit.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/Bit.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/Bit.java
deleted file mode 100644
index 533e3bd..0000000
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/Bit.java
+++ /dev/null
@@ -1,168 +0,0 @@
-/*******************************************************************************
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- ******************************************************************************/
-package org.apache.drill.exec.record.vector;
-
-import org.apache.drill.common.expression.types.DataType;
-import org.apache.drill.exec.memory.BufferAllocator;
-import org.apache.drill.exec.physical.RecordField.ValueMode;
-import org.apache.drill.exec.record.MaterializedField;
-import org.apache.hadoop.io.SequenceFile;
-
-/**
- * Describes a vector which holds a number of true/false values.
- */
-public class Bit extends AbstractFixedValueVector<Bit> {
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(Bit.class);
-
-  public Bit(MaterializedField field, BufferAllocator allocator) {
-    super(field, allocator, 1);
-  }
-
-  
-//  /** Returns true or false for the specified bit index.
-//   * The index should be less than the OpenBitSet size
-//   */
-//  public boolean get(int index) {
-//    assert index >= 0 && index < this.valueCount;
-//    int i = index >> 3;               // div 8
-//    // signed shift will keep a negative index and force an
-//    // array-index-out-of-bounds-exception, removing the need for an explicit check.
-//    int bit = index & 0x3f;           // mod 64
-//    long bitmask = 1L << bit;
-//    return (data.getLong(i) & bitmask) != 0;
-//  }
-  
-  public int getBit(int index) {
-    
-    assert index >= 0 && index < this.maxValueCount;
-    int i = 8*(index >> 6); // div 8
-    int bit = index & 0x3f; // mod 64
-    return ((int) (data.getLong(i) >>> bit)) & 0x01;
-  }
-  
-  /** Sets the bit at the specified index.
-   * The index should be less than the OpenBitSet size.
-   */
-   public void set(int index) {
-     assert index >= 0 && index < this.maxValueCount;
-     int wordNum = index >> 3;   
-     int bit = index & 0x3f;
-     long bitmask = 1L << bit;
-     data.setLong(wordNum, data.getLong(wordNum) | bitmask);
-   }
-   
-   public void clear(int index) {
-     assert index >= 0 && index < this.maxValueCount;
-     int wordNum = index >> 3;
-     int bit = index & 0x03f;
-     long bitmask = 1L << bit;
-     data.setLong(wordNum, data.getLong(wordNum) & ~bitmask);
-   }
-   
-   
-   
-   /** Clears a range of bits.  Clearing past the end does not change the size of the set.
-   *
-   * @param startBitIndex lower index
-   * @param lastBitIndex one-past the last bit to clear
-   */
-  private void clear2(int startBitIndex, int lastBitIndex) {
-    if (lastBitIndex <= startBitIndex) return;
-
-    int firstWordStart = (startBitIndex>>3);
-    if (firstWordStart >= this.longWords) return;
-
-    // since endIndex is one past the end, this is index of the last
-    // word to be changed.
-    int lastWordStart   = ((lastBitIndex-1)>>3);
-
-    long startmask = -1L << startBitIndex;
-    long endmask = -1L >>> -lastBitIndex;  // 64-(endIndex&0x3f) is the same as -endIndex due to wrap
-
-    // invert masks since we are clearing
-    startmask = ~startmask;
-    endmask = ~endmask;
-
-    if (firstWordStart == lastWordStart) {
-      data.setLong(firstWordStart,  data.getLong(firstWordStart) & (startmask | endmask));
-      return;
-    }
-    data.setLong(firstWordStart,  data.getLong(firstWordStart) & startmask);
-
-    int middle = Math.min(this.longWords, lastWordStart);
-    
-    for(int i =firstWordStart+8; i < middle; i += 8){
-      data.setLong(i, 0L);
-    }
-    if (lastWordStart < this.longWords) {
-      data.setLong(lastWordStart,  data.getLong(lastWordStart) & endmask);
-    }
-  }
-  
-  public void setAllFalse(){
-    clear(0, maxValueCount);
-  }
-
-  
-  public void clear(int startIndex, int endIndex) {
-    if (endIndex <= startIndex) return;
-
-    int startWord = (startIndex >> 6);
-    if (startWord >= longWords) return;
-
-    // since endIndex is one past the end, this is index of the last
-    // word to be changed.
-    int endWord = ((endIndex - 1) >> 6);
-
-    long startmask = -1L << startIndex;
-    long endmask = -1L >>> -endIndex; // 64-(endIndex&0x3f) is the same as -endIndex due to wrap
-
-    // invert masks since we are clearing
-    startmask = ~startmask;
-    endmask = ~endmask;
-    
-    int startWordPos = startWord * 8;
-    if (startWord == endWord) {
-      data.setLong(startWordPos, data.getLong(startWordPos) & (startmask | endmask));
-      return;
-    }
-
-    int endWordPos = endWord * 8;
-
-    data.setLong(startWordPos, data.getLong(startWordPos) & startmask);
-
-    int middle = Math.min(longWords, endWord)*8;
-    
-    
-    for(int i =startWordPos+8; i < middle; i += 8){
-      data.setLong(i, 0L);
-    }
-    
-    if (endWordPos < startWordPos) {
-      data.setLong(endWordPos, data.getLong(endWordPos) & endmask);
-    }
-  }
-
-
-  @Override
-  public Object getObject(int index) {
-    return this.getBit(index);
-  }
-  
-  
-}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/7075cca1/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/Fixed1.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/Fixed1.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/Fixed1.java
deleted file mode 100644
index 82c86d1..0000000
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/Fixed1.java
+++ /dev/null
@@ -1,43 +0,0 @@
-/*******************************************************************************
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- ******************************************************************************/
-package org.apache.drill.exec.record.vector;
-
-import org.apache.drill.exec.memory.BufferAllocator;
-import org.apache.drill.exec.record.MaterializedField;
-
-public class Fixed1 extends AbstractFixedValueVector<Fixed1>{
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(Fixed1.class);
-
-  public Fixed1(MaterializedField field, BufferAllocator allocator) {
-    super(field, allocator, 8);
-  }
-  
-  public void setByte(int index, byte b){
-    data.setByte(index, b);
-  }
-
-  public byte getByte(int index){
-    return data.getByte(index);
-  }
-  
-  @Override
-  public Object getObject(int index) {
-    return getByte(index);
-  }
-  
-}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/7075cca1/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/Fixed12.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/Fixed12.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/Fixed12.java
deleted file mode 100644
index c5f641a..0000000
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/Fixed12.java
+++ /dev/null
@@ -1,35 +0,0 @@
-/*******************************************************************************
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- ******************************************************************************/
-package org.apache.drill.exec.record.vector;
-
-import org.apache.drill.exec.memory.BufferAllocator;
-import org.apache.drill.exec.record.MaterializedField;
-
-public class Fixed12 extends AbstractFixedValueVector<Fixed12>{
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(Fixed12.class);
-
-  public Fixed12(MaterializedField field, BufferAllocator allocator) {
-    super(field, allocator, 12*8);
-  }
-
-  
-  @Override
-  public Object getObject(int index) {
-    return null;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/7075cca1/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/Fixed16.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/Fixed16.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/Fixed16.java
deleted file mode 100644
index 649832b..0000000
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/Fixed16.java
+++ /dev/null
@@ -1,37 +0,0 @@
-/*******************************************************************************
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- ******************************************************************************/
-package org.apache.drill.exec.record.vector;
-
-import org.apache.drill.exec.memory.BufferAllocator;
-import org.apache.drill.exec.record.MaterializedField;
-
-public class Fixed16 extends AbstractFixedValueVector<Fixed16>{
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(Fixed16.class);
-
-  public Fixed16(MaterializedField field, BufferAllocator allocator) {
-    super(field, allocator, 16*8);
-  }
-
-  @Override
-  public Object getObject(int index) {
-    return null;
-  }
-  
-  
-  
-}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/7075cca1/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/Fixed2.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/Fixed2.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/Fixed2.java
deleted file mode 100644
index bd0e313..0000000
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/Fixed2.java
+++ /dev/null
@@ -1,53 +0,0 @@
-/*******************************************************************************
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- ******************************************************************************/
-package org.apache.drill.exec.record.vector;
-
-import org.apache.drill.exec.memory.BufferAllocator;
-import org.apache.drill.exec.record.MaterializedField;
-
-public class Fixed2 extends AbstractFixedValueVector<Fixed2>{
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(Fixed2.class);
-
-  public Fixed2(MaterializedField field, BufferAllocator allocator) {
-    super(field, allocator, 2*8);
-  }
-
-  public final void setSmallInt(int index, short value){
-    index*=2;
-    data.setShort(index, value);
-  }
-  
-  public final short getSmallInt(int index){
-    index*=2;
-    return data.getShort(index);
-  }
-  
-  public final void setUInt2(int index, short value){
-    setSmallInt(index, value);
-  }
-  
-  public final short getUInt2(int index){
-    return getSmallInt(index);
-  }
-  
-  @Override
-  public Object getObject(int index) {
-    return getSmallInt(index);
-  }
-  
-}


[40/53] [abbrv] Types transition

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/schema/Field.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/schema/Field.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/schema/Field.java
index eef0634..85bbdf3 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/schema/Field.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/schema/Field.java
@@ -18,49 +18,37 @@
 
 package org.apache.drill.exec.schema;
 
-import com.google.common.base.Objects;
-import com.google.common.base.Strings;
+import org.apache.drill.common.expression.ExpressionPosition;
 import org.apache.drill.common.expression.SchemaPath;
-import org.apache.drill.exec.memory.BufferAllocator;
-import org.apache.drill.exec.proto.SchemaDefProtos;
+import org.apache.drill.common.types.TypeProtos.MajorType;
 import org.apache.drill.exec.record.MaterializedField;
-import org.apache.drill.exec.record.vector.*;
-import org.apache.drill.exec.store.BatchExceededException;
-import org.apache.drill.exec.store.VectorHolder;
 
-import java.nio.charset.Charset;
-
-import static com.google.common.base.Preconditions.checkArgument;
-import static com.google.common.base.Preconditions.checkState;
-import static org.apache.drill.exec.proto.SchemaDefProtos.*;
+import com.google.common.base.Objects;
+import com.google.common.base.Strings;
 
 public abstract class Field {
     final MajorType fieldType;
-    final int parentFieldId;
-    final int fieldId;
     final String prefixFieldName;
     RecordSchema schema;
     RecordSchema parentSchema;
     boolean read;
 
-    public Field(RecordSchema parentSchema, int parentFieldId, IdGenerator<Integer> generator, MajorType type, String prefixFieldName) {
-        this.fieldId = generator.getNextId();
+    public Field(RecordSchema parentSchema, MajorType type, String prefixFieldName) {
         fieldType = type;
         this.prefixFieldName = prefixFieldName;
         this.parentSchema = parentSchema;
-        this.parentFieldId = parentFieldId;
     }
 
+    public MaterializedField getAsMaterializedField(){
+      return MaterializedField.create(new SchemaPath(getFieldName(), ExpressionPosition.UNKNOWN), fieldType);
+    }
+    
     public abstract String getFieldName();
 
     public String getFullFieldName() {
         return Strings.isNullOrEmpty(prefixFieldName) ? getFieldName() : prefixFieldName + "." + getFieldName();
     }
 
-    public int getFieldId() {
-        return fieldId;
-    }
-
     public void setRead(boolean read) {
         this.read = read;
     }
@@ -69,7 +57,6 @@ public abstract class Field {
 
     Objects.ToStringHelper getAttributesStringHelper() {
         return Objects.toStringHelper(this).add("type", fieldType)
-                .add("id", fieldId)
                 .add("fullFieldName", getFullFieldName())
                 .add("schema", schema == null ? null : schema.toSchemaString()).omitNullValues();
     }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/schema/ListSchema.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/schema/ListSchema.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/schema/ListSchema.java
index e5bd1a4..77f1f37 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/schema/ListSchema.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/schema/ListSchema.java
@@ -18,15 +18,16 @@
 
 package org.apache.drill.exec.schema;
 
-import com.google.common.base.Predicate;
-import com.google.common.collect.Iterables;
-import com.google.common.collect.Lists;
-import org.apache.drill.exec.proto.SchemaDefProtos;
+import static com.google.common.base.Preconditions.checkArgument;
+import static com.google.common.base.Preconditions.checkNotNull;
 
 import java.util.List;
 
-import static com.google.common.base.Preconditions.checkArgument;
-import static com.google.common.base.Preconditions.checkNotNull;
+import org.apache.drill.common.types.TypeProtos.DataMode;
+
+import com.google.common.base.Predicate;
+import com.google.common.collect.Iterables;
+import com.google.common.collect.Lists;
 
 public class ListSchema implements RecordSchema {
     private List<Field> fields;
@@ -37,7 +38,7 @@ public class ListSchema implements RecordSchema {
 
     @Override
     public void addField(Field field) {
-        if (field.getFieldType().getMode() == SchemaDefProtos.DataMode.REPEATED || fields.isEmpty() || !isSingleTyped() ||
+        if (field.getFieldType().getMode() == DataMode.REPEATED || fields.isEmpty() || !isSingleTyped() ||
                 !Iterables.getOnlyElement(fields).equals(field.getFieldType())) {
             fields.add(field);
         }
@@ -58,7 +59,7 @@ public class ListSchema implements RecordSchema {
     @Override
     public void removeField(Field field, int index) {
         checkArgument(fields.size() > index);
-        checkArgument(checkNotNull(fields.get(index)).getFieldId() == field.getFieldId());
+//        checkArgument(checkNotNull(fields.get(index)).getFieldId() == field.getFieldId());
         fields.remove(index);
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/schema/NamedField.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/schema/NamedField.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/schema/NamedField.java
index b975ad7..556628c 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/schema/NamedField.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/schema/NamedField.java
@@ -18,26 +18,25 @@
 
 package org.apache.drill.exec.schema;
 
-import com.google.common.base.Objects;
-import org.apache.drill.exec.proto.SchemaDefProtos;
+import org.apache.drill.common.types.TypeProtos.MajorType;
 import org.apache.drill.exec.schema.json.jackson.JacksonHelper;
 
+import com.google.common.base.Objects;
+
 public class NamedField extends Field {
-    final SchemaDefProtos.MajorType keyType;
+    final MajorType keyType;
     String fieldName;
 
-    public NamedField(RecordSchema parentSchema, int parentFieldId, IdGenerator<Integer> generator, String prefixFieldName, String fieldName, SchemaDefProtos.MajorType fieldType) {
-        this(parentSchema, parentFieldId, generator, prefixFieldName, fieldName, fieldType, JacksonHelper.STRING_TYPE);
+    public NamedField(RecordSchema parentSchema, String prefixFieldName, String fieldName, MajorType fieldType) {
+        this(parentSchema, prefixFieldName, fieldName, fieldType, JacksonHelper.STRING_TYPE);
     }
 
     public NamedField(RecordSchema parentSchema,
-                      int parentFieldId,
-                      IdGenerator<Integer> generator,
                       String prefixFieldName,
                       String fieldName,
-                      SchemaDefProtos.MajorType fieldType,
-                      SchemaDefProtos.MajorType keyType) {
-        super(parentSchema, parentFieldId, generator, fieldType, prefixFieldName);
+                      MajorType fieldType,
+                      MajorType keyType) {
+        super(parentSchema, fieldType, prefixFieldName);
         this.fieldName = fieldName;
         this.keyType = keyType;
     }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/schema/OrderedField.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/schema/OrderedField.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/schema/OrderedField.java
index 5f514af..eec1e4b 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/schema/OrderedField.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/schema/OrderedField.java
@@ -18,19 +18,18 @@
 
 package org.apache.drill.exec.schema;
 
+import org.apache.drill.common.types.TypeProtos.MajorType;
+
 import com.google.common.base.Objects;
-import org.apache.drill.exec.proto.SchemaDefProtos;
 
 public class OrderedField extends Field {
     private final int index;
 
     public OrderedField(RecordSchema parentSchema,
-                        int parentFieldId,
-                        IdGenerator<Integer> generator,
-                        SchemaDefProtos.MajorType type,
+                        MajorType type,
                         String prefixFieldName,
                         int index) {
-        super(parentSchema, parentFieldId, generator, type, prefixFieldName);
+        super(parentSchema, type, prefixFieldName);
         this.index = index;
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/schema/json/jackson/JacksonHelper.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/schema/json/jackson/JacksonHelper.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/schema/json/jackson/JacksonHelper.java
index 64a9d58..d0b27fc 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/schema/json/jackson/JacksonHelper.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/schema/json/jackson/JacksonHelper.java
@@ -18,24 +18,24 @@
 
 package org.apache.drill.exec.schema.json.jackson;
 
-import com.fasterxml.jackson.core.JsonParser;
-import com.fasterxml.jackson.core.JsonToken;
-import org.apache.drill.exec.proto.SchemaDefProtos;
-
 import java.io.IOException;
 
-import static org.apache.drill.exec.proto.SchemaDefProtos.MajorType;
-import static org.apache.drill.exec.proto.SchemaDefProtos.MinorType;
+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 com.fasterxml.jackson.core.JsonParser;
+import com.fasterxml.jackson.core.JsonToken;
 
 public class JacksonHelper {
 
-    public static final MajorType STRING_TYPE = MajorType.newBuilder().setMinorType(MinorType.VARCHAR4).setMode(SchemaDefProtos.DataMode.OPTIONAL).build();
-    public static final MajorType BOOLEAN_TYPE = MajorType.newBuilder().setMinorType(MinorType.BOOLEAN).setMode(SchemaDefProtos.DataMode.OPTIONAL).build();
-    public static final MajorType ARRAY_TYPE = MajorType.newBuilder().setMinorType(MinorType.LATE).setMode(SchemaDefProtos.DataMode.REPEATED).build();
-    public static final MajorType MAP_TYPE = MajorType.newBuilder().setMinorType(MinorType.MAP).setMode(SchemaDefProtos.DataMode.REPEATED).build();
-    public static final MajorType INT_TYPE = MajorType.newBuilder().setMinorType(MinorType.INT).setMode(SchemaDefProtos.DataMode.OPTIONAL).build();
-    public static final MajorType FLOAT_TYPE = MajorType.newBuilder().setMinorType(MinorType.FLOAT4).setMode(SchemaDefProtos.DataMode.OPTIONAL).build();
-    public static final MajorType NULL_TYPE = MajorType.newBuilder().setMinorType(MinorType.LATE).setMode(SchemaDefProtos.DataMode.OPTIONAL).build();
+    public static final MajorType STRING_TYPE = MajorType.newBuilder().setMinorType(MinorType.VARCHAR4).setMode(DataMode.OPTIONAL).build();
+    public static final MajorType BOOLEAN_TYPE = MajorType.newBuilder().setMinorType(MinorType.BOOLEAN).setMode(DataMode.OPTIONAL).build();
+    public static final MajorType ARRAY_TYPE = MajorType.newBuilder().setMinorType(MinorType.LATE).setMode(DataMode.REPEATED).build();
+    public static final MajorType MAP_TYPE = MajorType.newBuilder().setMinorType(MinorType.MAP).setMode(DataMode.REPEATED).build();
+    public static final MajorType INT_TYPE = MajorType.newBuilder().setMinorType(MinorType.INT).setMode(DataMode.OPTIONAL).build();
+    public static final MajorType FLOAT_TYPE = MajorType.newBuilder().setMinorType(MinorType.FLOAT4).setMode(DataMode.OPTIONAL).build();
+    public static final MajorType NULL_TYPE = MajorType.newBuilder().setMinorType(MinorType.LATE).setMode(DataMode.OPTIONAL).build();
 
     public static MajorType getFieldType(JsonToken token) {
         switch(token) {

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/JSONRecordReader.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/JSONRecordReader.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/JSONRecordReader.java
index 07ae20a..8c31aa4 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/JSONRecordReader.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/JSONRecordReader.java
@@ -9,16 +9,24 @@ import java.io.IOException;
 import java.io.InputStreamReader;
 import java.nio.charset.Charset;
 import java.util.List;
+import java.util.Map;
 
 import org.apache.drill.common.exceptions.DrillRuntimeException;
 import org.apache.drill.common.exceptions.ExecutionSetupException;
+import org.apache.drill.common.expression.ExpressionPosition;
 import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.common.types.TypeProtos.MajorType;
+import org.apache.drill.common.types.TypeProtos.MinorType;
 import org.apache.drill.exec.exception.SchemaChangeException;
 import org.apache.drill.exec.memory.BufferAllocator;
 import org.apache.drill.exec.ops.FragmentContext;
 import org.apache.drill.exec.physical.impl.OutputMutator;
-import org.apache.drill.exec.proto.SchemaDefProtos;
 import org.apache.drill.exec.record.MaterializedField;
+import org.apache.drill.exec.record.vector.NullableBit;
+import org.apache.drill.exec.record.vector.NullableFixed4;
+import org.apache.drill.exec.record.vector.NullableVarLen4;
+import org.apache.drill.exec.record.vector.TypeHelper;
+import org.apache.drill.exec.record.vector.ValueVector;
 import org.apache.drill.exec.schema.DiffSchema;
 import org.apache.drill.exec.schema.Field;
 import org.apache.drill.exec.schema.IdGenerator;
@@ -29,13 +37,8 @@ import org.apache.drill.exec.schema.OrderedField;
 import org.apache.drill.exec.schema.RecordSchema;
 import org.apache.drill.exec.schema.SchemaIdGenerator;
 import org.apache.drill.exec.schema.json.jackson.JacksonHelper;
-import org.apache.drill.exec.vector.NullableBitVector;
-import org.apache.drill.exec.vector.NullableFloat4Vector;
-import org.apache.drill.exec.vector.NullableIntVector;
-import org.apache.drill.exec.vector.NullableVarChar4Vector;
-import org.apache.drill.exec.vector.TypeHelper;
-import org.apache.drill.exec.vector.ValueVector;
 
+import com.beust.jcommander.internal.Maps;
 import com.carrotsearch.hppc.IntObjectOpenHashMap;
 import com.carrotsearch.hppc.cursors.ObjectCursor;
 import com.fasterxml.jackson.core.JsonFactory;
@@ -55,7 +58,7 @@ public class JSONRecordReader implements RecordReader {
 
     private final String inputPath;
 
-    private final IntObjectOpenHashMap<VectorHolder> valueVectorMap;
+    private final Map<Field, VectorHolder> valueVectorMap;
 
     private JsonParser parser;
     private SchemaIdGenerator generator;
@@ -70,7 +73,7 @@ public class JSONRecordReader implements RecordReader {
         this.inputPath = inputPath;
         this.allocator = fragmentContext.getAllocator();
         this.batchSize = batchSize;
-        valueVectorMap = new IntObjectOpenHashMap<>();
+        valueVectorMap = Maps.newHashMap();
     }
 
     public JSONRecordReader(FragmentContext fragmentContext, String inputPath) {
@@ -134,7 +137,7 @@ public class JSONRecordReader implements RecordReader {
             // Garbage collect fields never referenced in this batch
             for (Field field : Iterables.concat(currentSchema.removeUnreadFields(), removedFields)) {
                 diffSchema.addRemovedField(field);
-                outputMutator.removeField(field.getFieldId());
+                outputMutator.removeField(field.getAsMaterializedField());
             }
 
         } catch (IOException | SchemaChangeException e) {
@@ -144,8 +147,8 @@ public class JSONRecordReader implements RecordReader {
     }
 
     private void resetBatch() {
-        for (ObjectCursor<VectorHolder> holder : valueVectorMap.values()) {
-            holder.value.reset();
+        for (VectorHolder value : valueVectorMap.values()) {
+            value.reset();
         }
 
         currentSchema.resetMarkedFields();
@@ -162,9 +165,6 @@ public class JSONRecordReader implements RecordReader {
         }
     }
 
-    private SchemaIdGenerator getGenerator() {
-        return generator;
-    }
 
     private RecordSchema getCurrentSchema() {
         return currentSchema;
@@ -193,8 +193,8 @@ public class JSONRecordReader implements RecordReader {
     public static enum ReadType {
         ARRAY(END_ARRAY) {
             @Override
-            public Field createField(RecordSchema parentSchema, int parentFieldId, IdGenerator<Integer> generator, String prefixFieldName, String fieldName, SchemaDefProtos.MajorType fieldType, int index) {
-                return new OrderedField(parentSchema, parentFieldId, generator, fieldType, prefixFieldName, index);
+            public Field createField(RecordSchema parentSchema, String prefixFieldName, String fieldName, MajorType fieldType, int index) {
+                return new OrderedField(parentSchema, fieldType, prefixFieldName, index);
             }
 
             @Override
@@ -205,13 +205,11 @@ public class JSONRecordReader implements RecordReader {
         OBJECT(END_OBJECT) {
             @Override
             public Field createField(RecordSchema parentSchema,
-                                     int parentFieldId,
-                                     IdGenerator<Integer> generator,
                                      String prefixFieldName,
                                      String fieldName,
-                                     SchemaDefProtos.MajorType fieldType,
+                                     MajorType fieldType,
                                      int index) {
-                return new NamedField(parentSchema, parentFieldId, generator, prefixFieldName, fieldName, fieldType);
+                return new NamedField(parentSchema, prefixFieldName, fieldName, fieldType);
             }
 
             @Override
@@ -246,7 +244,7 @@ public class JSONRecordReader implements RecordReader {
                 }
 
                 String fieldName = parser.getCurrentName();
-                SchemaDefProtos.MajorType fieldType = JacksonHelper.getFieldType(token);
+                MajorType fieldType = JacksonHelper.getFieldType(token);
                 ReadType readType = null;
                 switch (token) {
                     case START_ARRAY:
@@ -289,7 +287,7 @@ public class JSONRecordReader implements RecordReader {
         private boolean recordData(Field parentField,
                                    JSONRecordReader.ReadType readType,
                                    JSONRecordReader reader,
-                                   SchemaDefProtos.MajorType fieldType,
+                                   MajorType fieldType,
                                    String prefixFieldName,
                                    String fieldName,
                                    int rowIndex,
@@ -298,7 +296,6 @@ public class JSONRecordReader implements RecordReader {
             Field field = currentSchema.getField(fieldName, colIndex);
             boolean isFieldFound = field != null;
             List<Field> removedFields = reader.getRemovedFields();
-            int parentFieldId = parentField == null ? 0 : parentField.getFieldId();
             if (!isFieldFound || !field.getFieldType().equals(fieldType)) {
                 if (isFieldFound) {
                     if (field.hasSchema()) {
@@ -310,8 +307,6 @@ public class JSONRecordReader implements RecordReader {
 
                 field = createField(
                         currentSchema,
-                        parentFieldId,
-                        reader.getGenerator(),
                         prefixFieldName,
                         fieldName,
                         fieldType,
@@ -324,7 +319,7 @@ public class JSONRecordReader implements RecordReader {
 
             field.setRead(true);
 
-            VectorHolder holder = getOrCreateVectorHolder(reader, field, parentFieldId);
+            VectorHolder holder = getOrCreateVectorHolder(reader, field);
             if (readType != null) {
                 RecordSchema fieldSchema = field.getAssignedSchema();
                 reader.setCurrentSchema(fieldSchema);
@@ -352,7 +347,7 @@ public class JSONRecordReader implements RecordReader {
             return true;
         }
 
-        private static <T> boolean addValueToVector(int index, VectorHolder holder, BufferAllocator allocator, T val, SchemaDefProtos.MinorType minorType) {
+        private static <T> boolean addValueToVector(int index, VectorHolder holder, BufferAllocator allocator, T val, MinorType minorType) {
             switch (minorType) {
                 case INT: {
                     holder.incAndCheckLength(32);
@@ -398,18 +393,16 @@ public class JSONRecordReader implements RecordReader {
             }
         }
 
-        private VectorHolder getOrCreateVectorHolder(JSONRecordReader reader, Field field, int parentFieldId) throws SchemaChangeException {
-            return reader.getOrCreateVectorHolder(field, parentFieldId);
+        private VectorHolder getOrCreateVectorHolder(JSONRecordReader reader, Field field) throws SchemaChangeException {
+            return reader.getOrCreateVectorHolder(field);
         }
 
         public abstract RecordSchema createSchema() throws IOException;
 
         public abstract Field createField(RecordSchema parentSchema,
-                                          int parentFieldId,
-                                          IdGenerator<Integer> generator,
                                           String prefixFieldName,
                                           String fieldName,
-                                          SchemaDefProtos.MajorType fieldType,
+                                          MajorType fieldType,
                                           int index);
     }
 
@@ -417,20 +410,19 @@ public class JSONRecordReader implements RecordReader {
         diffSchema.recordNewField(field);
     }
 
-    private VectorHolder getOrCreateVectorHolder(Field field, int parentFieldId) throws SchemaChangeException {
-        if (!valueVectorMap.containsKey(field.getFieldId())) {
-            SchemaDefProtos.MajorType type = field.getFieldType();
-            int fieldId = field.getFieldId();
-            MaterializedField f = MaterializedField.create(new SchemaPath(field.getFieldName()), fieldId, parentFieldId, type);
-            
-            ValueVector v = TypeHelper.getNewVector(f, allocator);
-            VectorHolder holder = new VectorHolder(batchSize, v);
-            holder.allocateNew(batchSize);
-            
-            valueVectorMap.put(fieldId, holder);
-            outputMutator.addField(fieldId, v);
+    private VectorHolder getOrCreateVectorHolder(Field field) throws SchemaChangeException {
+      VectorHolder holder = valueVectorMap.get(field);
+      
+        if (holder == null) {
+            MajorType type = field.getFieldType();
+            MaterializedField f = MaterializedField.create(new SchemaPath(field.getFieldName(), ExpressionPosition.UNKNOWN), type);
+            ValueVector<?> v = TypeHelper.getNewVector(f, allocator);
+            v.allocateNew(batchSize);
+            holder = new VectorHolder(batchSize, v);
+            valueVectorMap.put(field, holder);
+            outputMutator.addField(v);
             return holder;
         }
-        return valueVectorMap.lget();
+        return holder;
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/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 edda714..2829dfd 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
@@ -25,6 +25,7 @@ import java.util.concurrent.ConcurrentMap;
 
 import org.apache.drill.common.exceptions.ExecutionSetupException;
 import org.apache.drill.exec.exception.FragmentSetupException;
+import org.apache.drill.exec.expr.fn.FunctionImplementationRegistry;
 import org.apache.drill.exec.ops.FragmentContext;
 import org.apache.drill.exec.physical.base.FragmentRoot;
 import org.apache.drill.exec.physical.impl.ImplCreator;
@@ -113,7 +114,7 @@ public class BitComHandlerImpl implements BitComHandler {
   @Override
   public void startNewRemoteFragment(PlanFragment fragment){
     logger.debug("Received remote fragment start instruction", fragment);
-    FragmentContext context = new FragmentContext(bee.getContext(), fragment.getHandle(), null, null);
+    FragmentContext context = new FragmentContext(bee.getContext(), fragment.getHandle(), null, null,new FunctionImplementationRegistry(bee.getContext().getConfig()));
     BitTunnel tunnel = bee.getContext().getBitCom().getTunnel(fragment.getForeman());
     RemotingFragmentRunnerListener listener = new RemotingFragmentRunnerListener(context, tunnel);
     try{

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/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 5bb1ff2..1a4bc6c 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
@@ -22,6 +22,7 @@ import java.util.Map;
 import java.util.concurrent.atomic.AtomicInteger;
 
 import org.apache.drill.common.exceptions.ExecutionSetupException;
+import org.apache.drill.exec.expr.fn.FunctionImplementationRegistry;
 import org.apache.drill.exec.ops.FragmentContext;
 import org.apache.drill.exec.physical.base.FragmentRoot;
 import org.apache.drill.exec.physical.impl.ImplCreator;
@@ -74,7 +75,7 @@ class RunningFragmentManager implements FragmentStatusListener{
     {
       IncomingBuffers buffers = new IncomingBuffers(rootOperator);
       
-      FragmentContext rootContext = new FragmentContext(bee.getContext(), rootFragment.getHandle(), rootClient, buffers);
+      FragmentContext rootContext = new FragmentContext(bee.getContext(), rootFragment.getHandle(), rootClient, buffers, new FunctionImplementationRegistry(bee.getContext().getConfig()));
       RootExec rootExec = ImplCreator.getExec(rootContext, rootOperator);
       // add fragment to local node.
       map.put(rootFragment.getHandle(), new FragmentData(rootFragment.getHandle(), null, true));

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/RemoteFragmentHandler.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/RemoteFragmentHandler.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/RemoteFragmentHandler.java
index 4a5dbf2..e4d0cfc 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/RemoteFragmentHandler.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/RemoteFragmentHandler.java
@@ -21,6 +21,7 @@ import java.io.IOException;
 
 import org.apache.drill.common.exceptions.ExecutionSetupException;
 import org.apache.drill.exec.exception.FragmentSetupException;
+import org.apache.drill.exec.expr.fn.FunctionImplementationRegistry;
 import org.apache.drill.exec.ops.FragmentContext;
 import org.apache.drill.exec.physical.base.FragmentLeaf;
 import org.apache.drill.exec.physical.base.FragmentRoot;
@@ -56,7 +57,7 @@ public class RemoteFragmentHandler implements IncomingFragmentHandler {
       this.fragment = fragment;
       this.root = context.getPlanReader().readFragmentOperator(fragment.getFragmentJson());
       this.buffers = new IncomingBuffers(root);
-      this.context = new FragmentContext(context, fragment.getHandle(), null, buffers);
+      this.context = new FragmentContext(context, fragment.getHandle(), null, buffers, new FunctionImplementationRegistry(context.getConfig()));
       this.runnerListener = new RemotingFragmentRunnerListener(this.context, foremanTunnel);
       this.reader = context.getPlanReader();
       

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/sandbox/prototype/exec/java-exec/src/main/protobuf/SchemaDef.proto
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/protobuf/SchemaDef.proto b/sandbox/prototype/exec/java-exec/src/main/protobuf/SchemaDef.proto
index de0009a..a360cea 100644
--- a/sandbox/prototype/exec/java-exec/src/main/protobuf/SchemaDef.proto
+++ b/sandbox/prototype/exec/java-exec/src/main/protobuf/SchemaDef.proto
@@ -4,61 +4,8 @@ option java_package = "org.apache.drill.exec.proto";
 option java_outer_classname = "SchemaDefProtos";
 option optimize_for = SPEED;
 
+import "Types.proto";
 
-// Schema Definitions //
-enum MinorType {
-    LATE = 0;   //  late binding type
-    MAP = 1;   //  an empty map column.  Useful for conceptual setup.  Children listed within here
-    REPEATMAP = 2;   //  a repeated map column (means that multiple children sit below this)
-    TINYINT = 3;   //  single byte signed integer
-    SMALLINT = 4;   //  two byte signed integer
-    INT = 5;   //  four byte signed integer
-    BIGINT = 6;   //  eight byte signed integer
-    DECIMAL4 = 7;   //  a decimal supporting precision between 1 and 8 (4 bits for decimal location, 1 sign)
-    DECIMAL8 = 8;   //  a decimal supporting precision between 9 and 18 (5 bits for decimal location, 1 sign)
-    DECIMAL12 = 9;   //  a decimal supporting precision between19 and 28 (5 bits for decimal location, 1 sign)
-    DECIMAL16 = 10;   //  a decimal supporting precision between 29 and 37 (6 bits for decimal location, 1 sign)
-    MONEY = 11;   //  signed decimal with two digit precision
-    DATE = 12;   //  days since 4713bc 
-    TIME = 13;   //  time in micros before or after 2000/1/1
-    TIMETZ = 14;   //  time in micros before or after 2000/1/1 with timezone
-    TIMESTAMP = 15;   //  unix epoch time in millis
-    DATETIME = 16;   //  TBD
-    INTERVAL = 17;   //  TBD
-    FLOAT4 = 18;   //  4 byte ieee 754 
-    FLOAT8 = 19;   //  8 byte ieee 754
-    BOOLEAN = 20;   //  single bit value
-    FIXEDCHAR = 21;   //  utf8 fixed length string, padded with spaces
-    VARCHAR1 = 22;   //  utf8 variable length string (up to 2^8 in length)
-    VARCHAR2 = 23;   //  utf8 variable length string (up to 2^16 in length)
-    VARCHAR4 = 24;   //  utf8 variable length string (up to 2^32 in length)
-    FIXEDBINARY = 25;   //  fixed length binary, padded with 0 bytes
-    VARBINARY1 = 26;   //  variable length binary (up to 2^8 in length)
-    VARBINARY2 = 27;   //  variable length binary (up to 2^16 in length)
-    VARBINARY4 = 28;   //  variable length binary (up to 2^32 in length)
-    UINT1 = 29;   //  unsigned 1 byte integer
-    UINT2 = 30;   //  unsigned 2 byte integer
-    UINT4 = 31;   //  unsigned 4 byte integer
-    UINT8 = 32;   //  unsigned 8 byte integer
-    PROTO2 = 33;   //  protobuf encoded complex type. (up to 2^16 in length)
-    PROTO4 = 34;   //  protobuf encoded complex type. (up to 2^32 in length)
-    MSGPACK2 = 35;   //  msgpack encoded complex type. (up to 2^16 in length)
-    MSGPACK4 = 36;   //  msgpack encoded complex type. (up to 2^32 in length)
-}
-
-message MajorType {
-  optional MinorType minor_type = 1;
-  optional DataMode mode = 2;
-  optional int32 width = 3; // optional width for fixed size values.
-  optional int32 precision = 4; // used for decimal types
-  optional int32 scale = 5; // used for decimal types 
-}
-
-enum DataMode {
-  OPTIONAL = 0; // nullable
-  REQUIRED = 1; // non-nullable
-  REPEATED = 2; // single, repeated-field
-}
 
 enum ValueMode {
 	VALUE_VECTOR = 0;
@@ -77,10 +24,8 @@ message NamePart {
 }
 
 message FieldDef {
-  optional int32 field_id = 1;
-  optional int32 parent_id = 2; // the field_id of the parent of this field.  populated when this is a repeated field.  a field_id of 0 means that the record is the parent of this repeated field.
-  repeated NamePart name = 3; // multipart description of entire field name
-  optional MajorType major_type = 4; // the type associated with this field.
-  repeated FieldDef field = 5; // only in the cases of type == MAP or REPEATMAP
+  repeated NamePart name = 1; // multipart description of entire field name
+  optional common.MajorType major_type = 2; // the type associated with this field.
+  repeated FieldDef field = 3; // only in the cases of type == MAP or REPEATMAP
   
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/sandbox/prototype/exec/java-exec/src/main/resources/drill-module.conf
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/resources/drill-module.conf b/sandbox/prototype/exec/java-exec/src/main/resources/drill-module.conf
index ad18d6e..3ce903d 100644
--- a/sandbox/prototype/exec/java-exec/src/main/resources/drill-module.conf
+++ b/sandbox/prototype/exec/java-exec/src/main/resources/drill-module.conf
@@ -16,12 +16,13 @@ drill.exec: {
 	root: "/drill",
 	refresh: 500,
 	timeout: 1000,
-	retry: {
-	  count: 7200,
-	  delay: 500
-	}    
-  }
-
+  	retry: {
+  	  count: 7200,
+  	  delay: 500
+  	}    
+  },
+  functions: ["org.apache.drill.expr.fn.impl"],
+  
   network: {
     start: 35000
   }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/SortTest.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/SortTest.java b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/SortTest.java
new file mode 100644
index 0000000..3e38a0e
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/SortTest.java
@@ -0,0 +1,61 @@
+package org.apache.drill.exec;
+
+import java.util.Random;
+
+import org.apache.hadoop.io.BytesWritable;
+import org.apache.hadoop.util.IndexedSortable;
+import org.apache.hadoop.util.QuickSort;
+
+public class SortTest {
+  private static final int RECORD_COUNT = 10*1000*1000;
+  private static final int KEY_SIZE = 10;
+  private static final int DATA_SIZE = 90;
+  private static final int RECORD_SIZE = KEY_SIZE + DATA_SIZE; 
+  
+  private byte[] data;
+  
+  public static void main(String[] args) throws Exception{
+    for(int i =0; i < 100; i++){
+      SortTest st = new SortTest();
+      long nanos = st.doSort();
+      System.out.print("Sort Completed in ");
+      System.out.print(nanos);
+      System.out.println(" ns.");
+    }
+  }
+  
+  SortTest(){
+    System.out.print("Generating data... ");
+    data = new byte[RECORD_SIZE*RECORD_COUNT];
+    Random r = new Random();
+    r.nextBytes(data);
+    System.out.print("Data generated. ");
+  }
+  
+  public long doSort(){
+    QuickSort qs = new QuickSort();
+    ByteSortable b = new ByteSortable();
+    long nano = System.nanoTime();
+    qs.sort(b, 0, RECORD_COUNT);
+    return System.nanoTime() - nano;
+  }
+  
+  private class ByteSortable implements IndexedSortable{
+    final byte[] space = new byte[RECORD_SIZE];
+    final BytesWritable.Comparator comparator = new BytesWritable.Comparator();
+    
+    @Override
+    public int compare(int index1, int index2) {
+      return comparator.compare(data, index1*RECORD_SIZE, KEY_SIZE, data, index2*RECORD_SIZE, KEY_SIZE);
+    }
+
+    @Override
+    public void swap(int index1, int index2) {
+      int start1 = index1*RECORD_SIZE;
+      int start2 = index2*RECORD_SIZE;
+      System.arraycopy(data, start1, space, 0, RECORD_SIZE);
+      System.arraycopy(data, start2, data, start1, RECORD_SIZE);
+      System.arraycopy(space, 0, data, start2, RECORD_SIZE);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/expr/ExpressionTest.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/expr/ExpressionTest.java b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/expr/ExpressionTest.java
new file mode 100644
index 0000000..16f7802
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/expr/ExpressionTest.java
@@ -0,0 +1,108 @@
+package org.apache.drill.exec.expr;
+
+import static org.junit.Assert.assertEquals;
+import mockit.Expectations;
+import mockit.Injectable;
+import mockit.NonStrictExpectations;
+
+import org.antlr.runtime.ANTLRStringStream;
+import org.antlr.runtime.CommonTokenStream;
+import org.antlr.runtime.RecognitionException;
+import org.apache.drill.common.config.DrillConfig;
+import org.apache.drill.common.expression.ErrorCollector;
+import org.apache.drill.common.expression.ErrorCollectorImpl;
+import org.apache.drill.common.expression.ExpressionPosition;
+import org.apache.drill.common.expression.FunctionRegistry;
+import org.apache.drill.common.expression.LogicalExpression;
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.common.expression.parser.ExprLexer;
+import org.apache.drill.common.expression.parser.ExprParser;
+import org.apache.drill.common.expression.parser.ExprParser.parse_return;
+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.exec.expr.CodeGenerator.HoldingContainer;
+import org.apache.drill.exec.expr.fn.FunctionImplementationRegistry;
+import org.apache.drill.exec.record.RecordBatch;
+import org.apache.drill.exec.record.RecordBatch.TypedFieldId;
+import org.apache.drill.exec.record.vector.Fixed4;
+import org.junit.After;
+import org.junit.Test;
+import org.slf4j.ILoggerFactory;
+import org.slf4j.LoggerFactory;
+
+import ch.qos.logback.classic.LoggerContext;
+
+public class ExpressionTest {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ExpressionTest.class);
+
+  @Test
+  public void testBasicExpression(@Injectable RecordBatch batch) throws Exception {
+    System.out.println(getExpressionCode("if(true) then 1 else 0 end", batch));
+  }
+
+  @Test
+  public void testSpecial(final @Injectable RecordBatch batch) throws Exception {
+    final TypedFieldId tfid = new TypedFieldId(MajorType.newBuilder().setMode(DataMode.OPTIONAL)
+        .setMinorType(MinorType.INT).build(), 0);
+
+    new NonStrictExpectations() {
+      {
+        batch.getValueVector(new SchemaPath("alpha", ExpressionPosition.UNKNOWN));
+        result = tfid;
+        batch.getValueVectorById(tfid.getFieldId(), Fixed4.class);
+        result = new Fixed4(null, null);
+      }
+
+    };
+    System.out.println(getExpressionCode("1 + 1", batch));
+  }
+
+  @Test
+  public void testSchemaExpression(final @Injectable RecordBatch batch) throws Exception {
+    final TypedFieldId tfid = new TypedFieldId(MajorType.newBuilder().setMode(DataMode.OPTIONAL)
+        .setMinorType(MinorType.BIGINT).build(), 0);
+
+    new Expectations() {
+      {
+        batch.getValueVector(new SchemaPath("alpha", ExpressionPosition.UNKNOWN));
+        result = tfid;
+        // batch.getValueVectorById(tfid); result = new Fixed4(null, null);
+      }
+
+    };
+    System.out.println(getExpressionCode("1 + alpha", batch));
+
+  }
+
+  // HELPER METHODS //
+
+  private LogicalExpression parseExpr(String expr) throws RecognitionException {
+    ExprLexer lexer = new ExprLexer(new ANTLRStringStream(expr));
+    CommonTokenStream tokens = new CommonTokenStream(lexer);
+    ExprParser parser = new ExprParser(tokens);
+    parser.setRegistry(new FunctionRegistry(DrillConfig.create()));
+    parse_return ret = parser.parse();
+    return ret.e;
+  }
+
+  private String getExpressionCode(String expression, RecordBatch batch) throws Exception {
+    LogicalExpression expr = parseExpr(expression);
+    ErrorCollector error = new ErrorCollectorImpl();
+    LogicalExpression materializedExpr = ExpressionTreeMaterializer.materialize(expr, batch, error);
+    if (error.getErrorCount() != 0) {
+      logger.error("Failure while materializing expression [{}].  Errors: {}", expression, error);
+      assertEquals(0, error.getErrorCount());
+    }
+
+    CodeGenerator cg = new CodeGenerator("setup", "eval", new FunctionImplementationRegistry(DrillConfig.create()));
+    cg.addNextWrite(new ValueVectorWriteExpression(-1, materializedExpr));
+    return cg.generate();
+  }
+
+  @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/ce0da88d/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/PerformanceTests.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/PerformanceTests.java b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/PerformanceTests.java
new file mode 100644
index 0000000..d2655e1
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/PerformanceTests.java
@@ -0,0 +1,32 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+
+package org.apache.drill.exec.physical.impl;
+
+import com.google.caliper.runner.CaliperMain;
+
+
+public class PerformanceTests {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(PerformanceTests.class);
+
+
+  public static void main(String[] args){
+    CaliperMain.main(TestExecutionAbstractions.class, args);
+    System.out.println("Hello");
+  }
+}

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

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/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 cac6aa2..3dc961b 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
@@ -29,7 +29,6 @@ import org.apache.drill.exec.record.RecordBatchLoader;
 import org.apache.drill.exec.rpc.user.QueryResultBatch;
 import org.apache.drill.exec.server.Drillbit;
 import org.apache.drill.exec.server.RemoteServiceSet;
-import org.apache.drill.exec.vector.ValueVector;
 import org.junit.Test;
 
 import com.carrotsearch.hppc.cursors.IntObjectCursor;
@@ -58,33 +57,37 @@ public class TestSimpleFragmentRun extends PopUnitTestBase {
       boolean schemaChanged = batchLoader.load(batch.getHeader().getDef(), batch.getData());
       boolean firstColumn = true;
 
-      // print headers.
-      if (schemaChanged) {
-        System.out.println("\n\n========NEW SCHEMA=========\n\n");
-        for (IntObjectCursor<ValueVector> v : batchLoader) {
+          // print headers.
+          if (schemaChanged) {
+            System.out.println("\n\n========NEW SCHEMA=========\n\n");
+            for (ValueVector<?> value : batchLoader) {
 
-          if (firstColumn) {
-            firstColumn = false;
-          } else {
-            System.out.print("\t");
+              if (firstColumn) {
+                firstColumn = false;
+              } else {
+                System.out.print("\t");
+              }
+              System.out.print(value.getField().getName());
+              System.out.print("[");
+              System.out.print(value.getField().getType().getMinorType());
+              System.out.print("]");
+            }
+            System.out.println();
           }
-          System.out.print(v.value.getField().getName());
-          System.out.print("[");
-          System.out.print(v.value.getField().getType().getMinorType());
-          System.out.print("]");
-        }
-        System.out.println();
-      }
 
 
-      for (int i = 0; i < batchLoader.getRecordCount(); i++) {
-        boolean first = true;
-        recordCount++;
-        for (IntObjectCursor<ValueVector> v : batchLoader) {
-          if (first) {
-            first = false;
-          } else {
-            System.out.print("\t");
+          for (int i = 0; i < batchLoader.getRecordCount(); i++) {
+            boolean first = true;
+            recordCount++;
+            for (ValueVector<?> value : batchLoader) {
+              if (first) {
+                first = false;
+              } else {
+                System.out.print("\t");
+              }
+              System.out.print(value.getObject(i));
+            }
+            if(!first) System.out.println();
           }
           System.out.print(v.value.getAccessor().getObject(i));
         }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/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 ab68ea2..99bf820 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
@@ -1,110 +1,203 @@
 package org.apache.drill.exec.record;
 
-import com.beust.jcommander.internal.Lists;
-import com.google.common.collect.Range;
-import org.apache.drill.common.expression.*;
-import org.apache.drill.common.expression.types.DataType;
-import org.apache.drill.common.physical.RecordField;
-import org.apache.drill.exec.exception.SchemaChangeException;
-import org.junit.Test;
-
-import java.util.List;
-
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 
+import java.util.List;
+
+import mockit.Injectable;
+import mockit.NonStrictExpectations;
+
+import org.apache.drill.common.expression.ArgumentValidator;
+import org.apache.drill.common.expression.ErrorCollector;
+import org.apache.drill.common.expression.ErrorCollectorImpl;
+import org.apache.drill.common.expression.ExpressionPosition;
+import org.apache.drill.common.expression.FieldReference;
+import org.apache.drill.common.expression.FunctionCall;
+import org.apache.drill.common.expression.FunctionDefinition;
+import org.apache.drill.common.expression.IfExpression;
+import org.apache.drill.common.expression.LogicalExpression;
+import org.apache.drill.common.expression.OutputTypeDeterminer;
+import org.apache.drill.common.expression.ValueExpressions;
+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 org.apache.drill.exec.exception.SchemaChangeException;
+import org.apache.drill.exec.expr.ExpressionTreeMaterializer;
+import org.apache.drill.exec.proto.SchemaDefProtos.FieldDef;
+import org.apache.drill.exec.proto.SchemaDefProtos.NamePart;
+import org.apache.drill.exec.record.RecordBatch.TypedFieldId;
+import org.junit.Test;
+
+import com.beust.jcommander.internal.Lists;
+import com.google.common.collect.Range;
+
 public class ExpressionTreeMaterializerTest {
-    @Test
-    public void testMaterializingConstantTree() throws SchemaChangeException {
-        ExpressionTreeMaterializer tm = new ExpressionTreeMaterializer();
-        ErrorCollector ec = new ErrorCollectorImpl();
-        BatchSchema schema = new BatchSchema.BatchSchemaBuilder().buildAndClear();
-        LogicalExpression expr = tm.Materialize(new ValueExpressions.LongExpression(1L), schema, ec);
-        assertTrue(expr instanceof ValueExpressions.LongExpression);
-        assertEquals(1L, ValueExpressions.LongExpression.class.cast(expr).getLong());
-        assertFalse(ec.hasErrors());
-    }
-
-    @Test
-    public void testMaterializingLateboundField() throws SchemaChangeException {
-        ExpressionTreeMaterializer tm = new ExpressionTreeMaterializer();
-        ErrorCollector ec = new ErrorCollectorImpl();
-        BatchSchema.BatchSchemaBuilder builder = new BatchSchema.BatchSchemaBuilder();
-        builder.addTypedField((short) 2, DataType.INT64, false, RecordField.ValueMode.RLE, Long.class);
-        LogicalExpression expr = tm.Materialize(new FieldReference("test"), builder.buildAndClear(), ec);
-        assertEquals(DataType.INT64, expr.getDataType());
-        assertFalse(ec.hasErrors());
-    }
-
-    @Test
-    public void testMaterializingLateboundTree() throws SchemaChangeException {
-        ExpressionTreeMaterializer tm = new ExpressionTreeMaterializer();
-        ErrorCollector ec = new ErrorCollectorImpl();
-        BatchSchema.BatchSchemaBuilder builder = new BatchSchema.BatchSchemaBuilder();
-        builder.addTypedField((short) 2, DataType.INT64, false, RecordField.ValueMode.RLE, Long.class);
-        LogicalExpression expr = new IfExpression.Builder().addCondition(
-                new IfExpression.IfCondition(new FieldReference("test"),
-                        new IfExpression.Builder().addCondition(new IfExpression.IfCondition(new ValueExpressions.LongExpression(1L), new FieldReference("test1"))).build()
-                )
-        ).build();
-        LogicalExpression newExpr = tm.Materialize(expr, builder.buildAndClear(), ec);
-        assertTrue(newExpr instanceof IfExpression);
-        IfExpression newIfExpr = (IfExpression) newExpr;
-        assertEquals(1, newIfExpr.conditions.size());
-        IfExpression.IfCondition ifCondition = newIfExpr.conditions.get(0);
-        assertEquals(DataType.INT64, ifCondition.condition.getDataType());
-        assertTrue(ifCondition.expression instanceof IfExpression);
-        newIfExpr = (IfExpression) ifCondition.expression;
-        assertEquals(1, newIfExpr.conditions.size());
-        ifCondition = newIfExpr.conditions.get(0);
-        assertEquals(DataType.INT64, ifCondition.expression.getDataType());
-        assertEquals(1L, ((ValueExpressions.LongExpression) ifCondition.condition).getLong());
-        assertFalse(ec.hasErrors());
-    }
-
-    @Test
-    public void testMaterializingLateboundTreeValidated() throws SchemaChangeException {
-        ExpressionTreeMaterializer tm = new ExpressionTreeMaterializer();
-        ErrorCollector ec = new ErrorCollector() {
-            boolean errorFound = false;
-            @Override
-            public void addGeneralError(String expr, String s) {errorFound = true;}
-            @Override
-            public void addUnexpectedArgumentType(String expr, String name, DataType actual, DataType[] expected, int argumentIndex) {}
-            @Override
-            public void addUnexpectedArgumentCount(String expr, int actual, Range<Integer> expected) {}
-            @Override
-            public void addUnexpectedArgumentCount(String expr, int actual, int expected) {}
-            @Override
-            public void addNonNumericType(String expr, DataType actual) {}
-            @Override
-            public void addUnexpectedType(String expr, int index, DataType actual) {}
-            @Override
-            public void addExpectedConstantValue(String expr, int actual, String s) {}
-            @Override
-            public boolean hasErrors() { return errorFound; }
-            @Override
-            public String toErrorString() { return ""; }
-        };
-        BatchSchema.BatchSchemaBuilder builder = new BatchSchema.BatchSchemaBuilder();
-        builder.addTypedField((short) 2, DataType.INT64, false, RecordField.ValueMode.RLE, Long.class);
-        LogicalExpression expr = new FunctionCall(FunctionDefinition.simple("testFunc", new ArgumentValidator() {
-            @Override
-            public void validateArguments(String expr, List<LogicalExpression> expressions, ErrorCollector errors) {
-                errors.addGeneralError(expr, "Error!");
-            }
-
-            @Override
-            public String[] getArgumentNamesByPosition() {
-                return new String[0];
-            }
-        }, OutputTypeDeterminer.FIXED_BOOLEAN), Lists.newArrayList((LogicalExpression) new FieldReference("test")));
-        LogicalExpression newExpr = tm.Materialize(expr, builder.buildAndClear(), ec);
-        assertTrue(newExpr instanceof FunctionCall);
-        FunctionCall funcExpr = (FunctionCall) newExpr;
-        assertEquals(1, funcExpr.args.size());
-        assertEquals(DataType.INT64, funcExpr.args.get(0).getDataType());
-        assertTrue(ec.hasErrors());
-    }
+
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ExpressionTreeMaterializerTest.class);
+
+  final MajorType boolConstant = MajorType.newBuilder().setMode(DataMode.REQUIRED).setMinorType(MinorType.BOOLEAN).build();
+  final MajorType bigIntType = MajorType.newBuilder().setMode(DataMode.REQUIRED).setMinorType(MinorType.BIGINT).build();
+  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());
+  }
+
+
+  @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);
+    assertTrue(expr instanceof ValueExpressions.LongExpression);
+    assertEquals(1L, ValueExpressions.LongExpression.class.cast(expr).getLong());
+    assertFalse(ec.hasErrors());
+  }
+
+  @Test
+  public void testMaterializingLateboundField(final @Injectable RecordBatch batch) throws SchemaChangeException {
+    final SchemaBuilder builder = BatchSchema.newBuilder();
+    builder.addField(getField(2, "test", bigIntType));
+    final BatchSchema schema = builder.build();
+    
+    new NonStrictExpectations() {
+      {
+        batch.getValueVector(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);
+    assertEquals(bigIntType, expr.getMajorType());
+    assertFalse(ec.hasErrors());
+  }
+
+  @Test
+  public void testMaterializingLateboundTree(final @Injectable RecordBatch batch) throws SchemaChangeException {
+    new NonStrictExpectations() {
+      {
+        batch.getValueVector(new FieldReference("test", ExpressionPosition.UNKNOWN)); result = new TypedFieldId(Types.required(MinorType.BOOLEAN), -4);
+        batch.getValueVector(new FieldReference("test1", ExpressionPosition.UNKNOWN)); result = new TypedFieldId(Types.required(MinorType.BIGINT), -5);
+      }
+    };
+    
+    ErrorCollector ec = new ErrorCollectorImpl();
+
+    
+      LogicalExpression expr = new IfExpression.Builder()
+        .addCondition(
+            new IfExpression.IfCondition( //
+                new FieldReference("test", ExpressionPosition.UNKNOWN), //
+                new IfExpression.Builder() //
+                    .addCondition( //
+                        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();
+    LogicalExpression newExpr = ExpressionTreeMaterializer.materialize(expr, batch, ec);
+    assertTrue(newExpr instanceof IfExpression);
+    IfExpression newIfExpr = (IfExpression) newExpr;
+    assertEquals(1, newIfExpr.conditions.size());
+    IfExpression.IfCondition ifCondition = newIfExpr.conditions.get(0);
+    assertTrue(ifCondition.expression instanceof IfExpression);
+    newIfExpr = (IfExpression) ifCondition.expression;
+    assertEquals(1, newIfExpr.conditions.size());
+    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());
+    assertFalse(ec.hasErrors());
+  }
+
+  @Test
+  public void testMaterializingLateboundTreeValidated(final @Injectable RecordBatch batch) throws SchemaChangeException {
+    ErrorCollector ec = new ErrorCollector() {
+      int errorCount = 0;
+
+      @Override
+      public void addGeneralError(ExpressionPosition expr, String s) {
+        errorCount++;
+      }
+
+      @Override
+      public void addUnexpectedArgumentType(ExpressionPosition expr, String name, MajorType actual, MajorType[] expected,
+          int argumentIndex) {
+        errorCount++;
+      }
+
+      @Override
+      public void addUnexpectedArgumentCount(ExpressionPosition expr, int actual, Range<Integer> expected) {
+        errorCount++;
+      }
+
+      @Override
+      public void addUnexpectedArgumentCount(ExpressionPosition expr, int actual, int expected) {
+        errorCount++;
+      }
+
+      @Override
+      public void addNonNumericType(ExpressionPosition expr, MajorType actual) {
+        errorCount++;
+      }
+
+      @Override
+      public void addUnexpectedType(ExpressionPosition expr, int index, MajorType actual) {
+        errorCount++;
+      }
+
+      @Override
+      public void addExpectedConstantValue(ExpressionPosition expr, int actual, String s) {
+        errorCount++;
+      }
+
+      @Override
+      public boolean hasErrors() {
+        return errorCount > 0;
+      }
+
+      @Override
+      public String toErrorString() {
+        return String.format("Found %s errors.", errorCount);
+      }
+
+      @Override
+      public int getErrorCount() {
+        return errorCount;
+      }
+    };
+
+    new NonStrictExpectations() {
+      {
+        batch.getValueVector(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) {
+            errors.addGeneralError(expr, "Error!");
+          }
+
+          @Override
+          public String[] getArgumentNamesByPosition() {
+            return new String[0];
+          }
+        }, OutputTypeDeterminer.FIXED_BOOLEAN), Lists.newArrayList((LogicalExpression) new FieldReference("test",
+        ExpressionPosition.UNKNOWN)), ExpressionPosition.UNKNOWN);
+    LogicalExpression newExpr = ExpressionTreeMaterializer.materialize(functionCallExpr, batch, ec);
+    assertTrue(newExpr instanceof FunctionCall);
+    FunctionCall funcExpr = (FunctionCall) newExpr;
+    assertEquals(1, funcExpr.args.size());
+    assertEquals(bigIntType, funcExpr.args.get(0).getMajorType());
+    assertEquals(1, ec.getErrorCount());
+    System.out.println(ec.toErrorString());
+  }
 }


[26/53] [abbrv] git commit: Fix bug where handlers would get prematurely removed before all streams completed. Updated tests to validate execution of single bit multi-fragment and two bit multi-fragment queries.

Posted by ja...@apache.org.
Fix bug where handlers would get prematurely removed before all streams completed.  Updated tests to validate execution of single bit multi-fragment and two bit multi-fragment queries.


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

Branch: refs/heads/master
Commit: 719c8beb7d3e86e2b256950ab316e5c8526994b8
Parents: 5c07ccd
Author: Jacques Nadeau <ja...@apache.org>
Authored: Mon Jun 10 11:58:25 2013 -0700
Committer: Jacques Nadeau <ja...@apache.org>
Committed: Mon Jun 10 12:06:36 2013 -0700

----------------------------------------------------------------------
 .../planner/fragment/SimpleParallelizer.java    | 48 ++++++++------
 .../work/batch/AbstractFragmentCollector.java   |  7 +-
 .../drill/exec/work/batch/BatchCollector.java   |  2 +-
 .../drill/exec/work/batch/IncomingBuffers.java  | 11 +++-
 .../work/foreman/RunningFragmentManager.java    |  1 +
 .../impl/TestDistributedFragmentRun.java        | 40 +++++++++++-
 .../drill/exec/pop/TestFragmentChecker.java     |  2 +-
 .../apache/drill/exec/pop/TestFragmenter.java   |  2 +-
 .../resources/physical_double_exchange.json     | 55 ++++++++++++++++
 .../test/resources/physical_simpleexchange.json | 55 ----------------
 .../resources/physical_single_exchange.json     | 57 ++++++++++------
 .../physical_single_exchange_double_entry.json  | 68 ++++++++++++++++++++
 12 files changed, 244 insertions(+), 104 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/719c8beb/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/SimpleParallelizer.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/SimpleParallelizer.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/SimpleParallelizer.java
index 8adb447..e3bcff0 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/SimpleParallelizer.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/SimpleParallelizer.java
@@ -6,9 +6,9 @@
  * to you under the Apache License, Version 2.0 (the
  * "License"); you may not use this file except in compliance
  * with the License.  You may obtain a copy of the License at
- * 
+ *
  * http://www.apache.org/licenses/LICENSE-2.0
- * 
+ *
  * Unless required by applicable law or agreed to in writing, software
  * distributed under the License is distributed on an "AS IS" BASIS,
  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
@@ -37,34 +37,40 @@ import com.fasterxml.jackson.core.JsonProcessingException;
 import com.google.common.base.Preconditions;
 import com.google.common.collect.Lists;
 
+/**
+ * The simple parallelizer determines the level of parallelization of a plan based on the cost of the underlying
+ * operations.  It doesn't take into account system load or other factors.  Based on the cost of the query, the
+ * parallelization for each major fragment will be determined.  Once the amount of parallelization is done, assignment
+ * is done based on round robin assignment ordered by operator affinity (locality) to available execution Drillbits.
+ */
 public class SimpleParallelizer {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(SimpleParallelizer.class);
-
   private final Materializer materializer = new Materializer();
 
   /**
    * Generate a set of assigned fragments based on the provided planningSet. Do not allow parallelization stages to go
    * beyond the global max width.
-   * 
-   * @param context
-   *          The current QueryContext.
-   * @param planningSet
-   *          The set of queries with collected statistics that we'll work with.
-   * @param globalMaxWidth
-   *          The maximum level or paralellization any stage of the query can do. Note that while this might be the
-   *          number of active Drillbits, realistically, this could be well beyond that number of we want to do things
-   *          like speed results return.
-   * @return The list of generatoe PlanFragment protobuf objects to be assigned out to the individual nodes.
-   * @throws FragmentSetupException
+   *
+   * @param foremanNode     The driving/foreman node for this query.  (this node)
+   * @param queryId         The queryId for this query.
+   * @param activeEndpoints The list of endpoints to consider for inclusion in planning this query.
+   * @param reader          Tool used to read JSON plans
+   * @param rootNode        The root node of the PhysicalPlan that we will parallelizing.
+   * @param planningSet     The set of queries with collected statistics that we'll work with.
+   * @param globalMaxWidth  The maximum level or parallelization any stage of the query can do. Note that while this
+   *                        might be the number of active Drillbits, realistically, this could be well beyond that
+   *                        number of we want to do things like speed results return.
+   * @return The list of generated PlanFragment protobuf objects to be assigned out to the individual nodes.
+   * @throws ExecutionSetupException
    */
   public QueryWorkUnit getFragments(DrillbitEndpoint foremanNode, QueryId queryId, Collection<DrillbitEndpoint> activeEndpoints, PhysicalPlanReader reader, Fragment rootNode, PlanningSet planningSet,
-      int globalMaxWidth) throws ExecutionSetupException {
+                                    int globalMaxWidth) throws ExecutionSetupException {
     assignEndpoints(activeEndpoints, planningSet, globalMaxWidth);
     return generateWorkUnit(foremanNode, queryId, reader, rootNode, planningSet);
   }
 
   private QueryWorkUnit generateWorkUnit(DrillbitEndpoint foremanNode, QueryId queryId, PhysicalPlanReader reader, Fragment rootNode,
-      PlanningSet planningSet) throws ExecutionSetupException {
+                                         PlanningSet planningSet) throws ExecutionSetupException {
 
     List<PlanFragment> fragments = Lists.newArrayList();
 
@@ -82,8 +88,8 @@ public class SimpleParallelizer {
       if (isRootNode && wrapper.getWidth() != 1)
         throw new FragmentSetupException(
             String.format(
-                    "Failure while trying to setup fragment.  The root fragment must always have parallelization one.  In the current case, the width was set to %d.",
-                    wrapper.getWidth()));
+                "Failure while trying to setup fragment.  The root fragment must always have parallelization one.  In the current case, the width was set to %d.",
+                wrapper.getWidth()));
       // a fragment is self driven if it doesn't rely on any other exchanges.
       boolean isLeafFragment = node.getReceivingExchangePairs().size() == 0;
 
@@ -92,7 +98,7 @@ public class SimpleParallelizer {
         IndexedFragmentNode iNode = new IndexedFragmentNode(minorFragmentId, wrapper);
         PhysicalOperator op = physicalOperatorRoot.accept(materializer, iNode);
         Preconditions.checkArgument(op instanceof FragmentRoot);
-        FragmentRoot root = (FragmentRoot) op; 
+        FragmentRoot root = (FragmentRoot) op;
 
         // get plan as JSON
         String plan;
@@ -101,7 +107,7 @@ public class SimpleParallelizer {
         } catch (JsonProcessingException e) {
           throw new FragmentSetupException("Failure while trying to convert fragment into json.", e);
         }
-        
+
         FragmentHandle handle = FragmentHandle //
             .newBuilder() //
             .setMajorFragmentId(wrapper.getMajorFragmentId()) //
@@ -134,7 +140,7 @@ public class SimpleParallelizer {
   }
 
   private void assignEndpoints(Collection<DrillbitEndpoint> allNodes, PlanningSet planningSet,
-      int globalMaxWidth) throws PhysicalOperatorSetupException {
+                               int globalMaxWidth) throws PhysicalOperatorSetupException {
     // First we determine the amount of parallelization for a fragment. This will be between 1 and maxWidth based on
     // cost. (Later could also be based on cluster operation.) then we decide endpoints based on affinity (later this
     // could be based on endpoint load)

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/719c8beb/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/AbstractFragmentCollector.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/AbstractFragmentCollector.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/AbstractFragmentCollector.java
index ec03392..dd55377 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/AbstractFragmentCollector.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/AbstractFragmentCollector.java
@@ -29,6 +29,7 @@ import org.apache.drill.exec.rpc.RemoteConnection.ConnectionThrottle;
 import com.google.common.base.Preconditions;
 
 public abstract class AbstractFragmentCollector implements BatchCollector{
+
   private final List<DrillbitEndpoint> incoming;
   private final int oppositeMajorFragmentId;
   private final AtomicIntegerArray remainders;
@@ -83,7 +84,11 @@ public abstract class AbstractFragmentCollector implements BatchCollector{
     return decremented;
   }
 
-  
+
+  @Override
+  public int getTotalIncomingFragments() {
+    return incoming.size();
+  }
 
   protected abstract RawBatchBuffer getBuffer(int minorFragmentId);
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/719c8beb/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/BatchCollector.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/BatchCollector.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/BatchCollector.java
index b5a497e..89a889c 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/BatchCollector.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/BatchCollector.java
@@ -27,5 +27,5 @@ interface BatchCollector {
   public boolean batchArrived(ConnectionThrottle throttle, int minorFragmentId, RawFragmentBatch batch);
   public int getOppositeMajorFragmentId();
   public RawBatchBuffer[] getBuffers();
-
+  public int getTotalIncomingFragments();
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/719c8beb/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/IncomingBuffers.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/IncomingBuffers.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/IncomingBuffers.java
index 264c4b9..7f14069 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/IncomingBuffers.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/IncomingBuffers.java
@@ -33,7 +33,7 @@ import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
 
 /**
- * Determines when a particular fragment has enough data for each of its receiving exchanges to commence execution.
+ * Determines when a particular fragment has enough data for each of its receiving exchanges to commence execution.  Also monitors whether we've collected all incoming data.
  */
 public class IncomingBuffers {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(IncomingBuffers.class);
@@ -48,8 +48,15 @@ public class IncomingBuffers {
     root.accept(reqFrags, counts);
     
     logger.debug("Came up with a list of {} required fragments.  Fragments {}", remainingRequired.get(), counts);
-    streamsRemaining.set(remainingRequired.get());
     fragCounts = ImmutableMap.copyOf(counts);
+
+    // Determine the total number of incoming streams that will need to be completed before we are finished.
+    int totalStreams = 0;
+    for(BatchCollector bc : fragCounts.values()){
+      totalStreams += bc.getTotalIncomingFragments();
+    }
+    assert totalStreams >= remainingRequired.get() : String.format("Total Streams %d should be more than the minimum number of streams to commence (%d).  It isn't.", totalStreams, remainingRequired.get());
+    streamsRemaining.set(totalStreams);
   }
 
   public boolean batchArrived(ConnectionThrottle throttle, RawFragmentBatch batch) throws FragmentSetupException {

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/719c8beb/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 f069db7..5bb1ff2 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
@@ -96,6 +96,7 @@ class RunningFragmentManager implements FragmentStatusListener{
   }
     
   private void sendRemoteFragment(PlanFragment fragment){
+    logger.debug("Sending remote fragment to node {} with data {}", fragment.getAssignment(), fragment.getFragmentJson());
     map.put(fragment.getHandle(), new FragmentData(fragment.getHandle(), fragment.getAssignment(), false));
     FragmentSubmitListener listener = new FragmentSubmitListener(fragment.getAssignment(), fragment);
     tun.get(fragment.getAssignment()).sendFragment(listener, fragment);

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/719c8beb/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestDistributedFragmentRun.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestDistributedFragmentRun.java b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestDistributedFragmentRun.java
index d4147ee..9bcd84f 100644
--- a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestDistributedFragmentRun.java
+++ b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestDistributedFragmentRun.java
@@ -39,7 +39,7 @@ public class TestDistributedFragmentRun extends PopUnitTestBase{
   
   
   @Test 
-  public void oneBitOneExchangeRun() throws Exception{
+  public void oneBitOneExchangeOneEntryRun() throws Exception{
     RemoteServiceSet serviceSet = RemoteServiceSet.getLocalServiceSet();
 
     try(Drillbit bit1 = new Drillbit(CONFIG, serviceSet); DrillClient client = new DrillClient(CONFIG, serviceSet.getCoordinator());){
@@ -55,4 +55,42 @@ public class TestDistributedFragmentRun extends PopUnitTestBase{
     
 
   }
+
+
+  @Test
+  public void oneBitOneExchangeTwoEntryRun() throws Exception{
+    RemoteServiceSet serviceSet = RemoteServiceSet.getLocalServiceSet();
+
+    try(Drillbit bit1 = new Drillbit(CONFIG, serviceSet); DrillClient client = new DrillClient(CONFIG, serviceSet.getCoordinator());){
+      bit1.run();
+      client.connect();
+      List<QueryResultBatch> results = client.runQuery(QueryType.PHYSICAL, Files.toString(FileUtils.getResourceAsFile("/physical_single_exchange_double_entry.json"), Charsets.UTF_8));
+      int count = 0;
+      for(QueryResultBatch b : results){
+        count += b.getHeader().getRowCount();
+      }
+      assertEquals(200, count);
+    }
+
+
+  }
+
+  @Test
+    public void twoBitOneExchangeTwoEntryRun() throws Exception{
+      RemoteServiceSet serviceSet = RemoteServiceSet.getLocalServiceSet();
+
+      try(Drillbit bit1 = new Drillbit(CONFIG, serviceSet); Drillbit bit2 = new Drillbit(CONFIG, serviceSet); DrillClient client = new DrillClient(CONFIG, serviceSet.getCoordinator());){
+        bit1.run();
+        bit2.run();
+        client.connect();
+        List<QueryResultBatch> results = client.runQuery(QueryType.PHYSICAL, Files.toString(FileUtils.getResourceAsFile("/physical_single_exchange_double_entry.json"), Charsets.UTF_8));
+        int count = 0;
+      for(QueryResultBatch b : results){
+        count += b.getHeader().getRowCount();
+      }
+      assertEquals(200, count);
+    }
+
+
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/719c8beb/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/pop/TestFragmentChecker.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/pop/TestFragmentChecker.java b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/pop/TestFragmentChecker.java
index 710326e..ec751f1 100644
--- a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/pop/TestFragmentChecker.java
+++ b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/pop/TestFragmentChecker.java
@@ -40,7 +40,7 @@ public class TestFragmentChecker extends PopUnitTestBase{
   
   @Test
   public void checkSimpleExchangePlan() throws Exception{
-    print("/physical_simpleexchange.json", 2, 3);
+    print("/physical_double_exchange.json", 2, 3);
 
   }
   

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/719c8beb/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/pop/TestFragmenter.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/pop/TestFragmenter.java b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/pop/TestFragmenter.java
index 27bee40..a4d354b 100644
--- a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/pop/TestFragmenter.java
+++ b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/pop/TestFragmenter.java
@@ -45,7 +45,7 @@ public class TestFragmenter extends PopUnitTestBase {
   @Test
   public void ensureThreeFragments() throws FragmentSetupException, IOException {
     PhysicalPlanReader ppr = new PhysicalPlanReader(CONFIG, CONFIG.getMapper(), DrillbitEndpoint.getDefaultInstance());
-    Fragment b = getRootFragment(ppr, "/physical_simpleexchange.json");
+    Fragment b = getRootFragment(ppr, "/physical_double_exchange.json");
     logger.debug("Fragment Node {}", b);
     assertEquals(3, getFragmentCount(b));
     assertEquals(1, b.getReceivingExchangePairs().size());

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/719c8beb/sandbox/prototype/exec/java-exec/src/test/resources/physical_double_exchange.json
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/resources/physical_double_exchange.json b/sandbox/prototype/exec/java-exec/src/test/resources/physical_double_exchange.json
new file mode 100644
index 0000000..85823cf
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/test/resources/physical_double_exchange.json
@@ -0,0 +1,55 @@
+{
+    head:{
+        type:"APACHE_DRILL_PHYSICAL",
+        version:"1",
+        generator:{
+            type:"manual"
+        }
+    },
+    graph:[
+        {
+            @id:1,
+            pop:"mock-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"}
+            	]},
+            	{records: 100, types: [
+            	  {name: "blue", type: "INT", mode: "REQUIRED"},
+            	  {name: "red", type: "BIGINT", mode: "REQUIRED"},
+            	  {name: "green", type: "INT", mode: "REQUIRED"}
+            	]}
+            ]
+        },
+        {
+            @id:2,
+            pop: "hash-to-random-exchange",
+            child: 1
+        },
+        {
+            @id:3,
+            child: 2,
+            pop:"filter",
+            expr: "b > 5",
+            selectivity: 0.8
+        },
+        {
+            @id: 4,
+            child: 3,
+            pop: "mock-store"
+        },
+        {
+            @id:5,
+            child: 4,
+            pop: "union-exchange"
+        },
+        {
+            @id: 6,
+            child: 5,
+            pop: "screen"
+        }
+    ]
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/719c8beb/sandbox/prototype/exec/java-exec/src/test/resources/physical_simpleexchange.json
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/resources/physical_simpleexchange.json b/sandbox/prototype/exec/java-exec/src/test/resources/physical_simpleexchange.json
deleted file mode 100644
index 85823cf..0000000
--- a/sandbox/prototype/exec/java-exec/src/test/resources/physical_simpleexchange.json
+++ /dev/null
@@ -1,55 +0,0 @@
-{
-    head:{
-        type:"APACHE_DRILL_PHYSICAL",
-        version:"1",
-        generator:{
-            type:"manual"
-        }
-    },
-    graph:[
-        {
-            @id:1,
-            pop:"mock-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"}
-            	]},
-            	{records: 100, types: [
-            	  {name: "blue", type: "INT", mode: "REQUIRED"},
-            	  {name: "red", type: "BIGINT", mode: "REQUIRED"},
-            	  {name: "green", type: "INT", mode: "REQUIRED"}
-            	]}
-            ]
-        },
-        {
-            @id:2,
-            pop: "hash-to-random-exchange",
-            child: 1
-        },
-        {
-            @id:3,
-            child: 2,
-            pop:"filter",
-            expr: "b > 5",
-            selectivity: 0.8
-        },
-        {
-            @id: 4,
-            child: 3,
-            pop: "mock-store"
-        },
-        {
-            @id:5,
-            child: 4,
-            pop: "union-exchange"
-        },
-        {
-            @id: 6,
-            child: 5,
-            pop: "screen"
-        }
-    ]
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/719c8beb/sandbox/prototype/exec/java-exec/src/test/resources/physical_single_exchange.json
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/resources/physical_single_exchange.json b/sandbox/prototype/exec/java-exec/src/test/resources/physical_single_exchange.json
index 0e1921e..c781721 100644
--- a/sandbox/prototype/exec/java-exec/src/test/resources/physical_single_exchange.json
+++ b/sandbox/prototype/exec/java-exec/src/test/resources/physical_single_exchange.json
@@ -1,33 +1,48 @@
 {
-    head:{
-        type:"APACHE_DRILL_PHYSICAL",
-        version:"1",
-        generator:{
-            type:"manual"
+    "head": {
+        "type": "APACHE_DRILL_PHYSICAL",
+        "version": "1",
+        "generator": {
+            "type": "manual"
         }
     },
-    graph:[
+    "graph": [
         {
-            @id:1,
-            pop:"mock-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": 1,
+            "pop": "mock-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,
-            child: 1,
-            pop: "union-exchange"
+            "@id": 2,
+            "child": 1,
+            "pop": "union-exchange"
         },
         {
-            @id: 3,
-            child: 2,
-            pop: "screen"
+            "@id": 3,
+            "child": 2,
+            "pop": "screen"
         }
     ]
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/719c8beb/sandbox/prototype/exec/java-exec/src/test/resources/physical_single_exchange_double_entry.json
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/resources/physical_single_exchange_double_entry.json b/sandbox/prototype/exec/java-exec/src/test/resources/physical_single_exchange_double_entry.json
new file mode 100644
index 0000000..4146dd0
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/test/resources/physical_single_exchange_double_entry.json
@@ -0,0 +1,68 @@
+{
+    "head": {
+        "type": "APACHE_DRILL_PHYSICAL",
+        "version": "1",
+        "generator": {
+            "type": "manual"
+        }
+    },
+    "graph": [
+        {
+            "@id": 1,
+            "pop": "mock-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"
+                        }
+                    ]
+                },
+                {
+                    "records": 100,
+                    "types": [
+                        {
+                            "name": "blue",
+                            "type": "INT",
+                            "mode": "REQUIRED"
+                        },
+                        {
+                            "name": "red",
+                            "type": "BIGINT",
+                            "mode": "REQUIRED"
+                        },
+                        {
+                            "name": "green",
+                            "type": "INT",
+                            "mode": "REQUIRED"
+                        }
+                    ]
+                }
+            ]
+        },
+        {
+            "@id": 2,
+            "child": 1,
+            "pop": "union-exchange"
+        },
+        {
+            "@id": 3,
+            "child": 2,
+            "pop": "screen"
+        }
+    ]
+}
\ No newline at end of file


[41/53] [abbrv] Types transition

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockRecordReader.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockRecordReader.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockRecordReader.java
index 0f4619c..7b76d05 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockRecordReader.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockRecordReader.java
@@ -18,15 +18,18 @@
 package org.apache.drill.exec.physical.config;
 
 import org.apache.drill.common.exceptions.ExecutionSetupException;
+import org.apache.drill.common.expression.ExpressionPosition;
 import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.common.types.TypeProtos.DataMode;
+import org.apache.drill.common.types.TypeProtos.MajorType;
 import org.apache.drill.exec.exception.SchemaChangeException;
 import org.apache.drill.exec.ops.FragmentContext;
 import org.apache.drill.exec.physical.config.MockScanPOP.MockColumn;
 import org.apache.drill.exec.physical.config.MockScanPOP.MockScanEntry;
 import org.apache.drill.exec.physical.impl.OutputMutator;
-import org.apache.drill.exec.proto.SchemaDefProtos.DataMode;
-import org.apache.drill.exec.proto.SchemaDefProtos.MajorType;
 import org.apache.drill.exec.record.MaterializedField;
+import org.apache.drill.exec.record.vector.TypeHelper;
+import org.apache.drill.exec.record.vector.ValueVector;
 import org.apache.drill.exec.store.RecordReader;
 import org.apache.drill.exec.vector.FixedWidthVector;
 import org.apache.drill.exec.vector.NonRepeatedMutator;
@@ -57,12 +60,12 @@ public class MockRecordReader implements RecordReader {
     return x;
   }
 
-  private ValueVector getVector(int fieldId, String name, MajorType type, int length) {
+  private ValueVector<?> getVector(String name, MajorType type, int length) {
     assert context != null : "Context shouldn't be null.";
     if(type.getMode() != DataMode.REQUIRED) throw new UnsupportedOperationException();
     
-    MaterializedField f = MaterializedField.create(new SchemaPath(name), fieldId, 0, type);
-    ValueVector v;
+    MaterializedField f = MaterializedField.create(new SchemaPath(name, ExpressionPosition.UNKNOWN), type);
+    ValueVector<?> v;
     v = TypeHelper.getNewVector(f, context.getAllocator());
     if(v instanceof FixedWidthVector){
       ((FixedWidthVector)v).allocateNew(length);  
@@ -85,8 +88,8 @@ public class MockRecordReader implements RecordReader {
       batchRecordCount = 250000 / estimateRowSize;
 
       for (int i = 0; i < config.getTypes().length; i++) {
-        valueVectors[i] = getVector(i, config.getTypes()[i].getName(), config.getTypes()[i].getMajorType(), batchRecordCount);
-        output.addField(i, valueVectors[i]);
+        valueVectors[i] = getVector(config.getTypes()[i].getName(), config.getTypes()[i].getMajorType(), batchRecordCount);
+        output.addField(valueVectors[i]);
       }
       output.setNewSchema();
     } catch (SchemaChangeException e) {
@@ -128,7 +131,7 @@ public class MockRecordReader implements RecordReader {
   public void cleanup() {
     for (int i = 0; i < valueVectors.length; i++) {
       try {
-        output.removeField(valueVectors[i].getField().getFieldId());
+        output.removeField(valueVectors[i].getField());
       } catch (SchemaChangeException e) {
         logger.warn("Failure while trying to remove field.", e);
       }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockScanPOP.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockScanPOP.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockScanPOP.java
index 40227e5..3802ce2 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockScanPOP.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockScanPOP.java
@@ -21,6 +21,9 @@ import java.util.Collections;
 import java.util.LinkedList;
 import java.util.List;
 
+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.exec.physical.EndpointAffinity;
 import org.apache.drill.exec.physical.OperatorCost;
 import org.apache.drill.exec.physical.ReadEntry;
@@ -29,10 +32,7 @@ import org.apache.drill.exec.physical.base.PhysicalOperator;
 import org.apache.drill.exec.physical.base.Scan;
 import org.apache.drill.exec.physical.base.Size;
 import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
-import org.apache.drill.exec.proto.SchemaDefProtos.DataMode;
-import org.apache.drill.exec.proto.SchemaDefProtos.MajorType;
-import org.apache.drill.exec.proto.SchemaDefProtos.MinorType;
-import org.apache.drill.exec.vector.TypeHelper;
+import org.apache.drill.exec.record.vector.TypeHelper;
 
 import com.fasterxml.jackson.annotation.JsonCreator;
 import com.fasterxml.jackson.annotation.JsonIgnore;

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/FilterRecordBatch.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/FilterRecordBatch.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/FilterRecordBatch.java
index 6440d98..80d48f4 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/FilterRecordBatch.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/FilterRecordBatch.java
@@ -17,12 +17,21 @@
  ******************************************************************************/
 package org.apache.drill.exec.physical.impl;
 
+import org.apache.drill.common.expression.SchemaPath;
 import org.apache.drill.exec.ops.FragmentContext;
 import org.apache.drill.exec.record.BatchSchema;
 import org.apache.drill.exec.record.InvalidValueAccessor;
 import org.apache.drill.exec.record.RecordBatch;
+import org.apache.drill.exec.record.RecordBatch.TypedFieldId;
+import org.apache.drill.exec.record.WritableBatch;
 import org.apache.drill.exec.record.vector.SelectionVector;
+<<<<<<< HEAD
 import org.apache.drill.exec.vector.ValueVector;
+=======
+import org.apache.drill.exec.record.vector.SelectionVector2;
+import org.apache.drill.exec.record.vector.SelectionVector4;
+import org.apache.drill.exec.record.vector.ValueVector;
+>>>>>>> Build working
 
 public abstract class FilterRecordBatch implements RecordBatch {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(FilterRecordBatch.class);
@@ -57,8 +66,33 @@ public abstract class FilterRecordBatch implements RecordBatch {
     incoming.kill();
   }
 
+
+  @Override
+  public SelectionVector2 getSelectionVector2() {
+    return null;
+  }
+
+  @Override
+  public SelectionVector4 getSelectionVector4() {
+    return null;
+  }
+
+  @Override
+  public TypedFieldId getValueVector(SchemaPath path) {
+    return null;
+  }
+
+  @Override
+  public <T extends ValueVector<T>> T getValueVectorById(int fieldId, Class<?> vvClass) {
+    return null;
+  }
+
   @Override
+<<<<<<< HEAD
   public <T extends ValueVector> T getValueVector(int fieldId, Class<T> clazz) throws InvalidValueAccessor {
+=======
+  public WritableBatch getWritableBatch() {
+>>>>>>> Build working
     return null;
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/OutputMutator.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/OutputMutator.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/OutputMutator.java
index b3b9f5f..5247d08 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/OutputMutator.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/OutputMutator.java
@@ -18,10 +18,19 @@
 package org.apache.drill.exec.physical.impl;
 
 import org.apache.drill.exec.exception.SchemaChangeException;
+<<<<<<< HEAD
 import org.apache.drill.exec.vector.ValueVector;
 
 public interface OutputMutator {
   public void removeField(int fieldId) throws SchemaChangeException;
   public void addField(int fieldId, ValueVector vector) throws SchemaChangeException ;
+=======
+import org.apache.drill.exec.record.MaterializedField;
+import org.apache.drill.exec.record.vector.ValueVector;
+
+public interface OutputMutator {
+  public void removeField(MaterializedField field) throws SchemaChangeException;
+  public void addField(ValueVector<?> vector) throws SchemaChangeException ;
+>>>>>>> Build working
   public void setNewSchema() throws SchemaChangeException ;
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScanBatch.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScanBatch.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScanBatch.java
index 2f3e1fe..1e0c000 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScanBatch.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScanBatch.java
@@ -18,20 +18,37 @@
 package org.apache.drill.exec.physical.impl;
 
 import java.util.Iterator;
+<<<<<<< HEAD
+=======
+import java.util.List;
+import java.util.Map;
+>>>>>>> Build working
 
 import org.apache.drill.common.exceptions.ExecutionSetupException;
+import org.apache.drill.common.expression.SchemaPath;
 import org.apache.drill.exec.exception.SchemaChangeException;
 import org.apache.drill.exec.ops.FragmentContext;
 import org.apache.drill.exec.record.BatchSchema;
-import org.apache.drill.exec.record.InvalidValueAccessor;
+import org.apache.drill.exec.record.MaterializedField;
 import org.apache.drill.exec.record.RecordBatch;
 import org.apache.drill.exec.record.SchemaBuilder;
 import org.apache.drill.exec.record.WritableBatch;
+<<<<<<< HEAD
+=======
+import org.apache.drill.exec.record.vector.SelectionVector2;
+import org.apache.drill.exec.record.vector.SelectionVector4;
+import org.apache.drill.exec.record.vector.ValueVector;
+>>>>>>> Build working
 import org.apache.drill.exec.store.RecordReader;
 import org.apache.drill.exec.vector.ValueVector;
 
+<<<<<<< HEAD
 import com.carrotsearch.hppc.IntObjectOpenHashMap;
 import com.carrotsearch.hppc.procedures.IntObjectProcedure;
+=======
+import com.beust.jcommander.internal.Lists;
+import com.beust.jcommander.internal.Maps;
+>>>>>>> Build working
 
 /**
  * Record batch used for a particular scan. Operators against one or more
@@ -39,7 +56,14 @@ import com.carrotsearch.hppc.procedures.IntObjectProcedure;
 public class ScanBatch implements RecordBatch {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ScanBatch.class);
 
+<<<<<<< HEAD
   private IntObjectOpenHashMap<ValueVector> fields = new IntObjectOpenHashMap<ValueVector>();
+=======
+  final List<ValueVector<?>> vectors = Lists.newLinkedList();
+  final Map<MaterializedField, ValueVector<?>> fieldVectorMap = Maps.newHashMap();
+  
+  private VectorHolder holder = new VectorHolder(vectors);
+>>>>>>> Build working
   private BatchSchema schema;
   private int recordCount;
   private boolean schemaChanged = true;
@@ -83,6 +107,7 @@ public class ScanBatch implements RecordBatch {
   }
 
   private void releaseAssets() {
+<<<<<<< HEAD
     fields.forEach(new IntObjectProcedure<ValueVector>() {
       @Override
       public void apply(int key, ValueVector value) {
@@ -102,9 +127,13 @@ public class ScanBatch implements RecordBatch {
       throw new InvalidValueAccessor(String.format(
           "You requested a field accessor of type %s for field id %d but the actual type was %s.",
           clazz.getCanonicalName(), fieldId, vector.getClass().getCanonicalName()));
+=======
+    for(ValueVector<?> v : vectors){
+      v.close();
+>>>>>>> Build working
     }
   }
-
+  
   @Override
   public IterOutcome next() {
     while ((recordCount = currentReader.next()) == 0) {
@@ -132,11 +161,34 @@ public class ScanBatch implements RecordBatch {
     }
   }
 
+  
+  @Override
+  public SelectionVector2 getSelectionVector2() {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public SelectionVector4 getSelectionVector4() {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public TypedFieldId getValueVector(SchemaPath path) {
+    return holder.getValueVector(path);
+  }
+
+  @Override
+  public <T extends ValueVector<T>> T getValueVectorById(int fieldId, Class<?> clazz) {
+    return holder.getValueVector(fieldId, clazz);
+  }
+
+
   private class Mutator implements OutputMutator {
     private SchemaBuilder builder = BatchSchema.newBuilder();
     
-    public void removeField(int fieldId) throws SchemaChangeException {
+    public void removeField(MaterializedField field) throws SchemaChangeException {
       schemaChanged();
+<<<<<<< HEAD
       ValueVector v = fields.remove(fieldId);
       if (v == null) throw new SchemaChangeException("Failure attempting to remove an unknown field.");
       v.close();
@@ -146,8 +198,18 @@ public class ScanBatch implements RecordBatch {
       schemaChanged();
       ValueVector v = fields.put(fieldId, vector);
       vector.getField();
+=======
+      ValueVector<?> vector = fieldVectorMap.remove(field);
+      if (vector == null) throw new SchemaChangeException("Failure attempting to remove an unknown field.");
+      vectors.remove(vector);
+      vector.close();
+    }
+
+    public void addField(ValueVector<?> vector) {
+      vectors.add(vector);
+      fieldVectorMap.put(vector.getField(), vector);
+>>>>>>> Build working
       builder.addField(vector.getField());
-      if (v != null) v.close();
     }
 
     @Override
@@ -160,7 +222,7 @@ public class ScanBatch implements RecordBatch {
 
   @Override
   public WritableBatch getWritableBatch() {
-    return WritableBatch.get(this.getRecordCount(), fields);
+    return WritableBatch.get(this.getRecordCount(), vectors);
   }
   
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/VectorHolder.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/VectorHolder.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/VectorHolder.java
new file mode 100644
index 0000000..4209daa
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/VectorHolder.java
@@ -0,0 +1,39 @@
+package org.apache.drill.exec.physical.impl;
+
+import java.util.List;
+
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.exec.record.RecordBatch.TypedFieldId;
+import org.apache.drill.exec.record.vector.ValueVector;
+
+public class VectorHolder {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(VectorHolder.class);
+  
+  private List<ValueVector<?>> vectors;
+
+  public VectorHolder(List<ValueVector<?>> vectors) {
+    super();
+    this.vectors = vectors;
+  }
+  
+  public TypedFieldId getValueVector(SchemaPath path) {
+    for(int i =0; i < vectors.size(); i++){
+      ValueVector<?> vv = vectors.get(i);
+      if(vv.getField().matches(path)) return new TypedFieldId(vv.getField().getType(), i); 
+    }
+    return null;
+  }
+  
+  @SuppressWarnings("unchecked")
+  public <T extends ValueVector<T>> T getValueVector(int fieldId, Class<?> clazz) {
+    ValueVector<?> v = vectors.get(fieldId);
+    assert v != null;
+    if (v.getClass() != clazz){
+      logger.warn(String.format(
+          "Failure while reading vector.  Expected vector class of %s but was holding vector class %s.",
+          clazz.getCanonicalName(), v.getClass().getCanonicalName()));
+      return null;
+    }
+    return (T) v;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/WireRecordBatch.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/WireRecordBatch.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/WireRecordBatch.java
index fcbd272..f4921b5 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/WireRecordBatch.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/WireRecordBatch.java
@@ -17,17 +17,26 @@
  ******************************************************************************/
 package org.apache.drill.exec.physical.impl;
 
+<<<<<<< HEAD
+=======
+import org.apache.drill.common.expression.SchemaPath;
+>>>>>>> Build working
 import org.apache.drill.exec.exception.SchemaChangeException;
 import org.apache.drill.exec.ops.FragmentContext;
 import org.apache.drill.exec.proto.UserBitShared.RecordBatchDef;
 import org.apache.drill.exec.record.BatchSchema;
-import org.apache.drill.exec.record.InvalidValueAccessor;
 import org.apache.drill.exec.record.RawFragmentBatch;
 import org.apache.drill.exec.record.RawFragmentBatchProvider;
 import org.apache.drill.exec.record.RecordBatch;
 import org.apache.drill.exec.record.RecordBatchLoader;
 import org.apache.drill.exec.record.WritableBatch;
+<<<<<<< HEAD
 import org.apache.drill.exec.vector.ValueVector;
+=======
+import org.apache.drill.exec.record.vector.SelectionVector2;
+import org.apache.drill.exec.record.vector.SelectionVector4;
+import org.apache.drill.exec.record.vector.ValueVector;
+>>>>>>> Build working
 
 public class WireRecordBatch implements RecordBatch{
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(WireRecordBatch.class);
@@ -64,11 +73,32 @@ public class WireRecordBatch implements RecordBatch{
     fragProvider.kill(context);
   }
 
+  
+  @Override
+  public SelectionVector2 getSelectionVector2() {
+    throw new UnsupportedOperationException();
+  }
+
   @Override
+  public SelectionVector4 getSelectionVector4() {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public TypedFieldId getValueVector(SchemaPath path) {
+    return batchLoader.getValueVector(path);
+  }
+
+  @Override
+<<<<<<< HEAD
   public <T extends ValueVector> T getValueVector(int fieldId, Class<T> clazz) throws InvalidValueAccessor {
+=======
+  public <T extends ValueVector<T>> T getValueVectorById(int fieldId, Class<?> clazz) {
+>>>>>>> Build working
     return batchLoader.getValueVector(fieldId, clazz);
   }
 
+  
   @Override
   public IterOutcome next() {
     RawFragmentBatch batch = fragProvider.getNext();

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/filter/EvalSetupException.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/filter/EvalSetupException.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/filter/EvalSetupException.java
new file mode 100644
index 0000000..f547989
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/filter/EvalSetupException.java
@@ -0,0 +1,5 @@
+package org.apache.drill.exec.physical.impl.filter;
+
+public class EvalSetupException extends Exception{
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(EvalSetupException.class);
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/filter/EvaluationPredicate.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/filter/EvaluationPredicate.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/filter/EvaluationPredicate.java
new file mode 100644
index 0000000..3176c41
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/filter/EvaluationPredicate.java
@@ -0,0 +1,13 @@
+package org.apache.drill.exec.physical.impl.filter;
+
+import org.apache.drill.exec.record.vector.SelectionVector2;
+
+public class EvaluationPredicate {
+  private SelectionVector2 vector;
+  
+  EvaluationPredicate(String pred){
+    
+  }
+  
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/filter/ExampleFilter.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/filter/ExampleFilter.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/filter/ExampleFilter.java
new file mode 100644
index 0000000..158350f
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/filter/ExampleFilter.java
@@ -0,0 +1,113 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+
+package org.apache.drill.exec.physical.impl.filter;
+
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.exec.exception.SchemaChangeException;
+import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.record.BatchSchema;
+import org.apache.drill.exec.record.InvalidValueAccessor;
+import org.apache.drill.exec.record.RecordBatch;
+import org.apache.drill.exec.record.RecordBatch.TypedFieldId;
+import org.apache.drill.exec.record.WritableBatch;
+import org.apache.drill.exec.record.vector.SelectionVector2;
+import org.apache.drill.exec.record.vector.SelectionVector4;
+import org.apache.drill.exec.record.vector.ValueVector;
+
+public class ExampleFilter implements RecordBatch {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ExampleFilter.class);
+
+  //private EvalutationPredicates []
+  private RecordBatch incoming;
+  private BatchSchema outboundSchema;
+  private int recordCount;
+
+  private void reconfigureSchema() throws SchemaChangeException {
+    BatchSchema in = incoming.getSchema();
+    outboundSchema = BatchSchema.newBuilder().addFields(in).setSelectionVectorMode(BatchSchema.SelectionVectorMode.TWO_BYTE).build();
+  }
+
+  private int generateSelectionVector(){
+                    return -1;
+  }
+
+  @Override
+  public FragmentContext getContext() {
+    return incoming.getContext();
+  }
+
+  @Override
+  public BatchSchema getSchema() {
+    return outboundSchema;
+  }
+
+  @Override
+  public int getRecordCount() {
+    return recordCount;  //To change body of implemented methods use File | Settings | File Templates.
+  }
+
+  @Override
+  public void kill() {
+    //To change body of implemented methods use File | Settings | File Templates.
+  }
+
+  @Override
+  public SelectionVector2 getSelectionVector2() {
+    return null;
+  }
+
+  @Override
+  public SelectionVector4 getSelectionVector4() {
+    return null;
+  }
+
+  @Override
+  public TypedFieldId getValueVector(SchemaPath path) {
+    return null;
+  }
+
+  @Override
+  public <T extends ValueVector<T>> T getValueVectorById(int fieldId, Class<?> vvClass) {
+    return null;
+  }
+
+  @Override
+  public IterOutcome next() {
+    IterOutcome out = incoming.next();
+    switch (incoming.next()) {
+
+      case NONE:
+        return IterOutcome.NONE;
+      case OK_NEW_SCHEMA:
+        //reconfigureSchema();
+      case OK:
+        this.recordCount = generateSelectionVector();
+        return out;
+      case STOP:
+        return IterOutcome.STOP;
+      default:
+        throw new UnsupportedOperationException();
+    }
+  }
+
+  @Override
+  public WritableBatch getWritableBatch() {
+    return null;  //To change body of implemented methods use File | Settings | File Templates.
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/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
new file mode 100644
index 0000000..69daae0
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/ProjectEvaluator.java
@@ -0,0 +1,12 @@
+package org.apache.drill.exec.physical.impl.project;
+
+import org.apache.drill.exec.exception.SchemaChangeException;
+import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.record.RecordBatch;
+
+public interface ProjectEvaluator {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ProjectEvaluator.class);
+  
+  public abstract void setupEvaluators(FragmentContext context, RecordBatch incoming) throws SchemaChangeException;
+  public abstract void doPerRecordWork(int inIndex, int outIndex);
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/ProjectRecordBatch.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/ProjectRecordBatch.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/ProjectRecordBatch.java
new file mode 100644
index 0000000..cfdb7bc
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/ProjectRecordBatch.java
@@ -0,0 +1,218 @@
+package org.apache.drill.exec.physical.impl.project;
+
+import java.io.IOException;
+import java.util.List;
+
+import org.apache.drill.common.expression.ErrorCollector;
+import org.apache.drill.common.expression.ErrorCollectorImpl;
+import org.apache.drill.common.expression.LogicalExpression;
+import org.apache.drill.common.expression.PathSegment;
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.common.logical.data.NamedExpression;
+import org.apache.drill.common.types.TypeProtos.MajorType;
+import org.apache.drill.exec.exception.ClassTransformationException;
+import org.apache.drill.exec.exception.SchemaChangeException;
+import org.apache.drill.exec.expr.CodeGenerator;
+import org.apache.drill.exec.expr.ExpressionTreeMaterializer;
+import org.apache.drill.exec.expr.ValueVectorReadExpression;
+import org.apache.drill.exec.expr.ValueVectorWriteExpression;
+import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.physical.config.Project;
+import org.apache.drill.exec.proto.SchemaDefProtos.FieldDef;
+import org.apache.drill.exec.proto.SchemaDefProtos.NamePart;
+import org.apache.drill.exec.proto.SchemaDefProtos.NamePart.Type;
+import org.apache.drill.exec.record.BatchSchema;
+import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
+import org.apache.drill.exec.record.MaterializedField;
+import org.apache.drill.exec.record.RecordBatch;
+import org.apache.drill.exec.record.SchemaBuilder;
+import org.apache.drill.exec.record.WritableBatch;
+import org.apache.drill.exec.record.vector.SelectionVector2;
+import org.apache.drill.exec.record.vector.SelectionVector4;
+import org.apache.drill.exec.record.vector.TypeHelper;
+import org.apache.drill.exec.record.vector.ValueVector;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+
+public class ProjectRecordBatch implements RecordBatch{
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ProjectRecordBatch.class);
+
+  private final Project pop;
+  private final RecordBatch incoming;
+  private final FragmentContext context;
+  private BatchSchema outSchema;
+  private Projector projector;
+  private List<ValueVector<?>> allocationVectors;
+  private List<ValueVector<?>> outputVectors;
+  
+  
+  public ProjectRecordBatch(Project pop, RecordBatch incoming, FragmentContext context){
+    this.pop = pop;
+    this.incoming = incoming;
+    this.context = context;
+  }
+  
+  
+  @Override
+  public FragmentContext getContext() {
+    return context;
+  }
+
+  @Override
+  public BatchSchema getSchema() {
+    Preconditions.checkNotNull(outSchema);
+    return outSchema;
+  }
+
+  @Override
+  public int getRecordCount() {
+    return incoming.getRecordCount();
+  }
+
+  @Override
+  public void kill() {
+    incoming.kill();
+  }
+
+  @Override
+  public SelectionVector2 getSelectionVector2() {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public SelectionVector4 getSelectionVector4() {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public TypedFieldId getValueVector(SchemaPath path) {
+    return null;
+  }
+
+  @Override
+  public <T extends ValueVector<T>> T getValueVectorById(int fieldId, Class<?> vvClass) {
+    return null;
+  }
+
+  @Override
+  public IterOutcome next() {
+    
+    IterOutcome upstream = incoming.next();
+    switch(upstream){
+    case NONE:
+    case NOT_YET:
+    case STOP:
+      return upstream;
+    case OK_NEW_SCHEMA:
+      try{
+        projector = createNewProjector();
+      }catch(SchemaChangeException ex){
+        incoming.kill();
+        context.fail(ex);
+        return IterOutcome.STOP;
+      }
+      // fall through.
+    case OK:
+      int recordCount = incoming.getRecordCount();
+      for(ValueVector<?> v : this.allocationVectors){
+        v.allocateNew(recordCount);
+      }
+      projector.projectRecords(recordCount, 0);
+      return upstream; // change if upstream changed, otherwise normal.
+    default:
+      throw new UnsupportedOperationException();
+    }
+  }
+  
+
+  private Projector createNewProjector() throws SchemaChangeException{
+    this.allocationVectors = Lists.newArrayList();
+    if(outputVectors != null){
+      for(ValueVector<?> v : outputVectors){
+        v.close();
+      }
+    }
+    this.outputVectors = Lists.newArrayList();
+    
+    final List<NamedExpression> exprs = pop.getExprs();
+    final ErrorCollector collector = new ErrorCollectorImpl();
+    final List<TransferPairing<?>> transfers = Lists.newArrayList();
+    
+    final CodeGenerator cg = new CodeGenerator("setupEvaluators", "doPerRecordWork", context.getFunctionRegistry());
+    
+    for(int i =0; i < exprs.size(); i++){
+      final NamedExpression namedExpression = exprs.get(i);
+      final MaterializedField outputField = getMaterializedField(namedExpression);
+      final LogicalExpression expr = ExpressionTreeMaterializer.materialize(namedExpression.getExpr(), incoming, collector);
+      if(collector.hasErrors()){
+        throw new SchemaChangeException(String.format("Failure while trying to materialize incoming schema.  Errors:\n %s.", collector.toErrorString()));
+      }
+      
+      
+      
+      
+      // add value vector to transfer if direct reference and this is allowed, otherwise, add to evaluation stack.
+      if(expr instanceof ValueVectorReadExpression && incoming.getSchema().getSelectionVector() == SelectionVectorMode.NONE){
+        ValueVectorReadExpression vectorRead = (ValueVectorReadExpression) expr;
+        ValueVector<?> vvIn = incoming.getValueVectorById(vectorRead.getFieldId(), TypeHelper.getValueVectorClass(vectorRead.getMajorType()));
+        Preconditions.checkNotNull(incoming);
+
+        TransferPairing<?> tp = vvIn.getTransferPair(outputField);
+        transfers.add(tp);
+        outputVectors.add(tp.getTo());
+      }else{
+        // need to do evaluation.
+        ValueVector<?> vector = TypeHelper.getNewVector(outputField, context.getAllocator());
+        allocationVectors.add(vector);
+        outputVectors.add(vector);
+        ValueVectorWriteExpression write = new ValueVectorWriteExpression(outputVectors.size() - 1, expr);
+        cg.addNextWrite(write);
+      }
+      
+    }
+    
+    SchemaBuilder bldr = BatchSchema.newBuilder().setSelectionVectorMode(SelectionVectorMode.NONE);
+    for(ValueVector<?> v : outputVectors){
+      bldr.addField(v.getField());
+    }
+    this.outSchema = bldr.build();
+    
+    try {
+      return context.getImplementationClass(Projector.TEMPLATE_DEFINITION, cg);
+    } catch (ClassTransformationException | IOException e) {
+      throw new SchemaChangeException("Failure while attempting to load generated class", e);
+    }
+  }
+  
+  
+  @Override
+  public WritableBatch getWritableBatch() {
+    return null;
+  }
+  
+  
+  private MaterializedField getMaterializedField(NamedExpression ex){
+    return new MaterializedField(getFieldDef(ex.getRef(), ex.getExpr().getMajorType()));
+  }
+
+  private FieldDef getFieldDef(SchemaPath path, MajorType type){
+    return FieldDef //
+        .newBuilder() //
+        .addAllName(getNameParts(path.getRootSegment())) //
+        .setMajorType(type) //
+        .build();
+  }
+  
+  private List<NamePart> getNameParts(PathSegment seg){
+    List<NamePart> parts = Lists.newArrayList();
+    while(seg != null){
+      if(seg.isArray()){
+        parts.add(NamePart.newBuilder().setType(Type.ARRAY).build());
+      }else{
+        parts.add(NamePart.newBuilder().setType(Type.NAME).setName(seg.getNameSegment().getPath().toString()).build());
+      }
+    }
+    return parts;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/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
new file mode 100644
index 0000000..31c418c
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/Projector.java
@@ -0,0 +1,20 @@
+package org.apache.drill.exec.physical.impl.project;
+
+import java.util.List;
+
+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.record.RecordBatch;
+
+public interface Projector {
+
+  public abstract void setup(FragmentContext context, RecordBatch incoming, List<TransferPairing<?>> transfers)  throws SchemaChangeException;
+
+  
+  public abstract void projectRecords(int recordCount, int firstOutputIndex);
+
+  public static TemplateClassDefinition<Projector, Void> TEMPLATE_DEFINITION = new TemplateClassDefinition<Projector, Void>( //
+      Projector.class, "org.apache.drill.exec.physical.impl.project.ProjectTemplate", ProjectEvaluator.class, Void.class);
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/ProjectorTemplate.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/ProjectorTemplate.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/ProjectorTemplate.java
new file mode 100644
index 0000000..60af7d2
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/ProjectorTemplate.java
@@ -0,0 +1,101 @@
+package org.apache.drill.exec.physical.impl.project;
+
+import java.util.List;
+
+import org.apache.drill.common.expression.ErrorCollector;
+import org.apache.drill.common.expression.ErrorCollectorImpl;
+import org.apache.drill.common.expression.LogicalExpression;
+import org.apache.drill.common.expression.PathSegment;
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.common.logical.data.NamedExpression;
+import org.apache.drill.common.types.TypeProtos.MajorType;
+import org.apache.drill.exec.exception.SchemaChangeException;
+import org.apache.drill.exec.expr.CodeGenerator;
+import org.apache.drill.exec.expr.ExpressionTreeMaterializer;
+import org.apache.drill.exec.expr.ValueVectorReadExpression;
+import org.apache.drill.exec.expr.ValueVectorWriteExpression;
+import org.apache.drill.exec.expr.fn.FunctionImplementationRegistry;
+import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.physical.config.Project;
+import org.apache.drill.exec.proto.SchemaDefProtos.FieldDef;
+import org.apache.drill.exec.proto.SchemaDefProtos.NamePart;
+import org.apache.drill.exec.proto.SchemaDefProtos.NamePart.Type;
+import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
+import org.apache.drill.exec.record.MaterializedField;
+import org.apache.drill.exec.record.RecordBatch;
+import org.apache.drill.exec.record.vector.SelectionVector2;
+import org.apache.drill.exec.record.vector.SelectionVector4;
+import org.apache.drill.exec.record.vector.TypeHelper;
+import org.apache.drill.exec.record.vector.ValueVector;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Lists;
+
+public abstract class ProjectorTemplate implements Projector {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ProjectorTemplate.class);
+
+  private ImmutableList<TransferPairing<?>> transfers;
+  private SelectionVector2 vector2;
+  private SelectionVector4 vector4;
+  private SelectionVectorMode svMode;
+  
+  public ProjectorTemplate(final FragmentContext context, final RecordBatch incomingBatch, final Project pop, FunctionImplementationRegistry funcRegistry) throws SchemaChangeException{
+    super();
+  }
+
+  @Override
+  public final void projectRecords(final int recordCount, int firstOutputIndex) {
+    switch(svMode){
+    case FOUR_BYTE:
+      throw new UnsupportedOperationException();
+      
+      
+    case TWO_BYTE:
+      final int count = recordCount*2;
+      for(int i = 0; i < count; i+=2, firstOutputIndex++){
+        doPerRecordWork(vector2.getIndex(i), firstOutputIndex);
+      }
+      return;
+      
+      
+    case NONE:
+      
+      for(TransferPairing<?> t : transfers){
+        t.transfer();
+      }
+      final int countN = recordCount;
+      for (int i = 0; i < countN; i++, firstOutputIndex++) {
+        doPerRecordWork(i, firstOutputIndex);
+      }
+      return;
+      
+      
+    default:
+      throw new UnsupportedOperationException();
+    }
+  }
+
+  @Override
+  public final void setup(FragmentContext context, RecordBatch incoming, List<TransferPairing<?>> transfers)  throws SchemaChangeException{
+
+    this.svMode = incoming.getSchema().getSelectionVector(); 
+    switch(svMode){
+    case FOUR_BYTE:
+      this.vector4 = incoming.getSelectionVector4();
+      break;
+    case TWO_BYTE:
+      this.vector2 = incoming.getSelectionVector2();
+      break;
+    }
+    this.transfers = ImmutableList.copyOf(transfers);
+    setupEvaluators(context, incoming);
+  }
+
+  protected abstract void setupEvaluators(FragmentContext context, RecordBatch incoming) throws SchemaChangeException;
+  protected abstract void doPerRecordWork(int inIndex, int outIndex);
+
+  
+
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/TransferPairing.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/TransferPairing.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/TransferPairing.java
new file mode 100644
index 0000000..2b4ac81
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/TransferPairing.java
@@ -0,0 +1,35 @@
+package org.apache.drill.exec.physical.impl.project;
+
+import org.apache.drill.exec.record.vector.ValueVector;
+
+public class TransferPairing<T extends ValueVector<T>> {
+  
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TransferPairing.class);
+  
+  final T from;
+  final T to;
+  
+  protected TransferPairing(T from, T to) {
+    super();
+    this.from = from;
+    this.to = to;
+  }
+
+  public void transfer(){
+    from.transferTo(to);
+  }
+  
+  public static <T extends ValueVector<T>> TransferPairing<T> getTransferPairing(T from, T to){
+    return new TransferPairing<T>(from, to);
+  }
+
+  public T getFrom() {
+    return from;
+  }
+
+  public T getTo() {
+    return to;
+  }
+  
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/PhysicalPlanReader.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/PhysicalPlanReader.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/PhysicalPlanReader.java
index 3c2df61..1148c93 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/PhysicalPlanReader.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/PhysicalPlanReader.java
@@ -21,6 +21,7 @@ import java.io.IOException;
 
 import org.apache.drill.common.config.DrillConfig;
 import org.apache.drill.common.logical.LogicalPlan;
+import org.apache.drill.common.types.TypeProtos.MajorType;
 import org.apache.drill.exec.coord.DrillbitEndpointSerDe;
 import org.apache.drill.exec.physical.PhysicalPlan;
 import org.apache.drill.exec.physical.base.FragmentLeaf;
@@ -28,7 +29,6 @@ import org.apache.drill.exec.physical.base.FragmentRoot;
 import org.apache.drill.exec.physical.base.PhysicalOperator;
 import org.apache.drill.exec.physical.base.PhysicalOperatorUtil;
 import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
-import org.apache.drill.exec.proto.SchemaDefProtos.MajorType;
 import org.apache.drill.exec.record.MajorTypeSerDe;
 
 import com.fasterxml.jackson.core.JsonProcessingException;

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/BatchSchema.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/BatchSchema.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/BatchSchema.java
index b26e742..bb07e56 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/BatchSchema.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/BatchSchema.java
@@ -6,9 +6,9 @@
  * to you under the Apache License, Version 2.0 (the
  * "License"); you may not use this file except in compliance
  * with the License.  You may obtain a copy of the License at
- * 
+ *
  * http://www.apache.org/licenses/LICENSE-2.0
- * 
+ *
  * Unless required by applicable law or agreed to in writing, software
  * distributed under the License is distributed on an "AS IS" BASIS,
  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
@@ -23,13 +23,17 @@ import java.util.List;
 
 public class BatchSchema implements Iterable<MaterializedField> {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(BatchSchema.class);
-
+  final SelectionVectorMode selectionVector;
+  ;
   private final List<MaterializedField> fields;
-  final boolean hasSelectionVector;
 
-  BatchSchema(boolean hasSelectionVector, List<MaterializedField> fields) {
+  BatchSchema(SelectionVectorMode selectionVector, List<MaterializedField> fields) {
     this.fields = fields;
-    this.hasSelectionVector = hasSelectionVector;
+    this.selectionVector = selectionVector;
+  }
+
+  public static SchemaBuilder newBuilder() {
+    return new SchemaBuilder();
   }
 
   @Override
@@ -37,16 +41,24 @@ public class BatchSchema implements Iterable<MaterializedField> {
     return fields.iterator();
   }
 
-  public static SchemaBuilder newBuilder() {
-    return new SchemaBuilder();
+  public SelectionVectorMode getSelectionVector() {
+    return selectionVector;
   }
 
   @Override
   public String toString() {
-    return "BatchSchema [fields=" + fields + ", hasSelectionVector=" + hasSelectionVector + "]";
+    return "BatchSchema [fields=" + fields + ", selectionVector=" + selectionVector + "]";
   }
 
-  
-  
-  
+  public static enum SelectionVectorMode {
+    NONE(-1, false), TWO_BYTE(2, true), FOUR_BYTE(4, true);
+
+    public boolean hasSelectionVector;
+    public final int size;
+    SelectionVectorMode(int size, boolean hasSelectionVector) {
+      this.size = size;
+    }
+  }
+
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/ExpressionTreeMaterializer.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/ExpressionTreeMaterializer.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/ExpressionTreeMaterializer.java
deleted file mode 100644
index 391aec5..0000000
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/ExpressionTreeMaterializer.java
+++ /dev/null
@@ -1,138 +0,0 @@
-/*******************************************************************************
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- ******************************************************************************/
-
-package org.apache.drill.exec.record;
-
-import com.google.common.collect.Lists;
-import org.apache.drill.common.expression.*;
-import org.apache.drill.common.expression.types.DataType;
-import org.apache.drill.common.expression.visitors.ExprVisitor;
-
-import java.util.List;
-
-public class ExpressionTreeMaterializer {
-    public LogicalExpression Materialize(LogicalExpression expr, BatchSchema schema, ErrorCollector errorCollector) {
-        return expr.accept(new MaterializeVisitor(schema, errorCollector));
-    }
-
-    private class MaterializeVisitor implements ExprVisitor<LogicalExpression> {
-        private final ErrorCollector errorCollector;
-        private final BatchSchema schema;
-        private boolean isModified; // Flag to track if children is changed
-
-        public MaterializeVisitor(BatchSchema schema, ErrorCollector errorCollector) {
-            this.schema = schema;
-            this.errorCollector = errorCollector;
-            isModified = false;
-        }
-
-        private LogicalExpression validateNewExpr(LogicalExpression newExpr) {
-            StringBuilder stringBuilder = new StringBuilder();
-            newExpr.addToString(stringBuilder);
-            newExpr.resolveAndValidate(stringBuilder.toString(), errorCollector);
-            return newExpr;
-        }
-
-        @Override
-        public LogicalExpression visitFunctionCall(FunctionCall call) {
-            List<LogicalExpression> args = Lists.newArrayList(call.iterator());
-            boolean hasChanged = false;
-            for (int i = 0; i < args.size(); ++i) {
-                LogicalExpression newExpr = args.get(i).accept(this);
-                if (isModified) {
-                    hasChanged = true;
-                    args.set(i, newExpr);
-                    isModified = false;
-                }
-            }
-
-            if(hasChanged) {
-                isModified = true;
-                return validateNewExpr(new FunctionCall(call.getDefinition(), args));
-            }
-
-            return call;
-        }
-
-        @Override
-        public LogicalExpression visitIfExpression(IfExpression ifExpr) {
-            List<IfExpression.IfCondition> conditions = Lists.newArrayList(ifExpr.iterator());
-            boolean hasChanged = false;
-            LogicalExpression newElseExpr = null;
-            if(ifExpr.elseExpression != null) {
-                newElseExpr = ifExpr.elseExpression.accept(this);
-                hasChanged = isModified;
-            }
-
-            isModified = false;
-
-            for(int i = 0; i < conditions.size(); ++i) {
-                IfExpression.IfCondition condition = conditions.get(i);
-
-                LogicalExpression newCondition = condition.condition.accept(this);
-                boolean modified = isModified;
-                isModified = false;
-                LogicalExpression newExpr = condition.expression.accept(this);
-                if(modified || isModified) {
-                    conditions.set(i, new IfExpression.IfCondition(newCondition, newExpr));
-                    hasChanged = true;
-                    isModified = false;
-                }
-            }
-
-            if(hasChanged) {
-                isModified = true;
-                return validateNewExpr(IfExpression.newBuilder().setElse(newElseExpr).addConditions(conditions).build());
-            }
-
-            return ifExpr;
-        }
-
-        @Override
-        public LogicalExpression visitSchemaPath(SchemaPath path) {
-            for (MaterializedField field : schema) {
-                if (field.getType() != DataType.LATEBIND && field.matches(path)) {
-                    isModified = true;
-                    return validateNewExpr(new FieldReference(path.getPath().toString(), field.getType()));
-                }
-            }
-
-            return path;
-        }
-
-        @Override
-        public LogicalExpression visitLongExpression(ValueExpressions.LongExpression intExpr) {
-            return intExpr;
-        }
-
-        @Override
-        public LogicalExpression visitDoubleExpression(ValueExpressions.DoubleExpression dExpr) {
-            return dExpr;
-        }
-
-        @Override
-        public LogicalExpression visitBoolean(ValueExpressions.BooleanExpression e) {
-            return e;
-        }
-
-        @Override
-        public LogicalExpression visitQuotedString(ValueExpressions.QuotedString e) {
-            return e;
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/MajorTypeSerDe.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/MajorTypeSerDe.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/MajorTypeSerDe.java
index 718396e..8799546 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/MajorTypeSerDe.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/MajorTypeSerDe.java
@@ -19,9 +19,9 @@ package org.apache.drill.exec.record;
 
 import java.io.IOException;
 
-import org.apache.drill.exec.proto.SchemaDefProtos.DataMode;
-import org.apache.drill.exec.proto.SchemaDefProtos.MajorType;
-import org.apache.drill.exec.proto.SchemaDefProtos.MinorType;
+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 com.fasterxml.jackson.annotation.JsonCreator;
 import com.fasterxml.jackson.annotation.JsonIgnore;

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/MaterializedField.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/MaterializedField.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/MaterializedField.java
index b692a93..6cf7087 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/MaterializedField.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/MaterializedField.java
@@ -18,18 +18,17 @@
 package org.apache.drill.exec.record;
 
 import java.util.Iterator;
-import java.util.List;
 
 import org.apache.drill.common.expression.PathSegment;
 import org.apache.drill.common.expression.SchemaPath;
-import org.apache.drill.exec.physical.RecordField.ValueMode;
-import org.apache.drill.exec.proto.SchemaDefProtos.DataMode;
+import org.apache.drill.common.types.TypeProtos.DataMode;
+import org.apache.drill.common.types.TypeProtos.MajorType;
 import org.apache.drill.exec.proto.SchemaDefProtos.FieldDef;
-import org.apache.drill.exec.proto.SchemaDefProtos.MajorType;
 import org.apache.drill.exec.proto.SchemaDefProtos.NamePart;
 import org.apache.drill.exec.proto.SchemaDefProtos.NamePart.Type;
+import org.apache.drill.exec.record.vector.TypeHelper;
 
-public class MaterializedField implements Comparable<MaterializedField> {
+public class MaterializedField{
   private final FieldDef def;
 
   public MaterializedField(FieldDef def) {
@@ -40,12 +39,10 @@ public class MaterializedField implements Comparable<MaterializedField> {
     return new MaterializedField(def);
   }
   
-  public static MaterializedField create(SchemaPath path, int fieldId, int parentId, MajorType type) {
+  public static MaterializedField create(SchemaPath path, MajorType type) {
     FieldDef.Builder b = FieldDef.newBuilder();
-    b.setFieldId(fieldId);
     b.setMajorType(type);
     addSchemaPathToFieldDef(path, b);
-    b.setParentId(parentId);
     return create(b.build());
   }
 
@@ -90,10 +87,6 @@ public class MaterializedField implements Comparable<MaterializedField> {
     return def.getMajorType().getWidth();
   }
 
-  public int getFieldId() {
-    return def.getFieldId();
-  }
-
   public MajorType getType() {
     return def.getMajorType();
   }
@@ -120,6 +113,9 @@ public class MaterializedField implements Comparable<MaterializedField> {
       throw new UnsupportedOperationException();
     }
     return new MaterializedField(def.toBuilder().setMajorType(mt.toBuilder().setMode(newDataMode).build()).build());
+
+  public Class<?> getValueClass() {
+    return TypeHelper.getValueVectorClass(getType().getMinorType(), getDataMode());
   }
 
   public boolean matches(SchemaPath path) {
@@ -141,46 +137,8 @@ public class MaterializedField implements Comparable<MaterializedField> {
     }
     // we've reviewed all path segments. confirm that we don't have any extra name parts.
     return !iter.hasNext();
-
-  private void check(String name, Object val1, Object expected) throws SchemaChangeException{
-    if(expected.equals(val1)) return;
-    throw new SchemaChangeException("Expected and actual field definitions don't match. Actual %s: %s, expected %s: %s", name, val1, name, expected);
   }
   
-  public void checkMaterialization(MaterializedField expected) throws SchemaChangeException{
-    if(this.type == expected.type || expected.type == DataType.LATEBIND) throw new SchemaChangeException("Expected and actual field definitions don't match. Actual DataType: %s, expected DataTypes: %s", this.type, expected.type);
-    if(expected.valueClass != null) check("valueClass", this.valueClass, expected.valueClass);
-    check("fieldId", this.fieldId, expected.fieldId);
-    check("nullability", this.nullable, expected.nullable);
-    check("valueMode", this.mode, expected.mode);
-  }
-
-  // private void check(String name, Object val1, Object expected) throws SchemaChangeException{
-  // if(expected.equals(val1)) return;
-  // throw new
-  // SchemaChangeException("Expected and actual field definitions don't match. Actual %s: %s, expected %s: %s", name,
-  // val1, name, expected);
-  // }
-
-  // public void checkMaterialization(MaterializedField expected) throws SchemaChangeException{
-  // if(this.type == expected.type || expected.type == DataType.LATEBIND) throw new
-  // SchemaChangeException("Expected and actual field definitions don't match. Actual DataType: %s, expected DataTypes: %s",
-  // this.type, expected.type);
-  // if(expected.valueClass != null) check("valueClass", this.valueClass, expected.valueClass);
-  // check("fieldId", this.fieldId, expected.fieldId);
-  // check("nullability", this.nullable, expected.nullable);
-  // check("valueMode", this.mode, expected.mode);
-  // }
-  //
-  // public MaterializedField getNullableVersion(Class<?> valueClass){
-  // return new MaterializedField(path, fieldId, type, true, mode, valueClass);
-  // }
-
-  @Override
-  public int compareTo(MaterializedField o) {
-    return Integer.compare(this.getFieldId(), o.getFieldId());
-  }
-
   @Override
   public String toString() {
     return "MaterializedField [" + def.toString() + "]";

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/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
new file mode 100644
index 0000000..4ab908f
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/NullExpression.java
@@ -0,0 +1,32 @@
+package org.apache.drill.exec.record;
+
+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.DataMode;
+import org.apache.drill.common.types.TypeProtos.MajorType;
+import org.apache.drill.common.types.TypeProtos.MinorType;
+
+public class NullExpression implements LogicalExpression{
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(NullExpression.class);
+
+  public static final NullExpression INSTANCE = new NullExpression();
+  
+  final MajorType t = MajorType.newBuilder().setMode(DataMode.OPTIONAL).setMinorType(MinorType.NULL).build();
+  
+  @Override
+  public MajorType getMajorType() {
+    return t;
+  }
+
+  @Override
+  public <T, V, E extends Exception> T accept(ExprVisitor<T, V, E> visitor, V value) throws E {
+    return visitor.visitUnknown(this, value);
+  }
+
+  @Override
+  public ExpressionPosition getPosition() {
+    return ExpressionPosition.UNKNOWN;
+  }
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatch.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatch.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatch.java
index c6b7888..40447ec 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatch.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatch.java
@@ -17,8 +17,12 @@
  ******************************************************************************/
 package org.apache.drill.exec.record;
 
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.common.types.TypeProtos.MajorType;
 import org.apache.drill.exec.ops.FragmentContext;
-import org.apache.drill.exec.vector.ValueVector;
+import org.apache.drill.exec.record.vector.SelectionVector2;
+import org.apache.drill.exec.record.vector.SelectionVector4;
+import org.apache.drill.exec.record.vector.ValueVector;
 
 /**
  * A record batch contains a set of field values for a particular range of records. In the case of a record batch
@@ -39,6 +43,12 @@ public interface RecordBatch {
     NOT_YET // used by batches that haven't received incoming data yet.
   }
 
+  public static enum SetupOutcome {
+    OK,
+    OK_NEW_SCHEMA,
+    FAILED
+  }
+  
   /**
    * Access the FragmentContext of the current query fragment. Useful for reporting failure information or other query
    * level information.
@@ -67,12 +77,18 @@ public interface RecordBatch {
    */
   public void kill();
 
-
-  public abstract <T extends ValueVector> T getValueVector(int fieldId, Class<T> clazz) throws InvalidValueAccessor;
-
-//  public abstract void getDictReader(int fieldId, Class<?> clazz) throws InvalidValueAccessor;
-//
-//  public abstract void getRleReader(int fieldId, Class<?> clazz) throws InvalidValueAccessor;
+  public abstract SelectionVector2 getSelectionVector2();
+  public abstract SelectionVector4 getSelectionVector4();
+  
+  /**
+   * Get the value vector 
+   * @param path The path where the vector should be located.
+   * @return The local field id associated with this vector.
+   */
+  public abstract TypedFieldId getValueVector(SchemaPath path);
+  
+  
+  public abstract <T extends ValueVector<T>> T getValueVectorById(int fieldId, Class<?> vvClass);
 
   /**
    * Update the data in each Field reading interface for the next range of records. Once a RecordBatch returns an
@@ -89,4 +105,41 @@ public interface RecordBatch {
    */
   public WritableBatch getWritableBatch();
 
+  public static class TypedFieldId{
+    final MajorType type;
+    final int fieldId;
+    public TypedFieldId(MajorType type, int fieldId) {
+      super();
+      this.type = type;
+      this.fieldId = fieldId;
+    }
+    public MajorType getType() {
+      return type;
+    }
+    public int getFieldId() {
+      return fieldId;
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+      if (this == obj)
+        return true;
+      if (obj == null)
+        return false;
+      if (getClass() != obj.getClass())
+        return false;
+      TypedFieldId other = (TypedFieldId) obj;
+      if (fieldId != other.fieldId)
+        return false;
+      if (type == null) {
+        if (other.type != null)
+          return false;
+      } else if (!type.equals(other.type))
+        return false;
+      return true;
+    }
+    
+    
+  }
+  
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatchLoader.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatchLoader.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatchLoader.java
index a2dbd81..c3db9f0 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatchLoader.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatchLoader.java
@@ -21,22 +21,28 @@ import io.netty.buffer.ByteBuf;
 
 import java.util.Iterator;
 import java.util.List;
+import java.util.Map;
 
+import org.apache.drill.common.expression.SchemaPath;
 import org.apache.drill.exec.exception.SchemaChangeException;
 import org.apache.drill.exec.memory.BufferAllocator;
 import org.apache.drill.exec.proto.SchemaDefProtos.FieldDef;
 import org.apache.drill.exec.proto.UserBitShared.FieldMetadata;
 import org.apache.drill.exec.proto.UserBitShared.RecordBatchDef;
-import org.apache.drill.exec.vector.TypeHelper;
-import org.apache.drill.exec.vector.ValueVector;
+import org.apache.drill.exec.record.RecordBatch.TypedFieldId;
+import org.apache.drill.exec.record.vector.TypeHelper;
+import org.apache.drill.exec.record.vector.ValueVector;
 
+import com.beust.jcommander.internal.Lists;
+import com.beust.jcommander.internal.Maps;
 import com.carrotsearch.hppc.IntObjectOpenHashMap;
 import com.carrotsearch.hppc.cursors.IntObjectCursor;
+import com.google.common.collect.ImmutableList;
 
-public class RecordBatchLoader implements Iterable<IntObjectCursor<ValueVector>>{
+public class RecordBatchLoader implements Iterable<ValueVector<?>>{
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(RecordBatchLoader.class);
 
-  private IntObjectOpenHashMap<ValueVector> vectors = new IntObjectOpenHashMap<ValueVector>();
+  private List<ValueVector<?>> vectors = Lists.newArrayList();
   private final BufferAllocator allocator;
   private int recordCount; 
   private BatchSchema schema;
@@ -60,64 +66,72 @@ public class RecordBatchLoader implements Iterable<IntObjectCursor<ValueVector>>
 //    logger.debug("Loading record batch with def {} and data {}", def, buf);
     this.recordCount = def.getRecordCount();
     boolean schemaChanged = false;
+
+    Map<MaterializedField, ValueVector<?>> oldFields = Maps.newHashMap();
+    for(ValueVector<?> v : this.vectors){
+      oldFields.put(v.getField(), v);
+    }
     
-    IntObjectOpenHashMap<ValueVector> newVectors = new IntObjectOpenHashMap<ValueVector>();
+    List<ValueVector<?>> newVectors = Lists.newArrayList();
 
     List<FieldMetadata> fields = def.getFieldList();
     
     int bufOffset = 0;
     for (FieldMetadata fmd : fields) {
       FieldDef fieldDef = fmd.getDef();
-      ValueVector v = vectors.remove(fieldDef.getFieldId());
-      if (v != null) {
-        if (v.getField().getDef().equals(fieldDef)) {
-          ValueVector.Mutator m = v.getMutator();
-          v.load(fmd, buf.slice(bufOffset, fmd.getBufferLength()));
-          newVectors.put(fieldDef.getFieldId(), v);
-          continue;
-        } else {
-          v.close();
-          v = null;
-        }
+      ValueVector<?> v = oldFields.remove(fieldDef);
+      if(v != null){
+        newVectors.add(v);
+        continue;
       }
-      // if we arrive here, either the metadata didn't match, or we didn't find a vector.
+      
+      // if we arrive here, we didn't have a matching vector.
       schemaChanged = true;
       MaterializedField m = new MaterializedField(fieldDef);
       v = TypeHelper.getNewVector(m, allocator);
       v.load(fmd, buf.slice(bufOffset, fmd.getBufferLength()));
       newVectors.put(fieldDef.getFieldId(), v);
+      v.setTo(fmd, buf.slice(bufOffset, fmd.getBufferLength()));
+      newVectors.add(v);
     }
     
-    if(!vectors.isEmpty()){
+    if(!oldFields.isEmpty()){
       schemaChanged = true;
-      for(IntObjectCursor<ValueVector> cursor : newVectors){
-        cursor.value.close();
+      for(ValueVector<?> v : oldFields.values()){
+        v.close();
       }
-      
     }
     
-    if(schemaChanged){
-      // rebuild the schema.
-      SchemaBuilder b = BatchSchema.newBuilder();
-      for(IntObjectCursor<ValueVector> cursor : newVectors){
-        b.addField(cursor.value.getField());
-      }
-      b.setSelectionVector(false);
-      this.schema = b.build();
+    // rebuild the schema.
+    SchemaBuilder b = BatchSchema.newBuilder();
+    for(ValueVector<?> v : newVectors){
+      b.addField(v.getField());
     }
-    vectors = newVectors;
+    b.setSelectionVectorMode(BatchSchema.SelectionVectorMode.NONE);
+    this.schema = b.build();
+    vectors = ImmutableList.copyOf(newVectors);
     return schemaChanged;
 
   }
 
+  public TypedFieldId getValueVector(SchemaPath path) {
+    for(int i =0; i < vectors.size(); i++){
+      ValueVector<?> vv = vectors.get(i);
+      if(vv.getField().matches(path)) return new TypedFieldId(vv.getField().getType(), i); 
+    }
+    return null;
+  }
+  
   @SuppressWarnings("unchecked")
-  public <T extends ValueVector> T getValueVector(int fieldId, Class<T> clazz) throws InvalidValueAccessor {
-    ValueVector v = vectors.get(fieldId);
+  public <T extends ValueVector<T>> T getValueVector(int fieldId, Class<?> clazz) {
+    ValueVector<?> v = vectors.get(fieldId);
     assert v != null;
-    if (v.getClass() != clazz)
-      throw new InvalidValueAccessor(String.format(
+    if (v.getClass() != clazz){
+      logger.warn(String.format(
           "Failure while reading vector.  Expected vector class of %s but was holding vector class %s.",
           clazz.getCanonicalName(), v.getClass().getCanonicalName()));
+      return null;
+    }
     return (T) v;
   }
 
@@ -131,7 +145,7 @@ public class RecordBatchLoader implements Iterable<IntObjectCursor<ValueVector>>
   }
 
   @Override
-  public Iterator<IntObjectCursor<ValueVector>> iterator() {
+  public Iterator<ValueVector<?>> iterator() {
     return this.vectors.iterator();
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/SchemaBuilder.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/SchemaBuilder.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/SchemaBuilder.java
index 1e25b1a..20cc82c 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/SchemaBuilder.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/SchemaBuilder.java
@@ -17,15 +17,12 @@
  ******************************************************************************/
 package org.apache.drill.exec.record;
 
-import java.util.Collections;
 import java.util.List;
+import java.util.Set;
 
-import org.apache.drill.common.expression.types.DataType;
 import org.apache.drill.exec.exception.SchemaChangeException;
-import org.apache.drill.exec.physical.RecordField.ValueMode;
 
-import com.carrotsearch.hppc.IntObjectOpenHashMap;
-import com.carrotsearch.hppc.cursors.ObjectCursor;
+import com.beust.jcommander.internal.Sets;
 import com.google.common.collect.Lists;
 
 /**
@@ -33,40 +30,39 @@ import com.google.common.collect.Lists;
  * builder will always check that this schema is a equal or more materialized version of the current schema.
  */
 public class SchemaBuilder {
-  private IntObjectOpenHashMap<MaterializedField> fields = new IntObjectOpenHashMap<MaterializedField>();
-  private IntObjectOpenHashMap<MaterializedField> expectedFields = new IntObjectOpenHashMap<MaterializedField>();
+  private Set<MaterializedField> fields = Sets.newHashSet();
 
-  private boolean hasSelectionVector;
-
-  public SchemaBuilder(BatchSchema expected) {
-    for (MaterializedField f : expected) {
-      expectedFields.put(f.getFieldId(), f);
-    }
-    hasSelectionVector = expected.hasSelectionVector;
-  }
+  private BatchSchema.SelectionVectorMode selectionVectorMode;
 
   SchemaBuilder() {
   }
 
-  /**
-   * Add a field where we don't have type information. In this case, DataType will be set to LATEBIND and valueClass
-   * will be set to null.
-   * 
-   * @param fieldId
-   *          The desired fieldId. Should be unique for this BatchSchema.
-   * @param nullable
-   *          Whether this field supports nullability.
-   * @param mode
-   * @throws SchemaChangeException
-   */
+//  /**
+//   * Add a field where we don't have type information. In this case, DataType will be set to LATEBIND and valueClass
+//   * will be set to null.
+//   *
+//   * @param fieldId
+//   *          The desired fieldId. Should be unique for this BatchSchema.
+//   * @param nullable
+//   *          Whether this field supports nullability.
+//   * @param mode
+//   * @throws SchemaChangeException
+//   */
 //  public void addLateBindField(short fieldId, boolean nullable, ValueMode mode) throws SchemaChangeException {
 //    addTypedField(fieldId, DataType.LATEBIND, nullable, mode, Void.class);
 //  }
 
-  public void setSelectionVector(boolean hasSelectionVector) {
-    this.hasSelectionVector = hasSelectionVector;
+  public SchemaBuilder setSelectionVectorMode(BatchSchema.SelectionVectorMode selectionVectorMode) {
+    this.selectionVectorMode = selectionVectorMode;
+    return this;
   }
 
+  public SchemaBuilder addFields(Iterable<MaterializedField> fields){
+    for(MaterializedField f : fields){
+      addField(f);
+    }
+    return this;
+  }
   
 //  private void setTypedField(short fieldId, DataType type, boolean nullable, ValueMode mode, Class<?> valueClass)
 //      throws SchemaChangeException {
@@ -83,8 +79,9 @@ public class SchemaBuilder {
 //    fields.put(f.getFieldId(), f);
 //  }
   
-  public void addField(MaterializedField f){
-    fields.put(f.getFieldId(), f);
+  public SchemaBuilder addField(MaterializedField f){
+    fields.add(f);
+    return this;
   }
 
 //  public void addTypedField(short fieldId, DataType type, boolean nullable, ValueMode mode, Class<?> valueClass)
@@ -104,9 +101,9 @@ public class SchemaBuilder {
 //    setTypedField(fieldId, type, nullable, mode, valueClass);
 //  }
   
-  public void removeField(short fieldId) throws SchemaChangeException{
-    MaterializedField f = fields.remove(fieldId);
-    if(f == null) throw new SchemaChangeException("You attempted to remove an nonexistent field.");
+  public SchemaBuilder removeField(MaterializedField f) throws SchemaChangeException{
+    if(!fields.remove(f)) throw new SchemaChangeException("You attempted to remove an nonexistent field.");
+    return this;
   }
 
   /**
@@ -114,14 +111,8 @@ public class SchemaBuilder {
    * @return
    * @throws SchemaChangeException
    */
-  public BatchSchema build() throws SchemaChangeException {
-    // check if any fields are unaccounted for.
-
-    List<MaterializedField> fieldList = Lists.newArrayList();
-    for (ObjectCursor<MaterializedField> f : fields.values()) {
-      if (f != null) fieldList.add(f.value);
-    }
-    Collections.sort(fieldList);
-    return new BatchSchema(this.hasSelectionVector, fieldList);
+  public BatchSchema build(){
+    List<MaterializedField> fieldList = Lists.newArrayList(fields);
+    return new BatchSchema(this.selectionVectorMode, fieldList);
   }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/WritableBatch.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/WritableBatch.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/WritableBatch.java
index 4b97768..2e1754c 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/WritableBatch.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/WritableBatch.java
@@ -57,50 +57,27 @@ public class WritableBatch {
   public ByteBuf[] getBuffers(){
     return buffers;
   }
-  
-//  public static WritableBatch get(ValueVector[] vectors){
-//    WritableCreator c = new WritableCreator();
-//    for(int i =0; i < vectors.length; i++){
-//      c.apply(i, vectors[i]);
-//    }
-//    return c.get();
-//  }
-//  
-  
-  public static WritableBatch get(int recordCount, IntObjectOpenHashMap<ValueVector> fields){
-    WritableCreator creator = new WritableCreator(recordCount);
-    fields.forEach(creator);
-    return creator.get();
-    
-  }
-  
-  private static class WritableCreator implements IntObjectProcedure<ValueVector>{
+
+  public static WritableBatch get(int recordCount, List<ValueVector<?>> vectors){
     
     List<ByteBuf> buffers = Lists.newArrayList();
     List<FieldMetadata> metadata = Lists.newArrayList();
-    private int recordCount;
     
 
-    public WritableCreator(int recordCount) {
-      super();
-      this.recordCount = recordCount;
-    }
-    
-    @Override
-    public void apply(int key, ValueVector value) {
-      metadata.add(value.getMetadata());
-      for(ByteBuf b : value.getBuffers()){
+    for(ValueVector<?> vv : vectors){
+      metadata.add(vv.getMetadata());
+      for(ByteBuf b : vv.getBuffers()){
         buffers.add(b);
         b.retain();
       }
-      value.clear();
+      // allocate new buffer to release hold on old buffer.
+      vv.allocateNew(vv.capacity());
     }
 
-    public WritableBatch get(){
-      RecordBatchDef batchDef = RecordBatchDef.newBuilder().addAllField(metadata).setRecordCount(recordCount).build();
-      WritableBatch b = new WritableBatch(batchDef, buffers);
+    RecordBatchDef batchDef = RecordBatchDef.newBuilder().addAllField(metadata).setRecordCount(recordCount).build();
+    WritableBatch b = new WritableBatch(batchDef, buffers);
       return b;
-    }
     
   }
+  
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/NullableFixed8.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/NullableFixed8.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/NullableFixed8.java
new file mode 100644
index 0000000..3fc39eb
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/NullableFixed8.java
@@ -0,0 +1,36 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.record.vector;
+
+import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.record.MaterializedField;
+
+public final class NullableFixed8 extends NullableValueVector<NullableFixed8, Fixed8>{
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(NullableFixed8.class);
+
+  public NullableFixed8(MaterializedField field, BufferAllocator allocator) {
+    super(field, allocator);
+  }
+
+  @Override
+  protected Fixed8 getNewValueVector(BufferAllocator allocator) {
+    return new Fixed8(this.field, allocator);
+  }
+
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/SelectionVector.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/SelectionVector.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/SelectionVector.java
index 02b75ce..3546bd8 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/SelectionVector.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/SelectionVector.java
@@ -37,5 +37,13 @@ public class SelectionVector {
   public void allocateNew(int count) {
 
   }
+<<<<<<< HEAD:sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/SelectionVector.java
+=======
+  
+  public final int getInt(int index){
+    index*=4;
+    return data.getInt(index);
+  }
+>>>>>>> Build working:sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/NullableFixed4.java
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/SelectionVector2.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/SelectionVector2.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/SelectionVector2.java
new file mode 100644
index 0000000..c314cd4
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/SelectionVector2.java
@@ -0,0 +1,46 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.record.vector;
+
+import io.netty.buffer.ByteBuf;
+
+import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.record.DeadBuf;
+
+/**
+ * A selection vector that fronts, at most, a
+ */
+public class SelectionVector2{
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(SelectionVector2.class);
+
+  private final BufferAllocator allocator;
+  private ByteBuf buffer = DeadBuf.DEAD_BUFFER;
+
+  public SelectionVector2(BufferAllocator allocator) {
+    this.allocator = allocator;
+  }
+
+  public int getCount(){
+    return -1;
+  }
+
+  public int getIndex(int directIndex){
+    return buffer.getChar(directIndex);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/SelectionVector4.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/SelectionVector4.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/SelectionVector4.java
new file mode 100644
index 0000000..d857146
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/SelectionVector4.java
@@ -0,0 +1,41 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+
+package org.apache.drill.exec.record.vector;
+
+import io.netty.buffer.ByteBuf;
+
+import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.record.DeadBuf;
+
+public class SelectionVector4 {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(SelectionVector4.class);
+
+  private final BufferAllocator allocator;
+  private ByteBuf buffer = DeadBuf.DEAD_BUFFER;
+
+  public SelectionVector4(BufferAllocator allocator) {
+    this.allocator = allocator;
+  }
+
+  public int getCount(){
+    return -1;
+  }
+
+
+}


Re: [44/53] [abbrv] git commit: Types transition

Posted by Ted Dunning <te...@gmail.com>.
On Fri, Jul 19, 2013 at 6:58 PM, <ja...@apache.org> wrote:

> Types transition
>
>
> Project: http://git-wip-us.apache.org/repos/asf/incubator-drill/repo
> Commit:
> http://git-wip-us.apache.org/repos/asf/incubator-drill/commit/ce0da88d
> Tree: http://git-wip-us.apache.org/repos/asf/incubator-drill/tree/ce0da88d
> Diff: http://git-wip-us.apache.org/repos/asf/incubator-drill/diff/ce0da88d
>
> Branch: refs/heads/master
> Commit: ce0da88d2df38d079a11d2b269fad9e964e1c14a
> Parents: e68bba2
> Author: Jacques Nadeau <ja...@apache.org>
> Authored: Mon Jun 24 08:06:43 2013 -0700
> Committer: Jacques Nadeau <ja...@apache.org>
> Committed: Fri Jul 19 14:53:11 2013 -0700
>
> ----------------------------------------------------------------------
>  sandbox/prototype/common/pom.xml                | 216 ++---
>  .../drill/common/expression/parser/ExprParser.g |  64 +-
>  .../org/apache/drill/common/expression/Arg.java |  44 +-
>  .../common/expression/ArgumentValidator.java    |   2 +-
>  .../common/expression/ArgumentValidators.java   |  62 +-
>  .../expression/BasicArgumentValidator.java      |   8 +-
>  .../drill/common/expression/ErrorCollector.java |  18 +-
>  .../common/expression/ErrorCollectorImpl.java   |  40 +-
>  .../common/expression/ExpressionPosition.java   |  27 +
>  .../expression/ExpressionStringBuilder.java     | 102 +++
>  .../common/expression/ExpressionValidator.java  |  86 ++
>  .../drill/common/expression/FieldReference.java |  18 +-
>  .../drill/common/expression/FunctionBase.java   |  94 --
>  .../drill/common/expression/FunctionCall.java   |  55 +-
>  .../common/expression/FunctionDefinition.java   |   8 +-
>  .../common/expression/FunctionRegistry.java     |  13 +-
>  .../drill/common/expression/IfExpression.java   |  65 +-
>  .../common/expression/LogicalExpression.java    |  98 +-
>  .../expression/LogicalExpressionBase.java       |  56 +-
>  .../common/expression/OutputTypeDeterminer.java |  34 +-
>  .../drill/common/expression/PathSegment.java    |  26 +
>  .../drill/common/expression/SchemaPath.java     | 176 ++--
>  .../common/expression/ValueExpressions.java     | 320 ++++---
>  .../common/expression/fn/BooleanFunctions.java  |   6 +-
>  .../common/expression/fn/MathFunctions.java     |  10 +-
>  .../common/expression/fn/StringFunctions.java   |   7 +-
>  .../common/expression/fn/UnaryFunctions.java    |   6 +-
>  .../drill/common/expression/types/AtomType.java |  67 ++
>  .../drill/common/expression/types/DataType.java |   2 +-
>  .../visitors/AbstractExprVisitor.java           |  56 ++
>  .../expression/visitors/AggregateChecker.java   |  25 +-
>  .../expression/visitors/ConstantChecker.java    |  21 +-
>  .../common/expression/visitors/ExprVisitor.java |  18 +-
>  .../expression/visitors/SimpleExprVisitor.java  |  56 ++
>  .../org/apache/drill/common/types/Types.java    | 134 +++
>  .../common/src/main/protobuf/Types.proto        |  64 ++
>  sandbox/prototype/exec/java-exec/pom.xml        | 360 ++++----
>  .../org/apache/drill/exec/ExecConstants.java    |   1 +
>  .../drill/exec/compile/ClassTransformer.java    |   1 +
>  .../drill/exec/compile/CodeModelTools.java      |  31 +
>  .../apache/drill/exec/expr/CodeGenerator.java   | 201 +++++
>  .../apache/drill/exec/expr/DrillAggrFunc.java   |   9 +
>  .../org/apache/drill/exec/expr/DrillFunc.java   |   9 +
>  .../drill/exec/expr/EvaluationVisitor.java      | 201 +++++
>  .../exec/expr/ExpressionTreeMaterializer.java   | 128 +++
>  .../exec/expr/SingleClassStringWriter.java      |  40 +
>  .../exec/expr/ValueVectorReadExpression.java    |  41 +
>  .../exec/expr/ValueVectorWriteExpression.java   |  45 +
>  .../exec/expr/annotations/FunctionTemplate.java |  25 +
>  .../drill/exec/expr/annotations/Output.java     |  18 +
>  .../drill/exec/expr/annotations/Param.java      |  18 +
>  .../drill/exec/expr/annotations/WorkSpace.java  |  14 +
>  .../drill/exec/expr/fn/FunctionConverter.java   | 186 ++++
>  .../drill/exec/expr/fn/FunctionHolder.java      | 158 ++++
>  .../expr/fn/FunctionImplementationRegistry.java |  43 +
>  .../exec/expr/fn/MethodGrabbingVisitor.java     |  52 ++
>  .../exec/expr/fn/ModifiedUnparseVisitor.java    | 891 +++++++++++++++++++
>  .../drill/exec/expr/fn/impl/MathFunctions.java  |  48 +
>  .../drill/exec/expr/holders/ValueHolder.java    |   4 +
>  .../holders/ValueHolderImplmenetations.java     |  44 +
>  .../exec/expr/holders/VarBinaryHolder.java      |  12 +
>  .../apache/drill/exec/ops/FragmentContext.java  |  26 +-
>  .../apache/drill/exec/opt/BasicOptimizer.java   |  28 +-
>  .../apache/drill/exec/physical/RecordField.java |   8 +-
>  .../exec/physical/config/MockRecordReader.java  |  19 +-
>  .../drill/exec/physical/config/MockScanPOP.java |   8 +-
>  .../exec/physical/impl/FilterRecordBatch.java   |  34 +
>  .../drill/exec/physical/impl/OutputMutator.java |   9 +
>  .../drill/exec/physical/impl/ScanBatch.java     |  72 +-
>  .../drill/exec/physical/impl/VectorHolder.java  |  39 +
>  .../exec/physical/impl/WireRecordBatch.java     |  32 +-
>  .../impl/filter/EvalSetupException.java         |   5 +
>  .../impl/filter/EvaluationPredicate.java        |  13 +
>  .../physical/impl/filter/ExampleFilter.java     | 113 +++
>  .../physical/impl/project/ProjectEvaluator.java |  12 +
>  .../impl/project/ProjectRecordBatch.java        | 218 +++++
>  .../exec/physical/impl/project/Projector.java   |  20 +
>  .../impl/project/ProjectorTemplate.java         | 101 +++
>  .../physical/impl/project/TransferPairing.java  |  35 +
>  .../drill/exec/planner/PhysicalPlanReader.java  |   2 +-
>  .../apache/drill/exec/record/BatchSchema.java   |  36 +-
>  .../exec/record/ExpressionTreeMaterializer.java | 138 ---
>  .../drill/exec/record/MajorTypeSerDe.java       |   6 +-
>  .../drill/exec/record/MaterializedField.java    |  58 +-
>  .../drill/exec/record/NullExpression.java       |  32 +
>  .../apache/drill/exec/record/RecordBatch.java   |  67 +-
>  .../drill/exec/record/RecordBatchLoader.java    |  84 +-
>  .../apache/drill/exec/record/SchemaBuilder.java |  75 +-
>  .../apache/drill/exec/record/WritableBatch.java |  43 +-
>  .../exec/record/vector/NullableFixed8.java      |  36 +
>  .../exec/record/vector/SelectionVector.java     |   8 +
>  .../exec/record/vector/SelectionVector2.java    |  46 +
>  .../exec/record/vector/SelectionVector4.java    |  41 +
>  .../org/apache/drill/exec/schema/Field.java     |  31 +-
>  .../apache/drill/exec/schema/ListSchema.java    |  17 +-
>  .../apache/drill/exec/schema/NamedField.java    |  19 +-
>  .../apache/drill/exec/schema/OrderedField.java  |   9 +-
>  .../exec/schema/json/jackson/JacksonHelper.java |  26 +-
>  .../drill/exec/store/JSONRecordReader.java      |  84 +-
>  .../exec/work/batch/BitComHandlerImpl.java      |   3 +-
>  .../work/foreman/RunningFragmentManager.java    |   3 +-
>  .../work/fragment/RemoteFragmentHandler.java    |   3 +-
>  .../java-exec/src/main/protobuf/SchemaDef.proto |  63 +-
>  .../src/main/resources/drill-module.conf        |  13 +-
>  .../java/org/apache/drill/exec/SortTest.java    |  61 ++
>  .../apache/drill/exec/expr/ExpressionTest.java  | 108 +++
>  .../exec/physical/impl/PerformanceTests.java    |  32 +
>  .../impl/TestExecutionAbstractions.java         | 226 +++++
>  .../physical/impl/TestSimpleFragmentRun.java    |  51 +-
>  .../record/ExpressionTreeMaterializerTest.java  | 297 ++++---
>  .../drill/exec/store/JSONRecordReaderTest.java  | 469 +++++-----
>  .../org/apache/drill/exec/ref/RunOutcome.java   |   1 +
>  .../exec/ref/eval/BasicEvaluatorFactory.java    |  11 +-
>  .../apache/drill/exec/ref/eval/IfEvaluator.java |  10 +-
>  .../exec/ref/eval/SimpleEvaluationVisitor.java  |  24 +-
>  .../drill/exec/ref/eval/fn/MathEvaluators.java  |   5 +-
>  .../exec/ref/rops/CollapsingAggregateROP.java   |   4 +-
>  .../apache/drill/exec/ref/rops/ConstantROP.java |   3 -
>  .../apache/drill/exec/ref/rops/FlattenROP.java  |   7 +-
>  .../drill/exec/ref/rops/WindowFrameROP.java     |   5 +-
>  .../drill/exec/ref/values/BaseArrayValue.java   |  21 +-
>  .../drill/exec/ref/values/BaseMapValue.java     |   9 +-
>  .../apache/drill/exec/ref/values/DataValue.java |   4 +-
>  .../drill/exec/ref/values/NumericValue.java     |  17 +-
>  .../drill/exec/ref/values/ScalarValues.java     |  66 +-
>  .../drill/exec/ref/values/SimpleArrayValue.java |   4 +-
>  .../drill/exec/ref/values/SimpleMapValue.java   |   5 +-
>  .../drill/exec/ref/values/ValueReader.java      |  14 +-
>  .../drill/exec/ref/values/ValueUtils.java       |  17 +-
>  .../apache/drill/exec/ref/RunSimplePlan.java    |  14 +-
>  .../org/apache/drill/exec/ref/TestUtils.java    |   3 +-
>  .../exec/ref/rops/CollapsingAggregateTest.java  |   5 +-
>  .../drill/exec/ref/rops/ConstantROPTest.java    |   5 +-
>  .../drill/exec/ref/rops/OrderROPTest.java       |   5 +-
>  .../drill/exec/ref/rops/WindowFrameROPTest.java |  12 +-
>  sandbox/prototype/pom.xml                       | 381 ++++----
>  136 files changed, 6212 insertions(+), 2103 deletions(-)
> ----------------------------------------------------------------------
>
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/sandbox/prototype/common/pom.xml
> ----------------------------------------------------------------------
> diff --git a/sandbox/prototype/common/pom.xml
> b/sandbox/prototype/common/pom.xml
> index 4a8c5d8..4891191 100644
> --- a/sandbox/prototype/common/pom.xml
> +++ b/sandbox/prototype/common/pom.xml
> @@ -1,103 +1,119 @@
>  <?xml version="1.0"?>
>  <project
> -        xsi:schemaLocation="http://maven.apache.org/POM/4.0.0
> http://maven.apache.org/xsd/maven-4.0.0.xsd"
> -        xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="
> http://www.w3.org/2001/XMLSchema-instance">
> -    <modelVersion>4.0.0</modelVersion>
> -    <parent>
> -        <artifactId>prototype-parent</artifactId>
> -        <groupId>org.apache.drill</groupId>
> -        <version>1.0-SNAPSHOT</version>
> -    </parent>
> -
> -    <artifactId>common</artifactId>
> -    <packaging>jar</packaging>
> -    <name>common</name>
> -
> -    <dependencies>
> -
> -        <dependency>
> -            <groupId>com.google.protobuf</groupId>
> -            <artifactId>protobuf-java</artifactId>
> -            <version>2.5.0</version>
> -        </dependency>
> -
> -        <dependency>
> -            <groupId>com.typesafe</groupId>
> -            <artifactId>config</artifactId>
> -            <version>1.0.0</version>
> -        </dependency>
> -
> -        <dependency>
> -            <groupId>org.apache.commons</groupId>
> -            <artifactId>commons-lang3</artifactId>
> -            <version>3.1</version>
> -        </dependency>
> -
> -        <dependency>
> -            <groupId>org.msgpack</groupId>
> -            <artifactId>msgpack</artifactId>
> -            <version>0.6.6</version>
> -        </dependency>
> -
> -        <dependency>
> -            <groupId>org.reflections</groupId>
> -            <artifactId>reflections</artifactId>
> -            <version>0.9.8</version>
> -        </dependency>
> -
> -        <dependency>
> -            <groupId>com.fasterxml.jackson.core</groupId>
> -            <artifactId>jackson-annotations</artifactId>
> -            <version>2.1.1</version>
> -        </dependency>
> -        <dependency>
> -            <groupId>org.hibernate</groupId>
> -            <artifactId>hibernate-validator</artifactId>
> -            <version>4.3.1.Final</version>
> -        </dependency>
> -        <dependency>
> -            <groupId>com.fasterxml.jackson.core</groupId>
> -            <artifactId>jackson-databind</artifactId>
> -            <version>2.1.1</version>
> -        </dependency>
> -        <dependency>
> -            <groupId>org.antlr</groupId>
> -            <artifactId>antlr-runtime</artifactId>
> -            <version>3.4</version>
> -        </dependency>
> -
> -
> -    </dependencies>
> -
> -
> -    <build>
> -        <plugins>
> -            <plugin>
> -                <artifactId>maven-jar-plugin</artifactId>
> -                <executions>
> -                    <execution>
> -                        <id>test-jar</id>
> -                        <goals>
> -                            <goal>test-jar</goal>
> -                        </goals>
> -                    </execution>
> -                </executions>
> -            </plugin>
> -            <plugin>
> -                <groupId>org.antlr</groupId>
> -                <artifactId>antlr3-maven-plugin</artifactId>
> -                <version>3.4</version>
> -                <configuration>
> -
> -                </configuration>
> -                <executions>
> -                    <execution>
> -                        <goals>
> -                            <goal>antlr</goal>
> -                        </goals>
> -                    </execution>
> -                </executions>
> -            </plugin>
> -        </plugins>
> -    </build>
> +  xsi:schemaLocation="http://maven.apache.org/POM/4.0.0
> http://maven.apache.org/xsd/maven-4.0.0.xsd"
> +  xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="
> http://www.w3.org/2001/XMLSchema-instance">
> +  <modelVersion>4.0.0</modelVersion>
> +
> +  <parent>
> +    <artifactId>prototype-parent</artifactId>
> +    <groupId>org.apache.drill</groupId>
> +    <version>1.0-SNAPSHOT</version>
> +  </parent>
> +
> +  <artifactId>common</artifactId>
> +  <packaging>jar</packaging>
> +  <name>common</name>
> +
> +  <dependencies>
> +
> +    <dependency>
> +      <groupId>com.google.protobuf</groupId>
> +      <artifactId>protobuf-java</artifactId>
> +      <version>2.5.0</version>
> +    </dependency>
> +
> +    <dependency>
> +      <groupId>com.typesafe</groupId>
> +      <artifactId>config</artifactId>
> +      <version>1.0.0</version>
> +    </dependency>
> +
> +    <dependency>
> +      <groupId>org.apache.commons</groupId>
> +      <artifactId>commons-lang3</artifactId>
> +      <version>3.1</version>
> +    </dependency>
> +
> +    <dependency>
> +      <groupId>org.msgpack</groupId>
> +      <artifactId>msgpack</artifactId>
> +      <version>0.6.6</version>
> +    </dependency>
> +
> +    <dependency>
> +      <groupId>org.reflections</groupId>
> +      <artifactId>reflections</artifactId>
> +      <version>0.9.8</version>
> +    </dependency>
> +
> +    <dependency>
> +      <groupId>com.fasterxml.jackson.core</groupId>
> +      <artifactId>jackson-annotations</artifactId>
> +      <version>2.1.1</version>
> +    </dependency>
> +    <dependency>
> +      <groupId>org.hibernate</groupId>
> +      <artifactId>hibernate-validator</artifactId>
> +      <version>4.3.1.Final</version>
> +    </dependency>
> +    <dependency>
> +      <groupId>com.fasterxml.jackson.core</groupId>
> +      <artifactId>jackson-databind</artifactId>
> +      <version>2.1.1</version>
> +    </dependency>
> +    <dependency>
> +      <groupId>org.antlr</groupId>
> +      <artifactId>antlr-runtime</artifactId>
> +      <version>3.4</version>
> +    </dependency>
> +
> +
> +  </dependencies>
> +
> +
> +  <build>
> +    <plugins>
> +      <plugin>
> +        <artifactId>maven-jar-plugin</artifactId>
> +        <executions>
> +          <execution>
> +            <id>test-jar</id>
> +            <goals>
> +              <goal>test-jar</goal>
> +            </goals>
> +          </execution>
> +        </executions>
> +      </plugin>
> +      <plugin>
> +        <groupId>org.antlr</groupId>
> +        <artifactId>antlr3-maven-plugin</artifactId>
> +        <version>3.4</version>
> +        <configuration>
> +
> +        </configuration>
> +        <executions>
> +          <execution>
> +            <goals>
> +              <goal>antlr</goal>
> +            </goals>
> +          </execution>
> +        </executions>
> +      </plugin>
> +
> +
> +      <plugin>
> +        <groupId>com.github.igor-petruk.protobuf</groupId>
> +        <artifactId>protobuf-maven-plugin</artifactId>
> +        <version>0.6.3</version>
> +        <executions>
> +          <execution>
> +            <goals>
> +              <goal>run</goal>
> +            </goals>
> +          </execution>
> +        </executions>
> +      </plugin>
> +    </plugins>
> +
> +  </build>
>  </project>
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/sandbox/prototype/common/src/main/antlr3/org/apache/drill/common/expression/parser/ExprParser.g
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/common/src/main/antlr3/org/apache/drill/common/expression/parser/ExprParser.g
> b/sandbox/prototype/common/src/main/antlr3/org/apache/drill/common/expression/parser/ExprParser.g
> index f8eb725..6cf041c 100644
> ---
> a/sandbox/prototype/common/src/main/antlr3/org/apache/drill/common/expression/parser/ExprParser.g
> +++
> b/sandbox/prototype/common/src/main/antlr3/org/apache/drill/common/expression/parser/ExprParser.g
> @@ -40,7 +40,8 @@ import org.apache.drill.common.expression.*;
>
>  @members{
>    private FunctionRegistry registry;
> -
> +  private String fullExpression;
> +  private int tokenPos;
>    public void setRegistry(FunctionRegistry registry){
>      this.registry = registry;
>    }
> @@ -48,14 +49,22 @@ import org.apache.drill.common.expression.*;
>    public static void p(String s){
>      System.out.println(s);
>    }
> +
> +  public ExpressionPosition pos(Token token){
> +    return new ExpressionPosition(fullExpression, token.getTokenIndex());
> +  }
>  }
>
>  parse returns [LogicalExpression e]
> -  :  expression EOF {$e = $expression.e; }
> +  :  expression EOF {
> +    $e = $expression.e;
> +    if(fullExpression == null) fullExpression = $expression.text;
> +    tokenPos = $expression.start.getTokenIndex();
> +  }
>    ;
>
>  functionCall returns [LogicalExpression e]
> -  :  Identifier OParen exprList? CParen {$e =
> registry.createExpression($Identifier.text, $exprList.listE);  }
> +  :  Identifier OParen exprList? CParen {$e =
> registry.createExpression($Identifier.text, pos($Identifier),
> $exprList.listE);  }
>    ;
>
>  ifStatement returns [LogicalExpression e]
> @@ -65,7 +74,7 @@ ifStatement returns [LogicalExpression e]
>         @after {
>           $e = s.build();
>         }
> -  :  i1=ifStat {s.addCondition($i1.i); } (elseIfStat {
> s.addCondition($elseIfStat.i); } )* Else expression {
> s.setElse($expression.e); }End
> +  :  i1=ifStat {s.addCondition($i1.i); s.setPosition(pos($i1.start)); }
> (elseIfStat { s.addCondition($elseIfStat.i); } )* Else expression {
> s.setElse($expression.e); }End
>    ;
>
>  ifStat returns [IfExpression.IfCondition i]
> @@ -113,96 +122,103 @@ condExpr returns [LogicalExpression e]
>  orExpr returns [LogicalExpression e]
>         @init{
>           List<LogicalExpression> exprs = new
> ArrayList<LogicalExpression>();
> +         ExpressionPosition p = null;
>         }
>         @after{
>           if(exprs.size() == 1){
>             $e = exprs.get(0);
>           }else{
> -           $e = registry.createExpression("||", exprs);
> +           $e = registry.createExpression("||", p, exprs);
>           }
>         }
> -  :  a1=andExpr { exprs.add($a1.e); } (Or^ a2=andExpr { exprs.add($a2.e);
> })*
> +  :  a1=andExpr { exprs.add($a1.e); p = pos( $a1.start );} (Or^
> a2=andExpr { exprs.add($a2.e); })*
>    ;
>
>  andExpr returns [LogicalExpression e]
>         @init{
>           List<LogicalExpression> exprs = new
> ArrayList<LogicalExpression>();
> +         ExpressionPosition p = null;
>         }
>         @after{
>           if(exprs.size() == 1){
>             $e = exprs.get(0);
>           }else{
> -           $e = registry.createExpression("&&", exprs);
> +           $e = registry.createExpression("&&", p, exprs);
>           }
>         }
> -  :  e1=equExpr { exprs.add($e1.e);  } (And^ e2=equExpr {
> exprs.add($e2.e);  })*
> +  :  e1=equExpr { exprs.add($e1.e); p = pos( $e1.start );  } (And^
> e2=equExpr { exprs.add($e2.e);  })*
>    ;
>
>  equExpr returns [LogicalExpression e]
>         @init{
>           List<LogicalExpression> exprs = new
> ArrayList<LogicalExpression>();
>           List<String> cmps = new ArrayList();
> +         ExpressionPosition p = null;
>         }
>         @after{
> -         $e = registry.createByOp(exprs, cmps);
> +         $e = registry.createByOp(exprs, p, cmps);
>         }
> -  :  r1=relExpr {exprs.add($r1.e);} ( cmpr= (Equals | NEquals )
> r2=relExpr {exprs.add($r2.e); cmps.add($cmpr.text); })*
> +  :  r1=relExpr { exprs.add($r1.e); p = pos( $r1.start );
> +    } ( cmpr= (Equals | NEquals ) r2=relExpr {exprs.add($r2.e);
> cmps.add($cmpr.text); })*
>    ;
>
>  relExpr returns [LogicalExpression e]
> -  :  left=addExpr {$e = $left.e; } (cmpr = (GTEquals | LTEquals | GT |
> LT) right=addExpr {$e = registry.createExpression($cmpr.text, $left.e,
> $right.e); } )?
> +  :  left=addExpr {$e = $left.e; } (cmpr = (GTEquals | LTEquals | GT |
> LT) right=addExpr {$e = registry.createExpression($cmpr.text,
> pos($left.start), $left.e, $right.e); } )?
>    ;
>
>  addExpr returns [LogicalExpression e]
>         @init{
>           List<LogicalExpression> exprs = new
> ArrayList<LogicalExpression>();
>           List<String> ops = new ArrayList();
> +         ExpressionPosition p = null;
>         }
>         @after{
> -         $e = registry.createByOp(exprs, ops);
> +         $e = registry.createByOp(exprs, p, ops);
>         }
> -  :  m1=mulExpr  {exprs.add($m1.e);} ( op=(Plus|Minus) m2=mulExpr
> {exprs.add($m2.e); ops.add($op.text); })*
> +  :  m1=mulExpr  {exprs.add($m1.e); p = pos($m1.start); } (
> op=(Plus|Minus) m2=mulExpr {exprs.add($m2.e); ops.add($op.text); })*
>    ;
>
>  mulExpr returns [LogicalExpression e]
>         @init{
>           List<LogicalExpression> exprs = new
> ArrayList<LogicalExpression>();
>           List<String> ops = new ArrayList();
> +         ExpressionPosition p = null;
>         }
>         @after{
> -         $e = registry.createByOp(exprs, ops);
> +         $e = registry.createByOp(exprs, p, ops);
>         }
> -  :  p1=powExpr  {exprs.add($p1.e);} (op=(Asterisk|ForwardSlash|Percent)
> p2=powExpr {exprs.add($p2.e); ops.add($op.text); } )*
> +  :  p1=powExpr  {exprs.add($p1.e); p = pos($p1.start);}
> (op=(Asterisk|ForwardSlash|Percent) p2=powExpr {exprs.add($p2.e);
> ops.add($op.text); } )*
>    ;
>
>  powExpr returns [LogicalExpression e]
>         @init{
>           List<LogicalExpression> exprs = new
> ArrayList<LogicalExpression>();
>           List<String> ops = new ArrayList();
> +         ExpressionPosition p = null;
>         }
>         @after{
> -         $e = registry.createByOp(exprs, ops);
> +         $e = registry.createByOp(exprs, p, ops);
>         }
> -  :  u1=unaryExpr {exprs.add($u1.e);} (Caret u2=unaryExpr
> {exprs.add($u2.e); ops.add($Caret.text);} )*
> +  :  u1=unaryExpr {exprs.add($u1.e); p = pos($u1.start);} (Caret
> u2=unaryExpr {exprs.add($u2.e); ops.add($Caret.text);} )*
>    ;
>
>  unaryExpr returns [LogicalExpression e]
> -  :  Minus atom {$e = registry.createExpression("u-", $atom.e); }
> -  |  Excl atom {$e= registry.createExpression("!", $atom.e); }
> +  :  Minus atom {$e = registry.createExpression("u-", pos($atom.start),
> $atom.e); }
> +  |  Excl atom {$e= registry.createExpression("!", pos($atom.start),
> $atom.e); }
>    |  atom {$e = $atom.e; }
>    ;
>
>  atom returns [LogicalExpression e]
> -  :  Number {$e = ValueExpressions.getNumericExpression($Number.text); }
> -  |  Bool {$e = new ValueExpressions.BooleanExpression( $Bool.text ); }
> +  :  Number {$e = ValueExpressions.getNumericExpression($Number.text,
> pos($atom.start)); }
> +  |  Bool {$e = new ValueExpressions.BooleanExpression( $Bool.text,
> pos($atom.start)); }
>    |  lookup {$e = $lookup.e; }
>    ;
>
>
>  lookup returns [LogicalExpression e]
>    :  functionCall {$e = $functionCall.e ;}
> -  | Identifier {$e = new SchemaPath($Identifier.text); }
> -  | String {$e = new ValueExpressions.QuotedString($String.text); }
> +  | Identifier {$e = new SchemaPath($Identifier.text, pos($Identifier) );
> }
> +  | String {$e = new ValueExpressions.QuotedString($String.text,
> pos($String) ); }
>    | OParen expression CParen  {$e = $expression.e; }
> -  | SingleQuote Identifier SingleQuote {$e = new
> SchemaPath($Identifier.text); }
> +  | SingleQuote Identifier SingleQuote {$e = new
> SchemaPath($Identifier.text, pos($Identifier) ); }
>    ;
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/Arg.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/Arg.java
> b/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/Arg.java
> index de9057b..64ec351 100644
> ---
> a/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/Arg.java
> +++
> b/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/Arg.java
> @@ -17,24 +17,41 @@
>
> ******************************************************************************/
>  package org.apache.drill.common.expression;
>
> -import org.apache.drill.common.expression.types.DataType;
> +import java.util.List;
> +
>  import org.apache.drill.common.expression.visitors.ConstantChecker;
> +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 com.google.common.collect.Lists;
>
>
>  public class Arg {
>    private final String name;
> -  private final DataType[] allowedTypes;
> +  private final MajorType[] allowedTypes;
>    private final boolean constantsOnly;
>
> -  public Arg(DataType... allowedTypes){
> +
> +  /**
> +   * Create an arg that supports any of the listed minor types using opt
> or req.  Does not allow repeated types.
> +   * @param name
> +   * @param constantsOnly
> +   * @param types
> +   */
> +  public Arg(boolean constantsOnly, boolean allowNulls, String name,
> MinorType... types){
> +    this(constantsOnly, name, getMajorTypes(allowNulls, types));
> +  }
> +
> +  public Arg(MajorType... allowedTypes){
>      this(false, null, allowedTypes);
>    }
>
> -  public Arg(String name, DataType... allowedTypes) {
> +  public Arg(String name, MajorType... allowedTypes) {
>      this(false, name, allowedTypes);
>    }
>
> -  public Arg(boolean constantsOnly, String name, DataType...
> allowedTypes) {
> +  public Arg(boolean constantsOnly, String name, MajorType...
> allowedTypes) {
>      this.name = name;
>      this.allowedTypes = allowedTypes;
>      this.constantsOnly = constantsOnly;
> @@ -44,15 +61,15 @@ public class Arg {
>      return name;
>    }
>
> -  public void confirmDataType(String expr, int argIndex,
> LogicalExpression e, ErrorCollector errors){
> +  public void confirmDataType(ExpressionPosition expr, int argIndex,
> LogicalExpression e, ErrorCollector errors){
>      if(constantsOnly){
>        if(ConstantChecker.onlyIncludesConstants(e))
> errors.addExpectedConstantValue(expr, argIndex, name);
>      }
> -    DataType dt = e.getDataType();
> -    if(dt.isLateBind()){
> +    MajorType dt = e.getMajorType();
> +    if(dt.getMinorType() == MinorType.LATE){
>
>        // change among allowed types.
> -      for(DataType a : allowedTypes){
> +      for(MajorType a : allowedTypes){
>          if(dt == a) return;
>        }
>
> @@ -63,4 +80,13 @@ public class Arg {
>
>
>    }
> +
> +  private static MajorType[] getMajorTypes(boolean allowNulls,
> MinorType... types){
> +    List<MajorType> mts = Lists.newArrayList();
> +    for(MinorType t : types){
> +      if(allowNulls)
> mts.add(MajorType.newBuilder().setMinorType(t).setMode(DataMode.OPTIONAL).build());
> +
>  mts.add(MajorType.newBuilder().setMinorType(t).setMode(DataMode.REQUIRED).build());
> +    }
> +    return mts.toArray(new MajorType[mts.size()]);
> +  }
>  }
> \ No newline at end of file
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/ArgumentValidator.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/ArgumentValidator.java
> b/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/ArgumentValidator.java
> index dc22045..765dc3f 100644
> ---
> a/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/ArgumentValidator.java
> +++
> b/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/ArgumentValidator.java
> @@ -23,6 +23,6 @@ import java.util.List;
>   * Validates whether the set of arguments are acceptable
>   */
>  public interface ArgumentValidator {
> -  public void validateArguments(String expr, List<LogicalExpression>
> expressions, ErrorCollector errors);
> +  public void validateArguments(ExpressionPosition expr,
> List<LogicalExpression> expressions, ErrorCollector errors);
>    public String[] getArgumentNamesByPosition();
>  }
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/ArgumentValidators.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/ArgumentValidators.java
> b/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/ArgumentValidators.java
> index 25cb887..f30733d 100644
> ---
> a/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/ArgumentValidators.java
> +++
> b/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/ArgumentValidators.java
> @@ -20,8 +20,8 @@ package org.apache.drill.common.expression;
>  import java.util.List;
>
>  import org.apache.commons.lang3.ArrayUtils;
> -import org.apache.drill.common.expression.types.DataType;
> -import org.apache.drill.common.expression.types.DataType.Comparability;
> +import org.apache.drill.common.types.TypeProtos.MajorType;
> +import org.apache.drill.common.types.Types;
>
>  import com.google.common.base.Predicate;
>  import com.google.common.collect.Range;
> @@ -46,7 +46,7 @@ public class ArgumentValidators {
>      }
>
>      @Override
> -    public void validateArguments(String expr, List<LogicalExpression>
> expressions, ErrorCollector errors) {
> +    public void validateArguments(ExpressionPosition expr,
> List<LogicalExpression> expressions, ErrorCollector errors) {
>        // only need to check argument count since any type is allowed.
>        if (!argumentCount.contains(expressions.size()))
>          errors.addUnexpectedArgumentCount(expr, expressions.size(),
> argumentCount);
> @@ -61,17 +61,17 @@ public class ArgumentValidators {
>
>    private static class PredicateValidator implements ArgumentValidator {
>      private final Range<Integer> argumentCount;
> -    private Predicate<DataType> predicate;
> +    private Predicate<MajorType> predicate;
>      private boolean allSame;
>
> -    public PredicateValidator(int argCount, Predicate<DataType>
> predicate, boolean allSame) {
> +    public PredicateValidator(int argCount, Predicate<MajorType>
> predicate, boolean allSame) {
>        super();
>        this.argumentCount = Ranges.singleton(argCount);
>        this.predicate = predicate;
>        this.allSame = allSame;
>      }
>
> -    public PredicateValidator(int minArguments, int maxArguments,
> Predicate<DataType> predicate, boolean allSame) {
> +    public PredicateValidator(int minArguments, int maxArguments,
> Predicate<MajorType> predicate, boolean allSame) {
>        super();
>        this.argumentCount = Ranges.closedOpen(minArguments, maxArguments);
>        this.predicate = predicate;
> @@ -79,21 +79,21 @@ public class ArgumentValidators {
>      }
>
>      @Override
> -    public void validateArguments(String expr, List<LogicalExpression>
> expressions, ErrorCollector errors) {
> +    public void validateArguments(ExpressionPosition expr,
> List<LogicalExpression> expressions, ErrorCollector errors) {
>        int i = -1;
> -      DataType t = null;
> +      MajorType t = null;
>        for (LogicalExpression le : expressions) {
>          i++;
> -          DataType dataType = le.getDataType();
> -          if (t == null) t = dataType;
> +          MajorType majorType = le.getMajorType();
> +          if (t == null) t = majorType;
>
> -        if (!predicate.apply(dataType)) {
> -          errors.addUnexpectedType(expr, i, dataType);
> +        if (!predicate.apply(majorType)) {
> +          errors.addUnexpectedType(expr, i, majorType);
>            continue;
>          }
>
> -        if (allSame && t != DataType.LATEBIND && dataType !=
> DataType.LATEBIND && t != dataType) {
> -          errors.addUnexpectedType(expr, i, dataType);
> +        if (allSame && !Types.isLateBind(t) &&
> !Types.isLateBind(majorType) && !Types.softEquals(t, majorType, true)) {
> +          errors.addUnexpectedType(expr, i, majorType);
>          }
>
>        }
> @@ -109,50 +109,50 @@ public class ArgumentValidators {
>
>    public static class ComparableArguments extends PredicateValidator {
>
> -    public ComparableArguments(int argCount, DataType... allowedTypes) {
> +    public ComparableArguments(int argCount, MajorType... allowedTypes) {
>        super(argCount, new ComparableChecker(), true);
>      }
>
> -    public ComparableArguments(int minArguments, int maxArguments,
> DataType... allowedTypes) {
> +    public ComparableArguments(int minArguments, int maxArguments,
> MajorType... allowedTypes) {
>        super(minArguments, maxArguments, new ComparableChecker(), true);
>      }
>
> -    public static class ComparableChecker implements Predicate<DataType> {
> +    public static class ComparableChecker implements Predicate<MajorType>
> {
>
> -      public boolean apply(DataType dt) {
> -          Comparability comparability = dt.getComparability();
> -          return comparability.equals(Comparability.ORDERED) ||
> comparability.equals(Comparability.UNKNOWN);
> +      public boolean apply(MajorType dt) {
> +          Types.Comparability comparability = Types.getComparability(dt);
> +          return comparability.equals(Types.Comparability.ORDERED) ||
> comparability.equals(Types.Comparability.UNKNOWN);
>        }
>      }
>    }
>
>    public static class AllowedTypeList extends PredicateValidator {
>
> -    public AllowedTypeList(int argCount, DataType... allowedTypes) {
> +    public AllowedTypeList(int argCount, MajorType... allowedTypes) {
>        super(argCount, new AllowedTypeChecker(allowedTypes), false);
>      }
>
> -    public AllowedTypeList(int minArguments, int maxArguments,
> DataType... allowedTypes) {
> +    public AllowedTypeList(int minArguments, int maxArguments,
> MajorType... allowedTypes) {
>        super(minArguments, maxArguments, new
> AllowedTypeChecker(allowedTypes), false);
>      }
>
> -    public AllowedTypeList(int argCount, boolean allSame, DataType...
> allowedTypes) {
> +    public AllowedTypeList(int argCount, boolean allSame, MajorType...
> allowedTypes) {
>        super(argCount, new AllowedTypeChecker(allowedTypes), allSame);
>      }
>
> -    public AllowedTypeList(int minArguments, int maxArguments, boolean
> allSame, DataType... allowedTypes) {
> +    public AllowedTypeList(int minArguments, int maxArguments, boolean
> allSame, MajorType... allowedTypes) {
>        super(minArguments, maxArguments, new
> AllowedTypeChecker(allowedTypes), allSame);
>      }
>
> -    public static class AllowedTypeChecker implements Predicate<DataType>
> {
> +    public static class AllowedTypeChecker implements
> Predicate<MajorType> {
>
> -      private DataType[] allowedTypes;
> +      private MajorType[] allowedTypes;
>
> -      public AllowedTypeChecker(DataType... allowedTypes) {
> +      public AllowedTypeChecker(MajorType... allowedTypes) {
>          this.allowedTypes = allowedTypes;
>        }
>
> -      public boolean apply(DataType dt) {
> +      public boolean apply(MajorType dt) {
>          return ArrayUtils.contains(allowedTypes, dt);
>        }
>      }
> @@ -170,10 +170,10 @@ public class ArgumentValidators {
>        super(minArguments, maxArguments, new NumericTypeChecker(),
> allSame);
>      }
>
> -    public static class NumericTypeChecker implements Predicate<DataType>
> {
> +    public static class NumericTypeChecker implements
> Predicate<MajorType> {
>
> -      public boolean apply(DataType dt) {
> -        return dt.isNumericType();
> +      public boolean apply(MajorType dt) {
> +        return Types.isNumericType(dt);
>        }
>      }
>
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/BasicArgumentValidator.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/BasicArgumentValidator.java
> b/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/BasicArgumentValidator.java
> index eed49d3..26b7cf6 100644
> ---
> a/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/BasicArgumentValidator.java
> +++
> b/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/BasicArgumentValidator.java
> @@ -19,13 +19,13 @@ package org.apache.drill.common.expression;
>
>  import java.util.List;
>
> -import org.apache.drill.common.expression.types.DataType;
> +import org.apache.drill.common.types.TypeProtos.MajorType;
>
>  public class BasicArgumentValidator implements ArgumentValidator {
>
>    private final Arg[] args;
>
> -  public BasicArgumentValidator(DataType... types) {
> +  public BasicArgumentValidator(MajorType... types) {
>      this.args = new Arg[] { new Arg("single", types) };
>    }
>
> @@ -34,7 +34,7 @@ public class BasicArgumentValidator implements
> ArgumentValidator {
>    }
>
>    @Override
> -  public void validateArguments(String expr, List<LogicalExpression>
> expressions, ErrorCollector errors) {
> +  public void validateArguments(ExpressionPosition expr,
> List<LogicalExpression> expressions, ErrorCollector errors) {
>      if (expressions.size() != args.length)
> errors.addUnexpectedArgumentCount(expr, expressions.size(), args.length);
>
>      int i = 0;
> @@ -45,7 +45,7 @@ public class BasicArgumentValidator implements
> ArgumentValidator {
>      }
>    }
>
> -  public Arg arg(String name, DataType... allowedTypes) {
> +  public Arg arg(String name, MajorType... allowedTypes) {
>      return new Arg(name, allowedTypes);
>    }
>
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/ErrorCollector.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/ErrorCollector.java
> b/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/ErrorCollector.java
> index 21ecec4..b322345 100644
> ---
> a/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/ErrorCollector.java
> +++
> b/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/ErrorCollector.java
> @@ -17,27 +17,29 @@
>
> ******************************************************************************/
>  package org.apache.drill.common.expression;
>
> -import org.apache.drill.common.expression.types.DataType;
> +import org.apache.drill.common.types.TypeProtos.MajorType;
>
>  import com.google.common.collect.Range;
>
>  public interface ErrorCollector {
>
> -    public void addGeneralError(String expr, String s);
> +    public void addGeneralError(ExpressionPosition expr, String s);
>
> -    public void addUnexpectedArgumentType(String expr, String name,
> DataType actual, DataType[] expected, int argumentIndex);
> +    public void addUnexpectedArgumentType(ExpressionPosition expr, String
> name, MajorType actual, MajorType[] expected, int argumentIndex);
>
> -    public void addUnexpectedArgumentCount(String expr, int actual,
> Range<Integer> expected);
> +    public void addUnexpectedArgumentCount(ExpressionPosition expr, int
> actual, Range<Integer> expected);
>
> -    public void addUnexpectedArgumentCount(String expr, int actual, int
> expected);
> +    public void addUnexpectedArgumentCount(ExpressionPosition expr, int
> actual, int expected);
>
> -    public void addNonNumericType(String expr, DataType actual);
> +    public void addNonNumericType(ExpressionPosition expr, MajorType
> actual);
>
> -    public void addUnexpectedType(String expr, int index, DataType
> actual);
> +    public void addUnexpectedType(ExpressionPosition expr, int index,
> MajorType actual);
>
> -    public void addExpectedConstantValue(String expr, int actual, String
> s);
> +    public void addExpectedConstantValue(ExpressionPosition expr, int
> actual, String s);
>
>      boolean hasErrors();
> +
> +    public int getErrorCount();
>
>      String toErrorString();
>  }
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/ErrorCollectorImpl.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/ErrorCollectorImpl.java
> b/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/ErrorCollectorImpl.java
> index cc90b82..054e067 100644
> ---
> a/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/ErrorCollectorImpl.java
> +++
> b/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/ErrorCollectorImpl.java
> @@ -1,12 +1,13 @@
>  package org.apache.drill.common.expression;
>
> +import java.util.Arrays;
> +import java.util.List;
> +
> +import org.apache.drill.common.types.TypeProtos.MajorType;
> +
>  import com.google.common.base.Joiner;
>  import com.google.common.collect.Lists;
>  import com.google.common.collect.Range;
> -import org.apache.drill.common.expression.types.DataType;
> -
> -import java.util.Arrays;
> -import java.util.List;
>
>  public class ErrorCollectorImpl implements ErrorCollector {
>      List<ExpressionValidationError> errors;
> @@ -15,17 +16,17 @@ public class ErrorCollectorImpl implements
> ErrorCollector {
>          errors = Lists.newArrayList();
>      }
>
> -    private String addExpr(String expr, String message) {
> -        return "Expression: [" + expr + "]. Error: " + message;
> +    private String addExpr(ExpressionPosition expr, String message) {
> +        return String.format("Error in expression at index %d.  Error:
> %s.  Full expression: %s.", expr.getCharIndex(), message,
> expr.getExpression());
>      }
>
>      @Override
> -    public void addGeneralError(String expr, String s) {
> +    public void addGeneralError(ExpressionPosition expr, String s) {
>          errors.add(new ExpressionValidationError(addExpr(expr, s)));
>      }
>
>      @Override
> -    public void addUnexpectedArgumentType(String expr, String name,
> DataType actual, DataType[] expected, int argumentIndex) {
> +    public void addUnexpectedArgumentType(ExpressionPosition expr, String
> name, MajorType actual, MajorType[] expected, int argumentIndex) {
>          errors.add(
>                  new ExpressionValidationError(
>                          addExpr(expr, String.format(
> @@ -37,35 +38,35 @@ public class ErrorCollectorImpl implements
> ErrorCollector {
>      }
>
>      @Override
> -    public void addUnexpectedArgumentCount(String expr, int actual,
> Range<Integer> expected) {
> +    public void addUnexpectedArgumentCount(ExpressionPosition expr, int
> actual, Range<Integer> expected) {
>          errors.add(new ExpressionValidationError(
>                  addExpr(expr, String.format("Unexpected argument count.
> Actual argument count: %d, Expected range: %s", actual, expected))
>          ));
>      }
>
>      @Override
> -    public void addUnexpectedArgumentCount(String expr, int actual, int
> expected) {
> +    public void addUnexpectedArgumentCount(ExpressionPosition expr, int
> actual, int expected) {
>          errors.add(new ExpressionValidationError(
>                  addExpr(expr, String.format("Unexpected argument count.
> Actual argument count: %d, Expected count: %d", actual, expected))
>          ));
>      }
>
>      @Override
> -    public void addNonNumericType(String expr, DataType actual) {
> +    public void addNonNumericType(ExpressionPosition expr, MajorType
> actual) {
>          errors.add(new ExpressionValidationError(
>                  addExpr(expr, String.format("Unexpected numeric type.
> Actual type: %s", actual))
>          ));
>      }
>
>      @Override
> -    public void addUnexpectedType(String expr, int index, DataType
> actual) {
> +    public void addUnexpectedType(ExpressionPosition expr, int index,
> MajorType actual) {
>          errors.add(new ExpressionValidationError(
>                  addExpr(expr, String.format("Unexpected argument type.
> Actual type: %s, Index: %d", actual, index))
>          ));
>      }
>
>      @Override
> -    public void addExpectedConstantValue(String expr, int actual, String
> s) {
> +    public void addExpectedConstantValue(ExpressionPosition expr, int
> actual, String s) {
>          errors.add(new ExpressionValidationError(
>                  addExpr(expr, String.format("Unexpected constant value.
> Name: %s, Actual: %s", s, actual))
>          ));
> @@ -76,8 +77,21 @@ public class ErrorCollectorImpl implements
> ErrorCollector {
>          return !errors.isEmpty();
>      }
>
> +
> +    @Override
> +    public int getErrorCount() {
> +      return errors.size();
> +    }
> +
>      @Override
>      public String toErrorString() {
>          return "\n" + Joiner.on("\n").join(errors);
>      }
> +
> +    @Override
> +    public String toString() {
> +      return toErrorString();
> +    }
> +
> +
>  }
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/ExpressionPosition.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/ExpressionPosition.java
> b/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/ExpressionPosition.java
> new file mode 100644
> index 0000000..2e8b529
> --- /dev/null
> +++
> b/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/ExpressionPosition.java
> @@ -0,0 +1,27 @@
> +package org.apache.drill.common.expression;
> +
> +public class ExpressionPosition {
> +  static final org.slf4j.Logger logger =
> org.slf4j.LoggerFactory.getLogger(ExpressionPosition.class);
> +
> +  public static final ExpressionPosition UNKNOWN = new
> ExpressionPosition("--UNKNOWN EXPRESSION--", -1);
> +
> +  private final String expression;
> +  private final int charIndex;
> +
> +  public ExpressionPosition(String expression, int charIndex) {
> +    super();
> +    this.expression = expression;
> +    this.charIndex = charIndex;
> +  }
> +
> +  public String getExpression() {
> +    return expression;
> +  }
> +
> +  public int getCharIndex() {
> +    return charIndex;
> +  }
> +
> +
> +
> +}
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/ExpressionStringBuilder.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/ExpressionStringBuilder.java
> b/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/ExpressionStringBuilder.java
> new file mode 100644
> index 0000000..09a7d57
> --- /dev/null
> +++
> b/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/ExpressionStringBuilder.java
> @@ -0,0 +1,102 @@
> +package org.apache.drill.common.expression;
> +
> +import org.apache.drill.common.expression.IfExpression.IfCondition;
> +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.AbstractExprVisitor;
> +
> +import com.google.common.collect.ImmutableList;
> +
> +public class ExpressionStringBuilder extends AbstractExprVisitor<Void,
> StringBuilder, RuntimeException>{
> +  static final org.slf4j.Logger logger =
> org.slf4j.LoggerFactory.getLogger(ExpressionStringBuilder.class);
> +
> +  @Override
> +  public Void visitFunctionCall(FunctionCall call, StringBuilder sb)
> throws RuntimeException {
> +    FunctionDefinition func = call.getDefinition();
> +    ImmutableList<LogicalExpression> args = call.args;
> +    if (func.isOperator()) {
> +      if (args.size() == 1) { // unary
> +        func.addRegisteredName(sb);
> +        sb.append("(");
> +        args.get(0).accept(this, sb);
> +        sb.append(")");
> +      } else {
> +        for (int i = 0; i < args.size(); i++) {
> +          if (i != 0) {
> +            sb.append(" ");
> +            func.addRegisteredName(sb);
> +          }
> +          sb.append(" (");
> +          args.get(i).accept(this, sb);
> +          sb.append(") ");
> +        }
> +      }
> +    } else { // normal function
> +
> +      func.addRegisteredName(sb);
> +      sb.append("(");
> +      for (int i = 0; i < args.size(); i++) {
> +        if (i != 0) sb.append(", ");
> +        args.get(i).accept(this, sb);
> +      }
> +      sb.append(") ");
> +    }
> +    return null;
> +  }
> +
> +  @Override
> +  public Void visitIfExpression(IfExpression ifExpr, StringBuilder sb)
> throws RuntimeException {
> +    ImmutableList<IfCondition> conditions = ifExpr.conditions;
> +    sb.append(" ( ");
> +    for(int i =0; i < conditions.size(); i++){
> +      IfCondition c = conditions.get(i);
> +      if(i !=0) sb.append(" else ");
> +      sb.append("if (");
> +      c.condition.accept(this, sb);
> +      sb.append(" ) then (");
> +      c.expression.accept(this, sb);
> +      sb.append(" ) ");
> +    }
> +    sb.append(" end ");
> +    sb.append(" ) ");
> +    return null;
> +  }
> +
> +  @Override
> +  public Void visitSchemaPath(SchemaPath path, StringBuilder sb) throws
> RuntimeException {
> +    sb.append("'");
> +    sb.append(path.getPath());
> +    sb.append("'");
> +    return null;
> +  }
> +
> +  @Override
> +  public Void visitLongConstant(LongExpression lExpr, StringBuilder sb)
> throws RuntimeException {
> +    sb.append(lExpr.getLong());
> +    return null;
> +  }
> +
> +  @Override
> +  public Void visitDoubleConstant(DoubleExpression dExpr, StringBuilder
> sb) throws RuntimeException {
> +    sb.append(dExpr.getDouble());
> +    return null;
> +  }
> +
> +  @Override
> +  public Void visitBooleanConstant(BooleanExpression e, StringBuilder sb)
> throws RuntimeException {
> +    sb.append(e.getBoolean());
> +    return null;
> +  }
> +
> +  @Override
> +  public Void visitQuotedStringConstant(QuotedString e, StringBuilder sb)
> throws RuntimeException {
> +    sb.append("\"");
> +    sb.append(e.value);
> +    sb.append("\"");
> +    return null;
> +  }
> +
> +
> +}
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/ExpressionValidator.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/ExpressionValidator.java
> b/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/ExpressionValidator.java
> new file mode 100644
> index 0000000..98a9075
> --- /dev/null
> +++
> b/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/ExpressionValidator.java
> @@ -0,0 +1,86 @@
> +package org.apache.drill.common.expression;
> +
> +import org.apache.drill.common.expression.IfExpression.IfCondition;
> +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 org.apache.drill.common.types.TypeProtos.DataMode;
> +import org.apache.drill.common.types.TypeProtos.MajorType;
> +import org.apache.drill.common.types.TypeProtos.MinorType;
> +
> +public class ExpressionValidator implements ExprVisitor<Void,
> ErrorCollector, RuntimeException> {
> +  static final org.slf4j.Logger logger =
> org.slf4j.LoggerFactory.getLogger(ExpressionValidator.class);
> +
> +  @Override
> +  public Void visitFunctionCall(FunctionCall call, ErrorCollector errors)
> throws RuntimeException {
> +    call.getDefinition().getArgumentValidator()
> +        .validateArguments(call.getPosition(), call.args, errors);
> +    return null;
> +  }
> +
> +  @Override
> +  public Void visitIfExpression(IfExpression ifExpr, ErrorCollector
> errors) throws RuntimeException {
> +    // confirm that all conditions are required boolean values.
> +    int i = 0;
> +    for (IfCondition c : ifExpr.conditions) {
> +      MajorType mt = c.condition.getMajorType();
> +      if (mt.getMode() != DataMode.REQUIRED || mt.getMinorType() !=
> MinorType.BOOLEAN){
> +        errors.addGeneralError(c.condition.getPosition(),String.format(
> +                        "Failure composing If Expression.  All conditions
> must return a required value and be of type boolean.  Condition %d was
> DatMode %s and Type %s.",
> +                        i, mt.getMode(), mt.getMinorType()));
> +      }
> +      i++;
> +    }
> +
> +    // confirm that all outcomes are the same type.
> +    final MajorType mt = ifExpr.elseExpression.getMajorType();
> +    i = 0;
> +    for (IfCondition c : ifExpr.conditions) {
> +      MajorType innerT = c.expression.getMajorType();
> +      if (
> +          (innerT.getMode() == DataMode.REPEATED && mt.getMode() !=
> DataMode.REPEATED) || //
> +          (innerT.getMinorType() != mt.getMinorType())
> +          ) {
> +        errors.addGeneralError(c.condition.getPosition(),String.format(
> +            "Failure composing If Expression.  All expressions must
> return the same MajorType as the else expression.  The %d if condition
> returned type type %s but the else expression was of type %s",
> +            i, innerT, mt));
> +      }
> +      i++;
> +    }
> +    return null;
> +  }
> +
> +  @Override
> +  public Void visitSchemaPath(SchemaPath path, ErrorCollector errors)
> throws RuntimeException {
> +    return null;
> +  }
> +
> +  @Override
> +  public Void visitLongConstant(LongExpression intExpr, ErrorCollector
> errors) throws RuntimeException {
> +    return null;
> +  }
> +
> +  @Override
> +  public Void visitDoubleConstant(DoubleExpression dExpr, ErrorCollector
> errors) throws RuntimeException {
> +    return null;
> +  }
> +
> +  @Override
> +  public Void visitBooleanConstant(BooleanExpression e, ErrorCollector
> errors) throws RuntimeException {
> +    return null;
> +  }
> +
> +  @Override
> +  public Void visitQuotedStringConstant(QuotedString e, ErrorCollector
> errors) throws RuntimeException {
> +    return null;
> +  }
> +
> +  @Override
> +  public Void visitUnknown(LogicalExpression e, ErrorCollector value)
> throws RuntimeException {
> +    return null;
> +  }
> +
> +
> +}
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/FieldReference.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/FieldReference.java
> b/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/FieldReference.java
> index 94800ba..f611614 100644
> ---
> a/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/FieldReference.java
> +++
> b/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/FieldReference.java
> @@ -21,6 +21,7 @@ import java.io.IOException;
>
>  import org.apache.drill.common.expression.FieldReference.De;
>  import org.apache.drill.common.expression.FieldReference.Se;
> +import org.apache.drill.common.types.TypeProtos.MajorType;
>
>  import com.fasterxml.jackson.core.JsonGenerationException;
>  import com.fasterxml.jackson.core.JsonGenerator;
> @@ -32,26 +33,25 @@ import
> com.fasterxml.jackson.databind.annotation.JsonDeserialize;
>  import com.fasterxml.jackson.databind.annotation.JsonSerialize;
>  import com.fasterxml.jackson.databind.deser.std.StdDeserializer;
>  import com.fasterxml.jackson.databind.ser.std.StdSerializer;
> -import org.apache.drill.common.expression.types.DataType;
>
>  @JsonSerialize(using = Se.class)
>  @JsonDeserialize(using = De.class)
>  public class FieldReference extends SchemaPath {
> -  DataType overrideType;
> +  MajorType overrideType;
>
> -  public FieldReference(String value) {
> -    super(value);
> +  public FieldReference(String value, ExpressionPosition pos) {
> +    super(value, pos);
>    }
>
> -  public FieldReference(String value, DataType dataType) {
> -    super(value);
> +  public FieldReference(String value, ExpressionPosition pos, MajorType
> dataType) {
> +    super(value, pos);
>      this.overrideType = dataType;
>    }
>
>      @Override
> -    public DataType getDataType() {
> +    public MajorType getMajorType() {
>          if(overrideType == null) {
> -            return super.getDataType();
> +            return super.getMajorType();
>          } else {
>              return overrideType;
>          }
> @@ -66,7 +66,7 @@ public class FieldReference extends SchemaPath {
>      @Override
>      public FieldReference deserialize(JsonParser jp,
> DeserializationContext ctxt) throws IOException,
>          JsonProcessingException {
> -      return new FieldReference(this._parseString(jp, ctxt));
> +      return new FieldReference(this._parseString(jp, ctxt),
> ExpressionPosition.UNKNOWN);
>      }
>
>    }
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/FunctionBase.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/FunctionBase.java
> b/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/FunctionBase.java
> deleted file mode 100644
> index fd8ae18..0000000
> ---
> a/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/FunctionBase.java
> +++ /dev/null
> @@ -1,94 +0,0 @@
>
> -/*******************************************************************************
> - * Licensed to the Apache Software Foundation (ASF) under one
> - * or more contributor license agreements.  See the NOTICE file
> - * distributed with this work for additional information
> - * regarding copyright ownership.  The ASF licenses this file
> - * to you under the Apache License, Version 2.0 (the
> - * "License"); you may not use this file except in compliance
> - * with the License.  You may obtain a copy of the License at
> - *
> - * http://www.apache.org/licenses/LICENSE-2.0
> - *
> - * Unless required by applicable law or agreed to in writing, software
> - * distributed under the License is distributed on an "AS IS" BASIS,
> - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
> implied.
> - * See the License for the specific language governing permissions and
> - * limitations under the License.
> -
> ******************************************************************************/
> -package org.apache.drill.common.expression;
> -
> -import java.util.Iterator;
> -import java.util.List;
> -
> -import org.apache.drill.common.expression.types.DataType;
> -import org.apache.drill.common.expression.visitors.ExprVisitor;
> -
> -import com.fasterxml.jackson.annotation.JsonIgnore;
> -import com.google.common.collect.ImmutableList;
> -
> -
> -public abstract class FunctionBase extends LogicalExpressionBase
> implements Iterable<LogicalExpression>{
> -       public final ImmutableList<LogicalExpression> expressions;
> -
> -
> -  protected void funcToString(StringBuilder sb, String functionName) {
> -    sb.append(" ");
> -    sb.append(functionName);
> -    sb.append("( ");
> -    for(int i= 0; i < expressions.size(); i++){
> -      if(i != 0) sb.append(", ");
> -      expressions.get(i).addToString(sb);
> -    }
> -    sb.append(" ) ");
> -  }
> -
> -  protected void opToString(StringBuilder sb, String operator) {
> -    sb.append(" ( ");
> -    expressions.get(0).addToString(sb);
> -    for(int i =1; i < expressions.size(); i++){
> -      sb.append(operator);
> -      expressions.get(i).addToString(sb);
> -    }
> -    sb.append(" ) ");
> -  }
> -
> -       public FunctionBase(List<LogicalExpression> expressions){
> -               if( !(expressions instanceof ImmutableList)){
> -                       expressions = ImmutableList.copyOf(expressions);
> -               }
> -               this.expressions = (ImmutableList<LogicalExpression>)
> expressions;
> -       }
> -
> -//     public static DataType getJointType(String parentName,
> LogicalExpression expr1, LogicalExpression expr2) throws
> ExpressionValidationException{
> -//             DataType dt =
> DataType.getCombinedCast(expr1.getDataType(), expr2.getDataType());
> -//             if(dt == null) throw new ExpressionValidationException();
> -//
> -//             return dt;
> -//     }
> -//
> -//     public LogicalExpression wrapWithCastIfNecessary(DataType dt)
> throws ExpressionValidationException{
> -//             if(this.getDataType() != dt) return new Cast(this, dt);
> -//             return this;
> -//     }
> -
> -
> -
> -
> -       @Override
> -       public Iterator<LogicalExpression> iterator() {
> -               return expressions.iterator();
> -       }
> -
> -       @Override
> -       @JsonIgnore
> -       public DataType getDataType() {
> -               throw new UnsupportedOperationException();
> -       }
> -
> -
> -
> -
> -
> -
> -
> -}
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/FunctionCall.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/FunctionCall.java
> b/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/FunctionCall.java
> index ee76a5c..e13e87e 100644
> ---
> a/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/FunctionCall.java
> +++
> b/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/FunctionCall.java
> @@ -20,26 +20,34 @@ package org.apache.drill.common.expression;
>  import java.util.Iterator;
>  import java.util.List;
>
> -import org.apache.drill.common.expression.types.DataType;
>  import org.apache.drill.common.expression.visitors.ExprVisitor;
> +import org.apache.drill.common.types.TypeProtos.MajorType;
>
>  import com.google.common.collect.ImmutableList;
>
>  public class FunctionCall extends LogicalExpressionBase implements
> Iterable<LogicalExpression> {
>    private final FunctionDefinition func;
>    public final ImmutableList<LogicalExpression> args;
> +  private final ExpressionPosition pos;
>
> -  public FunctionCall(FunctionDefinition func, List<LogicalExpression>
> args) {
> +  public FunctionCall(FunctionDefinition func, List<LogicalExpression>
> args, ExpressionPosition pos) {
> +    super(pos);
>      this.func = func;
>      if (!(args instanceof ImmutableList)) {
>        args = ImmutableList.copyOf(args);
>      }
>      this.args = (ImmutableList<LogicalExpression>) args;
> +    this.pos = pos;
>    }
>
>    @Override
> -  public <T> T accept(ExprVisitor<T> visitor) {
> -    return visitor.visitFunctionCall(this);
> +  public ExpressionPosition getPosition() {
> +    return pos;
> +  }
> +
> +  @Override
> +  public <T, V, E extends Exception> T accept(ExprVisitor<T, V, E>
> visitor, V value) throws E{
> +    return visitor.visitFunctionCall(this, value);
>    }
>
>    @Override
> @@ -52,43 +60,16 @@ public class FunctionCall extends
> LogicalExpressionBase implements Iterable<Logi
>    }
>
>    @Override
> -  public DataType getDataType() {
> +  public MajorType getMajorType() {
>      return func.getDataType(this.args);
>    }
>
>    @Override
> -  public void addToString(StringBuilder sb) {
> -    if (func.isOperator()) {
> -      if (args.size() == 1) { // unary
> -        func.addRegisteredName(sb);
> -        sb.append("(");
> -        args.get(0).addToString(sb);
> -        sb.append(")");
> -      } else {
> -        for (int i = 0; i < args.size(); i++) {
> -          if (i != 0) {
> -            sb.append(" ");
> -            func.addRegisteredName(sb);
> -          }
> -          sb.append(" (");
> -          args.get(i).addToString(sb);
> -          sb.append(") ");
> -        }
> -      }
> -    } else { // normal function
> -
> -      func.addRegisteredName(sb);
> -      sb.append("(");
> -      for (int i = 0; i < args.size(); i++) {
> -        if (i != 0) sb.append(", ");
> -        args.get(i).addToString(sb);
> -      }
> -      sb.append(") ");
> -    }
> +  public String toString() {
> +    final int maxLen = 10;
> +    return "FunctionCall [func=" + func + ", args="
> +        + (args != null ? args.subList(0, Math.min(args.size(), maxLen))
> : null) + ", pos=" + pos + "]";
>    }
>
> -    @Override
> -    public void resolveAndValidate(String expr, ErrorCollector errors) {
> -        func.getArgumentValidator().validateArguments(expr, args, errors);
> -    }
> +
>  }
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/FunctionDefinition.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/FunctionDefinition.java
> b/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/FunctionDefinition.java
> index 9d21763..4cbab6a 100644
> ---
> a/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/FunctionDefinition.java
> +++
> b/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/FunctionDefinition.java
> @@ -21,7 +21,7 @@ import java.util.Arrays;
>  import java.util.List;
>
>  import org.apache.commons.lang3.ArrayUtils;
> -import org.apache.drill.common.expression.types.DataType;
> +import org.apache.drill.common.types.TypeProtos.MajorType;
>
>  public class FunctionDefinition {
>
> @@ -42,7 +42,7 @@ public class FunctionDefinition {
>      this.isOperator = isOperator;
>    }
>
> -  public DataType getDataType(List<LogicalExpression> args){
> +  public MajorType getDataType(List<LogicalExpression> args){
>      return outputType.getOutputType(args);
>    }
>
> @@ -82,8 +82,8 @@ public class FunctionDefinition {
>      return this.name;
>    }
>
> -  public FunctionCall newCall(List<LogicalExpression> args){
> -    return new FunctionCall(this, args);
> +  public FunctionCall newCall(List<LogicalExpression> args,
> ExpressionPosition pos){
> +    return new FunctionCall(this, args, pos);
>    }
>
>    public void addRegisteredName(StringBuilder sb){
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/FunctionRegistry.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/FunctionRegistry.java
> b/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/FunctionRegistry.java
> index c22e448..ed2c63e 100644
> ---
> a/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/FunctionRegistry.java
> +++
> b/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/FunctionRegistry.java
> @@ -57,17 +57,18 @@ public class FunctionRegistry {
>      }
>    }
>
> -  public LogicalExpression createExpression(String functionName,
> List<LogicalExpression> args){
> +
> +  public LogicalExpression createExpression(String functionName,
> ExpressionPosition ep, List<LogicalExpression> args){
>      FunctionDefinition d = funcMap.get(functionName);
>      if(d == null) throw new
> ExpressionParsingException(String.format("Unable to find function
> definition for function named '%s'", functionName));
> -    return d.newCall(args);
> +    return d.newCall(args, ep);
>    }
>
> -  public LogicalExpression createExpression(String unaryName,
> LogicalExpression... e){
> -    return funcMap.get(unaryName).newCall(Lists.newArrayList(e));
> +  public LogicalExpression createExpression(String unaryName,
> ExpressionPosition ep, LogicalExpression... e){
> +    return funcMap.get(unaryName).newCall(Lists.newArrayList(e), ep);
>    }
>
> -  public LogicalExpression createByOp(List<LogicalExpression> args,
> List<String> opTypes) {
> +  public LogicalExpression createByOp(List<LogicalExpression> args,
> ExpressionPosition ep, List<String> opTypes) {
>      // logger.debug("Generating new comparison expressions.");
>      if (args.size() == 1) {
>        return args.get(0);
> @@ -81,7 +82,7 @@ public class FunctionRegistry {
>        List<LogicalExpression> l2 = new ArrayList<LogicalExpression>();
>        l2.add(first);
>        l2.add(args.get(i + 1));
> -      first = createExpression(opTypes.get(i), args);
> +      first = createExpression(opTypes.get(i), ep, args);
>      }
>      return first;
>    }
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/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 a553f4c..6f2e4a8 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
> @@ -21,11 +21,12 @@ import java.util.ArrayList;
>  import java.util.List;
>
>  import org.apache.drill.common.expression.IfExpression.IfCondition;
> -import org.apache.drill.common.expression.types.DataType;
>  import org.apache.drill.common.expression.visitors.ExprVisitor;
> +import org.apache.drill.common.types.TypeProtos.MajorType;
>  import org.slf4j.Logger;
>  import org.slf4j.LoggerFactory;
>
> +import com.google.common.base.Preconditions;
>  import com.google.common.collect.ImmutableList;
>  import com.google.common.collect.UnmodifiableIterator;
>
> @@ -35,7 +36,8 @@ public class IfExpression extends LogicalExpressionBase
> implements Iterable<IfCo
>         public final ImmutableList<IfCondition> conditions;
>         public final LogicalExpression elseExpression;
>
> -       private IfExpression(List<IfCondition> conditions,
> LogicalExpression elseExpression){
> +       private IfExpression(ExpressionPosition pos, List<IfCondition>
> conditions, LogicalExpression elseExpression){
> +         super(pos);
>                 this.conditions = ImmutableList.copyOf(conditions);
>                 this.elseExpression = elseExpression;
>         }
> @@ -53,28 +55,32 @@ public class IfExpression extends
> LogicalExpressionBase implements Iterable<IfCo
>
>         }
>
> -
>         @Override
> -  public <T> T accept(ExprVisitor<T> visitor) {
> -    return visitor.visitIfExpression(this);
> +  public <T, V, E extends Exception> T accept(ExprVisitor<T, V, E>
> visitor, V value) throws E{
> +    return visitor.visitIfExpression(this, value);
>    }
>
> -
>    public static class Builder{
>                 List<IfCondition> conditions = new
> ArrayList<IfCondition>();
>                 private LogicalExpression elseExpression;
> +               private ExpressionPosition pos =
> ExpressionPosition.UNKNOWN;
> +
> +               public Builder setPosition(ExpressionPosition pos){
> +                 this.pos = pos;
> +                 return this;
> +               }
>
>                 public Builder addCondition(IfCondition condition){
>                         conditions.add(condition);
>              return this;
>                 }
>
> -        public Builder addConditions(Iterable<IfCondition> conditions) {
> -            for(IfCondition condition : conditions) {
> -                addCondition(condition);
> -            }
> -            return this;
> -        }
> +    public Builder addConditions(Iterable<IfCondition> conditions) {
> +      for (IfCondition condition : conditions) {
> +        addCondition(condition);
> +      }
> +      return this;
> +    }
>
>                 public Builder setElse(LogicalExpression elseExpression) {
>                         this.elseExpression = elseExpression;
> @@ -82,39 +88,20 @@ public class IfExpression extends
> LogicalExpressionBase implements Iterable<IfCo
>                 }
>
>                 public IfExpression build(){
> -                       return new IfExpression(conditions,
> elseExpression);
> +                 Preconditions.checkNotNull(pos);
> +                 Preconditions.checkNotNull(conditions);
> +                 Preconditions.checkNotNull(conditions);
> +                       return new IfExpression(pos, conditions,
> elseExpression);
>                 }
>
>         }
>
> -
> -    @Override
> -    public DataType getDataType() {
> -        return DataType.BOOLEAN;
> -    }
> -
> -    @Override
> -  public void addToString(StringBuilder sb) {
> -         sb.append(" ( ");
> -         for(int i =0; i < conditions.size(); i++){
> -           IfCondition c = conditions.get(i);
> -           if(i !=0) sb.append(" else ");
> -           sb.append("if (");
> -           c.condition.addToString(sb);
> -           sb.append(" ) then (");
> -           c.expression.addToString(sb);
> -           sb.append(" ) ");
> -         }
> -         sb.append(" end ");
> -         sb.append(" ) ");
> +  @Override
> +  public MajorType getMajorType() {
> +    return this.elseExpression.getMajorType();
>    }
>
> -    @Override
> -    public void resolveAndValidate(String expr, ErrorCollector errors) {
> -    }
> -
> -
> -    public static Builder newBuilder(){
> +  public static Builder newBuilder(){
>                 return new Builder();
>         }
>
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/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 3df33a0..165f409 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
> @@ -26,8 +26,8 @@ import org.apache.drill.common.config.DrillConfig;
>  import org.apache.drill.common.expression.parser.ExprLexer;
>  import org.apache.drill.common.expression.parser.ExprParser;
>  import org.apache.drill.common.expression.parser.ExprParser.parse_return;
> -import org.apache.drill.common.expression.types.DataType;
>  import org.apache.drill.common.expression.visitors.ExprVisitor;
> +import org.apache.drill.common.types.TypeProtos.MajorType;
>  import org.slf4j.Logger;
>  import org.slf4j.LoggerFactory;
>
> @@ -44,65 +44,65 @@ 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 {
> -    static final Logger logger =
> LoggerFactory.getLogger(LogicalExpression.class);
> +  static final Logger logger =
> LoggerFactory.getLogger(LogicalExpression.class);
>
> -    public abstract DataType getDataType();
> +  public abstract MajorType getMajorType();
>
> -    public void addToString(StringBuilder sb);
> +  public <T, V, E extends Exception> T accept(ExprVisitor<T, V, E>
> visitor, V value) throws E;
>
> -    public void resolveAndValidate(String expr, ErrorCollector errors);
> +  public ExpressionPosition getPosition();
>
> -    public <T> T accept(ExprVisitor<T> visitor);
> +  public static class De extends StdDeserializer<LogicalExpression> {
> +    DrillConfig config;
>
> -    public static class De extends StdDeserializer<LogicalExpression> {
> -        DrillConfig config;
> -        ErrorCollector errorCollector;
> -
> -        public De(DrillConfig config) {
> -            super(LogicalExpression.class);
> -            this.config = config;
> -            this.errorCollector = config.getErrorCollector();
> -        }
> -
> -        @Override
> -        public LogicalExpression deserialize(JsonParser jp,
> DeserializationContext ctxt) throws IOException,
> -                JsonProcessingException {
> -            String expr = jp.getText();
> -
> -            if (expr == null || expr.isEmpty())
> -                return null;
> -            try {
> -                // logger.debug("Parsing expression string '{}'", expr);
> -                ExprLexer lexer = new ExprLexer(new
> ANTLRStringStream(expr));
> -
> -                CommonTokenStream tokens = new CommonTokenStream(lexer);
> -                ExprParser parser = new ExprParser(tokens);
> -                parser.setRegistry(new FunctionRegistry(config));
> -                parse_return ret = parser.parse();
> -                // logger.debug("Found expression '{}'", ret.e);
> -                ret.e.resolveAndValidate(expr, errorCollector);
> -                return ret.e;
> -            } catch (RecognitionException e) {
> -                throw new RuntimeException(e);
> -            }
> -        }
> +    public De(DrillConfig config) {
> +      super(LogicalExpression.class);
> +      this.config = config;
> +    }
>
> +    @Override
> +    public LogicalExpression deserialize(JsonParser jp,
> DeserializationContext ctxt) throws IOException,
> +        JsonProcessingException {
> +      String expr = jp.getText();
> +
> +      if (expr == null || expr.isEmpty())
> +        return null;
> +      try {
> +        // logger.debug("Parsing expression string '{}'", expr);
> +        ExprLexer lexer = new ExprLexer(new ANTLRStringStream(expr));
> +        CommonTokenStream tokens = new CommonTokenStream(lexer);
> +        ExprParser parser = new ExprParser(tokens);
> +
> +        //TODO: move functionregistry and error collector to injectables.
> +        //ctxt.findInjectableValue(valueId, forProperty, beanInstance)
> +
> +        parser.setRegistry(new FunctionRegistry(config));
> +        parse_return ret = parser.parse();
> +
> +        // ret.e.resolveAndValidate(expr, errorCollector);
> +        return ret.e;
> +      } catch (RecognitionException e) {
> +        throw new RuntimeException(e);
> +      }
>      }
>
> -    public static class Se extends StdSerializer<LogicalExpression> {
> +  }
>
> -        protected Se() {
> -            super(LogicalExpression.class);
> -        }
> +  public static class Se extends StdSerializer<LogicalExpression> {
>
> -        @Override
> -        public void serialize(LogicalExpression value, JsonGenerator
> jgen, SerializerProvider provider) throws IOException,
> -                JsonGenerationException {
> -            StringBuilder sb = new StringBuilder();
> -            value.addToString(sb);
> -            jgen.writeString(sb.toString());
> -        }
> +    protected Se() {
> +      super(LogicalExpression.class);
> +    }
>
> +    @Override
> +    public void serialize(LogicalExpression value, JsonGenerator jgen,
> SerializerProvider provider) throws IOException,
> +        JsonGenerationException {
> +      StringBuilder sb = new StringBuilder();
> +      ExpressionStringBuilder esb = new ExpressionStringBuilder();
> +      value.accept(esb, sb);
> +      jgen.writeString(sb.toString());
>      }
>
> +  }
> +
>  }
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/LogicalExpressionBase.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/LogicalExpressionBase.java
> b/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/LogicalExpressionBase.java
> index e973df7..aa8885f 100644
> ---
> a/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/LogicalExpressionBase.java
> +++
> b/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/LogicalExpressionBase.java
> @@ -6,9 +6,9 @@
>   * to you under the Apache License, Version 2.0 (the
>   * "License"); you may not use this file except in compliance
>   * with the License.  You may obtain a copy of the License at
> - *
> + *
>   * http://www.apache.org/licenses/LICENSE-2.0
> - *
> + *
>   * Unless required by applicable law or agreed to in writing, software
>   * distributed under the License is distributed on an "AS IS" BASIS,
>   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
> implied.
> @@ -17,40 +17,44 @@
>
> ******************************************************************************/
>  package org.apache.drill.common.expression;
>
> -import org.apache.drill.common.expression.types.DataType;
> -import org.apache.drill.common.expression.visitors.ExprVisitor;
> +import org.apache.drill.common.types.TypeProtos.MajorType;
>
>  import com.fasterxml.jackson.annotation.JsonProperty;
>  import com.fasterxml.jackson.annotation.JsonPropertyOrder;
>
>
> -@JsonPropertyOrder({"type"})
> -public abstract class LogicalExpressionBase implements LogicalExpression {
> +@JsonPropertyOrder({ "type" })
> +public abstract class LogicalExpressionBase implements LogicalExpression{
>
> +  private final ExpressionPosition pos;
>
> -//     public static DataType getJointType(String parentName,
> LogicalExpression expr1, LogicalExpression expr2) throws
> ExpressionValidationException{
> -//             DataType dt =
> DataType.getCombinedCast(expr1.getDataType(), expr2.getDataType());
> -//             if(dt == null) throw new ExpressionValidationException();
> -//
> -//             return dt;
> -//     }
> +       protected LogicalExpressionBase(ExpressionPosition pos) {
> +    super();
> +    this.pos = pos;
> +  }
>
> +  @Override
> +  public ExpressionPosition getPosition() {
> +    return pos;
> +  }
>
> -    protected void i(StringBuilder sb, int indent) {
> -        for (int i = 0; i < indent; i++) {
> -            sb.append("  ");
> -        }
> -    }
> +  protected void i(StringBuilder sb, int indent){
> +               for(int i = 0; i < indent; i++){
> +                       sb.append("  ");
> +               }
> +       }
> +
> +       @Override
> +       public MajorType getMajorType() {
> +               throw new UnsupportedOperationException(String.format("The
> type of %s doesn't currently support LogicalExpression.getDataType().",
> this.getClass().getCanonicalName()));
> +       }
>
> -//     @Override
> -//     public <T> T accept(ExprVisitor<T> visitor) {
> -//             return visitor.visit(this);
> -//     }
> -
> -    @JsonProperty("type")
> -    public String getDescription() {
> -        return this.getClass().getSimpleName();
> -    }
> +  @JsonProperty("type")
> +       public String getDescription(){
> +               return this.getClass().getSimpleName();
> +       }
> +
>
> +
>
>  }
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/OutputTypeDeterminer.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/OutputTypeDeterminer.java
> b/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/OutputTypeDeterminer.java
> index 67402e2..0bec184 100644
> ---
> a/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/OutputTypeDeterminer.java
> +++
> b/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/OutputTypeDeterminer.java
> @@ -19,37 +19,51 @@ package org.apache.drill.common.expression;
>
>  import java.util.List;
>
> -import org.apache.drill.common.expression.types.DataType;
> +import org.apache.drill.common.types.TypeProtos.DataMode;
> +import org.apache.drill.common.types.TypeProtos.MajorType;
> +import org.apache.drill.common.types.TypeProtos.MinorType;
>
>  public interface OutputTypeDeterminer {
>
> -  public static OutputTypeDeterminer FIXED_BOOLEAN = new
> FixedType(DataType.BOOLEAN);
> +  public static OutputTypeDeterminer FIXED_BOOLEAN = new
> FixedType(MajorType.newBuilder().setMinorType(MinorType.BOOLEAN).setMode(DataMode.REQUIRED).build());
>
> -  public DataType getOutputType(List<LogicalExpression> expressions);
> +  public MajorType getOutputType(List<LogicalExpression> expressions);
>
>
>    public static class FixedType implements OutputTypeDeterminer{
> -    public DataType outputType;
> +    public MajorType outputType;
>
>
> -    public FixedType(DataType outputType) {
> +    public FixedType(MajorType outputType) {
>        super();
>        this.outputType = outputType;
>      }
>
>
>      @Override
> -    public DataType getOutputType(List<LogicalExpression> expressions) {
> -      return null;
> +    public MajorType getOutputType(List<LogicalExpression> expressions) {
> +      return outputType;
>      }
>
>    }
>
>    public static class SameAsFirstInput implements OutputTypeDeterminer{
> -
>      @Override
> -    public DataType getOutputType(List<LogicalExpression> expressions) {
> -      return expressions.get(0).getDataType();
> +    public MajorType getOutputType(List<LogicalExpression> expressions) {
> +      return expressions.get(0).getMajorType();
> +    }
> +  }
> +
> +  public static class SameAsAnySoft implements OutputTypeDeterminer{
> +    @Override
> +    public MajorType getOutputType(List<LogicalExpression> expressions) {
> +      for(LogicalExpression e : expressions){
> +        if(e.getMajorType().getMode() == DataMode.OPTIONAL){
> +          return e.getMajorType();
> +        }
> +      }
> +      return expressions.get(0).getMajorType();
>      }
>    }
> +
>  }
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/PathSegment.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/PathSegment.java
> b/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/PathSegment.java
> index c06426e..554c0ca 100644
> ---
> a/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/PathSegment.java
> +++
> b/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/PathSegment.java
> @@ -98,6 +98,32 @@ public abstract class PathSegment{
>        return "NameSegment [path=" + path + ", getCollisionBehavior()=" +
> getCollisionBehavior() + ", getChild()="
>            + getChild() + "]";
>      }
> +
> +    @Override
> +    public int hashCode() {
> +      final int prime = 31;
> +      int result = 1;
> +      result = prime * result + ((path == null) ? 0 : path.hashCode());
> +      return result;
> +    }
> +
> +    @Override
> +    public boolean equals(Object obj) {
> +      if (this == obj)
> +        return true;
> +      if (obj == null)
> +        return false;
> +      if (getClass() != obj.getClass())
> +        return false;
> +      NameSegment other = (NameSegment) obj;
> +      if (path == null) {
> +        if (other.path != null)
> +          return false;
> +      } else if (!path.equals(other.path))
> +        return false;
> +      return true;
> +    }
> +
>
>
>    }
>
>

[44/53] [abbrv] git commit: Types transition

Posted by ja...@apache.org.
Types transition


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

Branch: refs/heads/master
Commit: ce0da88d2df38d079a11d2b269fad9e964e1c14a
Parents: e68bba2
Author: Jacques Nadeau <ja...@apache.org>
Authored: Mon Jun 24 08:06:43 2013 -0700
Committer: Jacques Nadeau <ja...@apache.org>
Committed: Fri Jul 19 14:53:11 2013 -0700

----------------------------------------------------------------------
 sandbox/prototype/common/pom.xml                | 216 ++---
 .../drill/common/expression/parser/ExprParser.g |  64 +-
 .../org/apache/drill/common/expression/Arg.java |  44 +-
 .../common/expression/ArgumentValidator.java    |   2 +-
 .../common/expression/ArgumentValidators.java   |  62 +-
 .../expression/BasicArgumentValidator.java      |   8 +-
 .../drill/common/expression/ErrorCollector.java |  18 +-
 .../common/expression/ErrorCollectorImpl.java   |  40 +-
 .../common/expression/ExpressionPosition.java   |  27 +
 .../expression/ExpressionStringBuilder.java     | 102 +++
 .../common/expression/ExpressionValidator.java  |  86 ++
 .../drill/common/expression/FieldReference.java |  18 +-
 .../drill/common/expression/FunctionBase.java   |  94 --
 .../drill/common/expression/FunctionCall.java   |  55 +-
 .../common/expression/FunctionDefinition.java   |   8 +-
 .../common/expression/FunctionRegistry.java     |  13 +-
 .../drill/common/expression/IfExpression.java   |  65 +-
 .../common/expression/LogicalExpression.java    |  98 +-
 .../expression/LogicalExpressionBase.java       |  56 +-
 .../common/expression/OutputTypeDeterminer.java |  34 +-
 .../drill/common/expression/PathSegment.java    |  26 +
 .../drill/common/expression/SchemaPath.java     | 176 ++--
 .../common/expression/ValueExpressions.java     | 320 ++++---
 .../common/expression/fn/BooleanFunctions.java  |   6 +-
 .../common/expression/fn/MathFunctions.java     |  10 +-
 .../common/expression/fn/StringFunctions.java   |   7 +-
 .../common/expression/fn/UnaryFunctions.java    |   6 +-
 .../drill/common/expression/types/AtomType.java |  67 ++
 .../drill/common/expression/types/DataType.java |   2 +-
 .../visitors/AbstractExprVisitor.java           |  56 ++
 .../expression/visitors/AggregateChecker.java   |  25 +-
 .../expression/visitors/ConstantChecker.java    |  21 +-
 .../common/expression/visitors/ExprVisitor.java |  18 +-
 .../expression/visitors/SimpleExprVisitor.java  |  56 ++
 .../org/apache/drill/common/types/Types.java    | 134 +++
 .../common/src/main/protobuf/Types.proto        |  64 ++
 sandbox/prototype/exec/java-exec/pom.xml        | 360 ++++----
 .../org/apache/drill/exec/ExecConstants.java    |   1 +
 .../drill/exec/compile/ClassTransformer.java    |   1 +
 .../drill/exec/compile/CodeModelTools.java      |  31 +
 .../apache/drill/exec/expr/CodeGenerator.java   | 201 +++++
 .../apache/drill/exec/expr/DrillAggrFunc.java   |   9 +
 .../org/apache/drill/exec/expr/DrillFunc.java   |   9 +
 .../drill/exec/expr/EvaluationVisitor.java      | 201 +++++
 .../exec/expr/ExpressionTreeMaterializer.java   | 128 +++
 .../exec/expr/SingleClassStringWriter.java      |  40 +
 .../exec/expr/ValueVectorReadExpression.java    |  41 +
 .../exec/expr/ValueVectorWriteExpression.java   |  45 +
 .../exec/expr/annotations/FunctionTemplate.java |  25 +
 .../drill/exec/expr/annotations/Output.java     |  18 +
 .../drill/exec/expr/annotations/Param.java      |  18 +
 .../drill/exec/expr/annotations/WorkSpace.java  |  14 +
 .../drill/exec/expr/fn/FunctionConverter.java   | 186 ++++
 .../drill/exec/expr/fn/FunctionHolder.java      | 158 ++++
 .../expr/fn/FunctionImplementationRegistry.java |  43 +
 .../exec/expr/fn/MethodGrabbingVisitor.java     |  52 ++
 .../exec/expr/fn/ModifiedUnparseVisitor.java    | 891 +++++++++++++++++++
 .../drill/exec/expr/fn/impl/MathFunctions.java  |  48 +
 .../drill/exec/expr/holders/ValueHolder.java    |   4 +
 .../holders/ValueHolderImplmenetations.java     |  44 +
 .../exec/expr/holders/VarBinaryHolder.java      |  12 +
 .../apache/drill/exec/ops/FragmentContext.java  |  26 +-
 .../apache/drill/exec/opt/BasicOptimizer.java   |  28 +-
 .../apache/drill/exec/physical/RecordField.java |   8 +-
 .../exec/physical/config/MockRecordReader.java  |  19 +-
 .../drill/exec/physical/config/MockScanPOP.java |   8 +-
 .../exec/physical/impl/FilterRecordBatch.java   |  34 +
 .../drill/exec/physical/impl/OutputMutator.java |   9 +
 .../drill/exec/physical/impl/ScanBatch.java     |  72 +-
 .../drill/exec/physical/impl/VectorHolder.java  |  39 +
 .../exec/physical/impl/WireRecordBatch.java     |  32 +-
 .../impl/filter/EvalSetupException.java         |   5 +
 .../impl/filter/EvaluationPredicate.java        |  13 +
 .../physical/impl/filter/ExampleFilter.java     | 113 +++
 .../physical/impl/project/ProjectEvaluator.java |  12 +
 .../impl/project/ProjectRecordBatch.java        | 218 +++++
 .../exec/physical/impl/project/Projector.java   |  20 +
 .../impl/project/ProjectorTemplate.java         | 101 +++
 .../physical/impl/project/TransferPairing.java  |  35 +
 .../drill/exec/planner/PhysicalPlanReader.java  |   2 +-
 .../apache/drill/exec/record/BatchSchema.java   |  36 +-
 .../exec/record/ExpressionTreeMaterializer.java | 138 ---
 .../drill/exec/record/MajorTypeSerDe.java       |   6 +-
 .../drill/exec/record/MaterializedField.java    |  58 +-
 .../drill/exec/record/NullExpression.java       |  32 +
 .../apache/drill/exec/record/RecordBatch.java   |  67 +-
 .../drill/exec/record/RecordBatchLoader.java    |  84 +-
 .../apache/drill/exec/record/SchemaBuilder.java |  75 +-
 .../apache/drill/exec/record/WritableBatch.java |  43 +-
 .../exec/record/vector/NullableFixed8.java      |  36 +
 .../exec/record/vector/SelectionVector.java     |   8 +
 .../exec/record/vector/SelectionVector2.java    |  46 +
 .../exec/record/vector/SelectionVector4.java    |  41 +
 .../org/apache/drill/exec/schema/Field.java     |  31 +-
 .../apache/drill/exec/schema/ListSchema.java    |  17 +-
 .../apache/drill/exec/schema/NamedField.java    |  19 +-
 .../apache/drill/exec/schema/OrderedField.java  |   9 +-
 .../exec/schema/json/jackson/JacksonHelper.java |  26 +-
 .../drill/exec/store/JSONRecordReader.java      |  84 +-
 .../exec/work/batch/BitComHandlerImpl.java      |   3 +-
 .../work/foreman/RunningFragmentManager.java    |   3 +-
 .../work/fragment/RemoteFragmentHandler.java    |   3 +-
 .../java-exec/src/main/protobuf/SchemaDef.proto |  63 +-
 .../src/main/resources/drill-module.conf        |  13 +-
 .../java/org/apache/drill/exec/SortTest.java    |  61 ++
 .../apache/drill/exec/expr/ExpressionTest.java  | 108 +++
 .../exec/physical/impl/PerformanceTests.java    |  32 +
 .../impl/TestExecutionAbstractions.java         | 226 +++++
 .../physical/impl/TestSimpleFragmentRun.java    |  51 +-
 .../record/ExpressionTreeMaterializerTest.java  | 297 ++++---
 .../drill/exec/store/JSONRecordReaderTest.java  | 469 +++++-----
 .../org/apache/drill/exec/ref/RunOutcome.java   |   1 +
 .../exec/ref/eval/BasicEvaluatorFactory.java    |  11 +-
 .../apache/drill/exec/ref/eval/IfEvaluator.java |  10 +-
 .../exec/ref/eval/SimpleEvaluationVisitor.java  |  24 +-
 .../drill/exec/ref/eval/fn/MathEvaluators.java  |   5 +-
 .../exec/ref/rops/CollapsingAggregateROP.java   |   4 +-
 .../apache/drill/exec/ref/rops/ConstantROP.java |   3 -
 .../apache/drill/exec/ref/rops/FlattenROP.java  |   7 +-
 .../drill/exec/ref/rops/WindowFrameROP.java     |   5 +-
 .../drill/exec/ref/values/BaseArrayValue.java   |  21 +-
 .../drill/exec/ref/values/BaseMapValue.java     |   9 +-
 .../apache/drill/exec/ref/values/DataValue.java |   4 +-
 .../drill/exec/ref/values/NumericValue.java     |  17 +-
 .../drill/exec/ref/values/ScalarValues.java     |  66 +-
 .../drill/exec/ref/values/SimpleArrayValue.java |   4 +-
 .../drill/exec/ref/values/SimpleMapValue.java   |   5 +-
 .../drill/exec/ref/values/ValueReader.java      |  14 +-
 .../drill/exec/ref/values/ValueUtils.java       |  17 +-
 .../apache/drill/exec/ref/RunSimplePlan.java    |  14 +-
 .../org/apache/drill/exec/ref/TestUtils.java    |   3 +-
 .../exec/ref/rops/CollapsingAggregateTest.java  |   5 +-
 .../drill/exec/ref/rops/ConstantROPTest.java    |   5 +-
 .../drill/exec/ref/rops/OrderROPTest.java       |   5 +-
 .../drill/exec/ref/rops/WindowFrameROPTest.java |  12 +-
 sandbox/prototype/pom.xml                       | 381 ++++----
 136 files changed, 6212 insertions(+), 2103 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/sandbox/prototype/common/pom.xml
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/pom.xml b/sandbox/prototype/common/pom.xml
index 4a8c5d8..4891191 100644
--- a/sandbox/prototype/common/pom.xml
+++ b/sandbox/prototype/common/pom.xml
@@ -1,103 +1,119 @@
 <?xml version="1.0"?>
 <project
-        xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"
-        xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance">
-    <modelVersion>4.0.0</modelVersion>
-    <parent>
-        <artifactId>prototype-parent</artifactId>
-        <groupId>org.apache.drill</groupId>
-        <version>1.0-SNAPSHOT</version>
-    </parent>
-
-    <artifactId>common</artifactId>
-    <packaging>jar</packaging>
-    <name>common</name>
-
-    <dependencies>
-
-        <dependency>
-            <groupId>com.google.protobuf</groupId>
-            <artifactId>protobuf-java</artifactId>
-            <version>2.5.0</version>
-        </dependency>
-
-        <dependency>
-            <groupId>com.typesafe</groupId>
-            <artifactId>config</artifactId>
-            <version>1.0.0</version>
-        </dependency>
-
-        <dependency>
-            <groupId>org.apache.commons</groupId>
-            <artifactId>commons-lang3</artifactId>
-            <version>3.1</version>
-        </dependency>
-
-        <dependency>
-            <groupId>org.msgpack</groupId>
-            <artifactId>msgpack</artifactId>
-            <version>0.6.6</version>
-        </dependency>
-
-        <dependency>
-            <groupId>org.reflections</groupId>
-            <artifactId>reflections</artifactId>
-            <version>0.9.8</version>
-        </dependency>
-
-        <dependency>
-            <groupId>com.fasterxml.jackson.core</groupId>
-            <artifactId>jackson-annotations</artifactId>
-            <version>2.1.1</version>
-        </dependency>
-        <dependency>
-            <groupId>org.hibernate</groupId>
-            <artifactId>hibernate-validator</artifactId>
-            <version>4.3.1.Final</version>
-        </dependency>
-        <dependency>
-            <groupId>com.fasterxml.jackson.core</groupId>
-            <artifactId>jackson-databind</artifactId>
-            <version>2.1.1</version>
-        </dependency>
-        <dependency>
-            <groupId>org.antlr</groupId>
-            <artifactId>antlr-runtime</artifactId>
-            <version>3.4</version>
-        </dependency>
-
-
-    </dependencies>
-
-
-    <build>
-        <plugins>
-            <plugin>
-                <artifactId>maven-jar-plugin</artifactId>
-                <executions>
-                    <execution>
-                        <id>test-jar</id>
-                        <goals>
-                            <goal>test-jar</goal>
-                        </goals>
-                    </execution>
-                </executions>
-            </plugin>
-            <plugin>
-                <groupId>org.antlr</groupId>
-                <artifactId>antlr3-maven-plugin</artifactId>
-                <version>3.4</version>
-                <configuration>
-
-                </configuration>
-                <executions>
-                    <execution>
-                        <goals>
-                            <goal>antlr</goal>
-                        </goals>
-                    </execution>
-                </executions>
-            </plugin>
-        </plugins>
-    </build>
+  xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"
+  xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance">
+  <modelVersion>4.0.0</modelVersion>
+  
+  <parent>
+    <artifactId>prototype-parent</artifactId>
+    <groupId>org.apache.drill</groupId>
+    <version>1.0-SNAPSHOT</version>
+  </parent>
+
+  <artifactId>common</artifactId>
+  <packaging>jar</packaging>
+  <name>common</name>
+
+  <dependencies>
+
+    <dependency>
+      <groupId>com.google.protobuf</groupId>
+      <artifactId>protobuf-java</artifactId>
+      <version>2.5.0</version>
+    </dependency>
+
+    <dependency>
+      <groupId>com.typesafe</groupId>
+      <artifactId>config</artifactId>
+      <version>1.0.0</version>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.commons</groupId>
+      <artifactId>commons-lang3</artifactId>
+      <version>3.1</version>
+    </dependency>
+
+    <dependency>
+      <groupId>org.msgpack</groupId>
+      <artifactId>msgpack</artifactId>
+      <version>0.6.6</version>
+    </dependency>
+
+    <dependency>
+      <groupId>org.reflections</groupId>
+      <artifactId>reflections</artifactId>
+      <version>0.9.8</version>
+    </dependency>
+
+    <dependency>
+      <groupId>com.fasterxml.jackson.core</groupId>
+      <artifactId>jackson-annotations</artifactId>
+      <version>2.1.1</version>
+    </dependency>
+    <dependency>
+      <groupId>org.hibernate</groupId>
+      <artifactId>hibernate-validator</artifactId>
+      <version>4.3.1.Final</version>
+    </dependency>
+    <dependency>
+      <groupId>com.fasterxml.jackson.core</groupId>
+      <artifactId>jackson-databind</artifactId>
+      <version>2.1.1</version>
+    </dependency>
+    <dependency>
+      <groupId>org.antlr</groupId>
+      <artifactId>antlr-runtime</artifactId>
+      <version>3.4</version>
+    </dependency>
+
+
+  </dependencies>
+
+
+  <build>
+    <plugins>
+      <plugin>
+        <artifactId>maven-jar-plugin</artifactId>
+        <executions>
+          <execution>
+            <id>test-jar</id>
+            <goals>
+              <goal>test-jar</goal>
+            </goals>
+          </execution>
+        </executions>
+      </plugin>
+      <plugin>
+        <groupId>org.antlr</groupId>
+        <artifactId>antlr3-maven-plugin</artifactId>
+        <version>3.4</version>
+        <configuration>
+
+        </configuration>
+        <executions>
+          <execution>
+            <goals>
+              <goal>antlr</goal>
+            </goals>
+          </execution>
+        </executions>
+      </plugin>
+
+
+      <plugin>
+        <groupId>com.github.igor-petruk.protobuf</groupId>
+        <artifactId>protobuf-maven-plugin</artifactId>
+        <version>0.6.3</version>
+        <executions>
+          <execution>
+            <goals>
+              <goal>run</goal>
+            </goals>
+          </execution>
+        </executions>
+      </plugin>
+    </plugins>
+
+  </build>
 </project>

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/sandbox/prototype/common/src/main/antlr3/org/apache/drill/common/expression/parser/ExprParser.g
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/antlr3/org/apache/drill/common/expression/parser/ExprParser.g b/sandbox/prototype/common/src/main/antlr3/org/apache/drill/common/expression/parser/ExprParser.g
index f8eb725..6cf041c 100644
--- a/sandbox/prototype/common/src/main/antlr3/org/apache/drill/common/expression/parser/ExprParser.g
+++ b/sandbox/prototype/common/src/main/antlr3/org/apache/drill/common/expression/parser/ExprParser.g
@@ -40,7 +40,8 @@ import org.apache.drill.common.expression.*;
 
 @members{
   private FunctionRegistry registry;
-  
+  private String fullExpression;
+  private int tokenPos;
   public void setRegistry(FunctionRegistry registry){
     this.registry = registry;
   }
@@ -48,14 +49,22 @@ import org.apache.drill.common.expression.*;
   public static void p(String s){
     System.out.println(s);
   }
+  
+  public ExpressionPosition pos(Token token){
+    return new ExpressionPosition(fullExpression, token.getTokenIndex());
+  }
 }
 
 parse returns [LogicalExpression e]
-  :  expression EOF {$e = $expression.e; }
+  :  expression EOF {
+    $e = $expression.e; 
+    if(fullExpression == null) fullExpression = $expression.text;
+    tokenPos = $expression.start.getTokenIndex();
+  }
   ;
  
 functionCall returns [LogicalExpression e]
-  :  Identifier OParen exprList? CParen {$e = registry.createExpression($Identifier.text, $exprList.listE);  }
+  :  Identifier OParen exprList? CParen {$e = registry.createExpression($Identifier.text, pos($Identifier), $exprList.listE);  }
   ;
 
 ifStatement returns [LogicalExpression e]
@@ -65,7 +74,7 @@ ifStatement returns [LogicalExpression e]
 	@after {
 	  $e = s.build();
 	}  
-  :  i1=ifStat {s.addCondition($i1.i); } (elseIfStat { s.addCondition($elseIfStat.i); } )* Else expression { s.setElse($expression.e); }End 
+  :  i1=ifStat {s.addCondition($i1.i); s.setPosition(pos($i1.start)); } (elseIfStat { s.addCondition($elseIfStat.i); } )* Else expression { s.setElse($expression.e); }End 
   ;
 
 ifStat returns [IfExpression.IfCondition i]
@@ -113,96 +122,103 @@ condExpr returns [LogicalExpression e]
 orExpr returns [LogicalExpression e]
 	@init{
 	  List<LogicalExpression> exprs = new ArrayList<LogicalExpression>();
+	  ExpressionPosition p = null;
 	}
 	@after{
 	  if(exprs.size() == 1){
 	    $e = exprs.get(0);
 	  }else{
-	    $e = registry.createExpression("||", exprs);
+	    $e = registry.createExpression("||", p, exprs);
 	  }
 	}
-  :  a1=andExpr { exprs.add($a1.e); } (Or^ a2=andExpr { exprs.add($a2.e); })*
+  :  a1=andExpr { exprs.add($a1.e); p = pos( $a1.start );} (Or^ a2=andExpr { exprs.add($a2.e); })*
   ;
 
 andExpr returns [LogicalExpression e]
 	@init{
 	  List<LogicalExpression> exprs = new ArrayList<LogicalExpression>();
+	  ExpressionPosition p = null;
 	}
 	@after{
 	  if(exprs.size() == 1){
 	    $e = exprs.get(0);
 	  }else{
-	    $e = registry.createExpression("&&", exprs);
+	    $e = registry.createExpression("&&", p, exprs);
 	  }
 	}
-  :  e1=equExpr { exprs.add($e1.e);  } (And^ e2=equExpr { exprs.add($e2.e);  })*
+  :  e1=equExpr { exprs.add($e1.e); p = pos( $e1.start );  } (And^ e2=equExpr { exprs.add($e2.e);  })*
   ;
 
 equExpr returns [LogicalExpression e]
 	@init{
 	  List<LogicalExpression> exprs = new ArrayList<LogicalExpression>();
 	  List<String> cmps = new ArrayList();
+	  ExpressionPosition p = null;
 	}
 	@after{
-	  $e = registry.createByOp(exprs, cmps);
+	  $e = registry.createByOp(exprs, p, cmps);
 	}
-  :  r1=relExpr {exprs.add($r1.e);} ( cmpr= (Equals | NEquals ) r2=relExpr {exprs.add($r2.e); cmps.add($cmpr.text); })*
+  :  r1=relExpr { exprs.add($r1.e); p = pos( $r1.start );
+    } ( cmpr= (Equals | NEquals ) r2=relExpr {exprs.add($r2.e); cmps.add($cmpr.text); })*
   ;
 
 relExpr returns [LogicalExpression e]
-  :  left=addExpr {$e = $left.e; } (cmpr = (GTEquals | LTEquals | GT | LT) right=addExpr {$e = registry.createExpression($cmpr.text, $left.e, $right.e); } )? 
+  :  left=addExpr {$e = $left.e; } (cmpr = (GTEquals | LTEquals | GT | LT) right=addExpr {$e = registry.createExpression($cmpr.text, pos($left.start), $left.e, $right.e); } )? 
   ;
 
 addExpr returns [LogicalExpression e]
 	@init{
 	  List<LogicalExpression> exprs = new ArrayList<LogicalExpression>();
 	  List<String> ops = new ArrayList();
+	  ExpressionPosition p = null;
 	}
 	@after{
-	  $e = registry.createByOp(exprs, ops);
+	  $e = registry.createByOp(exprs, p, ops);
 	}
-  :  m1=mulExpr  {exprs.add($m1.e);} ( op=(Plus|Minus) m2=mulExpr {exprs.add($m2.e); ops.add($op.text); })* 
+  :  m1=mulExpr  {exprs.add($m1.e); p = pos($m1.start); } ( op=(Plus|Minus) m2=mulExpr {exprs.add($m2.e); ops.add($op.text); })* 
   ;
 
 mulExpr returns [LogicalExpression e]
 	@init{
 	  List<LogicalExpression> exprs = new ArrayList<LogicalExpression>();
 	  List<String> ops = new ArrayList();
+	  ExpressionPosition p = null;
 	}
 	@after{
-	  $e = registry.createByOp(exprs, ops);
+	  $e = registry.createByOp(exprs, p, ops);
 	}
-  :  p1=powExpr  {exprs.add($p1.e);} (op=(Asterisk|ForwardSlash|Percent) p2=powExpr {exprs.add($p2.e); ops.add($op.text); } )*
+  :  p1=powExpr  {exprs.add($p1.e); p = pos($p1.start);} (op=(Asterisk|ForwardSlash|Percent) p2=powExpr {exprs.add($p2.e); ops.add($op.text); } )*
   ;
 
 powExpr returns [LogicalExpression e]
 	@init{
 	  List<LogicalExpression> exprs = new ArrayList<LogicalExpression>();
 	  List<String> ops = new ArrayList();
+	  ExpressionPosition p = null;
 	}
 	@after{
-	  $e = registry.createByOp(exprs, ops);
+	  $e = registry.createByOp(exprs, p, ops);
 	}
-  :  u1=unaryExpr {exprs.add($u1.e);} (Caret u2=unaryExpr {exprs.add($u2.e); ops.add($Caret.text);} )*
+  :  u1=unaryExpr {exprs.add($u1.e); p = pos($u1.start);} (Caret u2=unaryExpr {exprs.add($u2.e); ops.add($Caret.text);} )*
   ;
   
 unaryExpr returns [LogicalExpression e]
-  :  Minus atom {$e = registry.createExpression("u-", $atom.e); }
-  |  Excl atom {$e= registry.createExpression("!", $atom.e); }
+  :  Minus atom {$e = registry.createExpression("u-", pos($atom.start), $atom.e); }
+  |  Excl atom {$e= registry.createExpression("!", pos($atom.start), $atom.e); }
   |  atom {$e = $atom.e; }
   ;
 
 atom returns [LogicalExpression e]
-  :  Number {$e = ValueExpressions.getNumericExpression($Number.text); }
-  |  Bool {$e = new ValueExpressions.BooleanExpression( $Bool.text ); }
+  :  Number {$e = ValueExpressions.getNumericExpression($Number.text, pos($atom.start)); }
+  |  Bool {$e = new ValueExpressions.BooleanExpression( $Bool.text, pos($atom.start)); }
   |  lookup {$e = $lookup.e; }
   ;
 
 
 lookup returns [LogicalExpression e]
   :  functionCall {$e = $functionCall.e ;}
-  | Identifier {$e = new SchemaPath($Identifier.text); }
-  | String {$e = new ValueExpressions.QuotedString($String.text); }
+  | Identifier {$e = new SchemaPath($Identifier.text, pos($Identifier) ); }
+  | String {$e = new ValueExpressions.QuotedString($String.text, pos($String) ); }
   | OParen expression CParen  {$e = $expression.e; }
-  | SingleQuote Identifier SingleQuote {$e = new SchemaPath($Identifier.text); }
+  | SingleQuote Identifier SingleQuote {$e = new SchemaPath($Identifier.text, pos($Identifier) ); }
   ;

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/Arg.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/Arg.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/Arg.java
index de9057b..64ec351 100644
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/Arg.java
+++ b/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/Arg.java
@@ -17,24 +17,41 @@
  ******************************************************************************/
 package org.apache.drill.common.expression;
 
-import org.apache.drill.common.expression.types.DataType;
+import java.util.List;
+
 import org.apache.drill.common.expression.visitors.ConstantChecker;
+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 com.google.common.collect.Lists;
 
 
 public class Arg {
   private final String name;
-  private final DataType[] allowedTypes;
+  private final MajorType[] allowedTypes;
   private final boolean constantsOnly;
   
-  public Arg(DataType... allowedTypes){
+  
+  /**
+   * Create an arg that supports any of the listed minor types using opt or req.  Does not allow repeated types.
+   * @param name
+   * @param constantsOnly
+   * @param types
+   */
+  public Arg(boolean constantsOnly, boolean allowNulls, String name, MinorType... types){
+    this(constantsOnly, name, getMajorTypes(allowNulls, types));
+  }
+  
+  public Arg(MajorType... allowedTypes){
     this(false, null, allowedTypes);
   }
   
-  public Arg(String name, DataType... allowedTypes) {
+  public Arg(String name, MajorType... allowedTypes) {
     this(false, name, allowedTypes);
   }
 
-  public Arg(boolean constantsOnly, String name, DataType... allowedTypes) {
+  public Arg(boolean constantsOnly, String name, MajorType... allowedTypes) {
     this.name = name;
     this.allowedTypes = allowedTypes;
     this.constantsOnly = constantsOnly;
@@ -44,15 +61,15 @@ public class Arg {
     return name;
   }
   
-  public void confirmDataType(String expr, int argIndex, LogicalExpression e, ErrorCollector errors){
+  public void confirmDataType(ExpressionPosition expr, int argIndex, LogicalExpression e, ErrorCollector errors){
     if(constantsOnly){
       if(ConstantChecker.onlyIncludesConstants(e)) errors.addExpectedConstantValue(expr, argIndex, name);
     }
-    DataType dt = e.getDataType();
-    if(dt.isLateBind()){
+    MajorType dt = e.getMajorType();
+    if(dt.getMinorType() == MinorType.LATE){
       
       // change among allowed types.
-      for(DataType a : allowedTypes){
+      for(MajorType a : allowedTypes){
         if(dt == a) return;
       }
       
@@ -63,4 +80,13 @@ public class Arg {
     
     
   }
+  
+  private static MajorType[] getMajorTypes(boolean allowNulls, MinorType... types){
+    List<MajorType> mts = Lists.newArrayList();
+    for(MinorType t : types){
+      if(allowNulls) mts.add(MajorType.newBuilder().setMinorType(t).setMode(DataMode.OPTIONAL).build());
+      mts.add(MajorType.newBuilder().setMinorType(t).setMode(DataMode.REQUIRED).build());
+    }
+    return mts.toArray(new MajorType[mts.size()]);
+  }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/ArgumentValidator.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/ArgumentValidator.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/ArgumentValidator.java
index dc22045..765dc3f 100644
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/ArgumentValidator.java
+++ b/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/ArgumentValidator.java
@@ -23,6 +23,6 @@ import java.util.List;
  * Validates whether the set of arguments are acceptable
  */
 public interface ArgumentValidator {
-  public void validateArguments(String expr, List<LogicalExpression> expressions, ErrorCollector errors);
+  public void validateArguments(ExpressionPosition expr, List<LogicalExpression> expressions, ErrorCollector errors);
   public String[] getArgumentNamesByPosition();
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/ArgumentValidators.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/ArgumentValidators.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/ArgumentValidators.java
index 25cb887..f30733d 100644
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/ArgumentValidators.java
+++ b/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/ArgumentValidators.java
@@ -20,8 +20,8 @@ package org.apache.drill.common.expression;
 import java.util.List;
 
 import org.apache.commons.lang3.ArrayUtils;
-import org.apache.drill.common.expression.types.DataType;
-import org.apache.drill.common.expression.types.DataType.Comparability;
+import org.apache.drill.common.types.TypeProtos.MajorType;
+import org.apache.drill.common.types.Types;
 
 import com.google.common.base.Predicate;
 import com.google.common.collect.Range;
@@ -46,7 +46,7 @@ public class ArgumentValidators {
     }
 
     @Override
-    public void validateArguments(String expr, List<LogicalExpression> expressions, ErrorCollector errors) {
+    public void validateArguments(ExpressionPosition expr, List<LogicalExpression> expressions, ErrorCollector errors) {
       // only need to check argument count since any type is allowed.
       if (!argumentCount.contains(expressions.size()))
         errors.addUnexpectedArgumentCount(expr, expressions.size(), argumentCount);
@@ -61,17 +61,17 @@ public class ArgumentValidators {
 
   private static class PredicateValidator implements ArgumentValidator {
     private final Range<Integer> argumentCount;
-    private Predicate<DataType> predicate;
+    private Predicate<MajorType> predicate;
     private boolean allSame;
 
-    public PredicateValidator(int argCount, Predicate<DataType> predicate, boolean allSame) {
+    public PredicateValidator(int argCount, Predicate<MajorType> predicate, boolean allSame) {
       super();
       this.argumentCount = Ranges.singleton(argCount);
       this.predicate = predicate;
       this.allSame = allSame;
     }
 
-    public PredicateValidator(int minArguments, int maxArguments, Predicate<DataType> predicate, boolean allSame) {
+    public PredicateValidator(int minArguments, int maxArguments, Predicate<MajorType> predicate, boolean allSame) {
       super();
       this.argumentCount = Ranges.closedOpen(minArguments, maxArguments);
       this.predicate = predicate;
@@ -79,21 +79,21 @@ public class ArgumentValidators {
     }
 
     @Override
-    public void validateArguments(String expr, List<LogicalExpression> expressions, ErrorCollector errors) {
+    public void validateArguments(ExpressionPosition expr, List<LogicalExpression> expressions, ErrorCollector errors) {
       int i = -1;
-      DataType t = null;
+      MajorType t = null;
       for (LogicalExpression le : expressions) {
         i++;
-          DataType dataType = le.getDataType();
-          if (t == null) t = dataType;
+          MajorType majorType = le.getMajorType();
+          if (t == null) t = majorType;
 
-        if (!predicate.apply(dataType)) {
-          errors.addUnexpectedType(expr, i, dataType);
+        if (!predicate.apply(majorType)) {
+          errors.addUnexpectedType(expr, i, majorType);
           continue;
         }
 
-        if (allSame && t != DataType.LATEBIND && dataType != DataType.LATEBIND && t != dataType) {
-          errors.addUnexpectedType(expr, i, dataType);
+        if (allSame && !Types.isLateBind(t) && !Types.isLateBind(majorType) && !Types.softEquals(t, majorType, true)) {
+          errors.addUnexpectedType(expr, i, majorType);
         }
 
       }
@@ -109,50 +109,50 @@ public class ArgumentValidators {
 
   public static class ComparableArguments extends PredicateValidator {
 
-    public ComparableArguments(int argCount, DataType... allowedTypes) {
+    public ComparableArguments(int argCount, MajorType... allowedTypes) {
       super(argCount, new ComparableChecker(), true);
     }
 
-    public ComparableArguments(int minArguments, int maxArguments, DataType... allowedTypes) {
+    public ComparableArguments(int minArguments, int maxArguments, MajorType... allowedTypes) {
       super(minArguments, maxArguments, new ComparableChecker(), true);
     }
 
-    public static class ComparableChecker implements Predicate<DataType> {
+    public static class ComparableChecker implements Predicate<MajorType> {
 
-      public boolean apply(DataType dt) {
-          Comparability comparability = dt.getComparability();
-          return comparability.equals(Comparability.ORDERED) || comparability.equals(Comparability.UNKNOWN);
+      public boolean apply(MajorType dt) {
+          Types.Comparability comparability = Types.getComparability(dt);
+          return comparability.equals(Types.Comparability.ORDERED) || comparability.equals(Types.Comparability.UNKNOWN);
       }
     }
   }
 
   public static class AllowedTypeList extends PredicateValidator {
 
-    public AllowedTypeList(int argCount, DataType... allowedTypes) {
+    public AllowedTypeList(int argCount, MajorType... allowedTypes) {
       super(argCount, new AllowedTypeChecker(allowedTypes), false);
     }
 
-    public AllowedTypeList(int minArguments, int maxArguments, DataType... allowedTypes) {
+    public AllowedTypeList(int minArguments, int maxArguments, MajorType... allowedTypes) {
       super(minArguments, maxArguments, new AllowedTypeChecker(allowedTypes), false);
     }
 
-    public AllowedTypeList(int argCount, boolean allSame, DataType... allowedTypes) {
+    public AllowedTypeList(int argCount, boolean allSame, MajorType... allowedTypes) {
       super(argCount, new AllowedTypeChecker(allowedTypes), allSame);
     }
 
-    public AllowedTypeList(int minArguments, int maxArguments, boolean allSame, DataType... allowedTypes) {
+    public AllowedTypeList(int minArguments, int maxArguments, boolean allSame, MajorType... allowedTypes) {
       super(minArguments, maxArguments, new AllowedTypeChecker(allowedTypes), allSame);
     }
 
-    public static class AllowedTypeChecker implements Predicate<DataType> {
+    public static class AllowedTypeChecker implements Predicate<MajorType> {
 
-      private DataType[] allowedTypes;
+      private MajorType[] allowedTypes;
 
-      public AllowedTypeChecker(DataType... allowedTypes) {
+      public AllowedTypeChecker(MajorType... allowedTypes) {
         this.allowedTypes = allowedTypes;
       }
 
-      public boolean apply(DataType dt) {
+      public boolean apply(MajorType dt) {
         return ArrayUtils.contains(allowedTypes, dt);
       }
     }
@@ -170,10 +170,10 @@ public class ArgumentValidators {
       super(minArguments, maxArguments, new NumericTypeChecker(), allSame);
     }
 
-    public static class NumericTypeChecker implements Predicate<DataType> {
+    public static class NumericTypeChecker implements Predicate<MajorType> {
 
-      public boolean apply(DataType dt) {
-        return dt.isNumericType();
+      public boolean apply(MajorType dt) {
+        return Types.isNumericType(dt);
       }
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/BasicArgumentValidator.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/BasicArgumentValidator.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/BasicArgumentValidator.java
index eed49d3..26b7cf6 100644
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/BasicArgumentValidator.java
+++ b/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/BasicArgumentValidator.java
@@ -19,13 +19,13 @@ package org.apache.drill.common.expression;
 
 import java.util.List;
 
-import org.apache.drill.common.expression.types.DataType;
+import org.apache.drill.common.types.TypeProtos.MajorType;
 
 public class BasicArgumentValidator implements ArgumentValidator {
 
   private final Arg[] args;
 
-  public BasicArgumentValidator(DataType... types) {
+  public BasicArgumentValidator(MajorType... types) {
     this.args = new Arg[] { new Arg("single", types) };
   }
 
@@ -34,7 +34,7 @@ public class BasicArgumentValidator implements ArgumentValidator {
   }
 
   @Override
-  public void validateArguments(String expr, List<LogicalExpression> expressions, ErrorCollector errors) {
+  public void validateArguments(ExpressionPosition expr, List<LogicalExpression> expressions, ErrorCollector errors) {
     if (expressions.size() != args.length) errors.addUnexpectedArgumentCount(expr, expressions.size(), args.length);
 
     int i = 0;
@@ -45,7 +45,7 @@ public class BasicArgumentValidator implements ArgumentValidator {
     }
   }
 
-  public Arg arg(String name, DataType... allowedTypes) {
+  public Arg arg(String name, MajorType... allowedTypes) {
     return new Arg(name, allowedTypes);
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/ErrorCollector.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/ErrorCollector.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/ErrorCollector.java
index 21ecec4..b322345 100644
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/ErrorCollector.java
+++ b/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/ErrorCollector.java
@@ -17,27 +17,29 @@
  ******************************************************************************/
 package org.apache.drill.common.expression;
 
-import org.apache.drill.common.expression.types.DataType;
+import org.apache.drill.common.types.TypeProtos.MajorType;
 
 import com.google.common.collect.Range;
 
 public interface ErrorCollector {
 
-    public void addGeneralError(String expr, String s);
+    public void addGeneralError(ExpressionPosition expr, String s);
 
-    public void addUnexpectedArgumentType(String expr, String name, DataType actual, DataType[] expected, int argumentIndex);
+    public void addUnexpectedArgumentType(ExpressionPosition expr, String name, MajorType actual, MajorType[] expected, int argumentIndex);
 
-    public void addUnexpectedArgumentCount(String expr, int actual, Range<Integer> expected);
+    public void addUnexpectedArgumentCount(ExpressionPosition expr, int actual, Range<Integer> expected);
 
-    public void addUnexpectedArgumentCount(String expr, int actual, int expected);
+    public void addUnexpectedArgumentCount(ExpressionPosition expr, int actual, int expected);
 
-    public void addNonNumericType(String expr, DataType actual);
+    public void addNonNumericType(ExpressionPosition expr, MajorType actual);
 
-    public void addUnexpectedType(String expr, int index, DataType actual);
+    public void addUnexpectedType(ExpressionPosition expr, int index, MajorType actual);
 
-    public void addExpectedConstantValue(String expr, int actual, String s);
+    public void addExpectedConstantValue(ExpressionPosition expr, int actual, String s);
 
     boolean hasErrors();
+    
+    public int getErrorCount();
 
     String toErrorString();
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/ErrorCollectorImpl.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/ErrorCollectorImpl.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/ErrorCollectorImpl.java
index cc90b82..054e067 100644
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/ErrorCollectorImpl.java
+++ b/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/ErrorCollectorImpl.java
@@ -1,12 +1,13 @@
 package org.apache.drill.common.expression;
 
+import java.util.Arrays;
+import java.util.List;
+
+import org.apache.drill.common.types.TypeProtos.MajorType;
+
 import com.google.common.base.Joiner;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Range;
-import org.apache.drill.common.expression.types.DataType;
-
-import java.util.Arrays;
-import java.util.List;
 
 public class ErrorCollectorImpl implements ErrorCollector {
     List<ExpressionValidationError> errors;
@@ -15,17 +16,17 @@ public class ErrorCollectorImpl implements ErrorCollector {
         errors = Lists.newArrayList();
     }
 
-    private String addExpr(String expr, String message) {
-        return "Expression: [" + expr + "]. Error: " + message;
+    private String addExpr(ExpressionPosition expr, String message) {
+        return String.format("Error in expression at index %d.  Error: %s.  Full expression: %s.", expr.getCharIndex(), message, expr.getExpression());
     }
 
     @Override
-    public void addGeneralError(String expr, String s) {
+    public void addGeneralError(ExpressionPosition expr, String s) {
         errors.add(new ExpressionValidationError(addExpr(expr, s)));
     }
 
     @Override
-    public void addUnexpectedArgumentType(String expr, String name, DataType actual, DataType[] expected, int argumentIndex) {
+    public void addUnexpectedArgumentType(ExpressionPosition expr, String name, MajorType actual, MajorType[] expected, int argumentIndex) {
         errors.add(
                 new ExpressionValidationError(
                         addExpr(expr, String.format(
@@ -37,35 +38,35 @@ public class ErrorCollectorImpl implements ErrorCollector {
     }
 
     @Override
-    public void addUnexpectedArgumentCount(String expr, int actual, Range<Integer> expected) {
+    public void addUnexpectedArgumentCount(ExpressionPosition expr, int actual, Range<Integer> expected) {
         errors.add(new ExpressionValidationError(
                 addExpr(expr, String.format("Unexpected argument count. Actual argument count: %d, Expected range: %s", actual, expected))
         ));
     }
 
     @Override
-    public void addUnexpectedArgumentCount(String expr, int actual, int expected) {
+    public void addUnexpectedArgumentCount(ExpressionPosition expr, int actual, int expected) {
         errors.add(new ExpressionValidationError(
                 addExpr(expr, String.format("Unexpected argument count. Actual argument count: %d, Expected count: %d", actual, expected))
         ));
     }
 
     @Override
-    public void addNonNumericType(String expr, DataType actual) {
+    public void addNonNumericType(ExpressionPosition expr, MajorType actual) {
         errors.add(new ExpressionValidationError(
                 addExpr(expr, String.format("Unexpected numeric type. Actual type: %s", actual))
         ));
     }
 
     @Override
-    public void addUnexpectedType(String expr, int index, DataType actual) {
+    public void addUnexpectedType(ExpressionPosition expr, int index, MajorType actual) {
         errors.add(new ExpressionValidationError(
                 addExpr(expr, String.format("Unexpected argument type. Actual type: %s, Index: %d", actual, index))
         ));
     }
 
     @Override
-    public void addExpectedConstantValue(String expr, int actual, String s) {
+    public void addExpectedConstantValue(ExpressionPosition expr, int actual, String s) {
         errors.add(new ExpressionValidationError(
                 addExpr(expr, String.format("Unexpected constant value. Name: %s, Actual: %s", s, actual))
         ));
@@ -76,8 +77,21 @@ public class ErrorCollectorImpl implements ErrorCollector {
         return !errors.isEmpty();
     }
 
+    
+    @Override
+    public int getErrorCount() {
+      return errors.size();
+    }
+
     @Override
     public String toErrorString() {
         return "\n" + Joiner.on("\n").join(errors);
     }
+
+    @Override
+    public String toString() {
+      return toErrorString();
+    }
+    
+    
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/ExpressionPosition.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/ExpressionPosition.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/ExpressionPosition.java
new file mode 100644
index 0000000..2e8b529
--- /dev/null
+++ b/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/ExpressionPosition.java
@@ -0,0 +1,27 @@
+package org.apache.drill.common.expression;
+
+public class ExpressionPosition {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ExpressionPosition.class);
+  
+  public static final ExpressionPosition UNKNOWN = new ExpressionPosition("--UNKNOWN EXPRESSION--", -1);
+  
+  private final String expression;
+  private final int charIndex;
+  
+  public ExpressionPosition(String expression, int charIndex) {
+    super();
+    this.expression = expression;
+    this.charIndex = charIndex;
+  }
+
+  public String getExpression() {
+    return expression;
+  }
+
+  public int getCharIndex() {
+    return charIndex;
+  }
+  
+  
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/ExpressionStringBuilder.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/ExpressionStringBuilder.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/ExpressionStringBuilder.java
new file mode 100644
index 0000000..09a7d57
--- /dev/null
+++ b/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/ExpressionStringBuilder.java
@@ -0,0 +1,102 @@
+package org.apache.drill.common.expression;
+
+import org.apache.drill.common.expression.IfExpression.IfCondition;
+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.AbstractExprVisitor;
+
+import com.google.common.collect.ImmutableList;
+
+public class ExpressionStringBuilder extends AbstractExprVisitor<Void, StringBuilder, RuntimeException>{
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ExpressionStringBuilder.class);
+
+  @Override
+  public Void visitFunctionCall(FunctionCall call, StringBuilder sb) throws RuntimeException {
+    FunctionDefinition func = call.getDefinition();
+    ImmutableList<LogicalExpression> args = call.args;
+    if (func.isOperator()) {
+      if (args.size() == 1) { // unary
+        func.addRegisteredName(sb);
+        sb.append("(");
+        args.get(0).accept(this, sb);
+        sb.append(")");
+      } else {
+        for (int i = 0; i < args.size(); i++) {
+          if (i != 0) {
+            sb.append(" ");
+            func.addRegisteredName(sb);
+          }
+          sb.append(" (");
+          args.get(i).accept(this, sb);
+          sb.append(") ");
+        }
+      }
+    } else { // normal function
+
+      func.addRegisteredName(sb);
+      sb.append("(");
+      for (int i = 0; i < args.size(); i++) {
+        if (i != 0) sb.append(", ");
+        args.get(i).accept(this, sb);
+      }
+      sb.append(") ");
+    }
+    return null;
+  }
+
+  @Override
+  public Void visitIfExpression(IfExpression ifExpr, StringBuilder sb) throws RuntimeException {
+    ImmutableList<IfCondition> conditions = ifExpr.conditions;
+    sb.append(" ( ");
+    for(int i =0; i < conditions.size(); i++){
+      IfCondition c = conditions.get(i);
+      if(i !=0) sb.append(" else ");
+      sb.append("if (");
+      c.condition.accept(this, sb);
+      sb.append(" ) then (");
+      c.expression.accept(this, sb);
+      sb.append(" ) ");
+    }
+    sb.append(" end ");
+    sb.append(" ) ");
+    return null;
+  }
+
+  @Override
+  public Void visitSchemaPath(SchemaPath path, StringBuilder sb) throws RuntimeException {
+    sb.append("'");
+    sb.append(path.getPath());
+    sb.append("'");
+    return null;
+  }
+
+  @Override
+  public Void visitLongConstant(LongExpression lExpr, StringBuilder sb) throws RuntimeException {
+    sb.append(lExpr.getLong());
+    return null;
+  }
+
+  @Override
+  public Void visitDoubleConstant(DoubleExpression dExpr, StringBuilder sb) throws RuntimeException {
+    sb.append(dExpr.getDouble());
+    return null;
+  }
+
+  @Override
+  public Void visitBooleanConstant(BooleanExpression e, StringBuilder sb) throws RuntimeException {
+    sb.append(e.getBoolean());
+    return null;
+  }
+
+  @Override
+  public Void visitQuotedStringConstant(QuotedString e, StringBuilder sb) throws RuntimeException {
+    sb.append("\"");
+    sb.append(e.value);
+    sb.append("\"");
+    return null;
+  }
+  
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/ExpressionValidator.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/ExpressionValidator.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/ExpressionValidator.java
new file mode 100644
index 0000000..98a9075
--- /dev/null
+++ b/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/ExpressionValidator.java
@@ -0,0 +1,86 @@
+package org.apache.drill.common.expression;
+
+import org.apache.drill.common.expression.IfExpression.IfCondition;
+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 org.apache.drill.common.types.TypeProtos.DataMode;
+import org.apache.drill.common.types.TypeProtos.MajorType;
+import org.apache.drill.common.types.TypeProtos.MinorType;
+
+public class ExpressionValidator implements ExprVisitor<Void, ErrorCollector, RuntimeException> {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ExpressionValidator.class);
+
+  @Override
+  public Void visitFunctionCall(FunctionCall call, ErrorCollector errors) throws RuntimeException {
+    call.getDefinition().getArgumentValidator()
+        .validateArguments(call.getPosition(), call.args, errors);
+    return null;
+  }
+
+  @Override
+  public Void visitIfExpression(IfExpression ifExpr, ErrorCollector errors) throws RuntimeException {
+    // confirm that all conditions are required boolean values.
+    int i = 0;
+    for (IfCondition c : ifExpr.conditions) {
+      MajorType mt = c.condition.getMajorType();
+      if (mt.getMode() != DataMode.REQUIRED || mt.getMinorType() != MinorType.BOOLEAN){
+        errors.addGeneralError(c.condition.getPosition(),String.format(
+                        "Failure composing If Expression.  All conditions must return a required value and be of type boolean.  Condition %d was DatMode %s and Type %s.",
+                        i, mt.getMode(), mt.getMinorType()));
+      }
+      i++;
+    }
+
+    // confirm that all outcomes are the same type.
+    final MajorType mt = ifExpr.elseExpression.getMajorType();
+    i = 0;
+    for (IfCondition c : ifExpr.conditions) {
+      MajorType innerT = c.expression.getMajorType();
+      if (
+          (innerT.getMode() == DataMode.REPEATED && mt.getMode() != DataMode.REPEATED) || //
+          (innerT.getMinorType() != mt.getMinorType())
+          ) {
+        errors.addGeneralError(c.condition.getPosition(),String.format(
+            "Failure composing If Expression.  All expressions must return the same MajorType as the else expression.  The %d if condition returned type type %s but the else expression was of type %s",
+            i, innerT, mt));
+      }
+      i++;
+    }
+    return null;
+  }
+
+  @Override
+  public Void visitSchemaPath(SchemaPath path, ErrorCollector errors) throws RuntimeException {
+    return null;
+  }
+
+  @Override
+  public Void visitLongConstant(LongExpression intExpr, ErrorCollector errors) throws RuntimeException {
+    return null;
+  }
+
+  @Override
+  public Void visitDoubleConstant(DoubleExpression dExpr, ErrorCollector errors) throws RuntimeException {
+    return null;
+  }
+
+  @Override
+  public Void visitBooleanConstant(BooleanExpression e, ErrorCollector errors) throws RuntimeException {
+    return null;
+  }
+
+  @Override
+  public Void visitQuotedStringConstant(QuotedString e, ErrorCollector errors) throws RuntimeException {
+    return null;
+  }
+
+  @Override
+  public Void visitUnknown(LogicalExpression e, ErrorCollector value) throws RuntimeException {
+    return null;
+  }
+
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/FieldReference.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/FieldReference.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/FieldReference.java
index 94800ba..f611614 100644
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/FieldReference.java
+++ b/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/FieldReference.java
@@ -21,6 +21,7 @@ import java.io.IOException;
 
 import org.apache.drill.common.expression.FieldReference.De;
 import org.apache.drill.common.expression.FieldReference.Se;
+import org.apache.drill.common.types.TypeProtos.MajorType;
 
 import com.fasterxml.jackson.core.JsonGenerationException;
 import com.fasterxml.jackson.core.JsonGenerator;
@@ -32,26 +33,25 @@ import com.fasterxml.jackson.databind.annotation.JsonDeserialize;
 import com.fasterxml.jackson.databind.annotation.JsonSerialize;
 import com.fasterxml.jackson.databind.deser.std.StdDeserializer;
 import com.fasterxml.jackson.databind.ser.std.StdSerializer;
-import org.apache.drill.common.expression.types.DataType;
 
 @JsonSerialize(using = Se.class)
 @JsonDeserialize(using = De.class)
 public class FieldReference extends SchemaPath {
-  DataType overrideType;
+  MajorType overrideType;
 
-  public FieldReference(String value) {
-    super(value);
+  public FieldReference(String value, ExpressionPosition pos) {
+    super(value, pos);
   }
 
-  public FieldReference(String value, DataType dataType) {
-    super(value);
+  public FieldReference(String value, ExpressionPosition pos, MajorType dataType) {
+    super(value, pos);
     this.overrideType = dataType;
   }
 
     @Override
-    public DataType getDataType() {
+    public MajorType getMajorType() {
         if(overrideType == null) {
-            return super.getDataType();
+            return super.getMajorType();
         } else {
             return overrideType;
         }
@@ -66,7 +66,7 @@ public class FieldReference extends SchemaPath {
     @Override
     public FieldReference deserialize(JsonParser jp, DeserializationContext ctxt) throws IOException,
         JsonProcessingException {
-      return new FieldReference(this._parseString(jp, ctxt));
+      return new FieldReference(this._parseString(jp, ctxt), ExpressionPosition.UNKNOWN);
     }
 
   }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/FunctionBase.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/FunctionBase.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/FunctionBase.java
deleted file mode 100644
index fd8ae18..0000000
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/FunctionBase.java
+++ /dev/null
@@ -1,94 +0,0 @@
-/*******************************************************************************
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- ******************************************************************************/
-package org.apache.drill.common.expression;
-
-import java.util.Iterator;
-import java.util.List;
-
-import org.apache.drill.common.expression.types.DataType;
-import org.apache.drill.common.expression.visitors.ExprVisitor;
-
-import com.fasterxml.jackson.annotation.JsonIgnore;
-import com.google.common.collect.ImmutableList;
-
-
-public abstract class FunctionBase extends LogicalExpressionBase implements Iterable<LogicalExpression>{
-	public final ImmutableList<LogicalExpression> expressions;
-	
-	
-  protected void funcToString(StringBuilder sb, String functionName) {
-    sb.append(" ");
-    sb.append(functionName);
-    sb.append("( ");
-    for(int i= 0; i < expressions.size(); i++){
-      if(i != 0) sb.append(", ");
-      expressions.get(i).addToString(sb);
-    }
-    sb.append(" ) ");
-  }
-  
-  protected void opToString(StringBuilder sb, String operator) {
-    sb.append(" ( ");
-    expressions.get(0).addToString(sb);
-    for(int i =1; i < expressions.size(); i++){
-      sb.append(operator);
-      expressions.get(i).addToString(sb); 
-    }
-    sb.append(" ) ");
-  }
-	
-	public FunctionBase(List<LogicalExpression> expressions){
-		if( !(expressions instanceof ImmutableList)){
-			expressions = ImmutableList.copyOf(expressions);
-		}
-		this.expressions = (ImmutableList<LogicalExpression>) expressions;
-	}
-	
-//	public static DataType getJointType(String parentName, LogicalExpression expr1, LogicalExpression expr2) throws ExpressionValidationException{
-//		DataType dt = DataType.getCombinedCast(expr1.getDataType(), expr2.getDataType());
-//		if(dt == null) throw new ExpressionValidationException();
-//		
-//		return dt;
-//	}
-//	
-//	public LogicalExpression wrapWithCastIfNecessary(DataType dt) throws ExpressionValidationException{
-//		if(this.getDataType() != dt) return new Cast(this, dt);
-//		return this;
-//	}	
-
-	
-
-
-	@Override
-	public Iterator<LogicalExpression> iterator() {
-		return expressions.iterator();
-	}
-
-	@Override
-	@JsonIgnore
-	public DataType getDataType() {
-		throw new UnsupportedOperationException();	
-	}
-
-	
-	
-	
-	
-	
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/FunctionCall.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/FunctionCall.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/FunctionCall.java
index ee76a5c..e13e87e 100644
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/FunctionCall.java
+++ b/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/FunctionCall.java
@@ -20,26 +20,34 @@ package org.apache.drill.common.expression;
 import java.util.Iterator;
 import java.util.List;
 
-import org.apache.drill.common.expression.types.DataType;
 import org.apache.drill.common.expression.visitors.ExprVisitor;
+import org.apache.drill.common.types.TypeProtos.MajorType;
 
 import com.google.common.collect.ImmutableList;
 
 public class FunctionCall extends LogicalExpressionBase implements Iterable<LogicalExpression> {
   private final FunctionDefinition func;
   public final ImmutableList<LogicalExpression> args;
+  private final ExpressionPosition pos;
 
-  public FunctionCall(FunctionDefinition func, List<LogicalExpression> args) {
+  public FunctionCall(FunctionDefinition func, List<LogicalExpression> args, ExpressionPosition pos) {
+    super(pos);
     this.func = func;
     if (!(args instanceof ImmutableList)) {
       args = ImmutableList.copyOf(args);
     }
     this.args = (ImmutableList<LogicalExpression>) args;
+    this.pos = pos;
   }
 
   @Override
-  public <T> T accept(ExprVisitor<T> visitor) {
-    return visitor.visitFunctionCall(this);
+  public ExpressionPosition getPosition() {
+    return pos;
+  }
+
+  @Override
+  public <T, V, E extends Exception> T accept(ExprVisitor<T, V, E> visitor, V value) throws E{
+    return visitor.visitFunctionCall(this, value);
   }
 
   @Override
@@ -52,43 +60,16 @@ public class FunctionCall extends LogicalExpressionBase implements Iterable<Logi
   }
   
   @Override
-  public DataType getDataType() {
+  public MajorType getMajorType() {
     return func.getDataType(this.args);
   }
 
   @Override
-  public void addToString(StringBuilder sb) {
-    if (func.isOperator()) {
-      if (args.size() == 1) { // unary
-        func.addRegisteredName(sb);
-        sb.append("(");
-        args.get(0).addToString(sb);
-        sb.append(")");
-      } else {
-        for (int i = 0; i < args.size(); i++) {
-          if (i != 0) {
-            sb.append(" ");
-            func.addRegisteredName(sb);
-          }
-          sb.append(" (");
-          args.get(i).addToString(sb);
-          sb.append(") ");
-        }
-      }
-    } else { // normal function
-
-      func.addRegisteredName(sb);
-      sb.append("(");
-      for (int i = 0; i < args.size(); i++) {
-        if (i != 0) sb.append(", ");
-        args.get(i).addToString(sb);
-      }
-      sb.append(") ");
-    }
+  public String toString() {
+    final int maxLen = 10;
+    return "FunctionCall [func=" + func + ", args="
+        + (args != null ? args.subList(0, Math.min(args.size(), maxLen)) : null) + ", pos=" + pos + "]";
   }
 
-    @Override
-    public void resolveAndValidate(String expr, ErrorCollector errors) {
-        func.getArgumentValidator().validateArguments(expr, args, errors);
-    }
+  
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/FunctionDefinition.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/FunctionDefinition.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/FunctionDefinition.java
index 9d21763..4cbab6a 100644
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/FunctionDefinition.java
+++ b/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/FunctionDefinition.java
@@ -21,7 +21,7 @@ import java.util.Arrays;
 import java.util.List;
 
 import org.apache.commons.lang3.ArrayUtils;
-import org.apache.drill.common.expression.types.DataType;
+import org.apache.drill.common.types.TypeProtos.MajorType;
 
 public class FunctionDefinition {
 
@@ -42,7 +42,7 @@ public class FunctionDefinition {
     this.isOperator = isOperator;
   }
 
-  public DataType getDataType(List<LogicalExpression> args){
+  public MajorType getDataType(List<LogicalExpression> args){
     return outputType.getOutputType(args);
   }
   
@@ -82,8 +82,8 @@ public class FunctionDefinition {
     return this.name;
   }
   
-  public FunctionCall newCall(List<LogicalExpression> args){
-    return new FunctionCall(this, args);
+  public FunctionCall newCall(List<LogicalExpression> args, ExpressionPosition pos){
+    return new FunctionCall(this, args, pos);
   }
   
   public void addRegisteredName(StringBuilder sb){

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/FunctionRegistry.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/FunctionRegistry.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/FunctionRegistry.java
index c22e448..ed2c63e 100644
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/FunctionRegistry.java
+++ b/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/FunctionRegistry.java
@@ -57,17 +57,18 @@ public class FunctionRegistry {
     }
   }
   
-  public LogicalExpression createExpression(String functionName, List<LogicalExpression> args){
+  
+  public LogicalExpression createExpression(String functionName, ExpressionPosition ep, List<LogicalExpression> args){
     FunctionDefinition d = funcMap.get(functionName);
     if(d == null) throw new ExpressionParsingException(String.format("Unable to find function definition for function named '%s'", functionName));
-    return d.newCall(args);
+    return d.newCall(args, ep);
   }
   
-  public LogicalExpression createExpression(String unaryName, LogicalExpression... e){
-    return funcMap.get(unaryName).newCall(Lists.newArrayList(e));
+  public LogicalExpression createExpression(String unaryName, ExpressionPosition ep, LogicalExpression... e){
+    return funcMap.get(unaryName).newCall(Lists.newArrayList(e), ep);
   }
   
-  public LogicalExpression createByOp(List<LogicalExpression> args, List<String> opTypes) {
+  public LogicalExpression createByOp(List<LogicalExpression> args, ExpressionPosition ep, List<String> opTypes) {
     // logger.debug("Generating new comparison expressions.");
     if (args.size() == 1) {
       return args.get(0);
@@ -81,7 +82,7 @@ public class FunctionRegistry {
       List<LogicalExpression> l2 = new ArrayList<LogicalExpression>();
       l2.add(first);
       l2.add(args.get(i + 1));
-      first = createExpression(opTypes.get(i), args);
+      first = createExpression(opTypes.get(i), ep, args);
     }
     return first;
   }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/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 a553f4c..6f2e4a8 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
@@ -21,11 +21,12 @@ import java.util.ArrayList;
 import java.util.List;
 
 import org.apache.drill.common.expression.IfExpression.IfCondition;
-import org.apache.drill.common.expression.types.DataType;
 import org.apache.drill.common.expression.visitors.ExprVisitor;
+import org.apache.drill.common.types.TypeProtos.MajorType;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import com.google.common.base.Preconditions;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.UnmodifiableIterator;
 
@@ -35,7 +36,8 @@ public class IfExpression extends LogicalExpressionBase implements Iterable<IfCo
 	public final ImmutableList<IfCondition> conditions;
 	public final LogicalExpression elseExpression;
 	
-	private IfExpression(List<IfCondition> conditions, LogicalExpression elseExpression){
+	private IfExpression(ExpressionPosition pos, List<IfCondition> conditions, LogicalExpression elseExpression){
+	  super(pos);
 		this.conditions = ImmutableList.copyOf(conditions);
 		this.elseExpression = elseExpression;
 	}
@@ -53,28 +55,32 @@ public class IfExpression extends LogicalExpressionBase implements Iterable<IfCo
 
 	}
 	
-	
 	@Override
-  public <T> T accept(ExprVisitor<T> visitor) {
-    return visitor.visitIfExpression(this);
+  public <T, V, E extends Exception> T accept(ExprVisitor<T, V, E> visitor, V value) throws E{
+    return visitor.visitIfExpression(this, value);
   }
 
-
   public static class Builder{
 		List<IfCondition> conditions = new ArrayList<IfCondition>();
 		private LogicalExpression elseExpression;
+		private ExpressionPosition pos = ExpressionPosition.UNKNOWN;
+		
+		public Builder setPosition(ExpressionPosition pos){
+		  this.pos = pos;
+		  return this;
+		}
 		
 		public Builder addCondition(IfCondition condition){
 			conditions.add(condition);
             return this;
 		}
 
-        public Builder addConditions(Iterable<IfCondition> conditions) {
-            for(IfCondition condition : conditions) {
-                addCondition(condition);
-            }
-            return this;
-        }
+    public Builder addConditions(Iterable<IfCondition> conditions) {
+      for (IfCondition condition : conditions) {
+        addCondition(condition);
+      }
+      return this;
+    }
 		
 		public Builder setElse(LogicalExpression elseExpression) {
 			this.elseExpression = elseExpression;
@@ -82,39 +88,20 @@ public class IfExpression extends LogicalExpressionBase implements Iterable<IfCo
 		}
 		
 		public IfExpression build(){
-			return new IfExpression(conditions, elseExpression);
+		  Preconditions.checkNotNull(pos);
+		  Preconditions.checkNotNull(conditions);
+		  Preconditions.checkNotNull(conditions);
+			return new IfExpression(pos, conditions, elseExpression);
 		}
 		
 	}
 
-
-    @Override
-    public DataType getDataType() {
-        return DataType.BOOLEAN;
-    }
-
-    @Override
-  public void addToString(StringBuilder sb) {
-	  sb.append(" ( ");
-	  for(int i =0; i < conditions.size(); i++){
-	    IfCondition c = conditions.get(i);
-	    if(i !=0) sb.append(" else ");
-	    sb.append("if (");
-	    c.condition.addToString(sb);
-	    sb.append(" ) then (");
-	    c.expression.addToString(sb);
-	    sb.append(" ) ");
-	  }
-	  sb.append(" end ");
-	  sb.append(" ) ");
+  @Override
+  public MajorType getMajorType() {
+    return this.elseExpression.getMajorType();
   }
 
-    @Override
-    public void resolveAndValidate(String expr, ErrorCollector errors) {
-    }
-
-
-    public static Builder newBuilder(){
+  public static Builder newBuilder(){
 		return new Builder();
 	}
 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/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 3df33a0..165f409 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
@@ -26,8 +26,8 @@ import org.apache.drill.common.config.DrillConfig;
 import org.apache.drill.common.expression.parser.ExprLexer;
 import org.apache.drill.common.expression.parser.ExprParser;
 import org.apache.drill.common.expression.parser.ExprParser.parse_return;
-import org.apache.drill.common.expression.types.DataType;
 import org.apache.drill.common.expression.visitors.ExprVisitor;
+import org.apache.drill.common.types.TypeProtos.MajorType;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -44,65 +44,65 @@ 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 {
-    static final Logger logger = LoggerFactory.getLogger(LogicalExpression.class);
+  static final Logger logger = LoggerFactory.getLogger(LogicalExpression.class);
 
-    public abstract DataType getDataType();
+  public abstract MajorType getMajorType();
 
-    public void addToString(StringBuilder sb);
+  public <T, V, E extends Exception> T accept(ExprVisitor<T, V, E> visitor, V value) throws E;
 
-    public void resolveAndValidate(String expr, ErrorCollector errors);
+  public ExpressionPosition getPosition();
 
-    public <T> T accept(ExprVisitor<T> visitor);
+  public static class De extends StdDeserializer<LogicalExpression> {
+    DrillConfig config;
 
-    public static class De extends StdDeserializer<LogicalExpression> {
-        DrillConfig config;
-        ErrorCollector errorCollector;
-
-        public De(DrillConfig config) {
-            super(LogicalExpression.class);
-            this.config = config;
-            this.errorCollector = config.getErrorCollector();
-        }
-
-        @Override
-        public LogicalExpression deserialize(JsonParser jp, DeserializationContext ctxt) throws IOException,
-                JsonProcessingException {
-            String expr = jp.getText();
-
-            if (expr == null || expr.isEmpty())
-                return null;
-            try {
-                // logger.debug("Parsing expression string '{}'", expr);
-                ExprLexer lexer = new ExprLexer(new ANTLRStringStream(expr));
-
-                CommonTokenStream tokens = new CommonTokenStream(lexer);
-                ExprParser parser = new ExprParser(tokens);
-                parser.setRegistry(new FunctionRegistry(config));
-                parse_return ret = parser.parse();
-                // logger.debug("Found expression '{}'", ret.e);
-                ret.e.resolveAndValidate(expr, errorCollector);
-                return ret.e;
-            } catch (RecognitionException e) {
-                throw new RuntimeException(e);
-            }
-        }
+    public De(DrillConfig config) {
+      super(LogicalExpression.class);
+      this.config = config;
+    }
 
+    @Override
+    public LogicalExpression deserialize(JsonParser jp, DeserializationContext ctxt) throws IOException,
+        JsonProcessingException {
+      String expr = jp.getText();
+
+      if (expr == null || expr.isEmpty())
+        return null;
+      try {
+        // logger.debug("Parsing expression string '{}'", expr);
+        ExprLexer lexer = new ExprLexer(new ANTLRStringStream(expr));
+        CommonTokenStream tokens = new CommonTokenStream(lexer);
+        ExprParser parser = new ExprParser(tokens);
+
+        //TODO: move functionregistry and error collector to injectables.
+        //ctxt.findInjectableValue(valueId, forProperty, beanInstance)
+
+        parser.setRegistry(new FunctionRegistry(config));
+        parse_return ret = parser.parse();
+
+        // ret.e.resolveAndValidate(expr, errorCollector);
+        return ret.e;
+      } catch (RecognitionException e) {
+        throw new RuntimeException(e);
+      }
     }
 
-    public static class Se extends StdSerializer<LogicalExpression> {
+  }
 
-        protected Se() {
-            super(LogicalExpression.class);
-        }
+  public static class Se extends StdSerializer<LogicalExpression> {
 
-        @Override
-        public void serialize(LogicalExpression value, JsonGenerator jgen, SerializerProvider provider) throws IOException,
-                JsonGenerationException {
-            StringBuilder sb = new StringBuilder();
-            value.addToString(sb);
-            jgen.writeString(sb.toString());
-        }
+    protected Se() {
+      super(LogicalExpression.class);
+    }
 
+    @Override
+    public void serialize(LogicalExpression value, JsonGenerator jgen, SerializerProvider provider) throws IOException,
+        JsonGenerationException {
+      StringBuilder sb = new StringBuilder();
+      ExpressionStringBuilder esb = new ExpressionStringBuilder();
+      value.accept(esb, sb);
+      jgen.writeString(sb.toString());
     }
 
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/LogicalExpressionBase.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/LogicalExpressionBase.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/LogicalExpressionBase.java
index e973df7..aa8885f 100644
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/LogicalExpressionBase.java
+++ b/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/LogicalExpressionBase.java
@@ -6,9 +6,9 @@
  * to you under the Apache License, Version 2.0 (the
  * "License"); you may not use this file except in compliance
  * with the License.  You may obtain a copy of the License at
- *
+ * 
  * http://www.apache.org/licenses/LICENSE-2.0
- *
+ * 
  * Unless required by applicable law or agreed to in writing, software
  * distributed under the License is distributed on an "AS IS" BASIS,
  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
@@ -17,40 +17,44 @@
  ******************************************************************************/
 package org.apache.drill.common.expression;
 
-import org.apache.drill.common.expression.types.DataType;
-import org.apache.drill.common.expression.visitors.ExprVisitor;
+import org.apache.drill.common.types.TypeProtos.MajorType;
 
 import com.fasterxml.jackson.annotation.JsonProperty;
 import com.fasterxml.jackson.annotation.JsonPropertyOrder;
 
 
-@JsonPropertyOrder({"type"})
-public abstract class LogicalExpressionBase implements LogicalExpression {
+@JsonPropertyOrder({ "type" })
+public abstract class LogicalExpressionBase implements LogicalExpression{
 
+  private final ExpressionPosition pos;
 
-//	public static DataType getJointType(String parentName, LogicalExpression expr1, LogicalExpression expr2) throws ExpressionValidationException{
-//		DataType dt = DataType.getCombinedCast(expr1.getDataType(), expr2.getDataType());
-//		if(dt == null) throw new ExpressionValidationException();
-//		
-//		return dt;
-//	}
+	protected LogicalExpressionBase(ExpressionPosition pos) {
+    super();
+    this.pos = pos;
+  }
 
+  @Override
+  public ExpressionPosition getPosition() {
+    return pos;
+  }
 
-    protected void i(StringBuilder sb, int indent) {
-        for (int i = 0; i < indent; i++) {
-            sb.append("  ");
-        }
-    }
+  protected void i(StringBuilder sb, int indent){
+		for(int i = 0; i < indent; i++){
+			sb.append("  ");
+		}
+	}
+	
+	@Override
+	public MajorType getMajorType() {
+		throw new UnsupportedOperationException(String.format("The type of %s doesn't currently support LogicalExpression.getDataType().", this.getClass().getCanonicalName()));
+	}
 
-//	@Override
-//	public <T> T accept(ExprVisitor<T> visitor) {
-//		return visitor.visit(this);
-//	}
-
-    @JsonProperty("type")
-    public String getDescription() {
-        return this.getClass().getSimpleName();
-    }
+  @JsonProperty("type")
+	public String getDescription(){
+		return this.getClass().getSimpleName();
+	}
+	
 
+	
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/OutputTypeDeterminer.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/OutputTypeDeterminer.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/OutputTypeDeterminer.java
index 67402e2..0bec184 100644
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/OutputTypeDeterminer.java
+++ b/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/OutputTypeDeterminer.java
@@ -19,37 +19,51 @@ package org.apache.drill.common.expression;
 
 import java.util.List;
 
-import org.apache.drill.common.expression.types.DataType;
+import org.apache.drill.common.types.TypeProtos.DataMode;
+import org.apache.drill.common.types.TypeProtos.MajorType;
+import org.apache.drill.common.types.TypeProtos.MinorType;
 
 public interface OutputTypeDeterminer {
 
-  public static OutputTypeDeterminer FIXED_BOOLEAN = new FixedType(DataType.BOOLEAN);
+  public static OutputTypeDeterminer FIXED_BOOLEAN = new FixedType(MajorType.newBuilder().setMinorType(MinorType.BOOLEAN).setMode(DataMode.REQUIRED).build());
   
-  public DataType getOutputType(List<LogicalExpression> expressions);
+  public MajorType getOutputType(List<LogicalExpression> expressions);
   
   
   public static class FixedType implements OutputTypeDeterminer{
-    public DataType outputType;
+    public MajorType outputType;
     
     
-    public FixedType(DataType outputType) {
+    public FixedType(MajorType outputType) {
       super();
       this.outputType = outputType;
     }
 
 
     @Override
-    public DataType getOutputType(List<LogicalExpression> expressions) {
-      return null;
+    public MajorType getOutputType(List<LogicalExpression> expressions) {
+      return outputType;
     }
     
   }
   
   public static class SameAsFirstInput implements OutputTypeDeterminer{
-
     @Override
-    public DataType getOutputType(List<LogicalExpression> expressions) {
-      return expressions.get(0).getDataType();
+    public MajorType getOutputType(List<LogicalExpression> expressions) {
+      return expressions.get(0).getMajorType();
+    }
+  }
+  
+  public static class SameAsAnySoft implements OutputTypeDeterminer{
+    @Override
+    public MajorType getOutputType(List<LogicalExpression> expressions) {
+      for(LogicalExpression e : expressions){
+        if(e.getMajorType().getMode() == DataMode.OPTIONAL){
+          return e.getMajorType();
+        }
+      }
+      return expressions.get(0).getMajorType();
     }
   }
+  
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/PathSegment.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/PathSegment.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/PathSegment.java
index c06426e..554c0ca 100644
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/PathSegment.java
+++ b/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/PathSegment.java
@@ -98,6 +98,32 @@ public abstract class PathSegment{
       return "NameSegment [path=" + path + ", getCollisionBehavior()=" + getCollisionBehavior() + ", getChild()="
           + getChild() + "]";
     }
+
+    @Override
+    public int hashCode() {
+      final int prime = 31;
+      int result = 1;
+      result = prime * result + ((path == null) ? 0 : path.hashCode());
+      return result;
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+      if (this == obj)
+        return true;
+      if (obj == null)
+        return false;
+      if (getClass() != obj.getClass())
+        return false;
+      NameSegment other = (NameSegment) obj;
+      if (path == null) {
+        if (other.path != null)
+          return false;
+      } else if (!path.equals(other.path))
+        return false;
+      return true;
+    }
+    
     
     
   }


Re: [35/53] [abbrv] git commit: Separate allocate and load methods. rename setRecordCount to setValueCount add setGroupAndValueCount to RepeatedVectors. add a number of marker/cross-inheritance interfaces.

Posted by Ted Dunning <te...@gmail.com>.
On Fri, Jul 19, 2013 at 6:57 PM, <ja...@apache.org> wrote:

> Separate allocate and load methods.
> rename setRecordCount to setValueCount
> add setGroupAndValueCount to RepeatedVectors.
> add a number of marker/cross-inheritance interfaces.
>
>
> Project: http://git-wip-us.apache.org/repos/asf/incubator-drill/repo
> Commit:
> http://git-wip-us.apache.org/repos/asf/incubator-drill/commit/9ca9eb9b
> Tree: http://git-wip-us.apache.org/repos/asf/incubator-drill/tree/9ca9eb9b
> Diff: http://git-wip-us.apache.org/repos/asf/incubator-drill/diff/9ca9eb9b
>
> Branch: refs/heads/master
> Commit: 9ca9eb9b3d88e86e28d1b688d9cd943e6a7f08df
> Parents: 36793bb
> Author: Jacques Nadeau <ja...@apache.org>
> Authored: Mon Jul 15 10:50:07 2013 -0700
> Committer: Jacques Nadeau <ja...@apache.org>
> Committed: Mon Jul 15 13:13:59 2013 -0700
>
> ----------------------------------------------------------------------
>  .../templates/FixedValueVectors.java            | 118 +++++---
>  .../templates/NullableValueVectors.java         | 220 ++++++++++-----
>  .../templates/RepeatedValueVectors.java         | 273 ++++++++++++++-----
>  .../templates/VariableLengthVectors.java        | 177 +++++++-----
>  .../exec/physical/config/MockRecordReader.java  |  37 ++-
>  .../drill/exec/record/MaterializedField.java    |  16 ++
>  .../drill/exec/record/RecordBatchLoader.java    |   7 +-
>  .../apache/drill/exec/record/WritableBatch.java |   3 +-
>  .../drill/exec/store/JSONRecordReader.java      |   4 +-
>  .../apache/drill/exec/store/VectorHolder.java   |  21 +-
>  .../drill/exec/vector/BaseDataValueVector.java  |  47 ++++
>  .../drill/exec/vector/BaseValueVector.java      |  38 +++
>  .../org/apache/drill/exec/vector/BitVector.java | 112 +++++---
>  .../apache/drill/exec/vector/ByteHolder.java    |  12 +
>  .../drill/exec/vector/FixedWidthVector.java     |  23 ++
>  .../drill/exec/vector/NonRepeatedMutator.java   |   7 +
>  .../exec/vector/RepeatedFixedWidthVector.java   |  22 ++
>  .../vector/RepeatedVariableWidthVector.java     |  24 ++
>  .../apache/drill/exec/vector/ValueVector.java   | 179 ++++--------
>  .../drill/exec/vector/VariableWidthVector.java  |  29 ++
>  .../apache/drill/exec/work/foreman/Foreman.java |   5 +-
>  .../physical/impl/TestSimpleFragmentRun.java    |   2 +-
>  .../exec/record/vector/TestValueVector.java     |  81 +++---
>  .../drill/exec/store/JSONRecordReaderTest.java  |   2 +-
>  24 files changed, 995 insertions(+), 464 deletions(-)
> ----------------------------------------------------------------------
>
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/9ca9eb9b/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/FixedValueVectors.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/FixedValueVectors.java
> b/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/FixedValueVectors.java
> index 09dd5d8..7583d9f 100644
> ---
> a/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/FixedValueVectors.java
> +++
> b/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/FixedValueVectors.java
> @@ -12,6 +12,7 @@ import org.apache.drill.exec.memory.BufferAllocator;
>  import org.apache.drill.exec.proto.UserBitShared.FieldMetadata;
>  import org.apache.drill.exec.record.DeadBuf;
>  import org.apache.drill.exec.record.MaterializedField;
> +import org.apache.drill.exec.vector.BaseValueVector;
>  import org.apache.drill.exec.vector.MsgPack2Vector;
>
>  import java.util.Random;
> @@ -26,71 +27,100 @@ import java.util.Random;
>   * NB: this class is automatically generated from ValueVectorTypes.tdd
> using FreeMarker.
>   */
>  @SuppressWarnings("unused")
> -public final class ${minor.class}Vector extends ValueVector {
> +public final class ${minor.class}Vector extends BaseDataValueVector
> implements FixedWidthVector{
>    static final org.slf4j.Logger logger =
> org.slf4j.LoggerFactory.getLogger(${minor.class}Vector.class);
>
> +
> +  private final Accessor accessor = new Accessor();
> +  private final Mutator mutator = new Mutator();
> +
>    public ${minor.class}Vector(MaterializedField field, BufferAllocator
> allocator) {
>      super(field, allocator);
>    }
>
> +  public int getValueCapacity(){
> +    return (int) (data.capacity() *1.0 / ${type.width});
> +  }
> +
> +  public Accessor getAccessor(){
> +    return accessor;
> +  }
> +
> +  public Mutator getMutator(){
> +    return mutator;
> +  }
> +
>    /**
> -   * Allocate a new memory space for this vector.  Must be called prior
> to using the ValueVector.
> -   *
> +   * Allocate a new buffer that supports setting at least the provided
> number of values.  May actually be sized bigger depending on underlying
> buffer rounding size. Must be called prior to using the ValueVector.
>     * @param valueCount
> -   *          The number of values which can be contained within this
> vector.
>     */
>    public void allocateNew(int valueCount) {
> -    totalBytes = valueCount * ${type.width};
> -    allocateNew(totalBytes, allocator.buffer(totalBytes), valueCount);
> +    clear();
> +    this.data = allocator.buffer(valueCount * ${type.width});
> +    this.data.retain();
> +    this.data.readerIndex(0);
>    }
> -
> +
>    @Override
> -  public int getAllocatedSize() {
> -    return (int) Math.ceil(totalBytes);
> +  public FieldMetadata getMetadata() {
> +    return FieldMetadata.newBuilder()
> +             .setDef(getField().getDef())
> +             .setValueCount(recordCount)
> +             .setBufferLength(recordCount * ${type.width})
> +             .build();
>    }
>
> -  /**
> -   * Get the size requirement (in bytes) for the given number of values.
>  Only accurate
> -   * for fixed width value vectors.
> -   */
>    @Override
> -  public int getSizeFromCount(int valueCount) {
> -    return valueCount * ${type.width};
> +  public int load(int valueCount, ByteBuf buf){
> +    clear();
> +    this.recordCount = valueCount;
> +    int len = recordCount * ${type.width};
> +    data = buf.slice(0, len);
> +    data.retain();
> +    return len;
>    }
> -
> -  public Mutator getMutator() {
> -    return new Mutator();
> +
> +  @Override
> +  public void load(FieldMetadata metadata, ByteBuf buffer) {
> +    assert this.field.getDef().equals(metadata.getDef());
> +    int loaded = load(metadata.getValueCount(), buffer);
> +    assert metadata.getBufferLength() == loaded;
>    }
> +
> +  public final class Accessor extends BaseValueVector.BaseAccessor{
>
> - <#if (type.width > 8)>
> -
> -  public ${minor.javaType!type.javaType} get(int index) {
> -    ByteBuf dst = allocator.buffer(${type.width});
> -    data.getBytes(index * ${type.width}, dst, 0, ${type.width});
> -    return dst;
> -  }
> +    public int getRecordCount() {
> +      return recordCount;
> +    }
> +
> +    <#if (type.width > 8)>
>
> -  @Override
> -  public Object getObject(int index) {
> -    ByteBuf dst = allocator.buffer(${type.width});
> -    data.getBytes(index, dst, 0, ${type.width});
> -    return dst;
> -  }
> +    public ${minor.javaType!type.javaType} get(int index) {
> +      ByteBuf dst = allocator.buffer(${type.width});
> +      data.getBytes(index * ${type.width}, dst, 0, ${type.width});
> +      return dst;
> +    }
>
> +    @Override
> +    public Object getObject(int index) {
> +      ByteBuf dst = allocator.buffer(${type.width});
> +      data.getBytes(index, dst, 0, ${type.width});
> +      return dst;
> +    }
>
> - <#else> <#-- type.width <= 8 -->
> +    <#else> <#-- type.width <= 8 -->
>
> -  public ${minor.javaType!type.javaType} get(int index) {
> -    return data.get${(minor.javaType!type.javaType)?cap_first}(index *
> ${type.width});
> -  }
> +    public ${minor.javaType!type.javaType} get(int index) {
> +      return data.get${(minor.javaType!type.javaType)?cap_first}(index *
> ${type.width});
> +    }
>
> -  public Object getObject(int index) {
> -    return get(index);
> -  }
> +    public Object getObject(int index) {
> +      return get(index);
> +    }
>
>
> - </#if> <#-- type.width -->
> -
> +   </#if> <#-- type.width -->
> + }
>
>   /**
>    * ${minor.class}.Mutator implements a mutable vector of fixed width
> values.  Elements in the
> @@ -101,7 +131,7 @@ public final class ${minor.class}Vector extends
> ValueVector {
>    *
>    * NB: this class is automatically generated from ValueVectorTypes.tdd
> using FreeMarker.
>    */
> -  public class Mutator implements ValueVector.Mutator{
> +  public final class Mutator extends BaseValueVector.BaseMutator{
>
>      private Mutator(){};
>     /**
> @@ -147,9 +177,9 @@ public final class ${minor.class}Vector extends
> ValueVector {
>     }
>    </#if> <#-- type.width -->
>
> -   @Override
> -   public void setRecordCount(int recordCount) {
> -     ${minor.class}Vector.this.setRecordCount(recordCount);
> +   public void setValueCount(int recordCount) {
> +     ${minor.class}Vector.this.recordCount = recordCount;
> +     data.writerIndex(${type.width} * recordCount);
>     }
>
>
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/9ca9eb9b/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/NullableValueVectors.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/NullableValueVectors.java
> b/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/NullableValueVectors.java
> index c7de73f..3232f87 100644
> ---
> a/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/NullableValueVectors.java
> +++
> b/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/NullableValueVectors.java
> @@ -10,12 +10,15 @@ import io.netty.buffer.ByteBuf;
>
>  import java.io.Closeable;
>  import java.util.Random;
> +import java.util.Vector;
>
>  import org.apache.drill.exec.memory.BufferAllocator;
>  import org.apache.drill.exec.proto.SchemaDefProtos;
>  import org.apache.drill.exec.proto.UserBitShared.FieldMetadata;
>  import org.apache.drill.exec.record.DeadBuf;
>  import org.apache.drill.exec.record.MaterializedField;
> +import org.apache.drill.exec.vector.BaseValueVector;
> +import org.apache.drill.exec.vector.BitVector;
>  import org.apache.drill.exec.vector.UInt2Vector;
>  import org.apache.drill.exec.vector.UInt4Vector;
>
> @@ -27,93 +30,180 @@ import org.apache.drill.exec.vector.UInt4Vector;
>   * NB: this class is automatically generated from ValueVectorTypes.tdd
> using FreeMarker.
>   */
>  @SuppressWarnings("unused")
> -public final class Nullable${minor.class}Vector extends ValueVector {
> +public final class Nullable${minor.class}Vector extends BaseValueVector
> implements <#if type.major ==
> "VarLen">VariableWidth<#else>FixedWidth</#if>Vector {
>
> +  private int recordCount;
>    private final BitVector bits;
>    private final ${minor.class}Vector values;
> +  private final Accessor accessor = new Accessor();
> +  private final Mutator mutator = new Mutator();
>
>    public Nullable${minor.class}Vector(MaterializedField field,
> BufferAllocator allocator) {
>      super(field, allocator);
> -    bits = new BitVector(null, allocator);
> -    values = new ${minor.class}Vector(null, allocator);
> +    this.bits = new BitVector(null, allocator);
> +    this.values = new ${minor.class}Vector(null, allocator);
>    }
> -
> -  /**
> -   * Get the element at the specified position.
> -   *
> -   * @param   index   position of the value
> -   * @return  value of the element, if not null
> -   * @throws  NullValueException if the value is null
> -   */
> -  public <#if type.major ==
> "VarLen">byte[]<#else>${minor.javaType!type.javaType}</#if> get(int index) {
> -    assert !isNull(index);
> -    return values.get(index);
> +
> +  public int getValueCapacity(){
> +    return bits.getValueCapacity();
> +  }
> +
> +  @Override
> +  public ByteBuf[] getBuffers() {
> +    return new ByteBuf[]{bits.data, values.data};
>    }
>
> -
> -  public boolean isNull(int index) {
> -    return bits.get(index) == 0;
> +  @Override
> +  public void clear() {
> +    recordCount = 0;
> +    bits.clear();
> +    values.clear();
> +  }
> +
> +  int getBufferSize(){
> +    return values.getBufferSize() + bits.getBufferSize();
>    }
>
> -  public int isSet(int index){
> -    return bits.get(index);
> +  <#if type.major == "VarLen">
> +  @Override
> +  public FieldMetadata getMetadata() {
> +    return FieldMetadata.newBuilder()
> +             .setDef(getField().getDef())
> +             .setValueCount(recordCount)
> +             .setVarByteLength(values.getVarByteLength())
> +             .setBufferLength(getBufferSize())
> +             .build();
>    }
>
> -  /**
> -   * Allocate a new memory space for this vector.  Must be called prior
> to using the ValueVector.
> -   *
> -   * @param valueCount   The number of values which may be contained by
> this vector.
> -   */
> -  public void allocateNew(int totalBytes, ByteBuf sourceBuffer, int
> valueCount) {
> -    values.allocateNew(totalBytes, sourceBuffer, valueCount);
> +  @Override
> +  public void allocateNew(int totalBytes, int valueCount) {
> +    values.allocateNew(totalBytes, valueCount);
>      bits.allocateNew(valueCount);
> +    mutator.reset();
> +    accessor.reset();
>    }
>
>    @Override
> -  public int getAllocatedSize() {
> -    return bits.getAllocatedSize() + values.getAllocatedSize();
> +  public int load(int dataBytes, int valueCount, ByteBuf buf){
> +    clear();
> +    this.recordCount = valueCount;
> +    int loaded = bits.load(valueCount, buf);
> +
> +    // remove bits part of buffer.
> +    buf = buf.slice(loaded, buf.capacity() - loaded);
> +    loaded += values.load(dataBytes, valueCount, buf);
> +    return loaded;
>    }
> -
> -  /**
> -   * Get the size requirement (in bytes) for the given number of values.
>  Only accurate
> -   * for fixed width value vectors.
> -   */
> -  public int getTotalSizeFromCount(int valueCount) {
> -    return values.getSizeFromCount(valueCount) +
> bits.getSizeFromCount(valueCount);
> +
> +  @Override
> +  public void load(FieldMetadata metadata, ByteBuf buffer) {
> +    assert this.field.getDef().equals(metadata.getDef());
> +    int loaded = load(metadata.getVarByteLength(),
> metadata.getValueCount(), buffer);
> +    assert metadata.getBufferLength() == loaded;
>    }
>
> -  public int getSizeFromCount(int valueCount){
> -    return getTotalSizeFromCount(valueCount);
> +  @Override
> +  public int getByteCapacity(){
> +    return values.getByteCapacity();
>    }
>
> +  <#else>
>    @Override
> -  public MaterializedField getField() {
> -    return field;
> +  public FieldMetadata getMetadata() {
> +    return FieldMetadata.newBuilder()
> +             .setDef(getField().getDef())
> +             .setValueCount(recordCount)
> +             .setBufferLength(getBufferSize())
> +             .build();
>    }
> -
> +
>    @Override
> -  public ByteBuf[] getBuffers() {
> -    return new ByteBuf[]{bits.data, values.data};
> +  public void allocateNew(int valueCount) {
> +    values.allocateNew(valueCount);
> +    bits.allocateNew(valueCount);
> +    mutator.reset();
> +    accessor.reset();
>    }
> -
> -
> +
> +  @Override
> +  public int load(int valueCount, ByteBuf buf){
> +    clear();
> +    this.recordCount = valueCount;
> +    int loaded = bits.load(valueCount, buf);
> +
> +    // remove bits part of buffer.
> +    buf = buf.slice(loaded, buf.capacity() - loaded);
> +    loaded += values.load(valueCount, buf);
> +    return loaded;
> +  }
> +
>    @Override
> -  public Object getObject(int index) {
> -    return isNull(index) ? null : values.getObject(index);
> +  public void load(FieldMetadata metadata, ByteBuf buffer) {
> +    assert this.field.getDef().equals(metadata.getDef());
> +    int loaded = load(metadata.getValueCount(), buffer);
> +    assert metadata.getBufferLength() == loaded;
> +  }
> +
> +  </#if>
> +
> +  public Accessor getAccessor(){
> +    return accessor;
>    }
>
>    public Mutator getMutator(){
> -    return new Mutator();
> +    return mutator;
> +  }
> +
> +  public ${minor.class}Vector convertToRequiredVector(){
> +    ${minor.class}Vector v = new
> ${minor.class}Vector(getField().getOtherNullableVersion(), allocator);
> +    v.data = values.data;
> +    v.recordCount = this.recordCount;
> +    v.data.retain();
> +    clear();
> +    return v;
>    }
>
> -  public class Mutator implements ValueVector.Mutator{
> +
> +
> +  public final class Accessor implements ValueVector.Accessor{
>
> -    private final BitVector.Mutator bitMutator;
> -    private final ${minor.class}Vector.Mutator valueMutator;
> +    /**
> +     * Get the element at the specified position.
> +     *
> +     * @param   index   position of the value
> +     * @return  value of the element, if not null
> +     * @throws  NullValueException if the value is null
> +     */
> +    public <#if type.major ==
> "VarLen">byte[]<#else>${minor.javaType!type.javaType}</#if> get(int index) {
> +      assert !isNull(index);
> +      return values.getAccessor().get(index);
> +    }
> +
> +    public boolean isNull(int index) {
> +      return isSet(index) == 0;
> +    }
> +
> +    public int isSet(int index){
> +      return bits.getAccessor().get(index);
> +    }
> +
> +    @Override
> +    public Object getObject(int index) {
> +      return isNull(index) ? null : values.getAccessor().getObject(index);
> +    }
> +
> +    public int getRecordCount(){
> +      return recordCount;
> +    }
> +
> +    public void reset(){}
> +  }
> +
> +  public final class Mutator implements ValueVector.Mutator{
> +
> +    private int setCount;
>
>      private Mutator(){
> -      bitMutator = bits.getMutator();
> -      valueMutator = values.getMutator();
>      }
>
>      /**
> @@ -123,28 +213,30 @@ public final class Nullable${minor.class}Vector
> extends ValueVector {
>       * @param bytes   array of bytes to write
>       */
>      public void set(int index, <#if type.major == "VarLen">byte[]<#elseif
> (type.width < 4)>int<#else>${minor.javaType!type.javaType}</#if> value) {
> -      setNotNull(index);
> -      valueMutator.set(index, value);
> -    }
> -
> -    public void setNull(int index) {
> -      bitMutator.set(index, 0);
> +      setCount++;
> +      bits.getMutator().set(index, 1);
> +      values.getMutator().set(index, value);
>      }
>
> -    private void setNotNull(int index) {
> -      bitMutator.set(index, 1);
> +    public void setValueCount(int recordCount) {
> +      assert recordCount >= 0;
> +      Nullable${minor.class}Vector.this.recordCount = recordCount;
> +      values.getMutator().setValueCount(recordCount);
> +      bits.getMutator().setValueCount(recordCount);
>      }
>
> -    @Override
> -    public void setRecordCount(int recordCount) {
> -      Nullable${minor.class}Vector.this.setRecordCount(recordCount);
> -      bits.setRecordCount(recordCount);
> +    public boolean noNulls(){
> +      return recordCount == setCount;
>      }
>
>      public void randomizeData(){
>        throw new UnsupportedOperationException();
>      }
>
> +    public void reset(){
> +      setCount = 0;
> +    }
> +
>    }
>  }
>  </#list>
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/9ca9eb9b/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/RepeatedValueVectors.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/RepeatedValueVectors.java
> b/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/RepeatedValueVectors.java
> index 4acc4cc..363e4c8 100644
> ---
> a/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/RepeatedValueVectors.java
> +++
> b/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/RepeatedValueVectors.java
> @@ -7,12 +7,18 @@ import org.apache.drill.exec.vector.UInt4Vector;
>  <@pp.changeOutputFile name="Repeated${minor.class}Vector.java" />
>  package org.apache.drill.exec.vector;
>
> +
> +
> +
> +
> +
>  import static com.google.common.base.Preconditions.checkArgument;
>  import static com.google.common.base.Preconditions.checkState;
>  import io.netty.buffer.ByteBuf;
>
>  import java.io.Closeable;
>  import java.util.Random;
> +import java.util.Vector;
>
>  import org.apache.drill.exec.memory.BufferAllocator;
>  import org.apache.drill.exec.proto.SchemaDefProtos;
> @@ -30,100 +36,216 @@ import
> org.apache.drill.exec.record.MaterializedField;
>   * NB: this class is automatically generated from ValueVectorTypes.tdd
> using FreeMarker.
>   */
>
> - public final class Repeated${minor.class}Vector extends ValueVector {
> -
> -  private final UInt2Vector countVector;    // number of repeated
> elements in each record
> -  private final UInt4Vector offsetVector;   // offsets to start of each
> record
> -  private final ${minor.class}Vector valuesVector;
> + public final class Repeated${minor.class}Vector extends BaseValueVector
> implements Repeated<#if type.major ==
> "VarLen">VariableWidth<#else>FixedWidth</#if>Vector {
>
> +  private MaterializedField field;
> +
> +  private int parentValueCount;
> +  private int childValueCount;
> +
> +  private final UInt2Vector counts;    // number of repeated elements in
> each record
> +  private final UInt4Vector offsets;   // offsets to start of each record
> +  private final ${minor.class}Vector values;
> +  private final Mutator mutator = new Mutator();
> +  private final Accessor accessor = new Accessor();
> +
> +
>    public Repeated${minor.class}Vector(MaterializedField field,
> BufferAllocator allocator) {
>      super(field, allocator);
> -    this.countVector = new UInt2Vector(null, allocator);
> -    this.offsetVector = new UInt4Vector(null, allocator);
> -    this.valuesVector = new ${minor.class}Vector(null, allocator);
> +    this.counts = new UInt2Vector(null, allocator);
> +    this.offsets = new UInt4Vector(null, allocator);
> +    this.values = new ${minor.class}Vector(null, allocator);
>    }
>
> -  public void allocateNew(int totalBytes, ByteBuf sourceBuffer, int
> valueCount) {
> -    super.allocateNew(totalBytes, sourceBuffer, valueCount);
> -    countVector.allocateNew(valueCount);
> -    offsetVector.allocateNew(valueCount);
> +  public int getValueCapacity(){
> +    return values.getValueCapacity();
> +  }
> +
> +  int getBufferSize(){
> +    return counts.getBufferSize() + offsets.getBufferSize() +
> values.getBufferSize();
> +  }
> +
> +  <#if type.major == "VarLen">
> +  @Override
> +  public FieldMetadata getMetadata() {
> +    return FieldMetadata.newBuilder()
> +             .setDef(getField().getDef())
> +             .setGroupCount(this.parentValueCount)
> +             .setValueCount(this.childValueCount)
> +             .setVarByteLength(values.getVarByteLength())
> +             .setBufferLength(getBufferSize())
> +             .build();
> +  }
> +
> +  public void allocateNew(int totalBytes, int parentValueCount, int
> childValueCount) {
> +    counts.allocateNew(parentValueCount);
> +    offsets.allocateNew(parentValueCount);
> +    values.allocateNew(totalBytes, childValueCount);
> +    mutator.reset();
> +    accessor.reset();
>    }
>
> +
>
> -  /**
> -   * Get a value for the given record.  Each element in the repeated
> field is accessed by
> -   * the positionIndex param.
> -   *
> -   * @param  index           record containing the repeated field
> -   * @param  positionIndex   position within the repeated field
> -   * @return element at the given position in the given record
> -   */
> -  public <#if type.major == "VarLen">byte[]
> -         <#else>${minor.javaType!type.javaType}
> -         </#if> get(int index, int positionIndex) {
> -
> -    assert positionIndex < countVector.get(index);
> -    return valuesVector.get(offsetVector.get(index) + positionIndex);
> +
> +  @Override
> +  public int load(int dataBytes, int parentValueCount, int
> childValueCount, ByteBuf buf){
> +    clear();
> +    this.parentValueCount = parentValueCount;
> +    this.childValueCount = childValueCount;
> +    int loaded = 0;
> +    loaded += counts.load(parentValueCount, buf);
> +    loaded += offsets.load(parentValueCount, buf.slice(loaded,
> buf.capacity() - loaded));
> +    loaded += values.load(dataBytes, childValueCount, buf.slice(loaded,
> buf.capacity() - loaded));
> +    return loaded;
>    }
> -
> -  public MaterializedField getField() {
> -    return field;
> +
> +  @Override
> +  public void load(FieldMetadata metadata, ByteBuf buffer) {
> +    assert this.field.getDef().equals(metadata.getDef());
> +    int loaded = load(metadata.getVarByteLength(),
> metadata.getGroupCount(), metadata.getValueCount(), buffer);
> +    assert metadata.getBufferLength() == loaded;
> +  }
> +
> +  public int getByteCapacity(){
> +    return values.getByteCapacity();
>    }
>
> -  /**
> -   * Get the size requirement (in bytes) for the given number of values.
>  Only accurate
> -   * for fixed width value vectors.
> -   */
> -  public int getTotalSizeFromCount(int valueCount) {
> -    return valuesVector.getSizeFromCount(valueCount) +
> -           countVector.getSizeFromCount(valueCount) +
> -           offsetVector.getSizeFromCount(valueCount);
> +  <#else>
> +
> +  @Override
> +  public FieldMetadata getMetadata() {
> +    return FieldMetadata.newBuilder()
> +             .setDef(getField().getDef())
> +             .setGroupCount(this.parentValueCount)
> +             .setValueCount(this.childValueCount)
> +             .setBufferLength(getBufferSize())
> +             .build();
>    }
>
> -  public int getSizeFromCount(int valueCount){
> -    return getTotalSizeFromCount(valueCount);
> +  public void allocateNew(int parentValueCount, int childValueCount) {
> +    clear();
> +    values.allocateNew(childValueCount);
> +    counts.allocateNew(parentValueCount);
> +    offsets.allocateNew(parentValueCount);
> +    mutator.reset();
> +    accessor.reset();
>    }
> -
> -  /**
> -   * Get the explicitly specified size of the allocated buffer, if
> available.  Otherwise
> -   * calculate the size based on width and record count.
> -   */
> -  public int getAllocatedSize() {
> -    return valuesVector.getAllocatedSize() +
> -           countVector.getAllocatedSize() +
> -           offsetVector.getAllocatedSize();
> +
> +  public int load(int parentValueCount, int childValueCount, ByteBuf buf){
> +    clear();
> +    this.parentValueCount = parentValueCount;
> +    this.childValueCount = childValueCount;
> +    int loaded = 0;
> +    loaded += counts.load(parentValueCount, buf);
> +    loaded += offsets.load(parentValueCount, buf.slice(loaded,
> buf.capacity() - loaded));
> +    loaded += values.load(childValueCount, buf.slice(loaded,
> buf.capacity() - loaded));
> +    return loaded;
>    }
> -
> -  /**
> -   * Get the elements at the given index.
> -   */
> -  public int getCount(int index) {
> -    return countVector.get(index);
> +
> +  @Override
> +  public void load(FieldMetadata metadata, ByteBuf buffer) {
> +    assert this.field.getDef().equals(metadata.getDef());
> +    int loaded = load(metadata.getGroupCount(), metadata.getValueCount(),
> buffer);
> +    assert metadata.getBufferLength() == loaded;
>    }
> +  </#if>
> +
> +//  /**
> +//   * Get the size requirement (in bytes) for the given number of
> values.  Only accurate
> +//   * for fixed width value vectors.
> +//   */
> +//  public int getTotalSizeFromCount(int valueCount) {
> +//    return values.getSizeFromCount(valueCount) +
> +//           counts.getSizeFromCount(valueCount) +
> +//           offsets.getSizeFromCount(valueCount);
> +//  }
> +//
> +//  public int getSizeFromCount(int valueCount){
> +//    return getTotalSizeFromCount(valueCount);
> +//  }
> +
> +//  /**
> +//   * Get the explicitly specified size of the allocated buffer, if
> available.  Otherwise
> +//   * calculate the size based on width and record count.
> +//   */
> +//  public int getAllocatedSize() {
> +//    return values.getAllocatedSize() +
> +//           counts.getAllocatedSize() +
> +//           offsets.getAllocatedSize();
> +//  }
> +
> +
>
>    public ByteBuf[] getBuffers() {
> -    return new ByteBuf[]{countVector.data, offsetVector.data, data};
> +    return new ByteBuf[]{counts.data, offsets.data, values.data};
>    }
>
> -  public Object getObject(int index) {
> -    return data.slice(index, getSizeFromCount(countVector.get(index)));
> +  public void clear(){
> +    counts.clear();
> +    offsets.clear();
> +    values.clear();
> +    parentValueCount = 0;
> +    childValueCount = 0;
>    }
>
>    public Mutator getMutator(){
> -    return new Mutator();
> +    return mutator;
> +  }
> +
> +  public Accessor getAccessor(){
> +    return accessor;
>    }
>
> -  public class Mutator implements ValueVector.Mutator{
> +  public final class Accessor implements ValueVector.Accessor{
> +    /**
> +     * Get the elements at the given index.
> +     */
> +    public int getCount(int index) {
> +      return counts.getAccessor().get(index);
> +    }
> +
> +    public Object getObject(int index) {
> +      throw new UnsupportedOperationException();
> +    }
> +
> +    /**
> +     * Get a value for the given record.  Each element in the repeated
> field is accessed by
> +     * the positionIndex param.
> +     *
> +     * @param  index           record containing the repeated field
> +     * @param  positionIndex   position within the repeated field
> +     * @return element at the given position in the given record
> +     */
> +    public <#if type.major == "VarLen">byte[]
> +           <#else>${minor.javaType!type.javaType}
> +           </#if> get(int index, int positionIndex) {
>
> +      assert positionIndex < counts.getAccessor().get(index);
> +      return values.getAccessor().get(offsets.getAccessor().get(index) +
> positionIndex);
> +    }
> +
> +    public MaterializedField getField() {
> +      return field;
> +    }
> +
> +    public int getGroupCount(){
> +      return parentValueCount;
> +    }
> +
> +    public int getValueCount(){
> +      return childValueCount;
> +    }
>
> -    private final UInt2Vector.Mutator countMutator;
> -    private final ${minor.class}Vector.Mutator valuesMutator;
> -    private final UInt4Vector.Mutator offsetMutator;
> +    public void reset(){
> +
> +    }
> +  }
> +
> +  public final class Mutator implements ValueVector.Mutator{
> +
>
>      private Mutator(){
> -      this.countMutator = countVector.getMutator();
> -      this.offsetMutator = offsetVector.getMutator();
> -      this.valuesMutator = valuesVector.getMutator();
>      }
>
>      /**
> @@ -137,21 +259,28 @@ import
> org.apache.drill.exec.record.MaterializedField;
>                                 <#elseif type.major == "VarLen"> byte[]
>                                 <#else> int
>                                 </#if> value) {
> -      countMutator.set(index, countVector.get(index) + 1);
> -      offsetMutator.set(index, offsetVector.get(index - 1) +
> countVector.get(index-1));
> -      valuesMutator.set(offsetVector.get(index), value);
> +      counts.getMutator().set(index, counts.getAccessor().get(index) + 1);
> +      offsets.getMutator().set(index, offsets.getAccessor().get(index -
> 1) + counts.getAccessor().get(index-1));
> +      values.getMutator().set(offsets.getAccessor().get(index), value);
>      }
> +
>
> -    public void setRecordCount(int recordCount) {
> -      valuesMutator.setRecordCount(recordCount);
> -      offsetMutator.setRecordCount(recordCount);
> -      countMutator.setRecordCount(recordCount);
> +    public void setGroupAndValueCount(int groupCount, int valueCount) {
> +      parentValueCount = groupCount;
> +      childValueCount = valueCount;
> +      counts.getMutator().setValueCount(groupCount);
> +      offsets.getMutator().setValueCount(groupCount);
> +      values.getMutator().setValueCount(valueCount);
>      }
>
>      public void randomizeData(){
>        throw new UnsupportedOperationException();
>      }
>
> +    public void reset(){
> +
> +    }
> +
>    }
>  }
>  </#list>
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/9ca9eb9b/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/VariableLengthVectors.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/VariableLengthVectors.java
> b/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/VariableLengthVectors.java
> index 954836a..c615258 100644
> ---
> a/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/VariableLengthVectors.java
> +++
> b/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/VariableLengthVectors.java
> @@ -11,6 +11,7 @@ import static
> com.google.common.base.Preconditions.checkState;
>  import io.netty.buffer.ByteBuf;
>
>  import java.io.Closeable;
> +import java.nio.ByteBuffer;
>  import java.util.Random;
>
>  import org.apache.drill.exec.memory.BufferAllocator;
> @@ -18,10 +19,11 @@ import org.apache.drill.exec.proto.SchemaDefProtos;
>  import org.apache.drill.exec.proto.UserBitShared.FieldMetadata;
>  import org.apache.drill.exec.record.DeadBuf;
>  import org.apache.drill.exec.record.MaterializedField;
> +import org.apache.drill.exec.vector.ByteHolder;
>
>  /**
>   * ${minor.class}Vector implements a vector of variable width values.
>  Elements in the vector
> - * are accessed by position from the logical start of the vector.  A
> fixed width lengthVector
> + * are accessed by position from the logical start of the vector.  A
> fixed width offsetVector
>   * is used to convert an element's position to it's offset from the start
> of the (0-based)
>   * ByteBuf.  Size is inferred by adjacent elements.
>   *   The width of each element is ${type.width} byte(s)
> @@ -30,81 +32,125 @@ import org.apache.drill.exec.record.MaterializedField;
>   * NB: this class is automatically generated from ValueVectorTypes.tdd
> using FreeMarker.
>   */
>  @SuppressWarnings("unused")
> -public final class ${minor.class}Vector extends ValueVector {
> +public final class ${minor.class}Vector extends BaseDataValueVector
> implements VariableWidthVector{
>    static final org.slf4j.Logger logger =
> org.slf4j.LoggerFactory.getLogger(${minor.class}Vector.class);
>
> -  private final UInt${type.width}Vector lengthVector;
> -  private final UInt${type.width}Vector.Mutator lengthVectorMutator;
> -
> +  private final UInt${type.width}Vector offsetVector;
> +  private final Accessor accessor = new Accessor();
> +  private final Mutator mutator = new Mutator();
> +
>    public ${minor.class}Vector(MaterializedField field, BufferAllocator
> allocator) {
>      super(field, allocator);
> -    this.lengthVector = new UInt${type.width}Vector(null, allocator);
> -    this.lengthVectorMutator = lengthVector.getMutator();
> +    this.offsetVector = new UInt${type.width}Vector(null, allocator);
>    }
>
> -  public byte[] get(int index) {
> -    checkArgument(index >= 0);
> -    int startIdx = 0;
> -    int size = 0;
> -    if (index == 0) {
> -      size = lengthVector.get(1);
> -    } else {
> -      startIdx = lengthVector.get(index);
> -      size = lengthVector.get(index + 1) - startIdx;
> -    }
> -    checkState(size >= 0);
> -    byte[] dst = new byte[size];
> -    data.getBytes(startIdx, dst, 0, size);
> -    return dst;
> -  }
>
> -  @Override
> -  public int getAllocatedSize() {
> -    return lengthVector.getAllocatedSize() + totalBytes;
> +  int getSizeFromCount(int valueCount) {
> +    return valueCount * ${type.width};
>    }
> -
> +
> +  public int getValueCapacity(){
> +    return offsetVector.getValueCapacity();
> +  }
> +
> +  public int getByteCapacity(){
> +    return data.capacity();
> +  }
> +
>    /**
> -   * Get the size requirement (in bytes) for the given number of values.
>  Only accurate
> -   * for fixed width value vectors.
> +   * Return the number of bytes contained in the current var len byte
> vector.
> +   * @return
>     */
> -  public int getSizeFromCount(int valueCount) {
> -    return valueCount * ${type.width};
> +  public int getVarByteLength(){
> +    return offsetVector.getAccessor().get(recordCount);
>    }
> -
> +
>    @Override
> -  protected void clear() {
> -    super.clear();
> -    lengthVector.clear();
> +  public FieldMetadata getMetadata() {
> +    int len = recordCount * ${type.width} + getVarByteLength();
> +    return FieldMetadata.newBuilder()
> +             .setDef(getField().getDef())
> +             .setValueCount(recordCount)
> +             .setVarByteLength(getVarByteLength())
> +             .setBufferLength(len)
> +             .build();
>    }
>
> -  /**
> -   * Allocate a new memory space for this vector.  Must be called prior
> to using the ValueVector.
> -   *
> -   * @param valueCount
> -   *          The number of values which can be contained within this
> vector.
> -   */
> -  public void allocateNew(int totalBytes, ByteBuf sourceBuffer, int
> valueCount) {
> -    super.allocateNew(totalBytes, sourceBuffer, valueCount);
> -    lengthVector.allocateNew(valueCount);
> +  public int load(int dataBytes, int valueCount, ByteBuf buf){
> +    this.recordCount = valueCount;
> +    int loaded = offsetVector.load(valueCount+1, buf);
> +    data = buf.slice(loaded, dataBytes);
> +    data.retain();
> +    return loaded + dataBytes;
> +  }
> +
> +  @Override
> +  public void load(FieldMetadata metadata, ByteBuf buffer) {
> +    assert this.field.getDef().equals(metadata.getDef());
> +    int loaded = load(metadata.getVarByteLength(),
> metadata.getValueCount(), buffer);
> +    assert metadata.getBufferLength() == loaded;
> +  }
> +
> +  @Override
> +  public void clear() {
> +    super.clear();
> +    offsetVector.clear();
>    }
>
>    @Override
>    public ByteBuf[] getBuffers() {
> -    return new ByteBuf[]{lengthVector.data, data};
> +    return new ByteBuf[]{offsetVector.data, this.data};
>    }
> -
> -  public Object getObject(int index) {
> -    return get(index);
> +
> +  public void allocateNew(int totalBytes, int valueCount) {
> +    clear();
> +    assert totalBytes >= 0;
> +    data = allocator.buffer(totalBytes);
> +    data.retain();
> +    data.readerIndex(0);
> +    offsetVector.allocateNew(valueCount+1);
>    }
>
> +  public Accessor getAccessor(){
> +    return accessor;
> +  }
> +
>    public Mutator getMutator() {
> -    return new Mutator();
> +    return mutator;
>    }
>
> +  public final class Accessor extends BaseValueVector.BaseAccessor{
> +
> +    public byte[] get(int index) {
> +      assert index >= 0;
> +      int startIdx = offsetVector.getAccessor().get(index);
> +      int length = offsetVector.getAccessor().get(index + 1) - startIdx;
> +      assert length >= 0;
> +      byte[] dst = new byte[length];
> +      data.getBytes(startIdx, dst, 0, length);
> +      return dst;
> +    }
> +
> +    public void get(int index, ByteHolder holder){
> +      assert index >= 0;
> +      holder.start = offsetVector.getAccessor().get(index);
> +      holder.length = offsetVector.getAccessor().get(index + 1) -
> holder.start;
> +      assert holder.length >= 0;
> +      holder.buffer = offsetVector.data;
> +    }
> +
> +    public Object getObject(int index) {
> +      return get(index);
> +    }
> +
> +    public int getRecordCount() {
> +      return recordCount;
> +    }
> +  }
>
>    /**
>     * Mutable${minor.class} implements a vector of variable width values.
>  Elements in the vector
> -   * are accessed by position from the logical start of the vector.  A
> fixed width lengthVector
> +   * are accessed by position from the logical start of the vector.  A
> fixed width offsetVector
>     * is used to convert an element's position to it's offset from the
> start of the (0-based)
>     * ByteBuf.  Size is inferred by adjacent elements.
>     *   The width of each element is ${type.width} byte(s)
> @@ -112,7 +158,7 @@ public final class ${minor.class}Vector extends
> ValueVector {
>     *
>     * NB: this class is automatically generated from ValueVectorTypes.tdd
> using FreeMarker.
>     */
> -  public class Mutator implements ValueVector.Mutator{
> +  public final class Mutator extends BaseValueVector.BaseMutator{
>
>      /**
>       * Set the variable length element at the specified index to the
> supplied byte array.
> @@ -121,23 +167,24 @@ public final class ${minor.class}Vector extends
> ValueVector {
>       * @param bytes   array of bytes to write
>       */
>      public void set(int index, byte[] bytes) {
> -      checkArgument(index >= 0);
> -      if (index == 0) {
> -        lengthVectorMutator.set(0, 0);
> -        lengthVectorMutator.set(1, bytes.length);
> -        data.setBytes(0, bytes);
> -      } else {
> -        int currentOffset = lengthVector.get(index);
> -        // set the end offset of the buffer
> -        lengthVectorMutator.set(index + 1, currentOffset + bytes.length);
> -        data.setBytes(currentOffset, bytes);
> -      }
> +      assert index >= 0;
> +      int currentOffset = offsetVector.getAccessor().get(index);
> +      offsetVector.getMutator().set(index + 1, currentOffset +
> bytes.length);
> +      data.setBytes(currentOffset, bytes);
>      }
>
> -    @Override
> -    public void setRecordCount(int recordCount) {
> -      ${minor.class}Vector.this.setRecordCount(recordCount);
> -      lengthVector.setRecordCount(recordCount);
> +    public void set(int index, int start, int length, ByteBuf buffer){
> +      assert index >= 0;
> +      int currentOffset = offsetVector.getAccessor().get(index);
> +      offsetVector.getMutator().set(index + 1, currentOffset + length);
> +      ByteBuf bb = buffer.slice(start, length);
> +      data.setBytes(currentOffset, bb);
> +    }
> +
> +    public void setValueCount(int recordCount) {
> +      ${minor.class}Vector.this.recordCount = recordCount;
> +      data.writerIndex(recordCount * ${type.width});
> +      offsetVector.getMutator().setValueCount(recordCount+1);
>      }
>
>      @Override
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/9ca9eb9b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockRecordReader.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockRecordReader.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockRecordReader.java
> index cd3371d..0f4619c 100644
> ---
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockRecordReader.java
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockRecordReader.java
> @@ -28,8 +28,11 @@ import
> org.apache.drill.exec.proto.SchemaDefProtos.DataMode;
>  import org.apache.drill.exec.proto.SchemaDefProtos.MajorType;
>  import org.apache.drill.exec.record.MaterializedField;
>  import org.apache.drill.exec.store.RecordReader;
> +import org.apache.drill.exec.vector.FixedWidthVector;
> +import org.apache.drill.exec.vector.NonRepeatedMutator;
>  import org.apache.drill.exec.vector.TypeHelper;
>  import org.apache.drill.exec.vector.ValueVector;
> +import org.apache.drill.exec.vector.VariableWidthVector;
>
>  public class MockRecordReader implements RecordReader {
>    static final org.slf4j.Logger logger =
> org.slf4j.LoggerFactory.getLogger(MockRecordReader.class);
> @@ -39,6 +42,7 @@ public class MockRecordReader implements RecordReader {
>    private FragmentContext context;
>    private ValueVector[] valueVectors;
>    private int recordsRead;
> +  private int batchRecordCount;
>
>    public MockRecordReader(FragmentContext context, MockScanEntry config) {
>      this.context = context;
> @@ -60,7 +64,14 @@ public class MockRecordReader implements RecordReader {
>      MaterializedField f = MaterializedField.create(new SchemaPath(name),
> fieldId, 0, type);
>      ValueVector v;
>      v = TypeHelper.getNewVector(f, context.getAllocator());
> -    v.allocateNew(length);
> +    if(v instanceof FixedWidthVector){
> +      ((FixedWidthVector)v).allocateNew(length);
> +    }else if(v instanceof VariableWidthVector){
> +      ((VariableWidthVector)v).allocateNew(50*length, length);
> +    }else{
> +      throw new UnsupportedOperationException(String.format("Unable to
> get allocate vector %s", v.getClass().getName()));
> +    }
> +
>      return v;
>
>    }
> @@ -71,7 +82,7 @@ public class MockRecordReader implements RecordReader {
>        this.output = output;
>        int estimateRowSize = getEstimatedRecordSize(config.getTypes());
>        valueVectors = new ValueVector[config.getTypes().length];
> -      int batchRecordCount = 250000 / estimateRowSize;
> +      batchRecordCount = 250000 / estimateRowSize;
>
>        for (int i = 0; i < config.getTypes().length; i++) {
>          valueVectors[i] = getVector(i, config.getTypes()[i].getName(),
> config.getTypes()[i].getMajorType(), batchRecordCount);
> @@ -86,13 +97,29 @@ public class MockRecordReader implements RecordReader {
>
>    @Override
>    public int next() {
> -    int recordSetSize = Math.min(valueVectors[0].capacity(),
> this.config.getRecords()- recordsRead);
> +
> +    int recordSetSize = Math.min(batchRecordCount,
> this.config.getRecords()- recordsRead);
> +
>      recordsRead += recordSetSize;
>      for(ValueVector v : valueVectors){
> +      if(v instanceof FixedWidthVector){
> +        ((FixedWidthVector)v).allocateNew(recordSetSize);
> +      }else if(v instanceof VariableWidthVector){
> +        ((VariableWidthVector)v).allocateNew(50*recordSetSize,
> recordSetSize);
> +      }else{
> +        throw new UnsupportedOperationException();
> +      }
> +
>        logger.debug("MockRecordReader:  Generating random data for VV of
> type " + v.getClass().getName());
> -      v.randomizeData();
> +      ValueVector.Mutator m = v.getMutator();
> +      m.randomizeData();
> +
> +      if(m instanceof NonRepeatedMutator){
> +        ((NonRepeatedMutator)m).setValueCount(recordSetSize);
> +      }else{
> +        throw new UnsupportedOperationException();
> +      }
>
> -      v.getMutator().setRecordCount(recordSetSize);
>      }
>      return recordSetSize;
>    }
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/9ca9eb9b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/MaterializedField.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/MaterializedField.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/MaterializedField.java
> index d1858f1..05fb576 100644
> ---
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/MaterializedField.java
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/MaterializedField.java
> @@ -105,6 +105,22 @@ public class MaterializedField implements
> Comparable<MaterializedField> {
>    public DataMode getDataMode() {
>      return def.getMajorType().getMode();
>    }
> +
> +  public MaterializedField getOtherNullableVersion(){
> +    MajorType mt = def.getMajorType();
> +    DataMode newDataMode = null;
> +    switch(mt.getMode()){
> +    case OPTIONAL:
> +      newDataMode = DataMode.REQUIRED;
> +      break;
> +    case REQUIRED:
> +      newDataMode = DataMode.OPTIONAL;
> +      break;
> +    default:
> +      throw new UnsupportedOperationException();
> +    }
> +    return new
> MaterializedField(def.toBuilder().setMajorType(mt.toBuilder().setMode(newDataMode).build()).build());
> +  }
>
>    public boolean matches(SchemaPath path) {
>      Iterator<NamePart> iter = def.getNameList().iterator();
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/9ca9eb9b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatchLoader.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatchLoader.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatchLoader.java
> index be43026..a2dbd81 100644
> ---
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatchLoader.java
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatchLoader.java
> @@ -52,7 +52,7 @@ public class RecordBatchLoader implements
> Iterable<IntObjectCursor<ValueVector>>
>     * @param def
>     *          The definition for the record batch.
>     * @param buf
> -   *          The buffer that holds the data ssociated with the record
> batch
> +   *          The buffer that holds the data associated with the record
> batch
>     * @return Whether or not the schema changed since the previous load.
>     * @throws SchemaChangeException
>     */
> @@ -71,7 +71,8 @@ public class RecordBatchLoader implements
> Iterable<IntObjectCursor<ValueVector>>
>        ValueVector v = vectors.remove(fieldDef.getFieldId());
>        if (v != null) {
>          if (v.getField().getDef().equals(fieldDef)) {
> -          v.allocateNew(fmd.getBufferLength(), buf.slice(bufOffset,
> fmd.getBufferLength()), recordCount);
> +          ValueVector.Mutator m = v.getMutator();
> +          v.load(fmd, buf.slice(bufOffset, fmd.getBufferLength()));
>            newVectors.put(fieldDef.getFieldId(), v);
>            continue;
>          } else {
> @@ -83,7 +84,7 @@ public class RecordBatchLoader implements
> Iterable<IntObjectCursor<ValueVector>>
>        schemaChanged = true;
>        MaterializedField m = new MaterializedField(fieldDef);
>        v = TypeHelper.getNewVector(m, allocator);
> -      v.allocateNew(fmd.getBufferLength(), buf.slice(bufOffset,
> fmd.getBufferLength()), recordCount);
> +      v.load(fmd, buf.slice(bufOffset, fmd.getBufferLength()));
>        newVectors.put(fieldDef.getFieldId(), v);
>      }
>
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/9ca9eb9b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/WritableBatch.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/WritableBatch.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/WritableBatch.java
> index a367b6d..4b97768 100644
> ---
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/WritableBatch.java
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/WritableBatch.java
> @@ -93,8 +93,7 @@ public class WritableBatch {
>          buffers.add(b);
>          b.retain();
>        }
> -      // allocate new buffer to release hold on old buffer.
> -      value.allocateNew(value.capacity());
> +      value.clear();
>      }
>
>      public WritableBatch get(){
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/9ca9eb9b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/JSONRecordReader.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/JSONRecordReader.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/JSONRecordReader.java
> index e637518..07ae20a 100644
> ---
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/JSONRecordReader.java
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/JSONRecordReader.java
> @@ -422,9 +422,11 @@ public class JSONRecordReader implements RecordReader
> {
>              SchemaDefProtos.MajorType type = field.getFieldType();
>              int fieldId = field.getFieldId();
>              MaterializedField f = MaterializedField.create(new
> SchemaPath(field.getFieldName()), fieldId, parentFieldId, type);
> +
>              ValueVector v = TypeHelper.getNewVector(f, allocator);
> -            v.allocateNew(batchSize);
>              VectorHolder holder = new VectorHolder(batchSize, v);
> +            holder.allocateNew(batchSize);
> +
>              valueVectorMap.put(fieldId, holder);
>              outputMutator.addField(fieldId, v);
>              return holder;
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/9ca9eb9b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/VectorHolder.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/VectorHolder.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/VectorHolder.java
> index fa0cbd5..d594b9e 100644
> ---
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/VectorHolder.java
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/VectorHolder.java
> @@ -18,16 +18,20 @@
>
>  package org.apache.drill.exec.store;
>
> +import org.apache.drill.exec.vector.FixedWidthVector;
>  import org.apache.drill.exec.vector.ValueVector;
> +import org.apache.drill.exec.vector.VariableWidthVector;
>
>  public class VectorHolder {
>      private int length;
>      private ValueVector vector;
> +    private ValueVector.Mutator mutator;
>      private int currentLength;
>
>      VectorHolder(int length, ValueVector vector) {
>          this.length = length;
>          this.vector = vector;
> +        this.mutator = vector.getMutator();
>      }
>
>      public ValueVector getValueVector() {
> @@ -51,6 +55,21 @@ public class VectorHolder {
>
>      public void reset() {
>          currentLength = 0;
> -        vector.allocateNew(length);
> +        allocateNew(length);
> +
> +    }
> +
> +    public void allocateNew(int valueLength){
> +      if(vector instanceof FixedWidthVector){
> +        ((FixedWidthVector)vector).allocateNew(valueLength);
> +      }else if(vector instanceof VariableWidthVector){
> +        ((VariableWidthVector)vector).allocateNew(valueLength * 10,
> valueLength);
> +      }else{
> +        throw new UnsupportedOperationException();
> +      }
> +    }
> +
> +    public ValueVector.Mutator getMutator(){
> +      return mutator;
>      }
>  }
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/9ca9eb9b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/BaseDataValueVector.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/BaseDataValueVector.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/BaseDataValueVector.java
> new file mode 100644
> index 0000000..dd2b504
> --- /dev/null
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/BaseDataValueVector.java
> @@ -0,0 +1,47 @@
> +package org.apache.drill.exec.vector;
> +
> +import io.netty.buffer.ByteBuf;
> +
> +import org.apache.drill.exec.memory.BufferAllocator;
> +import org.apache.drill.exec.proto.UserBitShared.FieldMetadata;
> +import org.apache.drill.exec.record.DeadBuf;
> +import org.apache.drill.exec.record.MaterializedField;
> +
> +abstract class BaseDataValueVector extends BaseValueVector{
> +
> +  protected ByteBuf data = DeadBuf.DEAD_BUFFER;
> +  protected int recordCount;
> +
> +  public BaseDataValueVector(MaterializedField field, BufferAllocator
> allocator) {
> +    super(field, allocator);
> +
> +  }
> +
> +  /**
> +   * Release the underlying ByteBuf and reset the ValueVector
> +   */
> +  @Override
> +  public void clear() {
> +    if (data != DeadBuf.DEAD_BUFFER) {
> +      data.release();
> +      data = DeadBuf.DEAD_BUFFER;
> +      recordCount = 0;
> +    }
> +  }
> +
> +  @Override
> +  public ByteBuf[] getBuffers(){
> +    return new ByteBuf[]{data};
> +  }
> +
> +  public int getBufferSize() {
> +    return data.writerIndex();
> +  }
> +
> +  @Override
> +  public FieldMetadata getMetadata() {
> +    return null;
> +  }
> +
> +
> +}
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/9ca9eb9b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/BaseValueVector.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/BaseValueVector.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/BaseValueVector.java
> new file mode 100644
> index 0000000..a8678f5
> --- /dev/null
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/BaseValueVector.java
> @@ -0,0 +1,38 @@
> +package org.apache.drill.exec.vector;
> +
> +import org.apache.drill.exec.memory.BufferAllocator;
> +import org.apache.drill.exec.record.MaterializedField;
> +
> +abstract class BaseValueVector implements ValueVector{
> +  static final org.slf4j.Logger logger =
> org.slf4j.LoggerFactory.getLogger(BaseValueVector.class);
> +
> +  protected final BufferAllocator allocator;
> +  protected final MaterializedField field;
> +
> +  BaseValueVector(MaterializedField field, BufferAllocator allocator) {
> +    this.allocator = allocator;
> +    this.field = field;
> +  }
> +
> +  @Override
> +  public void close() {
> +    clear();
> +  }
> +
> +  @Override
> +  public MaterializedField getField() {
> +    return field;
> +  }
> +
> +  abstract class BaseAccessor implements ValueVector.Accessor{
> +
> +
> +    public void reset(){}
> +  }
> +
> +  abstract class BaseMutator implements NonRepeatedMutator{
> +    public void reset(){}
> +  }
> +
> +}
> +
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/9ca9eb9b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/BitVector.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/BitVector.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/BitVector.java
> index d18a29d..9d247f5 100644
> ---
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/BitVector.java
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/BitVector.java
> @@ -1,8 +1,11 @@
>  package org.apache.drill.exec.vector;
>
> +import io.netty.buffer.ByteBuf;
> +
>  import java.util.Random;
>
>  import org.apache.drill.exec.memory.BufferAllocator;
> +import org.apache.drill.exec.proto.UserBitShared.FieldMetadata;
>  import org.apache.drill.exec.record.DeadBuf;
>  import org.apache.drill.exec.record.MaterializedField;
>  /**
> @@ -13,64 +16,93 @@ import org.apache.drill.exec.record.MaterializedField;
>   *
>   * NB: this class is automatically generated from ValueVectorTypes.tdd
> using FreeMarker.
>   */
> -public final class BitVector extends ValueVector {
> +public final class BitVector extends BaseDataValueVector implements
> FixedWidthVector{
>    static final org.slf4j.Logger logger =
> org.slf4j.LoggerFactory.getLogger(BitVector.class);
>
> +  private int valueCapacity;
> +
>    public BitVector(MaterializedField field, BufferAllocator allocator) {
>      super(field, allocator);
>    }
>
> +  private int getSizeFromCount(int valueCount) {
> +    return (int) Math.ceil(valueCount / 8);
> +  }
> +
>    /**
> -   * Get the byte holding the desired bit, then mask all other bits.  Iff
> the result is 0, the
> -   * bit was not set.
> +   * Allocate a new memory space for this vector.  Must be called prior
> to using the ValueVector.
>     *
> -   * @param  index   position of the bit in the vector
> -   * @return 1 if set, otherwise 0
> +   * @param valueCount  The number of values which can be contained
> within this vector.
>     */
> -  public int get(int index) {
> -    // logger.debug("BIT GET: index: {}, byte: {}, mask: {}, masked byte:
> {}",
> -    //             index,
> -    //             data.getByte((int)Math.floor(index/8)),
> -    //             (int)Math.pow(2, (index % 8)),
> -    //             data.getByte((int)Math.floor(index/8)) &
> (int)Math.pow(2, (index % 8)));
> -    return ((data.getByte((int)Math.floor(index/8)) & (int)Math.pow(2,
> (index % 8))) == 0) ? 0 : 1;
> +  public void allocateNew(int valueCount) {
> +    clear();
> +    valueCapacity = valueCount;
> +    int valueSize = getSizeFromCount(valueCount);
> +    data = allocator.buffer(valueSize);
> +    for (int i = 0; i < getSizeFromCount(valueCount); i++) {
> +      data.setByte(i, 0);
> +    }
>    }
> -
> +
>    @Override
> -  public Object getObject(int index) {
> -    return new Boolean(get(index) != 0);
> +  public int load(int valueCount, ByteBuf buf){
> +    clear();
> +    this.recordCount = valueCount;
> +    int len = getSizeFromCount(valueCount);
> +    data = buf.slice(0, len);
> +    data.retain();
> +    return len;
>    }
> -
> -  /**
> -   * Get the size requirement (in bytes) for the given number of values.
> -   */
> +
>    @Override
> -  public int getSizeFromCount(int valueCount) {
> -    return (int) Math.ceil(valueCount / 8);
> +  public void load(FieldMetadata metadata, ByteBuf buffer) {
> +    assert this.field.getDef().equals(metadata.getDef());
> +    int loaded = load(metadata.getValueCount(), buffer);
> +    assert metadata.getBufferLength() == loaded;
>    }
> -
> +
>    @Override
> -  public int getAllocatedSize() {
> -    return totalBytes;
> +  public int getValueCapacity() {
> +    return valueCapacity;
>    }
>
>    public Mutator getMutator() {
>      return new Mutator();
>    }
>
> -  /**
> -   * Allocate a new memory space for this vector.  Must be called prior
> to using the ValueVector.
> -   *
> -   * @param valueCount  The number of values which can be contained
> within this vector.
> -   */
> -  @Override
> -  public void allocateNew(int valueCount) {
> -    allocateNew(getSizeFromCount(valueCount), null, valueCount);
> -    for (int i = 0; i < getSizeFromCount(valueCount); i++) {
> -      data.setByte(i, 0);
> -    }
> +  public Accessor getAccessor(){
> +    return new Accessor();
>    }
> +
> +
> +  public class Accessor extends BaseAccessor{
>
> +    /**
> +     * Get the byte holding the desired bit, then mask all other bits.
>  Iff the result is 0, the
> +     * bit was not set.
> +     *
> +     * @param  index   position of the bit in the vector
> +     * @return 1 if set, otherwise 0
> +     */
> +    public int get(int index) {
> +      // logger.debug("BIT GET: index: {}, byte: {}, mask: {}, masked
> byte: {}",
> +      //             index,
> +      //             data.getByte((int)Math.floor(index/8)),
> +      //             (int)Math.pow(2, (index % 8)),
> +      //             data.getByte((int)Math.floor(index/8)) &
> (int)Math.pow(2, (index % 8)));
> +      return ((data.getByte((int)Math.floor(index/8)) & (int)Math.pow(2,
> (index % 8))) == 0) ? 0 : 1;
> +    }
> +
> +    @Override
> +    public Object getObject(int index) {
> +      return new Boolean(get(index) != 0);
> +    }
> +
> +    public int getRecordCount() {
> +      return recordCount;
> +    }
> +
> +  }
>
>    /**
>     * MutableBit implements a vector of bit-width values.  Elements in the
> vector are accessed
> @@ -79,7 +111,7 @@ public final class BitVector extends ValueVector {
>     *
>     * NB: this class is automatically generated from ValueVectorTypes.tdd
> using FreeMarker.
>     */
> -  public class Mutator implements ValueVector.Mutator{
> +  public class Mutator extends BaseMutator{
>
>      private Mutator(){}
>
> @@ -102,10 +134,9 @@ public final class BitVector extends ValueVector {
>        data.setByte((int) Math.floor(index/8), currentByte);
>      }
>
> -
> -    @Override
> -    public void setRecordCount(int recordCount) {
> -      BitVector.this.setRecordCount(recordCount);
> +    public void setValueCount(int recordCount) {
> +      BitVector.this.recordCount = recordCount;
> +      data.writerIndex(getSizeFromCount(recordCount));
>      }
>
>      @Override
> @@ -119,5 +150,6 @@ public final class BitVector extends ValueVector {
>          }
>        }
>      }
> +
>    }
>  }
> \ No newline at end of file
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/9ca9eb9b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/ByteHolder.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/ByteHolder.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/ByteHolder.java
> new file mode 100644
> index 0000000..45d8019
> --- /dev/null
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/ByteHolder.java
> @@ -0,0 +1,12 @@
> +package org.apache.drill.exec.vector;
> +
> +import io.netty.buffer.ByteBuf;
> +
> +public class ByteHolder {
> +  static final org.slf4j.Logger logger =
> org.slf4j.LoggerFactory.getLogger(ByteHolder.class);
> +
> +  public ByteBuf buffer;
> +  public int start;
> +  public int length;
> +
> +}
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/9ca9eb9b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/FixedWidthVector.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/FixedWidthVector.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/FixedWidthVector.java
> new file mode 100644
> index 0000000..0e3e3e9
> --- /dev/null
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/FixedWidthVector.java
> @@ -0,0 +1,23 @@
> +package org.apache.drill.exec.vector;
> +
> +import io.netty.buffer.ByteBuf;
> +
> +public interface FixedWidthVector extends ValueVector{
> +
> +  /**
> +   * Allocate a new memory space for this vector.  Must be called prior
> to using the ValueVector.
> +   *
> +   * @param totalBytes   Desired size of the underlying data buffer.
> +   * @param valueCount   Number of values in the vector.
> +   */
> +  public void allocateNew(int valueCount);
> +
> +  /**
> +   * Load the records in the provided buffer based on the given number of
> values.
> +   * @param valueCount Number of values the buffer contains.
> +   * @param buf Incoming buffer.
> +   * @return The number of bytes of the buffer that were consumed.
> +   */
> +  public int load(int valueCount, ByteBuf buf);
> +
> +}
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/9ca9eb9b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/NonRepeatedMutator.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/NonRepeatedMutator.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/NonRepeatedMutator.java
> new file mode 100644
> index 0000000..e9bdcbd
> --- /dev/null
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/NonRepeatedMutator.java
> @@ -0,0 +1,7 @@
> +package org.apache.drill.exec.vector;
> +
> +import org.apache.drill.exec.vector.ValueVector.Mutator;
> +
> +public interface NonRepeatedMutator extends Mutator{
> +  public void setValueCount(int recordCount);
> +}
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/9ca9eb9b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/RepeatedFixedWidthVector.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/RepeatedFixedWidthVector.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/RepeatedFixedWidthVector.java
> new file mode 100644
> index 0000000..35261d7
> --- /dev/null
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/RepeatedFixedWidthVector.java
> @@ -0,0 +1,22 @@
> +package org.apache.drill.exec.vector;
> +
> +import io.netty.buffer.ByteBuf;
> +
> +public interface RepeatedFixedWidthVector extends ValueVector{
> +  /**
> +   * Allocate a new memory space for this vector.  Must be called prior
> to using the ValueVector.
> +   *
> +   * @param parentValueCount   Number of separate repeating groupings.
> +   * @param childValueCount   Number of supported values in the vector.
> +   */
> +  public void allocateNew(int parentValueCount, int childValueCount);
> +
> +  /**
> +   * Load the records in the provided buffer based on the given number of
> values.
> +   * @param parentValueCount   Number of separate repeating groupings.
> +   * @param valueCount Number atomic values the buffer contains.
> +   * @param buf Incoming buffer.
> +   * @return The number of bytes of the buffer that were consumed.
> +   */
> +  public int load(int parentValueCount, int childValueCount, ByteBuf buf);
> +}
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/9ca9eb9b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/RepeatedVariableWidthVector.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/RepeatedVariableWidthVector.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/RepeatedVariableWidthVector.java
> new file mode 100644
> index 0000000..4f22481
> --- /dev/null
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/RepeatedVariableWidthVector.java
> @@ -0,0 +1,24 @@
> +package org.apache.drill.exec.vector;
> +
> +import io.netty.buffer.ByteBuf;
> +
> +public interface RepeatedVariableWidthVector extends ValueVector{
> +  /**
> +   * Allocate a new memory space for this vector.  Must be called prior
> to using the ValueVector.
> +   *
> +   * @param totalBytes   Desired size of the underlying data buffer.
> +   * @param parentValueCount   Number of separate repeating groupings.
> +   * @param childValueCount   Number of supported values in the vector.
> +   */
> +  public void allocateNew(int totalBytes, int parentValueCount, int
> childValueCount);
> +
> +  /**
> +   * Load the records in the provided buffer based on the given number of
> values.
> +   * @param dataBytes   The number of bytes associated with the data
> array.
> +   * @param parentValueCount   Number of separate repeating groupings.
> +   * @param childValueCount   Number of supported values in the vector.
> +   * @param buf Incoming buffer.
> +   * @return The number of bytes of the buffer that were consumed.
> +   */
> +  public int load(int dataBytes, int parentValueCount, int
> childValueCount, ByteBuf buf);
> +}
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/9ca9eb9b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/ValueVector.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/ValueVector.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/ValueVector.java
> index 718478e..328182b 100644
> ---
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/ValueVector.java
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/ValueVector.java
> @@ -21,9 +21,8 @@ import io.netty.buffer.ByteBuf;
>
>  import java.io.Closeable;
>
> -import org.apache.drill.exec.memory.BufferAllocator;
> +import org.apache.drill.exec.proto.SchemaDefProtos.FieldDef;
>  import org.apache.drill.exec.proto.UserBitShared.FieldMetadata;
> -import org.apache.drill.exec.record.DeadBuf;
>  import org.apache.drill.exec.record.MaterializedField;
>
>  /**
> @@ -31,42 +30,25 @@ import org.apache.drill.exec.record.MaterializedField;
>   * value vectors.  The template approach was chosen due to the lack of
> multiple inheritence.  It
>   * is also important that all related logic be as efficient as possible.
>   */
> -public abstract class ValueVector implements Closeable {
> +public interface ValueVector extends Closeable {
>
> -  static final org.slf4j.Logger logger =
> org.slf4j.LoggerFactory.getLogger(ValueVector.class);
> +//  /**
> +//   * Get the explicitly specified size of the allocated buffer, if
> available.  Otherwise
> +//   * calculate the size based on width and record count.
> +//   */
> +//  public abstract int getAllocatedSize();
>
> -  protected final BufferAllocator allocator;
> -  protected ByteBuf data = DeadBuf.DEAD_BUFFER;
> -  protected MaterializedField field;
> -  protected int recordCount;
> -  protected int totalBytes;
> -
> -  ValueVector(MaterializedField field, BufferAllocator allocator) {
> -    this.allocator = allocator;
> -    this.field = field;
> -  }
>
>    /**
> -   * Get the explicitly specified size of the allocated buffer, if
> available.  Otherwise
> -   * calculate the size based on width and record count.
> +   * Alternative to clear().  Allows use as closeable in
> try-with-resources.
>     */
> -  public abstract int getAllocatedSize();
> -
> +  public void close();
> +
>    /**
> -   * Get the size requirement (in bytes) for the given number of values.
>  Takes derived
> -   * type specs into account.
> +   * Release the underlying ByteBuf and reset the ValueVector to empty.
>     */
> -  public abstract int getSizeFromCount(int valueCount);
> -
> -  /**
> -   * Get the Java Object representation of the element at the specified
> position
> -   *
> -   * @param index   Index of the value to get
> -   */
> -  public abstract Object getObject(int index);
> -
> +  public void clear();
>
> -  public abstract Mutator getMutator();
>
>    /**
>     * Return the underlying buffers associated with this vector. Note that
> this doesn't impact the
> @@ -76,117 +58,72 @@ public abstract class ValueVector implements
> Closeable {
>     *
>     * @return The underlying ByteBuf.
>     */
> -  public ByteBuf[] getBuffers() {
> -    return new ByteBuf[]{data};
> -  }
> -
> -  /**
> -   * Returns the maximum number of values contained within this vector.
> -   * @return Vector size
> -   */
> -  public int capacity() {
> -    return getRecordCount();
> -  }
> -
> +  public abstract ByteBuf[] getBuffers();
> +
>    /**
> -   * Release supporting resources.
> +   * Load the data provided in the buffer.  Typically used when
> deserializing from the wire.
> +   * @param metadata Metadata used to decode the incoming buffer.
> +   * @param buffer The buffer that contains the ValueVector.
>     */
> -  @Override
> -  public void close() {
> -    clear();
> -  }
> +  public void load(FieldMetadata metadata, ByteBuf buffer);
>
> -  /**
> -   * Get information about how this field is materialized.
> -   * @return
> -   */
> -  public MaterializedField getField() {
> -    return field;
> -  }
>
>    /**
> -   * Get the number of records allocated for this value vector.
> -   * @return number of allocated records
> +   * Given the current buffer allocation, return the maximum number of
> values that this buffer can contain.
> +   * @return Maximum values buffer can contain.  In the case of a
> Repeated field, this is the number of atoms, not repeated groups.
>     */
> -  public int getRecordCount() {
> -    return recordCount;
> -  }
> -
> +  public int getValueCapacity();
> +
>    /**
> -   * Get the metadata for this field.
> +   * Get information about how this field is materialized.
>     * @return
>     */
> -  public FieldMetadata getMetadata() {
> -    int len = 0;
> -    for(ByteBuf b : getBuffers()){
> -      len += b.writerIndex();
> -    }
> -    return FieldMetadata.newBuilder()
> -             .setDef(getField().getDef())
> -             .setValueCount(getRecordCount())
> -             .setBufferLength(len)
> -             .build();
> -  }
> -
> +  public MaterializedField getField();
> +
>    /**
> -   * Allocate a new memory space for this vector.  Must be called prior
> to using the ValueVector.
> -   *
> -   * @param totalBytes   Optional desired size of the underlying buffer.
>  Specifying 0 will
> -   *                     estimate the size based on valueCount.
> -   * @param sourceBuffer Optional ByteBuf to use for storage (null will
> allocate automatically).
> -   * @param valueCount   Number of values in the vector.
> +   * Get the metadata for this field.  Used in serialization
> +   * @return FieldMetadata for this field.
>     */
> -  public void allocateNew(int totalBytes, ByteBuf sourceBuffer, int
> valueCount) {
> -    clear();
> -    this.recordCount = valueCount;
> -    this.totalBytes = totalBytes > 0 ? totalBytes :
> getSizeFromCount(valueCount);
> -    this.data = (sourceBuffer != null) ? sourceBuffer :
> allocator.buffer(this.totalBytes);
> -    this.data.retain();
> -    data.readerIndex(0);
> -  }
> -
> +  public FieldMetadata getMetadata();
> +
>    /**
> -   * Allocate a new memory space for this vector.  Must be called prior
> to using the ValueVector.
> -   *
> -   * @param valueCount
> -   *          The number of elements which can be contained within this
> vector.
> +   * Get Accessor to read value vector data.
> +   * @return
>     */
> -  public void allocateNew(int valueCount) {
> -    allocateNew(0, null, valueCount);
> -  }
> -
> +  public abstract Accessor getAccessor();
> +
>    /**
> -   * Release the underlying ByteBuf and reset the ValueVector
> +   * Get a Mutator to update this vectors data.
> +   * @return
>     */
> -  protected void clear() {
> -    if (data != DeadBuf.DEAD_BUFFER) {
> -      data.release();
> -      data = DeadBuf.DEAD_BUFFER;
> -      recordCount = 0;
> -      totalBytes = 0;
> -    }
> -  }
> +  public abstract Mutator getMutator();
>
> -  //public abstract <T extends Mutator> T getMutator();
>
> -  /**
> -   * Define the number of records that are in this value vector.
> -   * @param recordCount Number of records active in this vector.
> -   */
> -  void setRecordCount(int recordCount) {
> -    data.writerIndex(getSizeFromCount(recordCount));
> -    this.recordCount = recordCount;
> +  public interface Accessor{
> +
> +//    /**
> +//     * Get the number of records allocated for this value vector.
> +//     * @return number of allocated records
> +//     */
> +//    public int getRecordCount();
> +
> +    /**
> +     * Get the Java Object representation of the element at the specified
> position.  Useful for testing.
> +     *
> +     * @param index   Index of the value to get
> +     */
> +    public abstract Object getObject(int index);
> +
> +    public void reset();
>    }
> -
> -  /**
> -   * For testing only -- randomize the buffer contents
> -   */
> -  public void randomizeData() { }
> -
>
> -  public static interface Mutator{
> +
> +
> +
> +
> +  public interface Mutator{
> +    public void reset();
>      public void randomizeData();
> -    public void setRecordCount(int recordCount);
>    }
>  }
>
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/9ca9eb9b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/VariableWidthVector.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/VariableWidthVector.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/VariableWidthVector.java
> new file mode 100644
> index 0000000..c26cbab
> --- /dev/null
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/VariableWidthVector.java
> @@ -0,0 +1,29 @@
> +package org.apache.drill.exec.vector;
> +
> +import io.netty.buffer.ByteBuf;
> +
> +public interface VariableWidthVector extends ValueVector{
> +
> +  /**
> +   * Allocate a new memory space for this vector.  Must be called prior
> to using the ValueVector.
> +   *
> +   * @param totalBytes   Desired size of the underlying data buffer.
> +   * @param valueCount   Number of values in the vector.
> +   */
> +  public void allocateNew(int totalBytes, int valueCount);
> +
> +  /**
> +   * Provide the maximum amount of variable width bytes that can be
> stored int his vector.
> +   * @return
> +   */
> +  public int getByteCapacity();
> +
> +  /**
> +   * Load the records in the provided buffer based on the given number of
> values.
> +   * @param dataBytes   The number of bytes associated with the data
> array.
> +   * @param valueCount Number of values the buffer contains.
> +   * @param buf Incoming buffer.
> +   * @return The number of bytes of the buffer that were consumed.
> +   */
> +  public int load(int dataBytes, int valueCount, ByteBuf buf);
> +}
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/9ca9eb9b/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 ba103ed..a90382a 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
> @@ -230,8 +230,7 @@ public class Foreman implements Runnable, Closeable,
> Comparable<Object>{
>    }
>
>    public QueryResult getResult(UserClientConnection connection,
> RequestResults req) {
> -
> -    return null;
> +    throw new UnsupportedOperationException();
>    }
>
>
> @@ -254,7 +253,7 @@ public class Foreman implements Runnable, Closeable,
> Comparable<Object>{
>      }
>
>      void cleanupAndSendResult(QueryResult result){
> -      ForemanManagerListener.this.cleanupAndSendResult(result);
> +      Foreman.this.cleanupAndSendResult(result);
>      }
>
>    }
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/9ca9eb9b/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 3fe0622..cac6aa2 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
> @@ -86,7 +86,7 @@ public class TestSimpleFragmentRun extends
> PopUnitTestBase {
>            } else {
>              System.out.print("\t");
>            }
> -          System.out.print(v.value.getObject(i));
> +          System.out.print(v.value.getAccessor().getObject(i));
>          }
>          if(!first) System.out.println();
>        }
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/9ca9eb9b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/record/vector/TestValueVector.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/record/vector/TestValueVector.java
> b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/record/vector/TestValueVector.java
> index ae4f644..6a1f3ad 100644
> ---
> a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/record/vector/TestValueVector.java
> +++
> b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/record/vector/TestValueVector.java
> @@ -1,7 +1,6 @@
>  package org.apache.drill.exec.record.vector;
>
>  import static org.junit.Assert.assertEquals;
> -import static org.junit.Assert.assertFalse;
>
>  import java.nio.charset.Charset;
>
> @@ -46,14 +45,14 @@ public class TestValueVector {
>      m.set(100, 102);
>      m.set(1022, 103);
>      m.set(1023, 104);
> -    assertEquals(100, v.get(0));
> -    assertEquals(101, v.get(1));
> -    assertEquals(102, v.get(100));
> -    assertEquals(103, v.get(1022));
> -    assertEquals(104, v.get(1023));
> +    assertEquals(100, v.getAccessor().get(0));
> +    assertEquals(101, v.getAccessor().get(1));
> +    assertEquals(102, v.getAccessor().get(100));
> +    assertEquals(103, v.getAccessor().get(1022));
> +    assertEquals(104, v.getAccessor().get(1023));
>
>      // Ensure unallocated space returns 0
> -    assertEquals(0, v.get(3));
> +    assertEquals(0, v.getAccessor().get(3));
>    }
>
>    @Test
> @@ -74,7 +73,7 @@ public class TestValueVector {
>      // Create a new value vector for 1024 integers
>      NullableVarChar2Vector v = new NullableVarChar2Vector(field,
> allocator);
>      NullableVarChar2Vector.Mutator m = v.getMutator();
> -    v.allocateNew(1024);
> +    v.allocateNew(1024*10, 1024);
>
>      // Create and set 3 sample strings
>      String str1 = new String("AAAAA1");
> @@ -85,14 +84,14 @@ public class TestValueVector {
>      m.set(2, str3.getBytes(Charset.forName("UTF-8")));
>
>      // Check the sample strings
> -    assertEquals(str1, new String(v.get(0), Charset.forName("UTF-8")));
> -    assertEquals(str2, new String(v.get(1), Charset.forName("UTF-8")));
> -    assertEquals(str3, new String(v.get(2), Charset.forName("UTF-8")));
> +    assertEquals(str1, new String(v.getAccessor().get(0),
> Charset.forName("UTF-8")));
> +    assertEquals(str2, new String(v.getAccessor().get(1),
> Charset.forName("UTF-8")));
> +    assertEquals(str3, new String(v.getAccessor().get(2),
> Charset.forName("UTF-8")));
>
>      // Ensure null value throws
>      boolean b = false;
>      try {
> -      v.get(3);
> +      v.getAccessor().get(3);
>      } catch(AssertionError e) {
>        b = true;
>      }finally{
> @@ -130,17 +129,17 @@ public class TestValueVector {
>      m.set(100, 102);
>      m.set(1022, 103);
>      m.set(1023, 104);
> -    assertEquals(100, v.get(0));
> -    assertEquals(101, v.get(1));
> -    assertEquals(102, v.get(100));
> -    assertEquals(103, v.get(1022));
> -    assertEquals(104, v.get(1023));
> +    assertEquals(100, v.getAccessor().get(0));
> +    assertEquals(101, v.getAccessor().get(1));
> +    assertEquals(102, v.getAccessor().get(100));
> +    assertEquals(103, v.getAccessor().get(1022));
> +    assertEquals(104, v.getAccessor().get(1023));
>
>      // Ensure null values throw
>      {
>        boolean b = false;
>        try {
> -        v.get(3);
> +        v.getAccessor().get(3);
>        } catch(AssertionError e) {
>          b = true;
>        }finally{
> @@ -155,7 +154,7 @@ public class TestValueVector {
>      {
>        boolean b = false;
>        try {
> -        v.get(0);
> +        v.getAccessor().get(0);
>        } catch(AssertionError e) {
>          b = true;
>        }finally{
> @@ -170,18 +169,18 @@ public class TestValueVector {
>      m.set(100, 102);
>      m.set(1022, 103);
>      m.set(1023, 104);
> -    assertEquals(100, v.get(0));
> -    assertEquals(101, v.get(1));
> -    assertEquals(102, v.get(100));
> -    assertEquals(103, v.get(1022));
> -    assertEquals(104, v.get(1023));
> +    assertEquals(100, v.getAccessor().get(0));
> +    assertEquals(101, v.getAccessor().get(1));
> +    assertEquals(102, v.getAccessor().get(100));
> +    assertEquals(103, v.getAccessor().get(1022));
> +    assertEquals(104, v.getAccessor().get(1023));
>
>      // Ensure null values throw
>
>      {
>        boolean b = false;
>        try {
> -        v.get(3);
> +        v.getAccessor().get(3);
>        } catch(AssertionError e) {
>          b = true;
>        }finally{
> @@ -219,17 +218,17 @@ public class TestValueVector {
>      m.set(100, 102.3f);
>      m.set(1022, 103.4f);
>      m.set(1023, 104.5f);
> -    assertEquals(100.1f, v.get(0), 0);
> -    assertEquals(101.2f, v.get(1), 0);
> -    assertEquals(102.3f, v.get(100), 0);
> -    assertEquals(103.4f, v.get(1022), 0);
> -    assertEquals(104.5f, v.get(1023), 0);
> +    assertEquals(100.1f, v.getAccessor().get(0), 0);
> +    assertEquals(101.2f, v.getAccessor().get(1), 0);
> +    assertEquals(102.3f, v.getAccessor().get(100), 0);
> +    assertEquals(103.4f, v.getAccessor().get(1022), 0);
> +    assertEquals(104.5f, v.getAccessor().get(1023), 0);
>
>      // Ensure null values throw
>      {
>        boolean b = false;
>        try {
> -        v.get(3);
> +        v.getAccessor().get(3);
>        } catch(AssertionError e) {
>          b = true;
>        }finally{
> @@ -243,7 +242,7 @@ public class TestValueVector {
>      {
>        boolean b = false;
>        try {
> -        v.get(0);
> +        v.getAccessor().get(0);
>        } catch(AssertionError e) {
>          b = true;
>        }finally{
> @@ -279,27 +278,27 @@ public class TestValueVector {
>      m.set(1, 0);
>      m.set(100, 0);
>      m.set(1022, 1);
> -    assertEquals(1, v.get(0));
> -    assertEquals(0, v.get(1));
> -    assertEquals(0, v.get(100));
> -    assertEquals(1, v.get(1022));
> +    assertEquals(1, v.getAccessor().get(0));
> +    assertEquals(0, v.getAccessor().get(1));
> +    assertEquals(0, v.getAccessor().get(100));
> +    assertEquals(1, v.getAccessor().get(1022));
>
>      // test setting the same value twice
>      m.set(0, 1);
>      m.set(0, 1);
>      m.set(1, 0);
>      m.set(1, 0);
> -    assertEquals(1, v.get(0));
> -    assertEquals(0, v.get(1));
> +    assertEquals(1, v.getAccessor().get(0));
> +    assertEquals(0, v.getAccessor().get(1));
>
>      // test toggling the values
>      m.set(0, 0);
>      m.set(1, 1);
> -    assertEquals(0, v.get(0));
> -    assertEquals(1, v.get(1));
> +    assertEquals(0, v.getAccessor().get(0));
> +    assertEquals(1, v.getAccessor().get(1));
>
>      // Ensure unallocated space returns 0
> -    assertEquals(0, v.get(3));
> +    assertEquals(0, v.getAccessor().get(3));
>    }
>
>  }
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/9ca9eb9b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/JSONRecordReaderTest.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/JSONRecordReaderTest.java
> b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/JSONRecordReaderTest.java
> index 117414c..4b35313 100644
> ---
> a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/JSONRecordReaderTest.java
> +++
> b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/JSONRecordReaderTest.java
> @@ -73,7 +73,7 @@ public class JSONRecordReaderTest {
>              return;
>          }
>
> -        T val = (T) valueVector.getObject(index);
> +        T val = (T) valueVector.getAccessor().getObject(index);
>          if (val instanceof byte[]) {
>              assertTrue(Arrays.equals((byte[]) value, (byte[]) val));
>          } else {
>
>

[35/53] [abbrv] git commit: Separate allocate and load methods. rename setRecordCount to setValueCount add setGroupAndValueCount to RepeatedVectors. add a number of marker/cross-inheritance interfaces.

Posted by ja...@apache.org.
Separate allocate and load methods.
rename setRecordCount to setValueCount
add setGroupAndValueCount to RepeatedVectors.
add a number of marker/cross-inheritance interfaces.


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

Branch: refs/heads/master
Commit: 9ca9eb9b3d88e86e28d1b688d9cd943e6a7f08df
Parents: 36793bb
Author: Jacques Nadeau <ja...@apache.org>
Authored: Mon Jul 15 10:50:07 2013 -0700
Committer: Jacques Nadeau <ja...@apache.org>
Committed: Mon Jul 15 13:13:59 2013 -0700

----------------------------------------------------------------------
 .../templates/FixedValueVectors.java            | 118 +++++---
 .../templates/NullableValueVectors.java         | 220 ++++++++++-----
 .../templates/RepeatedValueVectors.java         | 273 ++++++++++++++-----
 .../templates/VariableLengthVectors.java        | 177 +++++++-----
 .../exec/physical/config/MockRecordReader.java  |  37 ++-
 .../drill/exec/record/MaterializedField.java    |  16 ++
 .../drill/exec/record/RecordBatchLoader.java    |   7 +-
 .../apache/drill/exec/record/WritableBatch.java |   3 +-
 .../drill/exec/store/JSONRecordReader.java      |   4 +-
 .../apache/drill/exec/store/VectorHolder.java   |  21 +-
 .../drill/exec/vector/BaseDataValueVector.java  |  47 ++++
 .../drill/exec/vector/BaseValueVector.java      |  38 +++
 .../org/apache/drill/exec/vector/BitVector.java | 112 +++++---
 .../apache/drill/exec/vector/ByteHolder.java    |  12 +
 .../drill/exec/vector/FixedWidthVector.java     |  23 ++
 .../drill/exec/vector/NonRepeatedMutator.java   |   7 +
 .../exec/vector/RepeatedFixedWidthVector.java   |  22 ++
 .../vector/RepeatedVariableWidthVector.java     |  24 ++
 .../apache/drill/exec/vector/ValueVector.java   | 179 ++++--------
 .../drill/exec/vector/VariableWidthVector.java  |  29 ++
 .../apache/drill/exec/work/foreman/Foreman.java |   5 +-
 .../physical/impl/TestSimpleFragmentRun.java    |   2 +-
 .../exec/record/vector/TestValueVector.java     |  81 +++---
 .../drill/exec/store/JSONRecordReaderTest.java  |   2 +-
 24 files changed, 995 insertions(+), 464 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/9ca9eb9b/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/FixedValueVectors.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/FixedValueVectors.java b/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/FixedValueVectors.java
index 09dd5d8..7583d9f 100644
--- a/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/FixedValueVectors.java
+++ b/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/FixedValueVectors.java
@@ -12,6 +12,7 @@ import org.apache.drill.exec.memory.BufferAllocator;
 import org.apache.drill.exec.proto.UserBitShared.FieldMetadata;
 import org.apache.drill.exec.record.DeadBuf;
 import org.apache.drill.exec.record.MaterializedField;
+import org.apache.drill.exec.vector.BaseValueVector;
 import org.apache.drill.exec.vector.MsgPack2Vector;
 
 import java.util.Random;
@@ -26,71 +27,100 @@ import java.util.Random;
  * NB: this class is automatically generated from ValueVectorTypes.tdd using FreeMarker.
  */
 @SuppressWarnings("unused")
-public final class ${minor.class}Vector extends ValueVector {
+public final class ${minor.class}Vector extends BaseDataValueVector implements FixedWidthVector{
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(${minor.class}Vector.class);
 
+ 
+  private final Accessor accessor = new Accessor();
+  private final Mutator mutator = new Mutator();
+  
   public ${minor.class}Vector(MaterializedField field, BufferAllocator allocator) {
     super(field, allocator);
   }
 
+  public int getValueCapacity(){
+    return (int) (data.capacity() *1.0 / ${type.width});
+  }
+
+  public Accessor getAccessor(){
+    return accessor;
+  }
+  
+  public Mutator getMutator(){
+    return mutator;
+  }
+
   /**
-   * Allocate a new memory space for this vector.  Must be called prior to using the ValueVector.
-   *
+   * Allocate a new buffer that supports setting at least the provided number of values.  May actually be sized bigger depending on underlying buffer rounding size. Must be called prior to using the ValueVector.
    * @param valueCount
-   *          The number of values which can be contained within this vector.
    */
   public void allocateNew(int valueCount) {
-    totalBytes = valueCount * ${type.width};
-    allocateNew(totalBytes, allocator.buffer(totalBytes), valueCount);
+    clear();
+    this.data = allocator.buffer(valueCount * ${type.width});
+    this.data.retain();
+    this.data.readerIndex(0);
   }
-
+  
   @Override
-  public int getAllocatedSize() {
-    return (int) Math.ceil(totalBytes);
+  public FieldMetadata getMetadata() {
+    return FieldMetadata.newBuilder()
+             .setDef(getField().getDef())
+             .setValueCount(recordCount)
+             .setBufferLength(recordCount * ${type.width})
+             .build();
   }
 
-  /**
-   * Get the size requirement (in bytes) for the given number of values.  Only accurate
-   * for fixed width value vectors.
-   */
   @Override
-  public int getSizeFromCount(int valueCount) {
-    return valueCount * ${type.width};
+  public int load(int valueCount, ByteBuf buf){
+    clear();
+    this.recordCount = valueCount;
+    int len = recordCount * ${type.width};
+    data = buf.slice(0, len);
+    data.retain();
+    return len;
   }
-
-  public Mutator getMutator() {
-    return new Mutator();
+  
+  @Override
+  public void load(FieldMetadata metadata, ByteBuf buffer) {
+    assert this.field.getDef().equals(metadata.getDef());
+    int loaded = load(metadata.getValueCount(), buffer);
+    assert metadata.getBufferLength() == loaded;
   }
+  
+  public final class Accessor extends BaseValueVector.BaseAccessor{
 
- <#if (type.width > 8)>
-
-  public ${minor.javaType!type.javaType} get(int index) {
-    ByteBuf dst = allocator.buffer(${type.width});
-    data.getBytes(index * ${type.width}, dst, 0, ${type.width});
-    return dst;
-  }
+    public int getRecordCount() {
+      return recordCount;
+    }
+    
+    <#if (type.width > 8)>
 
-  @Override
-  public Object getObject(int index) {
-    ByteBuf dst = allocator.buffer(${type.width});
-    data.getBytes(index, dst, 0, ${type.width});
-    return dst;
-  }
+    public ${minor.javaType!type.javaType} get(int index) {
+      ByteBuf dst = allocator.buffer(${type.width});
+      data.getBytes(index * ${type.width}, dst, 0, ${type.width});
+      return dst;
+    }
 
+    @Override
+    public Object getObject(int index) {
+      ByteBuf dst = allocator.buffer(${type.width});
+      data.getBytes(index, dst, 0, ${type.width});
+      return dst;
+    }
 
- <#else> <#-- type.width <= 8 -->
+    <#else> <#-- type.width <= 8 -->
 
-  public ${minor.javaType!type.javaType} get(int index) {
-    return data.get${(minor.javaType!type.javaType)?cap_first}(index * ${type.width});
-  }
+    public ${minor.javaType!type.javaType} get(int index) {
+      return data.get${(minor.javaType!type.javaType)?cap_first}(index * ${type.width});
+    }
 
-  public Object getObject(int index) {
-    return get(index);
-  }
+    public Object getObject(int index) {
+      return get(index);
+    }
 
 
- </#if> <#-- type.width -->
- 
+   </#if> <#-- type.width -->
+ }
  
  /**
   * ${minor.class}.Mutator implements a mutable vector of fixed width values.  Elements in the
@@ -101,7 +131,7 @@ public final class ${minor.class}Vector extends ValueVector {
   *
   * NB: this class is automatically generated from ValueVectorTypes.tdd using FreeMarker.
   */
-  public class Mutator implements ValueVector.Mutator{
+  public final class Mutator extends BaseValueVector.BaseMutator{
 
     private Mutator(){};
    /**
@@ -147,9 +177,9 @@ public final class ${minor.class}Vector extends ValueVector {
    }
   </#if> <#-- type.width -->
   
-   @Override
-   public void setRecordCount(int recordCount) {
-     ${minor.class}Vector.this.setRecordCount(recordCount);
+   public void setValueCount(int recordCount) {
+     ${minor.class}Vector.this.recordCount = recordCount;
+     data.writerIndex(${type.width} * recordCount);
    }
 
 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/9ca9eb9b/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/NullableValueVectors.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/NullableValueVectors.java b/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/NullableValueVectors.java
index c7de73f..3232f87 100644
--- a/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/NullableValueVectors.java
+++ b/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/NullableValueVectors.java
@@ -10,12 +10,15 @@ import io.netty.buffer.ByteBuf;
 
 import java.io.Closeable;
 import java.util.Random;
+import java.util.Vector;
 
 import org.apache.drill.exec.memory.BufferAllocator;
 import org.apache.drill.exec.proto.SchemaDefProtos;
 import org.apache.drill.exec.proto.UserBitShared.FieldMetadata;
 import org.apache.drill.exec.record.DeadBuf;
 import org.apache.drill.exec.record.MaterializedField;
+import org.apache.drill.exec.vector.BaseValueVector;
+import org.apache.drill.exec.vector.BitVector;
 import org.apache.drill.exec.vector.UInt2Vector;
 import org.apache.drill.exec.vector.UInt4Vector;
 
@@ -27,93 +30,180 @@ import org.apache.drill.exec.vector.UInt4Vector;
  * NB: this class is automatically generated from ValueVectorTypes.tdd using FreeMarker.
  */
 @SuppressWarnings("unused")
-public final class Nullable${minor.class}Vector extends ValueVector {
+public final class Nullable${minor.class}Vector extends BaseValueVector implements <#if type.major == "VarLen">VariableWidth<#else>FixedWidth</#if>Vector {
 
+  private int recordCount;
   private final BitVector bits;
   private final ${minor.class}Vector values;
+  private final Accessor accessor = new Accessor();
+  private final Mutator mutator = new Mutator();
 
   public Nullable${minor.class}Vector(MaterializedField field, BufferAllocator allocator) {
     super(field, allocator);
-    bits = new BitVector(null, allocator);
-    values = new ${minor.class}Vector(null, allocator);
+    this.bits = new BitVector(null, allocator);
+    this.values = new ${minor.class}Vector(null, allocator);
   }
-
-  /**
-   * Get the element at the specified position.
-   *
-   * @param   index   position of the value
-   * @return  value of the element, if not null
-   * @throws  NullValueException if the value is null
-   */
-  public <#if type.major == "VarLen">byte[]<#else>${minor.javaType!type.javaType}</#if> get(int index) {
-    assert !isNull(index);
-    return values.get(index);
+  
+  public int getValueCapacity(){
+    return bits.getValueCapacity();
+  }
+  
+  @Override
+  public ByteBuf[] getBuffers() {
+    return new ByteBuf[]{bits.data, values.data};
   }
 
-
-  public boolean isNull(int index) {
-    return bits.get(index) == 0;
+  @Override
+  public void clear() {
+    recordCount = 0;
+    bits.clear();
+    values.clear();
+  }
+  
+  int getBufferSize(){
+    return values.getBufferSize() + bits.getBufferSize();
   }
 
-  public int isSet(int index){
-    return bits.get(index);
+  <#if type.major == "VarLen">
+  @Override
+  public FieldMetadata getMetadata() {
+    return FieldMetadata.newBuilder()
+             .setDef(getField().getDef())
+             .setValueCount(recordCount)
+             .setVarByteLength(values.getVarByteLength())
+             .setBufferLength(getBufferSize())
+             .build();
   }
   
-  /**
-   * Allocate a new memory space for this vector.  Must be called prior to using the ValueVector.
-   *
-   * @param valueCount   The number of values which may be contained by this vector.
-   */
-  public void allocateNew(int totalBytes, ByteBuf sourceBuffer, int valueCount) {
-    values.allocateNew(totalBytes, sourceBuffer, valueCount);
+  @Override
+  public void allocateNew(int totalBytes, int valueCount) {
+    values.allocateNew(totalBytes, valueCount);
     bits.allocateNew(valueCount);
+    mutator.reset();
+    accessor.reset();
   }
 
   @Override
-  public int getAllocatedSize() {
-    return bits.getAllocatedSize() + values.getAllocatedSize();
+  public int load(int dataBytes, int valueCount, ByteBuf buf){
+    clear();
+    this.recordCount = valueCount;
+    int loaded = bits.load(valueCount, buf);
+    
+    // remove bits part of buffer.
+    buf = buf.slice(loaded, buf.capacity() - loaded);
+    loaded += values.load(dataBytes, valueCount, buf);
+    return loaded;
   }
-
-  /**
-   * Get the size requirement (in bytes) for the given number of values.  Only accurate
-   * for fixed width value vectors.
-   */
-  public int getTotalSizeFromCount(int valueCount) {
-    return values.getSizeFromCount(valueCount) + bits.getSizeFromCount(valueCount);
+  
+  @Override
+  public void load(FieldMetadata metadata, ByteBuf buffer) {
+    assert this.field.getDef().equals(metadata.getDef());
+    int loaded = load(metadata.getVarByteLength(), metadata.getValueCount(), buffer);
+    assert metadata.getBufferLength() == loaded;
   }
   
-  public int getSizeFromCount(int valueCount){
-    return getTotalSizeFromCount(valueCount);
+  @Override
+  public int getByteCapacity(){
+    return values.getByteCapacity();
   }
 
+  <#else>
   @Override
-  public MaterializedField getField() {
-    return field;
+  public FieldMetadata getMetadata() {
+    return FieldMetadata.newBuilder()
+             .setDef(getField().getDef())
+             .setValueCount(recordCount)
+             .setBufferLength(getBufferSize())
+             .build();
   }
-
+  
   @Override
-  public ByteBuf[] getBuffers() {
-    return new ByteBuf[]{bits.data, values.data};
+  public void allocateNew(int valueCount) {
+    values.allocateNew(valueCount);
+    bits.allocateNew(valueCount);
+    mutator.reset();
+    accessor.reset();
   }
-
-
+  
+  @Override
+  public int load(int valueCount, ByteBuf buf){
+    clear();
+    this.recordCount = valueCount;
+    int loaded = bits.load(valueCount, buf);
+    
+    // remove bits part of buffer.
+    buf = buf.slice(loaded, buf.capacity() - loaded);
+    loaded += values.load(valueCount, buf);
+    return loaded;
+  }
+  
   @Override
-  public Object getObject(int index) {
-    return isNull(index) ? null : values.getObject(index);
+  public void load(FieldMetadata metadata, ByteBuf buffer) {
+    assert this.field.getDef().equals(metadata.getDef());
+    int loaded = load(metadata.getValueCount(), buffer);
+    assert metadata.getBufferLength() == loaded;
+  }
+  
+  </#if>
+  
+  public Accessor getAccessor(){
+    return accessor;
   }
   
   public Mutator getMutator(){
-    return new Mutator();
+    return mutator;
+  }
+  
+  public ${minor.class}Vector convertToRequiredVector(){
+    ${minor.class}Vector v = new ${minor.class}Vector(getField().getOtherNullableVersion(), allocator);
+    v.data = values.data;
+    v.recordCount = this.recordCount;
+    v.data.retain();
+    clear();
+    return v;
   }
   
-  public class Mutator implements ValueVector.Mutator{
+  
+  
+  public final class Accessor implements ValueVector.Accessor{
 
-    private final BitVector.Mutator bitMutator;
-    private final ${minor.class}Vector.Mutator valueMutator;
+    /**
+     * Get the element at the specified position.
+     *
+     * @param   index   position of the value
+     * @return  value of the element, if not null
+     * @throws  NullValueException if the value is null
+     */
+    public <#if type.major == "VarLen">byte[]<#else>${minor.javaType!type.javaType}</#if> get(int index) {
+      assert !isNull(index);
+      return values.getAccessor().get(index);
+    }
+
+    public boolean isNull(int index) {
+      return isSet(index) == 0;
+    }
+
+    public int isSet(int index){
+      return bits.getAccessor().get(index);
+    }
+    
+    @Override
+    public Object getObject(int index) {
+      return isNull(index) ? null : values.getAccessor().getObject(index);
+    }
+    
+    public int getRecordCount(){
+      return recordCount;
+    }
+    
+    public void reset(){}
+  }
+  
+  public final class Mutator implements ValueVector.Mutator{
+    
+    private int setCount;
     
     private Mutator(){
-      bitMutator = bits.getMutator();
-      valueMutator = values.getMutator();
     }
 
     /**
@@ -123,28 +213,30 @@ public final class Nullable${minor.class}Vector extends ValueVector {
      * @param bytes   array of bytes to write
      */
     public void set(int index, <#if type.major == "VarLen">byte[]<#elseif (type.width < 4)>int<#else>${minor.javaType!type.javaType}</#if> value) {
-      setNotNull(index);
-      valueMutator.set(index, value);
-    }
-
-    public void setNull(int index) {
-      bitMutator.set(index, 0);
+      setCount++;
+      bits.getMutator().set(index, 1);
+      values.getMutator().set(index, value);
     }
 
-    private void setNotNull(int index) {
-      bitMutator.set(index, 1);
+    public void setValueCount(int recordCount) {
+      assert recordCount >= 0;
+      Nullable${minor.class}Vector.this.recordCount = recordCount;
+      values.getMutator().setValueCount(recordCount);
+      bits.getMutator().setValueCount(recordCount);
     }
     
-    @Override
-    public void setRecordCount(int recordCount) {
-      Nullable${minor.class}Vector.this.setRecordCount(recordCount);
-      bits.setRecordCount(recordCount);
+    public boolean noNulls(){
+      return recordCount == setCount;
     }
     
     public void randomizeData(){
       throw new UnsupportedOperationException();
     }
     
+    public void reset(){
+      setCount = 0;
+    }
+    
   }
 }
 </#list>

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/9ca9eb9b/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/RepeatedValueVectors.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/RepeatedValueVectors.java b/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/RepeatedValueVectors.java
index 4acc4cc..363e4c8 100644
--- a/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/RepeatedValueVectors.java
+++ b/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/RepeatedValueVectors.java
@@ -7,12 +7,18 @@ import org.apache.drill.exec.vector.UInt4Vector;
 <@pp.changeOutputFile name="Repeated${minor.class}Vector.java" />
 package org.apache.drill.exec.vector;
 
+
+
+
+
+
 import static com.google.common.base.Preconditions.checkArgument;
 import static com.google.common.base.Preconditions.checkState;
 import io.netty.buffer.ByteBuf;
 
 import java.io.Closeable;
 import java.util.Random;
+import java.util.Vector;
 
 import org.apache.drill.exec.memory.BufferAllocator;
 import org.apache.drill.exec.proto.SchemaDefProtos;
@@ -30,100 +36,216 @@ import org.apache.drill.exec.record.MaterializedField;
  * NB: this class is automatically generated from ValueVectorTypes.tdd using FreeMarker.
  */
 
- public final class Repeated${minor.class}Vector extends ValueVector {
-
-  private final UInt2Vector countVector;    // number of repeated elements in each record
-  private final UInt4Vector offsetVector;   // offsets to start of each record
-  private final ${minor.class}Vector valuesVector;
+ public final class Repeated${minor.class}Vector extends BaseValueVector implements Repeated<#if type.major == "VarLen">VariableWidth<#else>FixedWidth</#if>Vector {
 
+  private MaterializedField field;
+  
+  private int parentValueCount;
+  private int childValueCount;
+  
+  private final UInt2Vector counts;    // number of repeated elements in each record
+  private final UInt4Vector offsets;   // offsets to start of each record
+  private final ${minor.class}Vector values;
+  private final Mutator mutator = new Mutator();
+  private final Accessor accessor = new Accessor();
+  
+  
   public Repeated${minor.class}Vector(MaterializedField field, BufferAllocator allocator) {
     super(field, allocator);
-    this.countVector = new UInt2Vector(null, allocator);
-    this.offsetVector = new UInt4Vector(null, allocator);
-    this.valuesVector = new ${minor.class}Vector(null, allocator);
+    this.counts = new UInt2Vector(null, allocator);
+    this.offsets = new UInt4Vector(null, allocator);
+    this.values = new ${minor.class}Vector(null, allocator);
   }
 
-  public void allocateNew(int totalBytes, ByteBuf sourceBuffer, int valueCount) {
-    super.allocateNew(totalBytes, sourceBuffer, valueCount);
-    countVector.allocateNew(valueCount);
-    offsetVector.allocateNew(valueCount);
+  public int getValueCapacity(){
+    return values.getValueCapacity();
+  }
+  
+  int getBufferSize(){
+    return counts.getBufferSize() + offsets.getBufferSize() + values.getBufferSize();
+  }
+  
+  <#if type.major == "VarLen">
+  @Override
+  public FieldMetadata getMetadata() {
+    return FieldMetadata.newBuilder()
+             .setDef(getField().getDef())
+             .setGroupCount(this.parentValueCount)
+             .setValueCount(this.childValueCount)
+             .setVarByteLength(values.getVarByteLength())
+             .setBufferLength(getBufferSize())
+             .build();
+  }
+  
+  public void allocateNew(int totalBytes, int parentValueCount, int childValueCount) {
+    counts.allocateNew(parentValueCount);
+    offsets.allocateNew(parentValueCount);
+    values.allocateNew(totalBytes, childValueCount);
+    mutator.reset();
+    accessor.reset();
   }
 
+  
 
-  /**
-   * Get a value for the given record.  Each element in the repeated field is accessed by
-   * the positionIndex param.
-   *
-   * @param  index           record containing the repeated field
-   * @param  positionIndex   position within the repeated field
-   * @return element at the given position in the given record
-   */
-  public <#if type.major == "VarLen">byte[]
-         <#else>${minor.javaType!type.javaType}
-         </#if> get(int index, int positionIndex) {
-
-    assert positionIndex < countVector.get(index);
-    return valuesVector.get(offsetVector.get(index) + positionIndex);
+  
+  @Override
+  public int load(int dataBytes, int parentValueCount, int childValueCount, ByteBuf buf){
+    clear();
+    this.parentValueCount = parentValueCount;
+    this.childValueCount = childValueCount;
+    int loaded = 0;
+    loaded += counts.load(parentValueCount, buf);
+    loaded += offsets.load(parentValueCount, buf.slice(loaded, buf.capacity() - loaded));
+    loaded += values.load(dataBytes, childValueCount, buf.slice(loaded, buf.capacity() - loaded));
+    return loaded;
   }
-
-  public MaterializedField getField() {
-    return field;
+  
+  @Override
+  public void load(FieldMetadata metadata, ByteBuf buffer) {
+    assert this.field.getDef().equals(metadata.getDef());
+    int loaded = load(metadata.getVarByteLength(), metadata.getGroupCount(), metadata.getValueCount(), buffer);
+    assert metadata.getBufferLength() == loaded;
+  }
+  
+  public int getByteCapacity(){
+    return values.getByteCapacity();
   }
 
-  /**
-   * Get the size requirement (in bytes) for the given number of values.  Only accurate
-   * for fixed width value vectors.
-   */
-  public int getTotalSizeFromCount(int valueCount) {
-    return valuesVector.getSizeFromCount(valueCount) +
-           countVector.getSizeFromCount(valueCount) +
-           offsetVector.getSizeFromCount(valueCount);
+  <#else>
+  
+  @Override
+  public FieldMetadata getMetadata() {
+    return FieldMetadata.newBuilder()
+             .setDef(getField().getDef())
+             .setGroupCount(this.parentValueCount)
+             .setValueCount(this.childValueCount)
+             .setBufferLength(getBufferSize())
+             .build();
   }
   
-  public int getSizeFromCount(int valueCount){
-    return getTotalSizeFromCount(valueCount);
+  public void allocateNew(int parentValueCount, int childValueCount) {
+    clear();
+    values.allocateNew(childValueCount);
+    counts.allocateNew(parentValueCount);
+    offsets.allocateNew(parentValueCount);
+    mutator.reset();
+    accessor.reset();
   }
-
-  /**
-   * Get the explicitly specified size of the allocated buffer, if available.  Otherwise
-   * calculate the size based on width and record count.
-   */
-  public int getAllocatedSize() {
-    return valuesVector.getAllocatedSize() +
-           countVector.getAllocatedSize() +
-           offsetVector.getAllocatedSize();
+  
+  public int load(int parentValueCount, int childValueCount, ByteBuf buf){
+    clear();
+    this.parentValueCount = parentValueCount;
+    this.childValueCount = childValueCount;
+    int loaded = 0;
+    loaded += counts.load(parentValueCount, buf);
+    loaded += offsets.load(parentValueCount, buf.slice(loaded, buf.capacity() - loaded));
+    loaded += values.load(childValueCount, buf.slice(loaded, buf.capacity() - loaded));
+    return loaded;
   }
-
-  /**
-   * Get the elements at the given index.
-   */
-  public int getCount(int index) {
-    return countVector.get(index);
+  
+  @Override
+  public void load(FieldMetadata metadata, ByteBuf buffer) {
+    assert this.field.getDef().equals(metadata.getDef());
+    int loaded = load(metadata.getGroupCount(), metadata.getValueCount(), buffer);
+    assert metadata.getBufferLength() == loaded;
   }
+  </#if>
+  
+//  /**
+//   * Get the size requirement (in bytes) for the given number of values.  Only accurate
+//   * for fixed width value vectors.
+//   */
+//  public int getTotalSizeFromCount(int valueCount) {
+//    return values.getSizeFromCount(valueCount) +
+//           counts.getSizeFromCount(valueCount) +
+//           offsets.getSizeFromCount(valueCount);
+//  }
+//  
+//  public int getSizeFromCount(int valueCount){
+//    return getTotalSizeFromCount(valueCount);
+//  }
+
+//  /**
+//   * Get the explicitly specified size of the allocated buffer, if available.  Otherwise
+//   * calculate the size based on width and record count.
+//   */
+//  public int getAllocatedSize() {
+//    return values.getAllocatedSize() +
+//           counts.getAllocatedSize() +
+//           offsets.getAllocatedSize();
+//  }
+
+
 
   public ByteBuf[] getBuffers() {
-    return new ByteBuf[]{countVector.data, offsetVector.data, data};
+    return new ByteBuf[]{counts.data, offsets.data, values.data};
   }
 
-  public Object getObject(int index) {
-    return data.slice(index, getSizeFromCount(countVector.get(index)));
+  public void clear(){
+    counts.clear();
+    offsets.clear();
+    values.clear();
+    parentValueCount = 0;
+    childValueCount = 0;
   }
 
   public Mutator getMutator(){
-    return new Mutator();
+    return mutator;
+  }
+  
+  public Accessor getAccessor(){
+    return accessor;
   }
   
-  public class Mutator implements ValueVector.Mutator{
+  public final class Accessor implements ValueVector.Accessor{
+    /**
+     * Get the elements at the given index.
+     */
+    public int getCount(int index) {
+      return counts.getAccessor().get(index);
+    }
+    
+    public Object getObject(int index) {
+      throw new UnsupportedOperationException();
+    }
+
+    /**
+     * Get a value for the given record.  Each element in the repeated field is accessed by
+     * the positionIndex param.
+     *
+     * @param  index           record containing the repeated field
+     * @param  positionIndex   position within the repeated field
+     * @return element at the given position in the given record
+     */
+    public <#if type.major == "VarLen">byte[]
+           <#else>${minor.javaType!type.javaType}
+           </#if> get(int index, int positionIndex) {
 
+      assert positionIndex < counts.getAccessor().get(index);
+      return values.getAccessor().get(offsets.getAccessor().get(index) + positionIndex);
+    }
+
+    public MaterializedField getField() {
+      return field;
+    }
+    
+    public int getGroupCount(){
+      return parentValueCount;
+    }
+    
+    public int getValueCount(){
+      return childValueCount;
+    }
     
-    private final UInt2Vector.Mutator countMutator;
-    private final ${minor.class}Vector.Mutator valuesMutator;
-    private final UInt4Vector.Mutator offsetMutator;
+    public void reset(){
+      
+    }
+  }
+  
+  public final class Mutator implements ValueVector.Mutator{
+
     
     private Mutator(){
-      this.countMutator = countVector.getMutator();
-      this.offsetMutator = offsetVector.getMutator();
-      this.valuesMutator = valuesVector.getMutator();
     }
 
     /**
@@ -137,21 +259,28 @@ import org.apache.drill.exec.record.MaterializedField;
                                <#elseif type.major == "VarLen"> byte[]
                                <#else> int
                                </#if> value) {
-      countMutator.set(index, countVector.get(index) + 1);
-      offsetMutator.set(index, offsetVector.get(index - 1) + countVector.get(index-1));
-      valuesMutator.set(offsetVector.get(index), value);
+      counts.getMutator().set(index, counts.getAccessor().get(index) + 1);
+      offsets.getMutator().set(index, offsets.getAccessor().get(index - 1) + counts.getAccessor().get(index-1));
+      values.getMutator().set(offsets.getAccessor().get(index), value);
     }
+
     
-    public void setRecordCount(int recordCount) {
-      valuesMutator.setRecordCount(recordCount);
-      offsetMutator.setRecordCount(recordCount);
-      countMutator.setRecordCount(recordCount);
+    public void setGroupAndValueCount(int groupCount, int valueCount) {
+      parentValueCount = groupCount;
+      childValueCount = valueCount;
+      counts.getMutator().setValueCount(groupCount);
+      offsets.getMutator().setValueCount(groupCount);
+      values.getMutator().setValueCount(valueCount);
     }
     
     public void randomizeData(){
       throw new UnsupportedOperationException();
     }
     
+    public void reset(){
+      
+    }
+    
   }
 }
 </#list>

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/9ca9eb9b/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/VariableLengthVectors.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/VariableLengthVectors.java b/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/VariableLengthVectors.java
index 954836a..c615258 100644
--- a/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/VariableLengthVectors.java
+++ b/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/VariableLengthVectors.java
@@ -11,6 +11,7 @@ import static com.google.common.base.Preconditions.checkState;
 import io.netty.buffer.ByteBuf;
 
 import java.io.Closeable;
+import java.nio.ByteBuffer;
 import java.util.Random;
 
 import org.apache.drill.exec.memory.BufferAllocator;
@@ -18,10 +19,11 @@ import org.apache.drill.exec.proto.SchemaDefProtos;
 import org.apache.drill.exec.proto.UserBitShared.FieldMetadata;
 import org.apache.drill.exec.record.DeadBuf;
 import org.apache.drill.exec.record.MaterializedField;
+import org.apache.drill.exec.vector.ByteHolder;
 
 /**
  * ${minor.class}Vector implements a vector of variable width values.  Elements in the vector
- * are accessed by position from the logical start of the vector.  A fixed width lengthVector
+ * are accessed by position from the logical start of the vector.  A fixed width offsetVector
  * is used to convert an element's position to it's offset from the start of the (0-based)
  * ByteBuf.  Size is inferred by adjacent elements.
  *   The width of each element is ${type.width} byte(s)
@@ -30,81 +32,125 @@ import org.apache.drill.exec.record.MaterializedField;
  * NB: this class is automatically generated from ValueVectorTypes.tdd using FreeMarker.
  */
 @SuppressWarnings("unused")
-public final class ${minor.class}Vector extends ValueVector {
+public final class ${minor.class}Vector extends BaseDataValueVector implements VariableWidthVector{
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(${minor.class}Vector.class);
 
-  private final UInt${type.width}Vector lengthVector;
-  private final UInt${type.width}Vector.Mutator lengthVectorMutator;
-
+  private final UInt${type.width}Vector offsetVector;
+  private final Accessor accessor = new Accessor();
+  private final Mutator mutator = new Mutator();
+  
   public ${minor.class}Vector(MaterializedField field, BufferAllocator allocator) {
     super(field, allocator);
-    this.lengthVector = new UInt${type.width}Vector(null, allocator);
-    this.lengthVectorMutator = lengthVector.getMutator();
+    this.offsetVector = new UInt${type.width}Vector(null, allocator);
   }
 
-  public byte[] get(int index) {
-    checkArgument(index >= 0);
-    int startIdx = 0;
-    int size = 0;
-    if (index == 0) {
-      size = lengthVector.get(1);
-    } else {
-      startIdx = lengthVector.get(index);
-      size = lengthVector.get(index + 1) - startIdx;
-    }
-    checkState(size >= 0);
-    byte[] dst = new byte[size];
-    data.getBytes(startIdx, dst, 0, size);
-    return dst;
-  }
 
-  @Override
-  public int getAllocatedSize() {
-    return lengthVector.getAllocatedSize() + totalBytes;
+  int getSizeFromCount(int valueCount) {
+    return valueCount * ${type.width};
   }
-
+  
+  public int getValueCapacity(){
+    return offsetVector.getValueCapacity();
+  }
+  
+  public int getByteCapacity(){
+    return data.capacity(); 
+  }
+  
   /**
-   * Get the size requirement (in bytes) for the given number of values.  Only accurate
-   * for fixed width value vectors.
+   * Return the number of bytes contained in the current var len byte vector.
+   * @return
    */
-  public int getSizeFromCount(int valueCount) {
-    return valueCount * ${type.width};
+  public int getVarByteLength(){
+    return offsetVector.getAccessor().get(recordCount); 
   }
-
+  
   @Override
-  protected void clear() {
-    super.clear();
-    lengthVector.clear();
+  public FieldMetadata getMetadata() {
+    int len = recordCount * ${type.width} + getVarByteLength();
+    return FieldMetadata.newBuilder()
+             .setDef(getField().getDef())
+             .setValueCount(recordCount)
+             .setVarByteLength(getVarByteLength())
+             .setBufferLength(len)
+             .build();
   }
 
-  /**
-   * Allocate a new memory space for this vector.  Must be called prior to using the ValueVector.
-   *
-   * @param valueCount
-   *          The number of values which can be contained within this vector.
-   */
-  public void allocateNew(int totalBytes, ByteBuf sourceBuffer, int valueCount) {
-    super.allocateNew(totalBytes, sourceBuffer, valueCount);
-    lengthVector.allocateNew(valueCount);
+  public int load(int dataBytes, int valueCount, ByteBuf buf){
+    this.recordCount = valueCount;
+    int loaded = offsetVector.load(valueCount+1, buf);
+    data = buf.slice(loaded, dataBytes);
+    data.retain();
+    return loaded + dataBytes;
+  }
+  
+  @Override
+  public void load(FieldMetadata metadata, ByteBuf buffer) {
+    assert this.field.getDef().equals(metadata.getDef());
+    int loaded = load(metadata.getVarByteLength(), metadata.getValueCount(), buffer);
+    assert metadata.getBufferLength() == loaded;
+  }
+  
+  @Override
+  public void clear() {
+    super.clear();
+    offsetVector.clear();
   }
 
   @Override
   public ByteBuf[] getBuffers() {
-    return new ByteBuf[]{lengthVector.data, data};
+    return new ByteBuf[]{offsetVector.data, this.data};
   }
-
-  public Object getObject(int index) {
-    return get(index);
+  
+  public void allocateNew(int totalBytes, int valueCount) {
+    clear();
+    assert totalBytes >= 0;
+    data = allocator.buffer(totalBytes);
+    data.retain();
+    data.readerIndex(0);
+    offsetVector.allocateNew(valueCount+1);
   }
 
+  public Accessor getAccessor(){
+    return accessor;
+  }
+  
   public Mutator getMutator() {
-    return new Mutator();
+    return mutator;
   }
   
+  public final class Accessor extends BaseValueVector.BaseAccessor{
+    
+    public byte[] get(int index) {
+      assert index >= 0;
+      int startIdx = offsetVector.getAccessor().get(index);
+      int length = offsetVector.getAccessor().get(index + 1) - startIdx;
+      assert length >= 0;
+      byte[] dst = new byte[length];
+      data.getBytes(startIdx, dst, 0, length);
+      return dst;
+    }
+    
+    public void get(int index, ByteHolder holder){
+      assert index >= 0;
+      holder.start = offsetVector.getAccessor().get(index);
+      holder.length = offsetVector.getAccessor().get(index + 1) - holder.start;
+      assert holder.length >= 0;
+      holder.buffer = offsetVector.data;
+    }
+    
+    public Object getObject(int index) {
+      return get(index);
+    }
+    
+    public int getRecordCount() {
+      return recordCount;
+    }
+  }
   
   /**
    * Mutable${minor.class} implements a vector of variable width values.  Elements in the vector
-   * are accessed by position from the logical start of the vector.  A fixed width lengthVector
+   * are accessed by position from the logical start of the vector.  A fixed width offsetVector
    * is used to convert an element's position to it's offset from the start of the (0-based)
    * ByteBuf.  Size is inferred by adjacent elements.
    *   The width of each element is ${type.width} byte(s)
@@ -112,7 +158,7 @@ public final class ${minor.class}Vector extends ValueVector {
    *
    * NB: this class is automatically generated from ValueVectorTypes.tdd using FreeMarker.
    */
-  public class Mutator implements ValueVector.Mutator{
+  public final class Mutator extends BaseValueVector.BaseMutator{
 
     /**
      * Set the variable length element at the specified index to the supplied byte array.
@@ -121,23 +167,24 @@ public final class ${minor.class}Vector extends ValueVector {
      * @param bytes   array of bytes to write
      */
     public void set(int index, byte[] bytes) {
-      checkArgument(index >= 0);
-      if (index == 0) {
-        lengthVectorMutator.set(0, 0);
-        lengthVectorMutator.set(1, bytes.length);
-        data.setBytes(0, bytes);
-      } else {
-        int currentOffset = lengthVector.get(index);
-        // set the end offset of the buffer
-        lengthVectorMutator.set(index + 1, currentOffset + bytes.length);
-        data.setBytes(currentOffset, bytes);
-      }
+      assert index >= 0;
+      int currentOffset = offsetVector.getAccessor().get(index);
+      offsetVector.getMutator().set(index + 1, currentOffset + bytes.length);
+      data.setBytes(currentOffset, bytes);
     }
 
-    @Override
-    public void setRecordCount(int recordCount) {
-      ${minor.class}Vector.this.setRecordCount(recordCount);
-      lengthVector.setRecordCount(recordCount);
+    public void set(int index, int start, int length, ByteBuf buffer){
+      assert index >= 0;
+      int currentOffset = offsetVector.getAccessor().get(index);
+      offsetVector.getMutator().set(index + 1, currentOffset + length);
+      ByteBuf bb = buffer.slice(start, length);
+      data.setBytes(currentOffset, bb);
+    }
+
+    public void setValueCount(int recordCount) {
+      ${minor.class}Vector.this.recordCount = recordCount;
+      data.writerIndex(recordCount * ${type.width});
+      offsetVector.getMutator().setValueCount(recordCount+1);
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/9ca9eb9b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockRecordReader.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockRecordReader.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockRecordReader.java
index cd3371d..0f4619c 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockRecordReader.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockRecordReader.java
@@ -28,8 +28,11 @@ import org.apache.drill.exec.proto.SchemaDefProtos.DataMode;
 import org.apache.drill.exec.proto.SchemaDefProtos.MajorType;
 import org.apache.drill.exec.record.MaterializedField;
 import org.apache.drill.exec.store.RecordReader;
+import org.apache.drill.exec.vector.FixedWidthVector;
+import org.apache.drill.exec.vector.NonRepeatedMutator;
 import org.apache.drill.exec.vector.TypeHelper;
 import org.apache.drill.exec.vector.ValueVector;
+import org.apache.drill.exec.vector.VariableWidthVector;
 
 public class MockRecordReader implements RecordReader {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(MockRecordReader.class);
@@ -39,6 +42,7 @@ public class MockRecordReader implements RecordReader {
   private FragmentContext context;
   private ValueVector[] valueVectors;
   private int recordsRead;
+  private int batchRecordCount;
 
   public MockRecordReader(FragmentContext context, MockScanEntry config) {
     this.context = context;
@@ -60,7 +64,14 @@ public class MockRecordReader implements RecordReader {
     MaterializedField f = MaterializedField.create(new SchemaPath(name), fieldId, 0, type);
     ValueVector v;
     v = TypeHelper.getNewVector(f, context.getAllocator());
-    v.allocateNew(length);
+    if(v instanceof FixedWidthVector){
+      ((FixedWidthVector)v).allocateNew(length);  
+    }else if(v instanceof VariableWidthVector){
+      ((VariableWidthVector)v).allocateNew(50*length, length);
+    }else{
+      throw new UnsupportedOperationException(String.format("Unable to get allocate vector %s", v.getClass().getName()));
+    }
+    
     return v;
 
   }
@@ -71,7 +82,7 @@ public class MockRecordReader implements RecordReader {
       this.output = output;
       int estimateRowSize = getEstimatedRecordSize(config.getTypes());
       valueVectors = new ValueVector[config.getTypes().length];
-      int batchRecordCount = 250000 / estimateRowSize;
+      batchRecordCount = 250000 / estimateRowSize;
 
       for (int i = 0; i < config.getTypes().length; i++) {
         valueVectors[i] = getVector(i, config.getTypes()[i].getName(), config.getTypes()[i].getMajorType(), batchRecordCount);
@@ -86,13 +97,29 @@ public class MockRecordReader implements RecordReader {
 
   @Override
   public int next() {
-    int recordSetSize = Math.min(valueVectors[0].capacity(), this.config.getRecords()- recordsRead);
+    
+    int recordSetSize = Math.min(batchRecordCount, this.config.getRecords()- recordsRead);
+
     recordsRead += recordSetSize;
     for(ValueVector v : valueVectors){
+      if(v instanceof FixedWidthVector){
+        ((FixedWidthVector)v).allocateNew(recordSetSize);
+      }else if(v instanceof VariableWidthVector){
+        ((VariableWidthVector)v).allocateNew(50*recordSetSize, recordSetSize);
+      }else{
+        throw new UnsupportedOperationException();
+      }
+      
       logger.debug("MockRecordReader:  Generating random data for VV of type " + v.getClass().getName());
-      v.randomizeData();
+      ValueVector.Mutator m = v.getMutator();
+      m.randomizeData();
+      
+      if(m instanceof NonRepeatedMutator){
+        ((NonRepeatedMutator)m).setValueCount(recordSetSize);  
+      }else{
+        throw new UnsupportedOperationException();
+      }
       
-      v.getMutator().setRecordCount(recordSetSize);
     }
     return recordSetSize;
   }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/9ca9eb9b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/MaterializedField.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/MaterializedField.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/MaterializedField.java
index d1858f1..05fb576 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/MaterializedField.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/MaterializedField.java
@@ -105,6 +105,22 @@ public class MaterializedField implements Comparable<MaterializedField> {
   public DataMode getDataMode() {
     return def.getMajorType().getMode();
   }
+  
+  public MaterializedField getOtherNullableVersion(){
+    MajorType mt = def.getMajorType();
+    DataMode newDataMode = null;
+    switch(mt.getMode()){
+    case OPTIONAL:
+      newDataMode = DataMode.REQUIRED;
+      break;
+    case REQUIRED:
+      newDataMode = DataMode.OPTIONAL;
+      break;
+    default:
+      throw new UnsupportedOperationException();
+    }
+    return new MaterializedField(def.toBuilder().setMajorType(mt.toBuilder().setMode(newDataMode).build()).build());
+  }
 
   public boolean matches(SchemaPath path) {
     Iterator<NamePart> iter = def.getNameList().iterator();

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/9ca9eb9b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatchLoader.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatchLoader.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatchLoader.java
index be43026..a2dbd81 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatchLoader.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatchLoader.java
@@ -52,7 +52,7 @@ public class RecordBatchLoader implements Iterable<IntObjectCursor<ValueVector>>
    * @param def
    *          The definition for the record batch.
    * @param buf
-   *          The buffer that holds the data ssociated with the record batch
+   *          The buffer that holds the data associated with the record batch
    * @return Whether or not the schema changed since the previous load.
    * @throws SchemaChangeException 
    */
@@ -71,7 +71,8 @@ public class RecordBatchLoader implements Iterable<IntObjectCursor<ValueVector>>
       ValueVector v = vectors.remove(fieldDef.getFieldId());
       if (v != null) {
         if (v.getField().getDef().equals(fieldDef)) {
-          v.allocateNew(fmd.getBufferLength(), buf.slice(bufOffset, fmd.getBufferLength()), recordCount);
+          ValueVector.Mutator m = v.getMutator();
+          v.load(fmd, buf.slice(bufOffset, fmd.getBufferLength()));
           newVectors.put(fieldDef.getFieldId(), v);
           continue;
         } else {
@@ -83,7 +84,7 @@ public class RecordBatchLoader implements Iterable<IntObjectCursor<ValueVector>>
       schemaChanged = true;
       MaterializedField m = new MaterializedField(fieldDef);
       v = TypeHelper.getNewVector(m, allocator);
-      v.allocateNew(fmd.getBufferLength(), buf.slice(bufOffset, fmd.getBufferLength()), recordCount);
+      v.load(fmd, buf.slice(bufOffset, fmd.getBufferLength()));
       newVectors.put(fieldDef.getFieldId(), v);
     }
     

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/9ca9eb9b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/WritableBatch.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/WritableBatch.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/WritableBatch.java
index a367b6d..4b97768 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/WritableBatch.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/WritableBatch.java
@@ -93,8 +93,7 @@ public class WritableBatch {
         buffers.add(b);
         b.retain();
       }
-      // allocate new buffer to release hold on old buffer.
-      value.allocateNew(value.capacity());
+      value.clear();
     }
 
     public WritableBatch get(){

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/9ca9eb9b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/JSONRecordReader.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/JSONRecordReader.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/JSONRecordReader.java
index e637518..07ae20a 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/JSONRecordReader.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/JSONRecordReader.java
@@ -422,9 +422,11 @@ public class JSONRecordReader implements RecordReader {
             SchemaDefProtos.MajorType type = field.getFieldType();
             int fieldId = field.getFieldId();
             MaterializedField f = MaterializedField.create(new SchemaPath(field.getFieldName()), fieldId, parentFieldId, type);
+            
             ValueVector v = TypeHelper.getNewVector(f, allocator);
-            v.allocateNew(batchSize);
             VectorHolder holder = new VectorHolder(batchSize, v);
+            holder.allocateNew(batchSize);
+            
             valueVectorMap.put(fieldId, holder);
             outputMutator.addField(fieldId, v);
             return holder;

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/9ca9eb9b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/VectorHolder.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/VectorHolder.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/VectorHolder.java
index fa0cbd5..d594b9e 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/VectorHolder.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/VectorHolder.java
@@ -18,16 +18,20 @@
 
 package org.apache.drill.exec.store;
 
+import org.apache.drill.exec.vector.FixedWidthVector;
 import org.apache.drill.exec.vector.ValueVector;
+import org.apache.drill.exec.vector.VariableWidthVector;
 
 public class VectorHolder {
     private int length;
     private ValueVector vector;
+    private ValueVector.Mutator mutator;
     private int currentLength;
 
     VectorHolder(int length, ValueVector vector) {
         this.length = length;
         this.vector = vector;
+        this.mutator = vector.getMutator();
     }
 
     public ValueVector getValueVector() {
@@ -51,6 +55,21 @@ public class VectorHolder {
 
     public void reset() {
         currentLength = 0;
-        vector.allocateNew(length);
+        allocateNew(length);
+        
+    }
+    
+    public void allocateNew(int valueLength){
+      if(vector instanceof FixedWidthVector){
+        ((FixedWidthVector)vector).allocateNew(valueLength);  
+      }else if(vector instanceof VariableWidthVector){
+        ((VariableWidthVector)vector).allocateNew(valueLength * 10, valueLength);  
+      }else{
+        throw new UnsupportedOperationException();
+      }
+    }
+    
+    public ValueVector.Mutator getMutator(){
+      return mutator;
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/9ca9eb9b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/BaseDataValueVector.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/BaseDataValueVector.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/BaseDataValueVector.java
new file mode 100644
index 0000000..dd2b504
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/BaseDataValueVector.java
@@ -0,0 +1,47 @@
+package org.apache.drill.exec.vector;
+
+import io.netty.buffer.ByteBuf;
+
+import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.proto.UserBitShared.FieldMetadata;
+import org.apache.drill.exec.record.DeadBuf;
+import org.apache.drill.exec.record.MaterializedField;
+
+abstract class BaseDataValueVector extends BaseValueVector{
+
+  protected ByteBuf data = DeadBuf.DEAD_BUFFER;
+  protected int recordCount;
+  
+  public BaseDataValueVector(MaterializedField field, BufferAllocator allocator) {
+    super(field, allocator);
+    
+  }
+
+  /**
+   * Release the underlying ByteBuf and reset the ValueVector
+   */
+  @Override
+  public void clear() {
+    if (data != DeadBuf.DEAD_BUFFER) {
+      data.release();
+      data = DeadBuf.DEAD_BUFFER;
+      recordCount = 0;
+    }
+  }
+  
+  @Override
+  public ByteBuf[] getBuffers(){
+    return new ByteBuf[]{data};
+  }
+  
+  public int getBufferSize() {
+    return data.writerIndex();
+  }
+
+  @Override
+  public FieldMetadata getMetadata() {
+    return null;
+  }
+  
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/9ca9eb9b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/BaseValueVector.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/BaseValueVector.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/BaseValueVector.java
new file mode 100644
index 0000000..a8678f5
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/BaseValueVector.java
@@ -0,0 +1,38 @@
+package org.apache.drill.exec.vector;
+
+import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.record.MaterializedField;
+
+abstract class BaseValueVector implements ValueVector{
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(BaseValueVector.class);
+  
+  protected final BufferAllocator allocator;
+  protected final MaterializedField field;
+
+  BaseValueVector(MaterializedField field, BufferAllocator allocator) {
+    this.allocator = allocator;
+    this.field = field;
+  }
+  
+  @Override
+  public void close() {
+    clear();
+  }
+  
+  @Override
+  public MaterializedField getField() {
+    return field;
+  }
+  
+  abstract class BaseAccessor implements ValueVector.Accessor{
+
+    
+    public void reset(){}
+  }
+  
+  abstract class BaseMutator implements NonRepeatedMutator{
+    public void reset(){}
+  }
+  
+}
+

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/9ca9eb9b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/BitVector.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/BitVector.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/BitVector.java
index d18a29d..9d247f5 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/BitVector.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/BitVector.java
@@ -1,8 +1,11 @@
 package org.apache.drill.exec.vector;
 
+import io.netty.buffer.ByteBuf;
+
 import java.util.Random;
 
 import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.proto.UserBitShared.FieldMetadata;
 import org.apache.drill.exec.record.DeadBuf;
 import org.apache.drill.exec.record.MaterializedField;
 /**
@@ -13,64 +16,93 @@ import org.apache.drill.exec.record.MaterializedField;
  *
  * NB: this class is automatically generated from ValueVectorTypes.tdd using FreeMarker.
  */
-public final class BitVector extends ValueVector {
+public final class BitVector extends BaseDataValueVector implements FixedWidthVector{
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(BitVector.class);
 
+  private int valueCapacity;
+  
   public BitVector(MaterializedField field, BufferAllocator allocator) {
     super(field, allocator);
   }
 
+  private int getSizeFromCount(int valueCount) {
+    return (int) Math.ceil(valueCount / 8);
+  }
+  
   /**
-   * Get the byte holding the desired bit, then mask all other bits.  Iff the result is 0, the
-   * bit was not set.
+   * Allocate a new memory space for this vector.  Must be called prior to using the ValueVector.
    *
-   * @param  index   position of the bit in the vector
-   * @return 1 if set, otherwise 0
+   * @param valueCount  The number of values which can be contained within this vector.
    */
-  public int get(int index) {
-    // logger.debug("BIT GET: index: {}, byte: {}, mask: {}, masked byte: {}",
-    //             index,
-    //             data.getByte((int)Math.floor(index/8)),
-    //             (int)Math.pow(2, (index % 8)),
-    //             data.getByte((int)Math.floor(index/8)) & (int)Math.pow(2, (index % 8)));
-    return ((data.getByte((int)Math.floor(index/8)) & (int)Math.pow(2, (index % 8))) == 0) ? 0 : 1;
+  public void allocateNew(int valueCount) {
+    clear();
+    valueCapacity = valueCount;
+    int valueSize = getSizeFromCount(valueCount);
+    data = allocator.buffer(valueSize);
+    for (int i = 0; i < getSizeFromCount(valueCount); i++) {
+      data.setByte(i, 0);
+    }
   }
-
+  
   @Override
-  public Object getObject(int index) {
-    return new Boolean(get(index) != 0);
+  public int load(int valueCount, ByteBuf buf){
+    clear();
+    this.recordCount = valueCount;
+    int len = getSizeFromCount(valueCount);
+    data = buf.slice(0, len);
+    data.retain();
+    return len;
   }
-
-  /**
-   * Get the size requirement (in bytes) for the given number of values.
-   */
+  
   @Override
-  public int getSizeFromCount(int valueCount) {
-    return (int) Math.ceil(valueCount / 8);
+  public void load(FieldMetadata metadata, ByteBuf buffer) {
+    assert this.field.getDef().equals(metadata.getDef());
+    int loaded = load(metadata.getValueCount(), buffer);
+    assert metadata.getBufferLength() == loaded;
   }
-
+  
   @Override
-  public int getAllocatedSize() {
-    return totalBytes;
+  public int getValueCapacity() {
+    return valueCapacity;
   }
 
   public Mutator getMutator() {
     return new Mutator();
   }
 
-  /**
-   * Allocate a new memory space for this vector.  Must be called prior to using the ValueVector.
-   *
-   * @param valueCount  The number of values which can be contained within this vector.
-   */
-  @Override
-  public void allocateNew(int valueCount) {
-    allocateNew(getSizeFromCount(valueCount), null, valueCount);
-    for (int i = 0; i < getSizeFromCount(valueCount); i++) {
-      data.setByte(i, 0);
-    }
+  public Accessor getAccessor(){
+    return new Accessor();
   }
+  
+  
+  public class Accessor extends BaseAccessor{
 
+    /**
+     * Get the byte holding the desired bit, then mask all other bits.  Iff the result is 0, the
+     * bit was not set.
+     *
+     * @param  index   position of the bit in the vector
+     * @return 1 if set, otherwise 0
+     */
+    public int get(int index) {
+      // logger.debug("BIT GET: index: {}, byte: {}, mask: {}, masked byte: {}",
+      //             index,
+      //             data.getByte((int)Math.floor(index/8)),
+      //             (int)Math.pow(2, (index % 8)),
+      //             data.getByte((int)Math.floor(index/8)) & (int)Math.pow(2, (index % 8)));
+      return ((data.getByte((int)Math.floor(index/8)) & (int)Math.pow(2, (index % 8))) == 0) ? 0 : 1;
+    }
+    
+    @Override
+    public Object getObject(int index) {
+      return new Boolean(get(index) != 0);
+    }
+    
+    public int getRecordCount() {
+      return recordCount;
+    }
+    
+  }
   
   /**
    * MutableBit implements a vector of bit-width values.  Elements in the vector are accessed
@@ -79,7 +111,7 @@ public final class BitVector extends ValueVector {
    *
    * NB: this class is automatically generated from ValueVectorTypes.tdd using FreeMarker.
    */
-  public class Mutator implements ValueVector.Mutator{
+  public class Mutator extends BaseMutator{
 
     private Mutator(){}
     
@@ -102,10 +134,9 @@ public final class BitVector extends ValueVector {
       data.setByte((int) Math.floor(index/8), currentByte);
     }
 
-    
-    @Override
-    public void setRecordCount(int recordCount) {
-      BitVector.this.setRecordCount(recordCount);
+    public void setValueCount(int recordCount) {
+      BitVector.this.recordCount = recordCount;
+      data.writerIndex(getSizeFromCount(recordCount));
     }
 
     @Override
@@ -119,5 +150,6 @@ public final class BitVector extends ValueVector {
         }
       }
     }
+
   }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/9ca9eb9b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/ByteHolder.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/ByteHolder.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/ByteHolder.java
new file mode 100644
index 0000000..45d8019
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/ByteHolder.java
@@ -0,0 +1,12 @@
+package org.apache.drill.exec.vector;
+
+import io.netty.buffer.ByteBuf;
+
+public class ByteHolder {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ByteHolder.class);
+  
+  public ByteBuf buffer;
+  public int start;
+  public int length;
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/9ca9eb9b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/FixedWidthVector.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/FixedWidthVector.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/FixedWidthVector.java
new file mode 100644
index 0000000..0e3e3e9
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/FixedWidthVector.java
@@ -0,0 +1,23 @@
+package org.apache.drill.exec.vector;
+
+import io.netty.buffer.ByteBuf;
+
+public interface FixedWidthVector extends ValueVector{
+  
+  /**
+   * Allocate a new memory space for this vector.  Must be called prior to using the ValueVector.
+   *
+   * @param totalBytes   Desired size of the underlying data buffer.
+   * @param valueCount   Number of values in the vector.
+   */
+  public void allocateNew(int valueCount);
+  
+  /**
+   * Load the records in the provided buffer based on the given number of values.
+   * @param valueCount Number of values the buffer contains.
+   * @param buf Incoming buffer.
+   * @return The number of bytes of the buffer that were consumed.
+   */
+  public int load(int valueCount, ByteBuf buf);
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/9ca9eb9b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/NonRepeatedMutator.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/NonRepeatedMutator.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/NonRepeatedMutator.java
new file mode 100644
index 0000000..e9bdcbd
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/NonRepeatedMutator.java
@@ -0,0 +1,7 @@
+package org.apache.drill.exec.vector;
+
+import org.apache.drill.exec.vector.ValueVector.Mutator;
+
+public interface NonRepeatedMutator extends Mutator{
+  public void setValueCount(int recordCount);
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/9ca9eb9b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/RepeatedFixedWidthVector.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/RepeatedFixedWidthVector.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/RepeatedFixedWidthVector.java
new file mode 100644
index 0000000..35261d7
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/RepeatedFixedWidthVector.java
@@ -0,0 +1,22 @@
+package org.apache.drill.exec.vector;
+
+import io.netty.buffer.ByteBuf;
+
+public interface RepeatedFixedWidthVector extends ValueVector{
+  /**
+   * Allocate a new memory space for this vector.  Must be called prior to using the ValueVector.
+   *
+   * @param parentValueCount   Number of separate repeating groupings.
+   * @param childValueCount   Number of supported values in the vector.
+   */
+  public void allocateNew(int parentValueCount, int childValueCount);
+  
+  /**
+   * Load the records in the provided buffer based on the given number of values.
+   * @param parentValueCount   Number of separate repeating groupings.
+   * @param valueCount Number atomic values the buffer contains.
+   * @param buf Incoming buffer.
+   * @return The number of bytes of the buffer that were consumed.
+   */
+  public int load(int parentValueCount, int childValueCount, ByteBuf buf);
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/9ca9eb9b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/RepeatedVariableWidthVector.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/RepeatedVariableWidthVector.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/RepeatedVariableWidthVector.java
new file mode 100644
index 0000000..4f22481
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/RepeatedVariableWidthVector.java
@@ -0,0 +1,24 @@
+package org.apache.drill.exec.vector;
+
+import io.netty.buffer.ByteBuf;
+
+public interface RepeatedVariableWidthVector extends ValueVector{
+  /**
+   * Allocate a new memory space for this vector.  Must be called prior to using the ValueVector.
+   *
+   * @param totalBytes   Desired size of the underlying data buffer.
+   * @param parentValueCount   Number of separate repeating groupings.
+   * @param childValueCount   Number of supported values in the vector.
+   */
+  public void allocateNew(int totalBytes, int parentValueCount, int childValueCount);
+  
+  /**
+   * Load the records in the provided buffer based on the given number of values.
+   * @param dataBytes   The number of bytes associated with the data array.
+   * @param parentValueCount   Number of separate repeating groupings.
+   * @param childValueCount   Number of supported values in the vector.
+   * @param buf Incoming buffer.
+   * @return The number of bytes of the buffer that were consumed.
+   */
+  public int load(int dataBytes, int parentValueCount, int childValueCount, ByteBuf buf);
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/9ca9eb9b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/ValueVector.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/ValueVector.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/ValueVector.java
index 718478e..328182b 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/ValueVector.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/ValueVector.java
@@ -21,9 +21,8 @@ import io.netty.buffer.ByteBuf;
 
 import java.io.Closeable;
 
-import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.proto.SchemaDefProtos.FieldDef;
 import org.apache.drill.exec.proto.UserBitShared.FieldMetadata;
-import org.apache.drill.exec.record.DeadBuf;
 import org.apache.drill.exec.record.MaterializedField;
 
 /**
@@ -31,42 +30,25 @@ import org.apache.drill.exec.record.MaterializedField;
  * value vectors.  The template approach was chosen due to the lack of multiple inheritence.  It
  * is also important that all related logic be as efficient as possible.
  */
-public abstract class ValueVector implements Closeable {
+public interface ValueVector extends Closeable {
 
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ValueVector.class);
+//  /**
+//   * Get the explicitly specified size of the allocated buffer, if available.  Otherwise
+//   * calculate the size based on width and record count.
+//   */
+//  public abstract int getAllocatedSize();
 
-  protected final BufferAllocator allocator;
-  protected ByteBuf data = DeadBuf.DEAD_BUFFER;
-  protected MaterializedField field;
-  protected int recordCount;
-  protected int totalBytes;
-
-  ValueVector(MaterializedField field, BufferAllocator allocator) {
-    this.allocator = allocator;
-    this.field = field;
-  }
 
   /**
-   * Get the explicitly specified size of the allocated buffer, if available.  Otherwise
-   * calculate the size based on width and record count.
+   * Alternative to clear().  Allows use as closeable in try-with-resources.
    */
-  public abstract int getAllocatedSize();
-
+  public void close();
+  
   /**
-   * Get the size requirement (in bytes) for the given number of values.  Takes derived
-   * type specs into account.
+   * Release the underlying ByteBuf and reset the ValueVector to empty.
    */
-  public abstract int getSizeFromCount(int valueCount);
-
-  /**
-   * Get the Java Object representation of the element at the specified position
-   *
-   * @param index   Index of the value to get
-   */
-  public abstract Object getObject(int index);
-
+  public void clear();
   
-  public abstract Mutator getMutator();
   
   /**
    * Return the underlying buffers associated with this vector. Note that this doesn't impact the
@@ -76,117 +58,72 @@ public abstract class ValueVector implements Closeable {
    *
    * @return The underlying ByteBuf.
    */
-  public ByteBuf[] getBuffers() {
-    return new ByteBuf[]{data};
-  }
-
-  /**
-   * Returns the maximum number of values contained within this vector.
-   * @return Vector size
-   */
-  public int capacity() {
-    return getRecordCount();
-  }
-
+  public abstract ByteBuf[] getBuffers();
+  
   /**
-   * Release supporting resources.
+   * Load the data provided in the buffer.  Typically used when deserializing from the wire.
+   * @param metadata Metadata used to decode the incoming buffer.
+   * @param buffer The buffer that contains the ValueVector.
    */
-  @Override
-  public void close() {
-    clear();
-  }
+  public void load(FieldMetadata metadata, ByteBuf buffer);
 
-  /**
-   * Get information about how this field is materialized.
-   * @return
-   */
-  public MaterializedField getField() {
-    return field;
-  }
 
   /**
-   * Get the number of records allocated for this value vector.
-   * @return number of allocated records
+   * Given the current buffer allocation, return the maximum number of values that this buffer can contain.
+   * @return Maximum values buffer can contain.  In the case of a Repeated field, this is the number of atoms, not repeated groups.
    */
-  public int getRecordCount() {
-    return recordCount;
-  }
-
+  public int getValueCapacity();
+  
   /**
-   * Get the metadata for this field.
+   * Get information about how this field is materialized.
    * @return
    */
-  public FieldMetadata getMetadata() {
-    int len = 0;
-    for(ByteBuf b : getBuffers()){
-      len += b.writerIndex();
-    }
-    return FieldMetadata.newBuilder()
-             .setDef(getField().getDef())
-             .setValueCount(getRecordCount())
-             .setBufferLength(len)
-             .build();
-  }
-
+  public MaterializedField getField();
+  
   /**
-   * Allocate a new memory space for this vector.  Must be called prior to using the ValueVector.
-   *
-   * @param totalBytes   Optional desired size of the underlying buffer.  Specifying 0 will
-   *                     estimate the size based on valueCount.
-   * @param sourceBuffer Optional ByteBuf to use for storage (null will allocate automatically).
-   * @param valueCount   Number of values in the vector.
+   * Get the metadata for this field.  Used in serialization
+   * @return FieldMetadata for this field.
    */
-  public void allocateNew(int totalBytes, ByteBuf sourceBuffer, int valueCount) {
-    clear();
-    this.recordCount = valueCount;
-    this.totalBytes = totalBytes > 0 ? totalBytes : getSizeFromCount(valueCount);
-    this.data = (sourceBuffer != null) ? sourceBuffer : allocator.buffer(this.totalBytes);
-    this.data.retain();
-    data.readerIndex(0);
-  }
-
+  public FieldMetadata getMetadata();
+  
   /**
-   * Allocate a new memory space for this vector.  Must be called prior to using the ValueVector.
-   *
-   * @param valueCount
-   *          The number of elements which can be contained within this vector.
+   * Get Accessor to read value vector data.
+   * @return 
    */
-  public void allocateNew(int valueCount) {
-    allocateNew(0, null, valueCount);
-  }
-
+  public abstract Accessor getAccessor();
+  
   /**
-   * Release the underlying ByteBuf and reset the ValueVector
+   * Get a Mutator to update this vectors data.
+   * @return
    */
-  protected void clear() {
-    if (data != DeadBuf.DEAD_BUFFER) {
-      data.release();
-      data = DeadBuf.DEAD_BUFFER;
-      recordCount = 0;
-      totalBytes = 0;
-    }
-  }
+  public abstract Mutator getMutator();
 
-  //public abstract <T extends Mutator> T getMutator();
   
-  /**
-   * Define the number of records that are in this value vector.
-   * @param recordCount Number of records active in this vector.
-   */
-  void setRecordCount(int recordCount) {
-    data.writerIndex(getSizeFromCount(recordCount));
-    this.recordCount = recordCount;
+  public interface Accessor{
+
+//    /**
+//     * Get the number of records allocated for this value vector.
+//     * @return number of allocated records
+//     */
+//    public int getRecordCount();
+
+    /**
+     * Get the Java Object representation of the element at the specified position.  Useful for testing.
+     *
+     * @param index   Index of the value to get
+     */
+    public abstract Object getObject(int index);
+    
+    public void reset();
   }
-
-  /**
-   * For testing only -- randomize the buffer contents
-   */
-  public void randomizeData() { }
-
   
-  public static interface Mutator{
+  
+    
+  
+  
+  public interface Mutator{
+    public void reset();
     public void randomizeData();
-    public void setRecordCount(int recordCount);
   }
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/9ca9eb9b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/VariableWidthVector.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/VariableWidthVector.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/VariableWidthVector.java
new file mode 100644
index 0000000..c26cbab
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/VariableWidthVector.java
@@ -0,0 +1,29 @@
+package org.apache.drill.exec.vector;
+
+import io.netty.buffer.ByteBuf;
+
+public interface VariableWidthVector extends ValueVector{
+
+  /**
+   * Allocate a new memory space for this vector.  Must be called prior to using the ValueVector.
+   *
+   * @param totalBytes   Desired size of the underlying data buffer.
+   * @param valueCount   Number of values in the vector.
+   */
+  public void allocateNew(int totalBytes, int valueCount);
+  
+  /**
+   * Provide the maximum amount of variable width bytes that can be stored int his vector.
+   * @return
+   */
+  public int getByteCapacity();
+  
+  /**
+   * Load the records in the provided buffer based on the given number of values.
+   * @param dataBytes   The number of bytes associated with the data array.
+   * @param valueCount Number of values the buffer contains.
+   * @param buf Incoming buffer.
+   * @return The number of bytes of the buffer that were consumed.
+   */
+  public int load(int dataBytes, int valueCount, ByteBuf buf);
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/9ca9eb9b/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 ba103ed..a90382a 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
@@ -230,8 +230,7 @@ public class Foreman implements Runnable, Closeable, Comparable<Object>{
   }
 
   public QueryResult getResult(UserClientConnection connection, RequestResults req) {
-
-    return null;
+    throw new UnsupportedOperationException();
   }
 
 
@@ -254,7 +253,7 @@ public class Foreman implements Runnable, Closeable, Comparable<Object>{
     }
     
     void cleanupAndSendResult(QueryResult result){
-      ForemanManagerListener.this.cleanupAndSendResult(result);
+      Foreman.this.cleanupAndSendResult(result);
     }
     
   }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/9ca9eb9b/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 3fe0622..cac6aa2 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
@@ -86,7 +86,7 @@ public class TestSimpleFragmentRun extends PopUnitTestBase {
           } else {
             System.out.print("\t");
           }
-          System.out.print(v.value.getObject(i));
+          System.out.print(v.value.getAccessor().getObject(i));
         }
         if(!first) System.out.println();
       }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/9ca9eb9b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/record/vector/TestValueVector.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/record/vector/TestValueVector.java b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/record/vector/TestValueVector.java
index ae4f644..6a1f3ad 100644
--- a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/record/vector/TestValueVector.java
+++ b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/record/vector/TestValueVector.java
@@ -1,7 +1,6 @@
 package org.apache.drill.exec.record.vector;
 
 import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
 
 import java.nio.charset.Charset;
 
@@ -46,14 +45,14 @@ public class TestValueVector {
     m.set(100, 102);
     m.set(1022, 103);
     m.set(1023, 104);
-    assertEquals(100, v.get(0));
-    assertEquals(101, v.get(1));
-    assertEquals(102, v.get(100));
-    assertEquals(103, v.get(1022));
-    assertEquals(104, v.get(1023));
+    assertEquals(100, v.getAccessor().get(0));
+    assertEquals(101, v.getAccessor().get(1));
+    assertEquals(102, v.getAccessor().get(100));
+    assertEquals(103, v.getAccessor().get(1022));
+    assertEquals(104, v.getAccessor().get(1023));
 
     // Ensure unallocated space returns 0
-    assertEquals(0, v.get(3));
+    assertEquals(0, v.getAccessor().get(3));
   }
 
   @Test
@@ -74,7 +73,7 @@ public class TestValueVector {
     // Create a new value vector for 1024 integers
     NullableVarChar2Vector v = new NullableVarChar2Vector(field, allocator);
     NullableVarChar2Vector.Mutator m = v.getMutator();
-    v.allocateNew(1024);
+    v.allocateNew(1024*10, 1024);
 
     // Create and set 3 sample strings
     String str1 = new String("AAAAA1");
@@ -85,14 +84,14 @@ public class TestValueVector {
     m.set(2, str3.getBytes(Charset.forName("UTF-8")));
 
     // Check the sample strings
-    assertEquals(str1, new String(v.get(0), Charset.forName("UTF-8")));
-    assertEquals(str2, new String(v.get(1), Charset.forName("UTF-8")));
-    assertEquals(str3, new String(v.get(2), Charset.forName("UTF-8")));
+    assertEquals(str1, new String(v.getAccessor().get(0), Charset.forName("UTF-8")));
+    assertEquals(str2, new String(v.getAccessor().get(1), Charset.forName("UTF-8")));
+    assertEquals(str3, new String(v.getAccessor().get(2), Charset.forName("UTF-8")));
 
     // Ensure null value throws
     boolean b = false;
     try {
-      v.get(3);
+      v.getAccessor().get(3);
     } catch(AssertionError e) { 
       b = true;
     }finally{
@@ -130,17 +129,17 @@ public class TestValueVector {
     m.set(100, 102);
     m.set(1022, 103);
     m.set(1023, 104);
-    assertEquals(100, v.get(0));
-    assertEquals(101, v.get(1));
-    assertEquals(102, v.get(100));
-    assertEquals(103, v.get(1022));
-    assertEquals(104, v.get(1023));
+    assertEquals(100, v.getAccessor().get(0));
+    assertEquals(101, v.getAccessor().get(1));
+    assertEquals(102, v.getAccessor().get(100));
+    assertEquals(103, v.getAccessor().get(1022));
+    assertEquals(104, v.getAccessor().get(1023));
 
     // Ensure null values throw
     {
       boolean b = false;
       try {
-        v.get(3);
+        v.getAccessor().get(3);
       } catch(AssertionError e) { 
         b = true;
       }finally{
@@ -155,7 +154,7 @@ public class TestValueVector {
     {
       boolean b = false;
       try {
-        v.get(0);
+        v.getAccessor().get(0);
       } catch(AssertionError e) { 
         b = true;
       }finally{
@@ -170,18 +169,18 @@ public class TestValueVector {
     m.set(100, 102);
     m.set(1022, 103);
     m.set(1023, 104);
-    assertEquals(100, v.get(0));
-    assertEquals(101, v.get(1));
-    assertEquals(102, v.get(100));
-    assertEquals(103, v.get(1022));
-    assertEquals(104, v.get(1023));
+    assertEquals(100, v.getAccessor().get(0));
+    assertEquals(101, v.getAccessor().get(1));
+    assertEquals(102, v.getAccessor().get(100));
+    assertEquals(103, v.getAccessor().get(1022));
+    assertEquals(104, v.getAccessor().get(1023));
 
     // Ensure null values throw
     
     {
       boolean b = false;
       try {
-        v.get(3);
+        v.getAccessor().get(3);
       } catch(AssertionError e) { 
         b = true;
       }finally{
@@ -219,17 +218,17 @@ public class TestValueVector {
     m.set(100, 102.3f);
     m.set(1022, 103.4f);
     m.set(1023, 104.5f);
-    assertEquals(100.1f, v.get(0), 0);
-    assertEquals(101.2f, v.get(1), 0);
-    assertEquals(102.3f, v.get(100), 0);
-    assertEquals(103.4f, v.get(1022), 0);
-    assertEquals(104.5f, v.get(1023), 0);
+    assertEquals(100.1f, v.getAccessor().get(0), 0);
+    assertEquals(101.2f, v.getAccessor().get(1), 0);
+    assertEquals(102.3f, v.getAccessor().get(100), 0);
+    assertEquals(103.4f, v.getAccessor().get(1022), 0);
+    assertEquals(104.5f, v.getAccessor().get(1023), 0);
 
     // Ensure null values throw
     {
       boolean b = false;
       try {
-        v.get(3);
+        v.getAccessor().get(3);
       } catch(AssertionError e) { 
         b = true;
       }finally{
@@ -243,7 +242,7 @@ public class TestValueVector {
     {
       boolean b = false;
       try {
-        v.get(0);
+        v.getAccessor().get(0);
       } catch(AssertionError e) { 
         b = true;
       }finally{
@@ -279,27 +278,27 @@ public class TestValueVector {
     m.set(1, 0);
     m.set(100, 0);
     m.set(1022, 1);
-    assertEquals(1, v.get(0));
-    assertEquals(0, v.get(1));
-    assertEquals(0, v.get(100));
-    assertEquals(1, v.get(1022));
+    assertEquals(1, v.getAccessor().get(0));
+    assertEquals(0, v.getAccessor().get(1));
+    assertEquals(0, v.getAccessor().get(100));
+    assertEquals(1, v.getAccessor().get(1022));
 
     // test setting the same value twice
     m.set(0, 1);
     m.set(0, 1);
     m.set(1, 0);
     m.set(1, 0);
-    assertEquals(1, v.get(0));
-    assertEquals(0, v.get(1));
+    assertEquals(1, v.getAccessor().get(0));
+    assertEquals(0, v.getAccessor().get(1));
 
     // test toggling the values
     m.set(0, 0);
     m.set(1, 1);
-    assertEquals(0, v.get(0));
-    assertEquals(1, v.get(1));
+    assertEquals(0, v.getAccessor().get(0));
+    assertEquals(1, v.getAccessor().get(1));
 
     // Ensure unallocated space returns 0
-    assertEquals(0, v.get(3));
+    assertEquals(0, v.getAccessor().get(3));
   }
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/9ca9eb9b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/JSONRecordReaderTest.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/JSONRecordReaderTest.java b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/JSONRecordReaderTest.java
index 117414c..4b35313 100644
--- a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/JSONRecordReaderTest.java
+++ b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/JSONRecordReaderTest.java
@@ -73,7 +73,7 @@ public class JSONRecordReaderTest {
             return;
         }
 
-        T val = (T) valueVector.getObject(index);
+        T val = (T) valueVector.getAccessor().getObject(index);
         if (val instanceof byte[]) {
             assertTrue(Arrays.equals((byte[]) value, (byte[]) val));
         } else {


Re: [47/53] [abbrv] Working Project merge build

Posted by Ted Dunning <te...@gmail.com>.
On Fri, Jul 19, 2013 at 6:58 PM, <ja...@apache.org> wrote:

>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/3d41be41/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/BufBitSet.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/BufBitSet.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/BufBitSet.java
> deleted file mode 100644
> index 574389f..0000000
> ---
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/BufBitSet.java
> +++ /dev/null
> @@ -1,847 +0,0 @@
> -/*
> - * Licensed to the Apache Software Foundation (ASF) under one or more
> - * contributor license agreements.  See the NOTICE file distributed with
> - * this work for additional information regarding copyright ownership.
> - * The ASF licenses this file to You under the Apache License, Version 2.0
> - * (the "License"); you may not use this file except in compliance with
> - * the License.  You may obtain a copy of the License at
> - *
> - *     http://www.apache.org/licenses/LICENSE-2.0
> - *
> - * Unless required by applicable law or agreed to in writing, software
> - * distributed under the License is distributed on an "AS IS" BASIS,
> - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
> implied.
> - * See the License for the specific language governing permissions and
> - * limitations under the License.
> - */
> -
> -package org.apache.drill.exec.record.vector; // from org.apache.solr.util
> rev 555343
> -
> -import io.netty.buffer.ByteBuf;
> -import io.netty.buffer.ByteBufAllocator;
> -
> -/**
> - * HEAVY WIP: ONLY PARTIALLY TRANSFERRED TO BUFFER METHODS. STILL NEEDS
> BIT SHIFT FIXES, GETLONG AND SETLONG updates to
> - * fix index postion AND OTHER THINGS.
> - *
> - * An "open" BitSet implementation that allows direct access to the array
> of words storing the bits.
> - * <p/>
> - * Unlike java.util.bitset, the fact that bits are packed into an array
> of longs is part of the interface. This allows
> - * efficient implementation of other algorithms by someone other than the
> author. It also allows one to efficiently
> - * implement alternate serialization or interchange formats.
> - * <p/>
> - * <code>BufBitSet</code> is faster than <code>java.util.BitSet</code> in
> most operations and *much* faster at
> - * calculating cardinality of sets and results of set operations. It can
> also handle sets of larger cardinality (up to
> - * 64 * 2**32-1)
> - * <p/>
> - * The goals of <code>BufBitSet</code> are the fastest implementation
> possible, and maximum code reuse. Extra safety
> - * and encapsulation may always be built on top, but if that's built in,
> the cost can never be removed (and hence people
> - * re-implement their own version in order to get better performance). If
> you want a "safe", totally encapsulated (and
> - * slower and limited) BitSet class, use <code>java.util.BitSet</code>.
> - * <p/>
> - */
> -
> -public class BufBitSet {
> -  private ByteBufAllocator allocator;
> -  private ByteBuf buf;
> -  // protected long[] bits;
> -  protected int wlen; // number of words (elements) used in the array
> -
> -  // Used only for assert:
> -  private long numBits;
> -
> -  // /** Constructs an BufBitSet large enough to hold
> <code>numBits</code>.
> -  // */
> -  // public BufBitSet(long numBits) {
> -  // this.numBits = numBits;
> -  // wlen = buf.capacity();
> -  // }
> -  //
> -  // public BufBitSet() {
> -  // this(64);
> -  // }
> -
> -  public BufBitSet(long numBits, ByteBufAllocator allocator) {
> -    this.allocator = allocator;
> -    this.numBits = numBits;
> -    int words = bits2words(numBits);
> -    this.wlen = words;
> -    buf = allocator.buffer(wlen);
> -  }
> -
> -  private BufBitSet(ByteBufAllocator allocator, ByteBuf buf) {
> -    this.allocator = allocator;
> -    this.numBits = buf.capacity() * 8;
> -    int words = buf.capacity();
> -    this.wlen = words;
> -    this.buf = buf;
> -  }
> -
> -  /** Returns the current capacity in bits (1 greater than the index of
> the last bit) */
> -  public long capacity() {
> -    return buf.capacity() << 6;
> -  }
> -
> -  /**
> -   * Returns the current capacity of this set. Included for
> compatibility. This is *not* equal to {@link #cardinality}
> -   */
> -  public long size() {
> -    return capacity();
> -  }
> -
> -  public int length() {
> -    return buf.capacity() << 6;
> -  }
> -
> -  /** Returns true if there are no set bits */
> -  public boolean isEmpty() {
> -    return cardinality() == 0;
> -  }
> -
> -  // /** Expert: returns the long[] storing the bits */
> -  // public long[] getBits() { return bits; }
> -  //
> -  // /** Expert: sets a new long[] to use as the bit storage */
> -  // public void setBits(long[] bits) { this.bits = bits; }
> -
> -  /** Expert: gets the number of longs in the array that are in use */
> -  public int getNumWords() {
> -    return wlen;
> -  }
> -
> -  /** Expert: sets the number of longs in the array that are in use */
> -  public void setNumWords(int nWords) {
> -    this.wlen = nWords;
> -  }
> -
> -  /** Returns true or false for the specified bit index. */
> -  public boolean get(int index) {
> -    int i = index >> 6; // div 64
> -    // signed shift will keep a negative index and force an
> -    // array-index-out-of-bounds-exception, removing the need for an
> explicit check.
> -    if (i >= buf.capacity()) return false;
> -
> -    int bit = index & 0x3f; // mod 64
> -    long bitmask = 1L << bit;
> -    return (buf.getLong(i) & bitmask) != 0;
> -  }
> -
> -  /**
> -   * Returns true or false for the specified bit index. The index should
> be less than the BufBitSet size
> -   */
> -  public boolean fastGet(int index) {
> -    assert index >= 0 && index < numBits;
> -    int i = index >> 6; // div 64
> -    // signed shift will keep a negative index and force an
> -    // array-index-out-of-bounds-exception, removing the need for an
> explicit check.
> -    int bit = index & 0x3f; // mod 64
> -    long bitmask = 1L << bit;
> -    return (buf.getLong(i) & bitmask) != 0;
> -  }
> -
> -  /**
> -   * Returns true or false for the specified bit index
> -   */
> -  public boolean get(long index) {
> -    int i = (int) (index >> 6); // div 64
> -    if (i >= buf.capacity()) return false;
> -    int bit = (int) index & 0x3f; // mod 64
> -    long bitmask = 1L << bit;
> -    return (buf.getLong(i) & bitmask) != 0;
> -  }
> -
> -  /**
> -   * Returns true or false for the specified bit index. The index should
> be less than the BufBitSet size.
> -   */
> -  public boolean fastGet(long index) {
> -    assert index >= 0 && index < numBits;
> -    int i = (int) (index >> 6); // div 64
> -    int bit = (int) index & 0x3f; // mod 64
> -    long bitmask = 1L << bit;
> -    return (buf.getLong(i) & bitmask) != 0;
> -  }
> -
> -  /*
> -   * // alternate implementation of get() public boolean get1(int index)
> { int i = index >> 6; // div 64 int bit = index
> -   * & 0x3f; // mod 64 return ((buf.getLong(i)>>>bit) & 0x01) != 0; //
> this does a long shift and a bittest (on x86) vs
> -   * // a long shift, and a long AND, (the test for zero is prob a no-op)
> // testing on a P4 indicates this is slower
> -   * than (buf.getLong(i) & bitmask) != 0; }
> -   */
> -
> -  /**
> -   * returns 1 if the bit is set, 0 if not. The index should be less than
> the BufBitSet size
> -   */
> -  public int getBit(int index) {
> -    assert index >= 0 && index < numBits;
> -    int i = index >> 6; // div 64
> -    int bit = index & 0x3f; // mod 64
> -    return ((int) (buf.getLong(i) >>> bit)) & 0x01;
> -  }
> -
> -  /*
> -   * public boolean get2(int index) { int word = index >> 6; // div 64
> int bit = index & 0x0000003f; // mod 64 return
> -   * (buf.getLong(word) << bit) < 0; // hmmm, this would work if bit
> order were reversed // we could right shift and
> -   * check for parity bit, if it was available to us. }
> -   */
> -
> -  /** sets a bit, expanding the set size if necessary */
> -  public void set(long index) {
> -    int wordNum = expandingWordNum(index);
> -    int bit = (int) index & 0x3f;
> -    long bitmask = 1L << bit;
> -    buf.setLong(wordNum, buf.getLong(wordNum) | bitmask);
> -    buf.setLong(wordNum, buf.getLong(wordNum) | bitmask);
> -  }
> -
> -  /**
> -   * Sets the bit at the specified index. The index should be less than
> the BufBitSet size.
> -   */
> -  public void fastSet(int index) {
> -    assert index >= 0 && index < numBits;
> -    int wordNum = index >> 6; // div 64
> -    int bit = index & 0x3f; // mod 64
> -    long bitmask = 1L << bit;
> -    buf.setLong(wordNum, buf.getLong(wordNum) | bitmask);
> -  }
> -
> -  /**
> -   * Sets the bit at the specified index. The index should be less than
> the BufBitSet size.
> -   */
> -  public void fastSet(long index) {
> -    assert index >= 0 && index < numBits;
> -    int wordNum = (int) (index >> 6);
> -    int bit = (int) index & 0x3f;
> -    long bitmask = 1L << bit;
> -    buf.setLong(wordNum, buf.getLong(wordNum) | bitmask);
> -  }
> -
> -  /**
> -   * Sets a range of bits, expanding the set size if necessary
> -   *
> -   * @param startIndex
> -   *          lower index
> -   * @param endIndex
> -   *          one-past the last bit to set
> -   */
> -  public void set(long startIndex, long endIndex) {
> -    if (endIndex <= startIndex) return;
> -
> -    int startWord = (int) (startIndex >> 6);
> -
> -    // since endIndex is one past the end, this is index of the last
> -    // word to be changed.
> -    int endWord = expandingWordNum(endIndex - 1);
> -
> -    long startmask = -1L << startIndex;
> -    long endmask = -1L >>> -endIndex; // 64-(endIndex&0x3f) is the same
> as -endIndex due to wrap
> -
> -    if (startWord == endWord) {
> -      buf.setLong(startWord, buf.getLong(startWord) | (startmask &
> endmask));
> -      return;
> -    }
> -    buf.setLong(startWord, buf.getLong(startWord) | startmask);
> -
> -    fill(buf, startWord + 1, endWord, -1L);
> -    buf.setLong(endWord, buf.getLong(endWord) | endmask);
> -  }
> -
> -  private void fill(ByteBuf buf, int start, int end, long val) {
> -    for (int i = 0; i < buf.capacity(); i += 8) {
> -      buf.setLong(i, val);
> -    }
> -  }
> -
> -  private final void setLongWord(int pos, long value) {
> -    buf.setLong(pos * 8, value);
> -  }
> -
> -  private final long getLongWord(int pos) {
> -    return buf.getLong(pos * 8);
> -  }
> -
> -  protected int expandingWordNum(long index) {
> -    int wordNum = (int) (index >> 6);
> -    if (wordNum >= wlen) {
> -      ensureCapacity(index + 1);
> -      wlen = wordNum + 1;
> -    }
> -    assert (numBits = Math.max(numBits, index + 1)) >= 0;
> -    return wordNum;
> -  }
> -
> -  /**
> -   * clears a bit. The index should be less than the BufBitSet size.
> -   */
> -  public void fastClear(int index) {
> -    assert index >= 0 && index < numBits;
> -    int wordNum = index >> 6;
> -    int bit = index & 0x03f;
> -    long bitmask = 1L << bit;
> -    buf.setLong(wordNum, buf.getLong(wordNum) & ~bitmask);
> -    // hmmm, it takes one more instruction to clear than it does to
> set... any
> -    // way to work around this? If there were only 63 bits per word, we
> could
> -    // use a right shift of 10111111...111 in binary to position the 0 in
> the
> -    // correct place (using sign extension).
> -    // Could also use Long.rotateRight() or rotateLeft() *if* they were
> converted
> -    // by the JVM into a native instruction.
> -    // buf.getLong(word) &= Long.rotateLeft(0xfffffffe,bit);
> -  }
> -
> -  /**
> -   * clears a bit. The index should be less than the BufBitSet size.
> -   */
> -  public void fastClear(long index) {
> -    assert index >= 0 && index < numBits;
> -    int wordNum = (int) (index >> 6); // div 64
> -    int bit = (int) index & 0x3f; // mod 64
> -    long bitmask = 1L << bit;
> -    buf.setLong(wordNum, buf.getLong(wordNum) & ~bitmask);
> -  }
> -
> -  /** clears a bit, allowing access beyond the current set size without
> changing the size. */
> -  public void clear(long index) {
> -    int wordNum = (int) (index >> 6); // div 64
> -    if (wordNum >= wlen) return;
> -    int bit = (int) index & 0x3f; // mod 64
> -    long bitmask = 1L << bit;
> -    buf.setLong(wordNum, buf.getLong(wordNum) & ~bitmask);
> -  }
> -
> -  /**
> -   * Clears a range of bits. Clearing past the end does not change the
> size of the set.
> -   *
> -   * @param startIndex
> -   *          lower index
> -   * @param endIndex
> -   *          one-past the last bit to clear
> -   */
> -  public void clear(int startIndex, int endIndex) {
> -    if (endIndex <= startIndex) return;
> -
> -    int startWord = (startIndex >> 6);
> -    if (startWord >= wlen) return;
> -
> -    // since endIndex is one past the end, this is index of the last
> -    // word to be changed.
> -    int endWord = ((endIndex - 1) >> 6);
> -
> -    long startmask = -1L << startIndex;
> -    long endmask = -1L >>> -endIndex; // 64-(endIndex&0x3f) is the same
> as -endIndex due to wrap
> -
> -    // invert masks since we are clearing
> -    startmask = ~startmask;
> -    endmask = ~endmask;
> -
> -    if (startWord == endWord) {
> -      buf.setLong(startWord, buf.getLong(startWord) & (startmask |
> endmask));
> -      return;
> -    }
> -
> -    buf.setLong(startWord, buf.getLong(startWord) & startmask);
> -
> -    int middle = Math.min(wlen, endWord);
> -    fill(buf, startWord + 1, middle, 0L);
> -    if (endWord < wlen) {
> -      buf.setLong(endWord, buf.getLong(endWord) & endmask);
> -    }
> -  }
> -
> -  /**
> -   * Clears a range of bits. Clearing past the end does not change the
> size of the set.
> -   *
> -   * @param startIndex
> -   *          lower index
> -   * @param endIndex
> -   *          one-past the last bit to clear
> -   */
> -  public void clear(long startIndex, long endIndex) {
> -    if (endIndex <= startIndex) return;
> -
> -    int startWord = (int) (startIndex >> 6);
> -    if (startWord >= wlen) return;
> -
> -    // since endIndex is one past the end, this is index of the last
> -    // word to be changed.
> -    int endWord = (int) ((endIndex - 1) >> 6);
> -
> -    long startmask = -1L << startIndex;
> -    long endmask = -1L >>> -endIndex; // 64-(endIndex&0x3f) is the same
> as -endIndex due to wrap
> -
> -    // invert masks since we are clearing
> -    startmask = ~startmask;
> -    endmask = ~endmask;
> -
> -    if (startWord == endWord) {
> -      buf.setLong(startWord, buf.getLong(startWord) & (startmask |
> endmask));
> -      return;
> -    }
> -
> -    buf.setLong(startWord, buf.getLong(startWord) & startmask);
> -
> -    int middle = Math.min(wlen, endWord);
> -    fill(buf, startWord + 1, middle, 0L);
> -    if (endWord < wlen) {
> -      buf.setLong(endWord, buf.getLong(endWord) & endmask);
> -    }
> -  }
> -
> -  /**
> -   * Sets a bit and returns the previous value. The index should be less
> than the BufBitSet size.
> -   */
> -  public boolean getAndSet(int index) {
> -    assert index >= 0 && index < numBits;
> -    int wordNum = index >> 6; // div 64
> -    int bit = index & 0x3f; // mod 64
> -    long bitmask = 1L << bit;
> -    long longVal = buf.getLong(wordNum);
> -    boolean val = (longVal & bitmask) != 0;
> -    buf.setLong(wordNum, longVal | bitmask);
> -    return val;
> -  }
> -
> -  /**
> -   * flips a bit. The index should be less than the BufBitSet size.
> -   */
> -  public void fastFlip(int index) {
> -    assert index >= 0 && index < numBits;
> -    int wordNum = index >> 6; // div 64
> -    int bit = index & 0x3f; // mod 64
> -    long bitmask = 1L << bit;
> -    buf.setLong(wordNum, (buf.getLong(wordNum) ^ bitmask));
> -  }
> -
> -  /**
> -   * flips a bit. The index should be less than the BufBitSet size.
> -   */
> -  public void fastFlip(long index) {
> -    assert index >= 0 && index < numBits;
> -    int wordNum = (int) (index >> 6); // div 64
> -    int bit = (int) index & 0x3f; // mod 64
> -    long bitmask = 1L << bit;
> -    buf.setLong(wordNum, (buf.getLong(wordNum) ^ bitmask));
> -  }
> -
> -  /** flips a bit, expanding the set size if necessary */
> -  public void flip(long index) {
> -    int wordNum = expandingWordNum(index);
> -    int bit = (int) index & 0x3f; // mod 64
> -    long bitmask = 1L << bit;
> -    buf.setLong(wordNum, (buf.getLong(wordNum) ^ bitmask));
> -  }
> -
> -  /**
> -   * flips a bit and returns the resulting bit value. The index should be
> less than the BufBitSet size.
> -   */
> -  public boolean flipAndGet(int index) {
> -    assert index >= 0 && index < numBits;
> -    int wordNum = index >> 6; // div 64
> -    int bit = index & 0x3f; // mod 64
> -    long bitmask = 1L << bit;
> -    long longVal = buf.getLong(wordNum);
> -    buf.setLong(wordNum, longVal ^ bitmask);
> -    return (longVal & bitmask) != 0;
> -  }
> -
> -  /**
> -   * flips a bit and returns the resulting bit value. The index should be
> less than the BufBitSet size.
> -   */
> -  public boolean flipAndGet(long index) {
> -    assert index >= 0 && index < numBits;
> -    int wordNum = (int) (index >> 6); // div 64
> -    int bit = (int) index & 0x3f; // mod 64
> -    long bitmask = 1L << bit;
> -    long longVal = buf.getLong(wordNum);
> -    buf.setLong(wordNum, longVal ^ bitmask);
> -    return (longVal & bitmask) != 0;
> -  }
> -
> -  /**
> -   * Flips a range of bits, expanding the set size if necessary
> -   *
> -   * @param startIndex
> -   *          lower index
> -   * @param endIndex
> -   *          one-past the last bit to flip
> -   */
> -  public void flip(long startIndex, long endIndex) {
> -    if (endIndex <= startIndex) return;
> -    int startWord = (int) (startIndex >> 6);
> -
> -    // since endIndex is one past the end, this is index of the last
> -    // word to be changed.
> -    int endWord = expandingWordNum(endIndex - 1);
> -
> -    /***
> -     * Grrr, java shifting wraps around so -1L>>>64 == -1 for that
> reason, make sure not to use endmask if the bits to
> -     * flip will be zero in the last word (redefine endWord to be the
> last changed...) long startmask = -1L <<
> -     * (startIndex & 0x3f); // example: 11111...111000 long endmask = -1L
> >>> (64-(endIndex & 0x3f)); // example:
> -     * 00111...111111
> -     ***/
> -
> -    long startmask = -1L << startIndex;
> -    long endmask = -1L >>> -endIndex; // 64-(endIndex&0x3f) is the same
> as -endIndex due to wrap
> -
> -    if (startWord == endWord) {
> -      buf.setLong(startWord, buf.getLong(startWord) ^ (startmask &
> endmask));
> -      return;
> -    }
> -
> -    buf.setLong(startWord, buf.getLong(startWord) ^ startmask);
> -
> -    for (int i = startWord + 1; i < endWord; i++) {
> -      buf.setLong(i, ~buf.getLong(i));
> -    }
> -
> -    buf.setLong(endWord, buf.getLong(endWord) ^ endmask);
> -  }
> -
> -  /*
> -   * public static int pop(long v0, long v1, long v2, long v3) { //
> derived from pop_array by setting last four elems to
> -   * 0. // exchanges one pop() call for 10 elementary operations //
> saving about 7 instructions... is there a better
> -   * way? long twosA=v0 & v1; long ones=v0^v1;
> -   *
> -   * long u2=ones^v2; long twosB =(ones&v2)|(u2&v3); ones=u2^v3;
> -   *
> -   * long fours=(twosA&twosB); long twos=twosA^twosB;
> -   *
> -   * return (pop(fours)<<2) + (pop(twos)<<1) + pop(ones);
> -   *
> -   * }
> -   */
> -
> -  /** @return the number of set bits */
> -  public long cardinality() {
> -    return BitUtil.pop_array(buf, 0, wlen);
> -  }
> -
> -  /**
> -   * Returns the popcount or cardinality of the intersection of the two
> sets. Neither set is modified.
> -   */
> -  public static long intersectionCount(BufBitSet a, BufBitSet b) {
> -    return BitUtil.pop_intersect(a.buf, b.buf, 0, Math.min(a.wlen,
> b.wlen));
> -  }
> -
> -  /**
> -   * Returns the popcount or cardinality of the union of the two sets.
> Neither set is modified.
> -   */
> -  public static long unionCount(BufBitSet a, BufBitSet b) {
> -    long tot = BitUtil.pop_union(a.buf, b.buf, 0, Math.min(a.wlen,
> b.wlen));
> -    if (a.wlen < b.wlen) {
> -      tot += BitUtil.pop_array(b.buf, a.wlen, b.wlen - a.wlen);
> -    } else if (a.wlen > b.wlen) {
> -      tot += BitUtil.pop_array(a.buf, b.wlen, a.wlen - b.wlen);
> -    }
> -    return tot;
> -  }
> -
> -  /**
> -   * Returns the popcount or cardinality of "a and not b" or
> "intersection(a, not(b))". Neither set is modified.
> -   */
> -  public static long andNotCount(BufBitSet a, BufBitSet b) {
> -    long tot = BitUtil.pop_andnot(a.buf, b.buf, 0, Math.min(a.wlen,
> b.wlen));
> -    if (a.wlen > b.wlen) {
> -      tot += BitUtil.pop_array(a.buf, b.wlen, a.wlen - b.wlen);
> -    }
> -    return tot;
> -  }
> -
> -  /**
> -   * Returns the popcount or cardinality of the exclusive-or of the two
> sets. Neither set is modified.
> -   */
> -  public static long xorCount(BufBitSet a, BufBitSet b) {
> -    long tot = BitUtil.pop_xor(a.buf, b.buf, 0, Math.min(a.wlen, b.wlen));
> -    if (a.wlen < b.wlen) {
> -      tot += BitUtil.pop_array(b.buf, a.wlen, b.wlen - a.wlen);
> -    } else if (a.wlen > b.wlen) {
> -      tot += BitUtil.pop_array(a.buf, b.wlen, a.wlen - b.wlen);
> -    }
> -    return tot;
> -  }
> -
> -  /**
> -   * Returns the index of the first set bit starting at the index
> specified. -1 is returned if there are no more set
> -   * bits.
> -   */
> -  public int nextSetBit(int index) {
> -    int i = index >> 6;
> -    if (i >= wlen) return -1;
> -    int subIndex = index & 0x3f; // index within the word
> -    long word = buf.getLong(i) >> subIndex; // skip all the bits to the
> right of index
> -
> -    if (word != 0) {
> -      return (i << 6) + subIndex + Long.numberOfTrailingZeros(word);
> -    }
> -
> -    while (++i < wlen) {
> -      word = buf.getLong(i);
> -      if (word != 0) return (i << 6) + Long.numberOfTrailingZeros(word);
> -    }
> -
> -    return -1;
> -  }
> -
> -  /**
> -   * Returns the index of the first set bit starting at the index
> specified. -1 is returned if there are no more set
> -   * bits.
> -   */
> -  public long nextSetBit(long index) {
> -    int i = (int) (index >>> 6);
> -    if (i >= wlen) return -1;
> -    int subIndex = (int) index & 0x3f; // index within the word
> -    long word = buf.getLong(i) >>> subIndex; // skip all the bits to the
> right of index
> -
> -    if (word != 0) {
> -      return (((long) i) << 6) + (subIndex +
> Long.numberOfTrailingZeros(word));
> -    }
> -
> -    while (++i < wlen) {
> -      word = buf.getLong(i);
> -      if (word != 0) return (((long) i) << 6) +
> Long.numberOfTrailingZeros(word);
> -    }
> -
> -    return -1;
> -  }
> -
> -  /**
> -   * Returns the index of the first set bit starting downwards at the
> index specified. -1 is returned if there are no
> -   * more set bits.
> -   */
> -  public int prevSetBit(int index) {
> -    int i = index >> 6;
> -    final int subIndex;
> -    long word;
> -    if (i >= wlen) {
> -      i = wlen - 1;
> -      if (i < 0) return -1;
> -      subIndex = 63; // last possible bit
> -      word = buf.getLong(i);
> -    } else {
> -      if (i < 0) return -1;
> -      subIndex = index & 0x3f; // index within the word
> -      word = (buf.getLong(i) << (63 - subIndex)); // skip all the bits to
> the left of index
> -    }
> -
> -    if (word != 0) {
> -      return (i << 6) + subIndex - Long.numberOfLeadingZeros(word); //
> See LUCENE-3197
> -    }
> -
> -    while (--i >= 0) {
> -      word = buf.getLong(i);
> -      if (word != 0) {
> -        return (i << 6) + 63 - Long.numberOfLeadingZeros(word);
> -      }
> -    }
> -
> -    return -1;
> -  }
> -
> -  /**
> -   * Returns the index of the first set bit starting downwards at the
> index specified. -1 is returned if there are no
> -   * more set bits.
> -   */
> -  public long prevSetBit(long index) {
> -    int i = (int) (index >> 6);
> -    final int subIndex;
> -    long word;
> -    if (i >= wlen) {
> -      i = wlen - 1;
> -      if (i < 0) return -1;
> -      subIndex = 63; // last possible bit
> -      word = buf.getLong(i);
> -    } else {
> -      if (i < 0) return -1;
> -      subIndex = (int) index & 0x3f; // index within the word
> -      word = (buf.getLong(i) << (63 - subIndex)); // skip all the bits to
> the left of index
> -    }
> -
> -    if (word != 0) {
> -      return (((long) i) << 6) + subIndex -
> Long.numberOfLeadingZeros(word); // See LUCENE-3197
> -    }
> -
> -    while (--i >= 0) {
> -      word = buf.getLong(i);
> -      if (word != 0) {
> -        return (((long) i) << 6) + 63 - Long.numberOfLeadingZeros(word);
> -      }
> -    }
> -
> -    return -1;
> -  }
> -
> -  BufBitSet cloneTest() {
> -    BufBitSet obs = new BufBitSet(allocator, buf.copy());
> -    return obs;
> -  }
> -
> -  /** this = this AND other */
> -  public void intersect(BufBitSet other) {
> -    int newLen = Math.min(this.wlen, other.wlen);
> -    ByteBuf thisArr = this.buf;
> -    ByteBuf otherArr = other.buf;
> -    // testing against zero can be more efficient
> -    int pos = newLen;
> -    while (--pos >= 0) {
> -      thisArr.setLong(pos, thisArr.getLong(pos) & otherArr.getLong(pos));
> -    }
> -    if (this.wlen > newLen) {
> -      // fill zeros from the new shorter length to the old length
> -      fill(buf, newLen, this.wlen, 0);
> -    }
> -    this.wlen = newLen;
> -  }
> -
> -  /** this = this OR other */
> -  public void union(BufBitSet other) {
> -    int newLen = Math.max(wlen, other.wlen);
> -    ensureCapacityWords(newLen);
> -    assert (numBits = Math.max(other.numBits, numBits)) >= 0;
> -
> -    ByteBuf thisArr = this.buf;
> -    ByteBuf otherArr = other.buf;
> -
> -    int pos = Math.min(wlen, other.wlen);
> -    while (--pos >= 0) {
> -      thisArr.setLong(pos, thisArr.getLong(pos) | otherArr.getLong(pos));
> -    }
> -    if (this.wlen < newLen) {
> -      System.arraycopy(otherArr, this.wlen, thisArr, this.wlen, newLen -
> this.wlen);
> -    }
> -    this.wlen = newLen;
> -  }
> -
> -  /** Remove all elements set in other. this = this AND_NOT other */
> -  public void remove(BufBitSet other) {
> -    int idx = Math.min(wlen, other.wlen);
> -    ByteBuf thisArr = this.buf;
> -    ByteBuf otherArr = other.buf;
> -    while (--idx >= 0) {
> -      thisArr.setLong(idx, thisArr.getLong(idx) & ~otherArr.getLong(idx));
> -    }
> -  }
> -
> -  /** this = this XOR other */
> -  public void xor(BufBitSet other) {
> -    int newLen = Math.max(wlen, other.wlen);
> -    ensureCapacityWords(newLen);
> -    assert (numBits = Math.max(other.numBits, numBits)) >= 0;
> -
> -    ByteBuf thisArr = this.buf;
> -    ByteBuf otherArr = other.buf;
> -    int pos = Math.min(wlen, other.wlen);
> -    while (--pos >= 0) {
> -      thisArr.setLong(pos, thisArr.getLong(pos) ^ otherArr.getLong(pos));
> -    }
> -    if (this.wlen < newLen) {
> -      otherArr.readerIndex(wlen);
> -      otherArr.writeBytes(thisArr);
> -    }
> -    this.wlen = newLen;
> -
> -  }
> -
> -  // some BitSet compatability methods
> -
> -  // ** see {@link intersect} */
> -  public void and(BufBitSet other) {
> -    intersect(other);
> -  }
> -
> -  // ** see {@link union} */
> -  public void or(BufBitSet other) {
> -    union(other);
> -  }
> -
> -  // ** see {@link andNot} */
> -  public void andNot(BufBitSet other) {
> -    remove(other);
> -  }
> -
> -  /** returns true if the sets have any elements in common */
> -  public boolean intersects(BufBitSet other) {
> -    int pos = Math.min(this.wlen, other.wlen);
> -    ByteBuf thisArr = this.buf;
> -    ByteBuf otherArr = other.buf;
> -    while (--pos >= 0) {
> -      if ((thisArr.getLong(pos) & otherArr.getLong(pos)) != 0) return
> true;
> -    }
> -    return false;
> -  }
> -
> -  public void ensureCapacityWords(int numWords) {
> -    if (buf.capacity() < numWords) {
> -      ByteBuf newBuf = allocator.buffer(numWords * 8);
> -      buf.writeBytes(newBuf);
> -      buf.release();
> -      buf = newBuf;
> -      this.numBits = numWords * 64;
> -    }
> -  }
> -
> -  /**
> -   * Ensure that the long[] is big enough to hold numBits, expanding it
> if necessary. getNumWords() is unchanged by this
> -   * call.
> -   */
> -  public void ensureCapacity(long numBits) {
> -    ensureCapacityWords(bits2words(numBits));
> -  }
> -
> -  /**
> -   * Lowers numWords, the number of words in use, by checking for
> trailing zero words.
> -   */
> -  public void trimTrailingZeros() {
> -    int idx = wlen - 1;
> -    while (idx >= 0 && buf.getLong(idx) == 0)
> -      idx--;
> -    wlen = idx + 1;
> -  }
> -
> -  /** returns the number of 64 bit words it would take to hold numBits */
> -  public static int bits2words(long numBits) {
> -    return (int) (((numBits - 1) >>> 6) + 1);
> -  }
> -
> -  /** returns true if both sets have the same bits set */
> -  @Override
> -  public boolean equals(Object o) {
> -    if (this == o) return true;
> -    if (!(o instanceof BufBitSet)) return false;
> -    BufBitSet a;
> -    BufBitSet b = (BufBitSet) o;
> -    // make a the larger set.
> -    if (b.wlen > this.wlen) {
> -      a = b;
> -      b = this;
> -    } else {
> -      a = this;
> -    }
> -
> -    // check for any set bits out of the range of b
> -    for (int i = a.wlen - 1; i >= b.wlen; i--) {
> -      if (a.buf.getLong(i) != 0) return false;
> -    }
> -
> -    for (int i = b.wlen - 1; i >= 0; i--) {
> -      if (a.buf.getLong(i) != b.buf.getLong(i)) return false;
> -    }
> -
> -    return true;
> -  }
> -
> -  @Override
> -  public int hashCode() {
> -    // Start with a zero hash and use a mix that results in zero if the
> input is zero.
> -    // This effectively truncates trailing zeros without an explicit
> check.
> -    long h = 0;
> -    for (int i = buf.capacity(); --i >= 0;) {
> -      h ^= buf.getLong(i);
> -      h = (h << 1) | (h >>> 63); // rotate left
> -    }
> -    // fold leftmost bits into right and add a constant to prevent
> -    // empty sets from returning 0, which is too common.
> -    return (int) ((h >> 32) ^ h) + 0x98761234;
> -  }
> -
> -  public void release() {
> -    this.buf.release();
> -  }
> -}
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/3d41be41/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/NullValueException.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/NullValueException.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/NullValueException.java
> deleted file mode 100644
> index 027b698..0000000
> ---
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/NullValueException.java
> +++ /dev/null
> @@ -1,9 +0,0 @@
> -package org.apache.drill.exec.record.vector;
> -
> -import org.apache.drill.common.exceptions.DrillRuntimeException;
> -
> -public class NullValueException extends DrillRuntimeException {
> -  public NullValueException(int index) {
> -    super("Element at index position: " + index + " is null");
> -  }
> -}
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/3d41be41/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/NullableFixed8.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/NullableFixed8.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/NullableFixed8.java
> deleted file mode 100644
> index 48aa0c2..0000000
> ---
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/NullableFixed8.java
> +++ /dev/null
> @@ -1,43 +0,0 @@
>
> -/*******************************************************************************
> - * Licensed to the Apache Software Foundation (ASF) under one
> - * or more contributor license agreements.  See the NOTICE file
> - * distributed with this work for additional information
> - * regarding copyright ownership.  The ASF licenses this file
> - * to you under the Apache License, Version 2.0 (the
> - * "License"); you may not use this file except in compliance
> - * with the License.  You may obtain a copy of the License at
> - *
> - * http://www.apache.org/licenses/LICENSE-2.0
> - *
> - * Unless required by applicable law or agreed to in writing, software
> - * distributed under the License is distributed on an "AS IS" BASIS,
> - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
> implied.
> - * See the License for the specific language governing permissions and
> - * limitations under the License.
> -
> ******************************************************************************/
> -package org.apache.drill.exec.record.vector;
> -
> -import org.apache.drill.exec.memory.BufferAllocator;
> -import org.apache.drill.exec.record.MaterializedField;
> -
> -public final class NullableFixed8 extends
> NullableValueVector<NullableFixed8, Fixed8>{
> -  static final org.slf4j.Logger logger =
> org.slf4j.LoggerFactory.getLogger(NullableFixed8.class);
> -
> -  public NullableFixed8(MaterializedField field, BufferAllocator
> allocator) {
> -    super(field, allocator);
> -  }
> -
> -  @Override
> -  protected Fixed8 getNewValueVector(BufferAllocator allocator) {
> -    return new Fixed8(this.field, allocator);
> -  }
> -
> -  public long get(int index){
> -    return 1l;
> -  }
> -
> -  public void set(int index, long value){
> -
> -  }
> -
> -}
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/3d41be41/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/SelectionVector.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/SelectionVector.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/SelectionVector.java
> deleted file mode 100644
> index 3546bd8..0000000
> ---
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/SelectionVector.java
> +++ /dev/null
> @@ -1,49 +0,0 @@
>
> -/*******************************************************************************
> - * Licensed to the Apache Software Foundation (ASF) under one
> - * or more contributor license agreements.  See the NOTICE file
> - * distributed with this work for additional information
> - * regarding copyright ownership.  The ASF licenses this file
> - * to you under the Apache License, Version 2.0 (the
> - * "License"); you may not use this file except in compliance
> - * with the License.  You may obtain a copy of the License at
> - *
> - * http://www.apache.org/licenses/LICENSE-2.0
> - *
> - * Unless required by applicable law or agreed to in writing, software
> - * distributed under the License is distributed on an "AS IS" BASIS,
> - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
> implied.
> - * See the License for the specific language governing permissions and
> - * limitations under the License.
> -
> ******************************************************************************/
> -package org.apache.drill.exec.record.vector;
> -
> -import org.apache.drill.exec.memory.BufferAllocator;
> -import org.apache.drill.exec.record.MaterializedField;
> -
> -/**
> - * Convenience/Clarification Fixed2 wrapper.
> - */
> -public class SelectionVector {
> -  static final org.slf4j.Logger logger =
> org.slf4j.LoggerFactory.getLogger(SelectionVector.class);
> -
> -  public SelectionVector(MaterializedField field, BufferAllocator
> allocator) {
> -
> -  }
> -
> -  public int capacity() {
> -    return -1;
> -  }
> -
> -  public void allocateNew(int count) {
> -
> -  }
> -<<<<<<<
> HEAD:sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/SelectionVector.java
> -=======
> -
> -  public final int getInt(int index){
> -    index*=4;
> -    return data.getInt(index);
> -  }
> ->>>>>>> Build
> working:sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/NullableFixed4.java
> -
> -}
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/3d41be41/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/SelectionVector2.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/SelectionVector2.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/SelectionVector2.java
> deleted file mode 100644
> index b808dc6..0000000
> ---
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/SelectionVector2.java
> +++ /dev/null
> @@ -1,49 +0,0 @@
>
> -/*******************************************************************************
> - * Licensed to the Apache Software Foundation (ASF) under one
> - * or more contributor license agreements.  See the NOTICE file
> - * distributed with this work for additional information
> - * regarding copyright ownership.  The ASF licenses this file
> - * to you under the Apache License, Version 2.0 (the
> - * "License"); you may not use this file except in compliance
> - * with the License.  You may obtain a copy of the License at
> - *
> - * http://www.apache.org/licenses/LICENSE-2.0
> - *
> - * Unless required by applicable law or agreed to in writing, software
> - * distributed under the License is distributed on an "AS IS" BASIS,
> - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
> implied.
> - * See the License for the specific language governing permissions and
> - * limitations under the License.
> -
> ******************************************************************************/
> -package org.apache.drill.exec.record.vector;
> -
> -import io.netty.buffer.ByteBuf;
> -
> -import org.apache.drill.exec.memory.BufferAllocator;
> -import org.apache.drill.exec.record.DeadBuf;
> -
> -/**
> - * A selection vector that fronts, at most, a
> - */
> -public class SelectionVector2{
> -  static final org.slf4j.Logger logger =
> org.slf4j.LoggerFactory.getLogger(SelectionVector2.class);
> -
> -  private final BufferAllocator allocator;
> -  private ByteBuf buffer = DeadBuf.DEAD_BUFFER;
> -
> -  public SelectionVector2(BufferAllocator allocator) {
> -    this.allocator = allocator;
> -  }
> -
> -  public int getCount(){
> -    return -1;
> -  }
> -
> -  public int getIndex(int directIndex){
> -    return buffer.getChar(directIndex);
> -  }
> -
> -  public void setIndex(int directIndex, char value){
> -    buffer.setChar(directIndex, value);
> -  }
> -}
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/3d41be41/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/SelectionVector4.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/SelectionVector4.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/SelectionVector4.java
> deleted file mode 100644
> index d857146..0000000
> ---
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/SelectionVector4.java
> +++ /dev/null
> @@ -1,41 +0,0 @@
>
> -/*******************************************************************************
> - * Licensed to the Apache Software Foundation (ASF) under one
> - * or more contributor license agreements.  See the NOTICE file
> - * distributed with this work for additional information
> - * regarding copyright ownership.  The ASF licenses this file
> - * to you under the Apache License, Version 2.0 (the
> - * "License"); you may not use this file except in compliance
> - * with the License.  You may obtain a copy of the License at
> - *
> - * http://www.apache.org/licenses/LICENSE-2.0
> - *
> - * Unless required by applicable law or agreed to in writing, software
> - * distributed under the License is distributed on an "AS IS" BASIS,
> - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
> implied.
> - * See the License for the specific language governing permissions and
> - * limitations under the License.
> -
> ******************************************************************************/
> -
> -package org.apache.drill.exec.record.vector;
> -
> -import io.netty.buffer.ByteBuf;
> -
> -import org.apache.drill.exec.memory.BufferAllocator;
> -import org.apache.drill.exec.record.DeadBuf;
> -
> -public class SelectionVector4 {
> -  static final org.slf4j.Logger logger =
> org.slf4j.LoggerFactory.getLogger(SelectionVector4.class);
> -
> -  private final BufferAllocator allocator;
> -  private ByteBuf buffer = DeadBuf.DEAD_BUFFER;
> -
> -  public SelectionVector4(BufferAllocator allocator) {
> -    this.allocator = allocator;
> -  }
> -
> -  public int getCount(){
> -    return -1;
> -  }
> -
> -
> -}
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/3d41be41/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/JSONRecordReader.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/JSONRecordReader.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/JSONRecordReader.java
> index 8c31aa4..8513dfe 100644
> ---
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/JSONRecordReader.java
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/JSONRecordReader.java
> @@ -22,14 +22,8 @@ import org.apache.drill.exec.memory.BufferAllocator;
>  import org.apache.drill.exec.ops.FragmentContext;
>  import org.apache.drill.exec.physical.impl.OutputMutator;
>  import org.apache.drill.exec.record.MaterializedField;
> -import org.apache.drill.exec.record.vector.NullableBit;
> -import org.apache.drill.exec.record.vector.NullableFixed4;
> -import org.apache.drill.exec.record.vector.NullableVarLen4;
> -import org.apache.drill.exec.record.vector.TypeHelper;
> -import org.apache.drill.exec.record.vector.ValueVector;
>  import org.apache.drill.exec.schema.DiffSchema;
>  import org.apache.drill.exec.schema.Field;
> -import org.apache.drill.exec.schema.IdGenerator;
>  import org.apache.drill.exec.schema.ListSchema;
>  import org.apache.drill.exec.schema.NamedField;
>  import org.apache.drill.exec.schema.ObjectSchema;
> @@ -37,10 +31,15 @@ import org.apache.drill.exec.schema.OrderedField;
>  import org.apache.drill.exec.schema.RecordSchema;
>  import org.apache.drill.exec.schema.SchemaIdGenerator;
>  import org.apache.drill.exec.schema.json.jackson.JacksonHelper;
> +import org.apache.drill.exec.vector.AllocationHelper;
> +import org.apache.drill.exec.vector.NullableBitVector;
> +import org.apache.drill.exec.vector.NullableFloat4Vector;
> +import org.apache.drill.exec.vector.NullableIntVector;
> +import org.apache.drill.exec.vector.NullableVarChar4Vector;
> +import org.apache.drill.exec.vector.TypeHelper;
> +import org.apache.drill.exec.vector.ValueVector;
>
>  import com.beust.jcommander.internal.Maps;
> -import com.carrotsearch.hppc.IntObjectOpenHashMap;
> -import com.carrotsearch.hppc.cursors.ObjectCursor;
>  import com.fasterxml.jackson.core.JsonFactory;
>  import com.fasterxml.jackson.core.JsonParser;
>  import com.fasterxml.jackson.core.JsonToken;
> @@ -416,8 +415,8 @@ public class JSONRecordReader implements RecordReader {
>          if (holder == null) {
>              MajorType type = field.getFieldType();
>              MaterializedField f = MaterializedField.create(new
> SchemaPath(field.getFieldName(), ExpressionPosition.UNKNOWN), type);
> -            ValueVector<?> v = TypeHelper.getNewVector(f, allocator);
> -            v.allocateNew(batchSize);
> +            ValueVector v = TypeHelper.getNewVector(f, allocator);
> +            AllocationHelper.allocate(v, batchSize, 50);
>              holder = new VectorHolder(batchSize, v);
>              valueVectorMap.put(field, holder);
>              outputMutator.addField(v);
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/3d41be41/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/AllocationHelper.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/AllocationHelper.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/AllocationHelper.java
> new file mode 100644
> index 0000000..1631e70
> --- /dev/null
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/AllocationHelper.java
> @@ -0,0 +1,15 @@
> +package org.apache.drill.exec.vector;
> +
> +public class AllocationHelper {
> +  static final org.slf4j.Logger logger =
> org.slf4j.LoggerFactory.getLogger(AllocationHelper.class);
> +
> +  public static void allocate(ValueVector v, int valueCount, int
> bytesPerValue){
> +    if(v instanceof FixedWidthVector){
> +      ((FixedWidthVector) v).allocateNew(valueCount);
> +    }else if(v instanceof VariableWidthVector){
> +      ((VariableWidthVector) v).allocateNew(valueCount * bytesPerValue,
> valueCount);
> +    }else{
> +      throw new UnsupportedOperationException();
> +    }
> +  }
> +}
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/3d41be41/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/BaseValueVector.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/BaseValueVector.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/BaseValueVector.java
> index a8678f5..e87e132 100644
> ---
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/BaseValueVector.java
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/BaseValueVector.java
> @@ -25,8 +25,6 @@ abstract class BaseValueVector implements ValueVector{
>    }
>
>    abstract class BaseAccessor implements ValueVector.Accessor{
> -
> -
>      public void reset(){}
>    }
>
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/3d41be41/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/BitVector.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/BitVector.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/BitVector.java
> index 9d247f5..9a9f438 100644
> ---
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/BitVector.java
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/BitVector.java
> @@ -8,6 +8,7 @@ import org.apache.drill.exec.memory.BufferAllocator;
>  import org.apache.drill.exec.proto.UserBitShared.FieldMetadata;
>  import org.apache.drill.exec.record.DeadBuf;
>  import org.apache.drill.exec.record.MaterializedField;
> +import org.apache.drill.exec.record.TransferPair;
>  /**
>   * Bit implements a vector of bit-width values.  Elements in the vector
> are accessed
>   * by position from the logical start of the vector.
> @@ -74,6 +75,32 @@ public final class BitVector extends
> BaseDataValueVector implements FixedWidthVe
>      return new Accessor();
>    }
>
> +  public TransferPair getTransferPair(){
> +    return new TransferImpl();
> +  }
> +
> +  public void transferTo(BitVector target){
> +    target.data = data;
> +    target.data.retain();
> +    target.recordCount = recordCount;
> +    clear();
> +  }
> +
> +  private class TransferImpl implements TransferPair{
> +    BitVector to;
> +
> +    public TransferImpl(){
> +      this.to = new BitVector(getField(), allocator);
> +    }
> +
> +    public BitVector getTo(){
> +      return to;
> +    }
> +
> +    public void transfer(){
> +      transferTo(to);
> +    }
> +  }
>
>    public class Accessor extends BaseAccessor{
>
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/3d41be41/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/ValueVector.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/ValueVector.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/ValueVector.java
> index 328182b..27089ac 100644
> ---
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/ValueVector.java
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/ValueVector.java
> @@ -21,9 +21,9 @@ import io.netty.buffer.ByteBuf;
>
>  import java.io.Closeable;
>
> -import org.apache.drill.exec.proto.SchemaDefProtos.FieldDef;
>  import org.apache.drill.exec.proto.UserBitShared.FieldMetadata;
>  import org.apache.drill.exec.record.MaterializedField;
> +import org.apache.drill.exec.record.TransferPair;
>
>  /**
>   * ValueVectorTypes defines a set of template-generated classes which
> implement type-specific
> @@ -50,6 +50,9 @@ public interface ValueVector extends Closeable {
>    public void clear();
>
>
> +  public TransferPair getTransferPair();
> +
> +
>    /**
>     * Return the underlying buffers associated with this vector. Note that
> this doesn't impact the
>     * reference counts for this buffer so it only should be used for
> in-context access. Also note
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/3d41be41/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/expr/ExpressionTest.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/expr/ExpressionTest.java
> b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/expr/ExpressionTest.java
> index 007abb3..623af0e 100644
> ---
> a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/expr/ExpressionTest.java
> +++
> b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/expr/ExpressionTest.java
> @@ -21,18 +21,13 @@ import
> org.apache.drill.common.expression.parser.ExprParser.parse_return;
>  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.exec.expr.CodeGenerator.HoldingContainer;
>  import org.apache.drill.exec.expr.fn.FunctionImplementationRegistry;
>  import org.apache.drill.exec.physical.impl.project.Projector;
>  import org.apache.drill.exec.record.RecordBatch;
>  import org.apache.drill.exec.record.RecordBatch.TypedFieldId;
> -import org.apache.drill.exec.record.vector.Fixed4;
> +import org.apache.drill.exec.vector.IntVector;
>  import org.junit.After;
>  import org.junit.Test;
> -import org.slf4j.ILoggerFactory;
> -import org.slf4j.LoggerFactory;
> -
> -import ch.qos.logback.classic.LoggerContext;
>
>  public class ExpressionTest {
>    static final org.slf4j.Logger logger =
> org.slf4j.LoggerFactory.getLogger(ExpressionTest.class);
> @@ -51,8 +46,8 @@ public class ExpressionTest {
>        {
>          batch.getValueVectorId(new SchemaPath("alpha",
> ExpressionPosition.UNKNOWN));
>          result = tfid;
> -        batch.getValueVectorById(tfid.getFieldId(), Fixed4.class);
> -        result = new Fixed4(null, null);
> +        batch.getValueVectorById(tfid.getFieldId(), IntVector.class);
> +        result = new IntVector(null, null);
>        }
>
>      };
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/3d41be41/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/SimpleRootExec.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/SimpleRootExec.java
> b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/SimpleRootExec.java
> index 093c58f..d125ec0 100644
> ---
> a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/SimpleRootExec.java
> +++
> b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/SimpleRootExec.java
> @@ -5,7 +5,7 @@ import
> org.apache.drill.exec.physical.impl.ScreenCreator.ScreenRoot;
>  import org.apache.drill.exec.record.RecordBatch;
>  import org.apache.drill.exec.record.RecordBatch.IterOutcome;
>  import org.apache.drill.exec.record.RecordBatch.TypedFieldId;
> -import org.apache.drill.exec.record.vector.ValueVector;
> +import org.apache.drill.exec.vector.ValueVector;
>
>  public class SimpleRootExec implements RootExec{
>    static final org.slf4j.Logger logger =
> org.slf4j.LoggerFactory.getLogger(SimpleRootExec.class);
> @@ -22,7 +22,7 @@ public class SimpleRootExec implements RootExec{
>    }
>
>
> -  public <T extends ValueVector<T>> T getValueVectorById(SchemaPath path,
> Class<?> vvClass){
> +  public <T extends ValueVector> T getValueVectorById(SchemaPath path,
> Class<?> vvClass){
>      TypedFieldId tfid = incoming.getValueVectorId(path);
>      return incoming.getValueVectorById(tfid.getFieldId(), vvClass);
>    }
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/3d41be41/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 3dc961b..7b002ea 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
> @@ -17,7 +17,7 @@
>
> ******************************************************************************/
>  package org.apache.drill.exec.physical.impl;
>
> -import static org.junit.Assert.assertEquals;
> +import static org.junit.Assert.*;
>
>  import java.util.List;
>
> @@ -29,9 +29,9 @@ import org.apache.drill.exec.record.RecordBatchLoader;
>  import org.apache.drill.exec.rpc.user.QueryResultBatch;
>  import org.apache.drill.exec.server.Drillbit;
>  import org.apache.drill.exec.server.RemoteServiceSet;
> +import org.apache.drill.exec.vector.ValueVector;
>  import org.junit.Test;
>
> -import com.carrotsearch.hppc.cursors.IntObjectCursor;
>  import com.google.common.base.Charsets;
>  import com.google.common.io.Files;
>
> @@ -57,42 +57,40 @@ public class TestSimpleFragmentRun extends
> PopUnitTestBase {
>        boolean schemaChanged =
> batchLoader.load(batch.getHeader().getDef(), batch.getData());
>        boolean firstColumn = true;
>
> -          // print headers.
> -          if (schemaChanged) {
> -            System.out.println("\n\n========NEW SCHEMA=========\n\n");
> -            for (ValueVector<?> value : batchLoader) {
> +      // print headers.
> +      if (schemaChanged) {
> +        System.out.println("\n\n========NEW SCHEMA=========\n\n");
> +        for (ValueVector value : batchLoader) {
>
> -              if (firstColumn) {
> -                firstColumn = false;
> -              } else {
> -                System.out.print("\t");
> -              }
> -              System.out.print(value.getField().getName());
> -              System.out.print("[");
> -              System.out.print(value.getField().getType().getMinorType());
> -              System.out.print("]");
> -            }
> -            System.out.println();
> +          if (firstColumn) {
> +            firstColumn = false;
> +          } else {
> +            System.out.print("\t");
>            }
> +          System.out.print(value.getField().getName());
> +          System.out.print("[");
> +          System.out.print(value.getField().getType().getMinorType());
> +          System.out.print("]");
> +        }
> +        System.out.println();
> +      }
>
>
> -          for (int i = 0; i < batchLoader.getRecordCount(); i++) {
> -            boolean first = true;
> -            recordCount++;
> -            for (ValueVector<?> value : batchLoader) {
> -              if (first) {
> -                first = false;
> -              } else {
> -                System.out.print("\t");
> -              }
> -              System.out.print(value.getObject(i));
> -            }
> -            if(!first) System.out.println();
> +      for (int i = 0; i < batchLoader.getRecordCount(); i++) {
> +        boolean first = true;
> +        recordCount++;
> +        for (ValueVector value : batchLoader) {
> +          if (first) {
> +            first = false;
> +          } else {
> +            System.out.print("\t");
>            }
> -          System.out.print(v.value.getAccessor().getObject(i));
> +          System.out.print(value.getAccessor().getObject(i));
>          }
>          if(!first) System.out.println();
>        }
> +
> +
>
>      }
>      logger.debug("Received results {}", results);
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/3d41be41/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/project/TestSimpleProjection.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/project/TestSimpleProjection.java
> b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/project/TestSimpleProjection.java
> index 4144a54..e3f03f2 100644
> ---
> a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/project/TestSimpleProjection.java
> +++
> b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/project/TestSimpleProjection.java
> @@ -18,9 +18,9 @@ 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.FragmentHandle;
> -import org.apache.drill.exec.record.vector.Fixed8;
>  import org.apache.drill.exec.rpc.user.UserServer.UserClientConnection;
>  import org.apache.drill.exec.server.DrillbitContext;
> +import org.apache.drill.exec.vector.BigIntVector;
>  import org.junit.After;
>  import org.junit.Test;
>
> @@ -50,13 +50,18 @@ public class TestSimpleProjection {
>      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()){
> -      Fixed8 c1 = exec.getValueVectorById(new SchemaPath("col1",
> ExpressionPosition.UNKNOWN), Fixed8.class);
> -      Fixed8 c2 = exec.getValueVectorById(new SchemaPath("col2",
> ExpressionPosition.UNKNOWN), Fixed8.class);
> +      BigIntVector c1 = exec.getValueVectorById(new SchemaPath("col1",
> ExpressionPosition.UNKNOWN), BigIntVector.class);
> +      BigIntVector c2 = exec.getValueVectorById(new SchemaPath("col2",
> ExpressionPosition.UNKNOWN), BigIntVector.class);
>        int x = 0;
> -      for(int i =0; i < c1.getRecordCount(); i++){
> -        assertEquals(c1.get(i)+1, c2.get(i));
> -        x += c1.get(i);
> +      BigIntVector.Accessor a1, a2;
> +      a1 = c1.getAccessor();
> +      a2 = c2.getAccessor();
> +
> +      for(int i =0; i < c1.getAccessor().getRecordCount(); i++){
> +        assertEquals(a1.get(i)+1, a2.get(i));
> +        x += a1.get(i);
>        }
> +
>        System.out.println(x);
>      }
>    }
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/3d41be41/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/record/vector/TestOpenBitSet.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/record/vector/TestOpenBitSet.java
> b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/record/vector/TestOpenBitSet.java
> deleted file mode 100644
> index 66f69de..0000000
> ---
> a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/record/vector/TestOpenBitSet.java
> +++ /dev/null
> @@ -1,361 +0,0 @@
>
> -/*******************************************************************************
> - * Licensed to the Apache Software Foundation (ASF) under one
> - * or more contributor license agreements.  See the NOTICE file
> - * distributed with this work for additional information
> - * regarding copyright ownership.  The ASF licenses this file
> - * to you under the Apache License, Version 2.0 (the
> - * "License"); you may not use this file except in compliance
> - * with the License.  You may obtain a copy of the License at
> - *
> - * http://www.apache.org/licenses/LICENSE-2.0
> - *
> - * Unless required by applicable law or agreed to in writing, software
> - * distributed under the License is distributed on an "AS IS" BASIS,
> - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
> implied.
> - * See the License for the specific language governing permissions and
> - * limitations under the License.
> -
> ******************************************************************************/
> -package org.apache.drill.exec.record.vector;
> -
> -import static org.junit.Assert.assertEquals;
> -import static org.junit.Assert.assertFalse;
> -import static org.junit.Assert.assertTrue;
> -import static org.junit.Assert.fail;
> -import io.netty.buffer.ByteBufAllocator;
> -import io.netty.buffer.UnpooledByteBufAllocator;
> -
> -import java.util.BitSet;
> -import java.util.Random;
> -
> -import org.junit.Ignore;
> -import org.junit.Test;
> -
> -@Ignore
> -public class TestOpenBitSet {
> -  static final org.slf4j.Logger logger =
> org.slf4j.LoggerFactory.getLogger(TestOpenBitSet.class);
> -
> -  Random random = new Random();
> -  ByteBufAllocator allocator = UnpooledByteBufAllocator.DEFAULT;
> -
> -  public int atLeast(int val){
> -    return val + random.nextInt(val);
> -  }
> -
> -
> -  public Random random() {
> -    return new Random();
> -  }
> -
> -  void doGet(BitSet a, BufBitSet b) {
> -    int max = a.size();
> -    for (int i = 0; i < max; i++) {
> -      if (a.get(i) != b.get(i)) {
> -        fail("mismatch: BitSet=[" + i + "]=" + a.get(i));
> -      }
> -      if (a.get(i) != b.get((long) i)) {
> -        fail("mismatch: BitSet=[" + i + "]=" + a.get(i));
> -      }
> -    }
> -  }
> -
> -  void doGetFast(BitSet a, BufBitSet b, int max) {
> -    for (int i = 0; i < max; i++) {
> -      if (a.get(i) != b.fastGet(i)) {
> -        fail("mismatch: BitSet=[" + i + "]=" + a.get(i));
> -      }
> -      if (a.get(i) != b.fastGet((long) i)) {
> -        fail("mismatch: BitSet=[" + i + "]=" + a.get(i));
> -      }
> -    }
> -  }
> -
> -  void doNextSetBit(BitSet a, BufBitSet b) {
> -    int aa = -1, bb = -1;
> -    do {
> -      aa = a.nextSetBit(aa + 1);
> -      bb = b.nextSetBit(bb + 1);
> -      assertEquals(aa, bb);
> -    } while (aa >= 0);
> -  }
> -
> -  void doNextSetBitLong(BitSet a, BufBitSet b) {
> -    int aa = -1, bb = -1;
> -    do {
> -      aa = a.nextSetBit(aa + 1);
> -      bb = (int) b.nextSetBit((long) (bb + 1));
> -      assertEquals(aa, bb);
> -    } while (aa >= 0);
> -  }
> -
> -  void doPrevSetBit(BitSet a, BufBitSet b) {
> -    int aa = a.size() + random().nextInt(100);
> -    int bb = aa;
> -    do {
> -      // aa = a.prevSetBit(aa-1);
> -      aa--;
> -      while ((aa >= 0) && (!a.get(aa))) {
> -        aa--;
> -      }
> -      bb = b.prevSetBit(bb - 1);
> -      assertEquals(aa, bb);
> -    } while (aa >= 0);
> -  }
> -
> -  void doPrevSetBitLong(BitSet a, BufBitSet b) {
> -    int aa = a.size() + random().nextInt(100);
> -    int bb = aa;
> -    do {
> -      // aa = a.prevSetBit(aa-1);
> -      aa--;
> -      while ((aa >= 0) && (!a.get(aa))) {
> -        aa--;
> -      }
> -      bb = (int) b.prevSetBit((long) (bb - 1));
> -      assertEquals(aa, bb);
> -    } while (aa >= 0);
> -  }
> -
> -  // test interleaving different OpenBitSetIterator.next()/skipTo()
> -  void doIterate(BitSet a, BufBitSet b, int mode) {
> -    // if (mode == 1) doIterate1(a, b);
> -    // if (mode == 2) doIterate2(a, b);
> -  }
> -
> -  //
> -  // void doIterate1(BitSet a, OpenBitSet b) {
> -  // int aa = -1, bb = -1;
> -  // OpenBitSetIterator iterator = new OpenBitSetIterator(b);
> -  // do {
> -  // aa = a.nextSetBit(aa + 1);
> -  // bb = random().nextBoolean() ? iterator.nextDoc() :
> iterator.advance(bb + 1);
> -  // assertEquals(aa == -1 ? DocIdSetIterator.NO_MORE_DOCS : aa, bb);
> -  // } while (aa >= 0);
> -  // }
> -  //
> -  // void doIterate2(BitSet a, OpenBitSet b) {
> -  // int aa = -1, bb = -1;
> -  // OpenBitSetIterator iterator = new OpenBitSetIterator(b);
> -  // do {
> -  // aa = a.nextSetBit(aa + 1);
> -  // bb = random().nextBoolean() ? iterator.nextDoc() :
> iterator.advance(bb + 1);
> -  // assertEquals(aa == -1 ? DocIdSetIterator.NO_MORE_DOCS : aa, bb);
> -  // } while (aa >= 0);
> -  // }
> -
> -  void doRandomSets(int maxSize, int iter, int mode) {
> -    BitSet a0 = null;
> -    BufBitSet b0 = null;
> -
> -    for (int i = 0; i < iter; i++) {
> -      int sz = random().nextInt(maxSize);
> -      BitSet a = new BitSet(sz);
> -      BufBitSet b = new BufBitSet(sz, allocator);
> -
> -      // test the various ways of setting bits
> -      if (sz > 0) {
> -        int nOper = random().nextInt(sz);
> -        for (int j = 0; j < nOper; j++) {
> -          int idx;
> -
> -          idx = random().nextInt(sz);
> -          a.set(idx);
> -          b.fastSet(idx);
> -
> -          idx = random().nextInt(sz);
> -          a.set(idx);
> -          b.fastSet((long) idx);
> -
> -          idx = random().nextInt(sz);
> -          a.clear(idx);
> -          b.fastClear(idx);
> -
> -          idx = random().nextInt(sz);
> -          a.clear(idx);
> -          b.fastClear((long) idx);
> -
> -          idx = random().nextInt(sz);
> -          a.flip(idx);
> -          b.fastFlip(idx);
> -
> -          boolean val = b.flipAndGet(idx);
> -          boolean val2 = b.flipAndGet(idx);
> -          assertTrue(val != val2);
> -
> -          idx = random().nextInt(sz);
> -          a.flip(idx);
> -          b.fastFlip((long) idx);
> -
> -          val = b.flipAndGet((long) idx);
> -          val2 = b.flipAndGet((long) idx);
> -          assertTrue(val != val2);
> -
> -          val = b.getAndSet(idx);
> -          assertTrue(val2 == val);
> -          assertTrue(b.get(idx));
> -
> -          if (!val) b.fastClear(idx);
> -          assertTrue(b.get(idx) == val);
> -        }
> -      }
> -
> -      // test that the various ways of accessing the bits are equivalent
> -      doGet(a, b);
> -      doGetFast(a, b, sz);
> -
> -      // test ranges, including possible extension
> -      int fromIndex, toIndex;
> -      fromIndex = random().nextInt(sz + 80);
> -      toIndex = fromIndex + random().nextInt((sz >> 1) + 1);
> -      BitSet aa = (BitSet) a.clone();
> -      aa.flip(fromIndex, toIndex);
> -      BufBitSet bb = b.cloneTest();
> -      bb.flip(fromIndex, toIndex);
> -
> -      doIterate(aa, bb, mode); // a problem here is from flip or doIterate
> -
> -      fromIndex = random().nextInt(sz + 80);
> -      toIndex = fromIndex + random().nextInt((sz >> 1) + 1);
> -      aa = (BitSet) a.clone();
> -      aa.clear(fromIndex, toIndex);
> -      bb = b.cloneTest();
> -      bb.clear(fromIndex, toIndex);
> -
> -      doNextSetBit(aa, bb); // a problem here is from clear() or
> nextSetBit
> -      doNextSetBitLong(aa, bb);
> -
> -      doPrevSetBit(aa, bb);
> -      doPrevSetBitLong(aa, bb);
> -
> -      fromIndex = random().nextInt(sz + 80);
> -      toIndex = fromIndex + random().nextInt((sz >> 1) + 1);
> -      aa = (BitSet) a.clone();
> -      aa.set(fromIndex, toIndex);
> -      bb = b.cloneTest();
> -      bb.set(fromIndex, toIndex);
> -
> -      doNextSetBit(aa, bb); // a problem here is from set() or nextSetBit
> -      doNextSetBitLong(aa, bb);
> -
> -      doPrevSetBit(aa, bb);
> -      doPrevSetBitLong(aa, bb);
> -
> -      if (a0 != null) {
> -        assertEquals(a.equals(a0), b.equals(b0));
> -
> -        assertEquals(a.cardinality(), b.cardinality());
> -
> -        BitSet a_and = (BitSet) a.clone();
> -        a_and.and(a0);
> -        BitSet a_or = (BitSet) a.clone();
> -        a_or.or(a0);
> -        BitSet a_xor = (BitSet) a.clone();
> -        a_xor.xor(a0);
> -        BitSet a_andn = (BitSet) a.clone();
> -        a_andn.andNot(a0);
> -
> -        BufBitSet b_and = b.cloneTest();
> -        assertEquals(b, b_and);
> -        b_and.and(b0);
> -        BufBitSet b_or = b.cloneTest();
> -        b_or.or(b0);
> -        BufBitSet b_xor = b.cloneTest();
> -        b_xor.xor(b0);
> -        BufBitSet b_andn = b.cloneTest();
> -        b_andn.andNot(b0);
> -
> -        doIterate(a_and, b_and, mode);
> -        doIterate(a_or, b_or, mode);
> -        doIterate(a_xor, b_xor, mode);
> -        doIterate(a_andn, b_andn, mode);
> -
> -        assertEquals(a_and.cardinality(), b_and.cardinality());
> -        assertEquals(a_or.cardinality(), b_or.cardinality());
> -        assertEquals(a_xor.cardinality(), b_xor.cardinality());
> -        assertEquals(a_andn.cardinality(), b_andn.cardinality());
> -
> -        // test non-mutating popcounts
> -        assertEquals(b_and.cardinality(), BufBitSet.intersectionCount(b,
> b0));
> -        assertEquals(b_or.cardinality(), BufBitSet.unionCount(b, b0));
> -        assertEquals(b_xor.cardinality(), BufBitSet.xorCount(b, b0));
> -        assertEquals(b_andn.cardinality(), BufBitSet.andNotCount(b, b0));
> -      }
> -
> -      a0 = a;
> -      b0 = b;
> -    }
> -  }
> -
> -  // large enough to flush obvious bugs, small enough to run in <.5 sec
> as part of a
> -  // larger testsuite.
> -  @Test
> -  public void testSmall() {
> -    doRandomSets(atLeast(1200), atLeast(1000), 1);
> -    doRandomSets(atLeast(1200), atLeast(1000), 2);
> -  }
> -
> -  // uncomment to run a bigger test (~2 minutes).
> -  /*
> -   * public void testBig() { doRandomSets(2000,200000, 1);
> doRandomSets(2000,200000, 2); }
> -   */
> -
> -  @Test
> -  public void testEquals() {
> -    BufBitSet b1 = new BufBitSet(1111, allocator);
> -    BufBitSet b2 = new BufBitSet(2222, allocator);
> -    assertTrue(b1.equals(b2));
> -    assertTrue(b2.equals(b1));
> -    b1.set(10);
> -    assertFalse(b1.equals(b2));
> -    assertFalse(b2.equals(b1));
> -    b2.set(10);
> -    assertTrue(b1.equals(b2));
> -    assertTrue(b2.equals(b1));
> -    b2.set(2221);
> -    assertFalse(b1.equals(b2));
> -    assertFalse(b2.equals(b1));
> -    b1.set(2221);
> -    assertTrue(b1.equals(b2));
> -    assertTrue(b2.equals(b1));
> -
> -    // try different type of object
> -    assertFalse(b1.equals(new Object()));
> -  }
> -
> -  @Test
> -  public void testHashCodeEquals() {
> -    BufBitSet bs1 = new BufBitSet(200, allocator);
> -    BufBitSet bs2 = new BufBitSet(64, allocator);
> -    bs1.set(3);
> -    bs2.set(3);
> -    assertEquals(bs1, bs2);
> -    assertEquals(bs1.hashCode(), bs2.hashCode());
> -  }
> -
> -  private BufBitSet makeOpenBitSet(int[] a) {
> -    BufBitSet bs = new BufBitSet(64, allocator);
> -    for (int e : a) {
> -      bs.set(e);
> -    }
> -    return bs;
> -  }
> -
> -  private BitSet makeBitSet(int[] a) {
> -    BitSet bs = new BitSet();
> -    for (int e : a) {
> -      bs.set(e);
> -    }
> -    return bs;
> -  }
> -
> -  private void checkPrevSetBitArray(int[] a) {
> -    BufBitSet obs = makeOpenBitSet(a);
> -    BitSet bs = makeBitSet(a);
> -    doPrevSetBit(bs, obs);
> -  }
> -
> -  public void testPrevSetBit() {
> -    checkPrevSetBitArray(new int[] {});
> -    checkPrevSetBitArray(new int[] { 0 });
> -    checkPrevSetBitArray(new int[] { 0, 2 });
> -  }
> -}
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/3d41be41/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/record/vector/TestValueVector.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/record/vector/TestValueVector.java
> b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/record/vector/TestValueVector.java
> index 6a1f3ad..0fed756 100644
> ---
> a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/record/vector/TestValueVector.java
> +++
> b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/record/vector/TestValueVector.java
> @@ -4,8 +4,11 @@ import static org.junit.Assert.assertEquals;
>
>  import java.nio.charset.Charset;
>
> +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.exec.memory.DirectBufferAllocator;
> -import org.apache.drill.exec.proto.SchemaDefProtos;
> +import org.apache.drill.exec.proto.SchemaDefProtos.FieldDef;
>  import org.apache.drill.exec.record.MaterializedField;
>  import org.apache.drill.exec.vector.BitVector;
>  import org.apache.drill.exec.vector.NullableFloat4Vector;
> @@ -22,15 +25,13 @@ public class TestValueVector {
>    @Test
>    public void testFixedType() {
>      // Build a required uint field definition
> -    SchemaDefProtos.MajorType.Builder typeBuilder =
> SchemaDefProtos.MajorType.newBuilder();
> -    SchemaDefProtos.FieldDef.Builder defBuilder =
> SchemaDefProtos.FieldDef.newBuilder();
> +    MajorType.Builder typeBuilder = MajorType.newBuilder();
> +    FieldDef.Builder defBuilder = FieldDef.newBuilder();
>      typeBuilder
> -        .setMinorType(SchemaDefProtos.MinorType.UINT4)
> -        .setMode(SchemaDefProtos.DataMode.REQUIRED)
> +        .setMinorType(MinorType.UINT4)
> +        .setMode(DataMode.REQUIRED)
>          .setWidth(4);
>      defBuilder
> -        .setFieldId(1)
> -        .setParentId(0)
>          .setMajorType(typeBuilder.build());
>          MaterializedField field =
> MaterializedField.create(defBuilder.build());
>
> @@ -58,15 +59,13 @@ public class TestValueVector {
>    @Test
>    public void testNullableVarLen2() {
>      // Build an optional varchar field definition
> -    SchemaDefProtos.MajorType.Builder typeBuilder =
> SchemaDefProtos.MajorType.newBuilder();
> -    SchemaDefProtos.FieldDef.Builder defBuilder =
> SchemaDefProtos.FieldDef.newBuilder();
> +    MajorType.Builder typeBuilder = MajorType.newBuilder();
> +    FieldDef.Builder defBuilder = FieldDef.newBuilder();
>      typeBuilder
> -        .setMinorType(SchemaDefProtos.MinorType.VARCHAR2)
> -        .setMode(SchemaDefProtos.DataMode.OPTIONAL)
> +        .setMinorType(MinorType.VARCHAR2)
> +        .setMode(DataMode.OPTIONAL)
>          .setWidth(2);
>      defBuilder
> -        .setFieldId(1)
> -        .setParentId(0)
>          .setMajorType(typeBuilder.build());
>      MaterializedField field =
> MaterializedField.create(defBuilder.build());
>
> @@ -106,15 +105,13 @@ public class TestValueVector {
>    @Test
>    public void testNullableFixedType() {
>      // Build an optional uint field definition
> -    SchemaDefProtos.MajorType.Builder typeBuilder =
> SchemaDefProtos.MajorType.newBuilder();
> -    SchemaDefProtos.FieldDef.Builder defBuilder =
> SchemaDefProtos.FieldDef.newBuilder();
> +    MajorType.Builder typeBuilder = MajorType.newBuilder();
> +    FieldDef.Builder defBuilder = FieldDef.newBuilder();
>      typeBuilder
> -        .setMinorType(SchemaDefProtos.MinorType.UINT4)
> -        .setMode(SchemaDefProtos.DataMode.OPTIONAL)
> +        .setMinorType(MinorType.UINT4)
> +        .setMode(DataMode.OPTIONAL)
>          .setWidth(4);
>      defBuilder
> -        .setFieldId(1)
> -        .setParentId(0)
>          .setMajorType(typeBuilder.build());
>      MaterializedField field =
> MaterializedField.create(defBuilder.build());
>
> @@ -195,15 +192,13 @@ public class TestValueVector {
>    @Test
>    public void testNullableFloat() {
>      // Build an optional float field definition
> -    SchemaDefProtos.MajorType.Builder typeBuilder =
> SchemaDefProtos.MajorType.newBuilder();
> -    SchemaDefProtos.FieldDef.Builder defBuilder =
> SchemaDefProtos.FieldDef.newBuilder();
> +    MajorType.Builder typeBuilder = MajorType.newBuilder();
> +    FieldDef.Builder defBuilder = FieldDef.newBuilder();
>      typeBuilder
> -        .setMinorType(SchemaDefProtos.MinorType.FLOAT4)
> -        .setMode(SchemaDefProtos.DataMode.OPTIONAL)
> +        .setMinorType(MinorType.FLOAT4)
> +        .setMode(DataMode.OPTIONAL)
>          .setWidth(4);
>      defBuilder
> -        .setFieldId(1)
> -        .setParentId(0)
>          .setMajorType(typeBuilder.build());
>      MaterializedField field =
> MaterializedField.create(defBuilder.build());
>
> @@ -256,15 +251,13 @@ public class TestValueVector {
>    @Test
>    public void testBitVector() {
>      // Build a required boolean field definition
> -    SchemaDefProtos.MajorType.Builder typeBuilder =
> SchemaDefProtos.MajorType.newBuilder();
> -    SchemaDefProtos.FieldDef.Builder defBuilder =
> SchemaDefProtos.FieldDef.newBuilder();
> +    MajorType.Builder typeBuilder = MajorType.newBuilder();
> +    FieldDef.Builder defBuilder = FieldDef.newBuilder();
>      typeBuilder
> -        .setMinorType(SchemaDefProtos.MinorType.BOOLEAN)
> -        .setMode(SchemaDefProtos.DataMode.REQUIRED)
> +        .setMinorType(MinorType.BOOLEAN)
> +        .setMode(DataMode.REQUIRED)
>          .setWidth(4);
>      defBuilder
> -        .setFieldId(1)
> -        .setParentId(0)
>          .setMajorType(typeBuilder.build());
>      MaterializedField field =
> MaterializedField.create(defBuilder.build());
>
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/3d41be41/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/JSONRecordReaderTest.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/JSONRecordReaderTest.java
> b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/JSONRecordReaderTest.java
> index d8966ae..7c9e8f4 100644
> ---
> a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/JSONRecordReaderTest.java
> +++
> b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/JSONRecordReaderTest.java
> @@ -19,6 +19,7 @@ import org.apache.drill.exec.ops.FragmentContext;
>  import org.apache.drill.exec.physical.impl.OutputMutator;
>  import org.apache.drill.exec.proto.SchemaDefProtos;
>  import org.apache.drill.exec.proto.UserBitShared;
> +import org.apache.drill.exec.record.MaterializedField;
>  import org.apache.drill.exec.vector.ValueVector;
>  import org.junit.Ignore;
>  import org.junit.Test;
> @@ -34,7 +35,7 @@ public class JSONRecordReaderTest {
>
>    class MockOutputMutator implements OutputMutator {
>      List<MaterializedField> removedFields = Lists.newArrayList();
> -    List<ValueVector<?>> addFields = Lists.newArrayList();
> +    List<ValueVector> addFields = Lists.newArrayList();
>
>      @Override
>      public void removeField(MaterializedField field) throws
> SchemaChangeException {
> @@ -42,7 +43,7 @@ public class JSONRecordReaderTest {
>      }
>
>      @Override
> -    public void addField(ValueVector<?> vector) throws
> SchemaChangeException {
> +    public void addField(ValueVector vector) throws SchemaChangeException
> {
>        addFields.add(vector);
>      }
>
> @@ -54,7 +55,7 @@ public class JSONRecordReaderTest {
>        return removedFields;
>      }
>
> -    List<ValueVector<?>> getAddFields() {
> +    List<ValueVector> getAddFields() {
>        return addFields;
>      }
>    }
> @@ -69,7 +70,7 @@ public class JSONRecordReaderTest {
>        return;
>      }
>
> -    T val = (T) valueVector.getObject(index);
> +    T val = (T) valueVector.getAccessor().getObject(index);
>      if (val instanceof byte[]) {
>        assertTrue(Arrays.equals((byte[]) value, (byte[]) val));
>      } else {
> @@ -89,15 +90,15 @@ public class JSONRecordReaderTest {
>      JSONRecordReader jr = new JSONRecordReader(context,
> getResource("scan_json_test_1.json"));
>
>      MockOutputMutator mutator = new MockOutputMutator();
> -    List<ValueVector<?>> addFields = mutator.getAddFields();
> +    List<ValueVector> addFields = mutator.getAddFields();
>      jr.setup(mutator);
>      assertEquals(2, jr.next());
>      assertEquals(3, addFields.size());
>      assertField(addFields.get(0), 0, MinorType.INT, 123, "test");
> -    assertField(addFields.get(1), 0, MinorType.BOOLEAN, 1, "b");
> +    assertField(addFields.get(1), 0, MinorType.BOOLEAN, true, "b");
>      assertField(addFields.get(2), 0, MinorType.VARCHAR4,
> "hi!".getBytes(UTF_8), "c");
>      assertField(addFields.get(0), 1, MinorType.INT, 1234, "test");
> -    assertField(addFields.get(1), 1, MinorType.BOOLEAN, 0, "b");
> +    assertField(addFields.get(1), 1, MinorType.BOOLEAN, false, "b");
>      assertField(addFields.get(2), 1, MinorType.VARCHAR4,
> "drill!".getBytes(UTF_8), "c");
>
>      assertEquals(0, jr.next());
> @@ -116,7 +117,7 @@ public class JSONRecordReaderTest {
>
>      JSONRecordReader jr = new JSONRecordReader(context,
> getResource("scan_json_test_2.json"));
>      MockOutputMutator mutator = new MockOutputMutator();
> -    List<ValueVector<?>> addFields = mutator.getAddFields();
> +    List<ValueVector> addFields = mutator.getAddFields();
>
>      jr.setup(mutator);
>      assertEquals(3, jr.next());
> @@ -124,25 +125,25 @@ public class JSONRecordReaderTest {
>      assertField(addFields.get(0), 0, MinorType.INT, 123, "test");
>      assertField(addFields.get(1), 0, MinorType.INT, 1, "b");
>      assertField(addFields.get(2), 0, MinorType.FLOAT4, (float) 2.15, "c");
> -    assertField(addFields.get(3), 0, MinorType.BOOLEAN, 1, "bool");
> +    assertField(addFields.get(3), 0, MinorType.BOOLEAN, true, "bool");
>      assertField(addFields.get(4), 0, MinorType.VARCHAR4,
> "test1".getBytes(UTF_8), "str1");
>
>      assertField(addFields.get(0), 1, MinorType.INT, 1234, "test");
>      assertField(addFields.get(1), 1, MinorType.INT, 3, "b");
> -    assertField(addFields.get(3), 1, MinorType.BOOLEAN, 0, "bool");
> +    assertField(addFields.get(3), 1, MinorType.BOOLEAN, false, "bool");
>      assertField(addFields.get(4), 1, MinorType.VARCHAR4,
> "test2".getBytes(UTF_8), "str1");
>      assertField(addFields.get(5), 1, MinorType.INT, 4, "d");
>
>      assertField(addFields.get(0), 2, MinorType.INT, 12345, "test");
>      assertField(addFields.get(2), 2, MinorType.FLOAT4, (float) 5.16, "c");
> -    assertField(addFields.get(3), 2, MinorType.BOOLEAN, 1, "bool");
> +    assertField(addFields.get(3), 2, MinorType.BOOLEAN, true, "bool");
>      assertField(addFields.get(5), 2, MinorType.INT, 6, "d");
>      assertField(addFields.get(6), 2, MinorType.VARCHAR4,
> "test3".getBytes(UTF_8), "str2");
>      assertTrue(mutator.getRemovedFields().isEmpty());
>      assertEquals(0, jr.next());
>    }
>
> -  @Test
> +  @Test @Ignore
>    public void testChangedSchemaInTwoBatches(@Injectable final
> FragmentContext context) throws IOException,
>        ExecutionSetupException {
>      new Expectations() {
> @@ -155,7 +156,7 @@ public class JSONRecordReaderTest {
>      JSONRecordReader jr = new JSONRecordReader(context,
> getResource("scan_json_test_2.json"), 64); // batch only fits 1
>
>                           // int
>      MockOutputMutator mutator = new MockOutputMutator();
> -    List<ValueVector<?>> addFields = mutator.getAddFields();
> +    List<ValueVector> addFields = mutator.getAddFields();
>      List<MaterializedField> removedFields = mutator.getRemovedFields();
>
>      jr.setup(mutator);
> @@ -164,14 +165,14 @@ public class JSONRecordReaderTest {
>      assertField(addFields.get(0), 0, MinorType.INT, 123, "test");
>      assertField(addFields.get(1), 0, MinorType.INT, 1, "b");
>      assertField(addFields.get(2), 0, MinorType.FLOAT4, (float) 2.15, "c");
> -    assertField(addFields.get(3), 0, MinorType.BOOLEAN, 1, "bool");
> +    assertField(addFields.get(3), 0, MinorType.BOOLEAN, true, "bool");
>      assertField(addFields.get(4), 0, MinorType.VARCHAR4,
> "test1".getBytes(UTF_8), "str1");
>      assertTrue(removedFields.isEmpty());
>      assertEquals(1, jr.next());
>      assertEquals(6, addFields.size());
>      assertField(addFields.get(0), 0, MinorType.INT, 1234, "test");
>      assertField(addFields.get(1), 0, MinorType.INT, 3, "b");
> -    assertField(addFields.get(3), 0, MinorType.BOOLEAN, 0, "bool");
> +    assertField(addFields.get(3), 0, MinorType.BOOLEAN, false, "bool");
>      assertField(addFields.get(4), 0, MinorType.VARCHAR4,
> "test2".getBytes(UTF_8), "str1");
>      assertField(addFields.get(5), 0, MinorType.INT, 4, "d");
>      assertEquals(1, removedFields.size());
> @@ -180,7 +181,7 @@ public class JSONRecordReaderTest {
>      assertEquals(1, jr.next());
>      assertEquals(8, addFields.size()); // The reappearing of field 'c' is
> also included
>      assertField(addFields.get(0), 0, MinorType.INT, 12345, "test");
> -    assertField(addFields.get(3), 0, MinorType.BOOLEAN, 1, "bool");
> +    assertField(addFields.get(3), 0, MinorType.BOOLEAN, true, "bool");
>      assertField(addFields.get(5), 0, MinorType.INT, 6, "d");
>      assertField(addFields.get(6), 0, MinorType.FLOAT4, (float) 5.16, "c");
>      assertField(addFields.get(7), 0, MinorType.VARCHAR4,
> "test3".getBytes(UTF_8), "str2");
> @@ -190,7 +191,7 @@ public class JSONRecordReaderTest {
>      assertEquals(0, jr.next());
>    }
>
> -  @Test
> +  @Test @Ignore
>    public void testNestedFieldInSameBatch(@Injectable final
> FragmentContext context) throws ExecutionSetupException {
>      new Expectations() {
>        {
> @@ -202,7 +203,7 @@ public class JSONRecordReaderTest {
>      JSONRecordReader jr = new JSONRecordReader(context,
> getResource("scan_json_test_3.json"));
>
>      MockOutputMutator mutator = new MockOutputMutator();
> -    List<ValueVector<?>> addFields = mutator.getAddFields();
> +    List<ValueVector> addFields = mutator.getAddFields();
>      jr.setup(mutator);
>      assertEquals(2, jr.next());
>      assertEquals(5, addFields.size());
> @@ -210,12 +211,12 @@ public class JSONRecordReaderTest {
>      assertField(addFields.get(1), 0, MinorType.MAP, null, "a");
>      assertField(addFields.get(2), 0, MinorType.VARCHAR4,
> "test".getBytes(UTF_8), "b");
>      assertField(addFields.get(3), 0, MinorType.MAP, null, "a");
> -    assertField(addFields.get(4), 0, MinorType.BOOLEAN, 1, "d");
> +    assertField(addFields.get(4), 0, MinorType.BOOLEAN, true, "d");
>      assertField(addFields.get(0), 1, MinorType.INT, 1234, "test");
>      assertField(addFields.get(1), 1, MinorType.MAP, null, "a");
>      assertField(addFields.get(2), 1, MinorType.VARCHAR4,
> "test2".getBytes(UTF_8), "b");
>      assertField(addFields.get(3), 1, MinorType.MAP, null, "a");
> -    assertField(addFields.get(4), 1, MinorType.BOOLEAN, 0, "d");
> +    assertField(addFields.get(4), 1, MinorType.BOOLEAN, true, "d");
>
>      assertEquals(0, jr.next());
>      assertTrue(mutator.getRemovedFields().isEmpty());
>
>

[47/53] [abbrv] Working Project merge build

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/3d41be41/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/BufBitSet.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/BufBitSet.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/BufBitSet.java
deleted file mode 100644
index 574389f..0000000
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/BufBitSet.java
+++ /dev/null
@@ -1,847 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.drill.exec.record.vector; // from org.apache.solr.util rev 555343
-
-import io.netty.buffer.ByteBuf;
-import io.netty.buffer.ByteBufAllocator;
-
-/**
- * HEAVY WIP: ONLY PARTIALLY TRANSFERRED TO BUFFER METHODS. STILL NEEDS BIT SHIFT FIXES, GETLONG AND SETLONG updates to
- * fix index postion AND OTHER THINGS.
- * 
- * An "open" BitSet implementation that allows direct access to the array of words storing the bits.
- * <p/>
- * Unlike java.util.bitset, the fact that bits are packed into an array of longs is part of the interface. This allows
- * efficient implementation of other algorithms by someone other than the author. It also allows one to efficiently
- * implement alternate serialization or interchange formats.
- * <p/>
- * <code>BufBitSet</code> is faster than <code>java.util.BitSet</code> in most operations and *much* faster at
- * calculating cardinality of sets and results of set operations. It can also handle sets of larger cardinality (up to
- * 64 * 2**32-1)
- * <p/>
- * The goals of <code>BufBitSet</code> are the fastest implementation possible, and maximum code reuse. Extra safety
- * and encapsulation may always be built on top, but if that's built in, the cost can never be removed (and hence people
- * re-implement their own version in order to get better performance). If you want a "safe", totally encapsulated (and
- * slower and limited) BitSet class, use <code>java.util.BitSet</code>.
- * <p/>
- */
-
-public class BufBitSet {
-  private ByteBufAllocator allocator;
-  private ByteBuf buf;
-  // protected long[] bits;
-  protected int wlen; // number of words (elements) used in the array
-
-  // Used only for assert:
-  private long numBits;
-
-  // /** Constructs an BufBitSet large enough to hold <code>numBits</code>.
-  // */
-  // public BufBitSet(long numBits) {
-  // this.numBits = numBits;
-  // wlen = buf.capacity();
-  // }
-  //
-  // public BufBitSet() {
-  // this(64);
-  // }
-
-  public BufBitSet(long numBits, ByteBufAllocator allocator) {
-    this.allocator = allocator;
-    this.numBits = numBits;
-    int words = bits2words(numBits);
-    this.wlen = words;
-    buf = allocator.buffer(wlen);
-  }
-
-  private BufBitSet(ByteBufAllocator allocator, ByteBuf buf) {
-    this.allocator = allocator;
-    this.numBits = buf.capacity() * 8;
-    int words = buf.capacity();
-    this.wlen = words;
-    this.buf = buf;
-  }
-
-  /** Returns the current capacity in bits (1 greater than the index of the last bit) */
-  public long capacity() {
-    return buf.capacity() << 6;
-  }
-
-  /**
-   * Returns the current capacity of this set. Included for compatibility. This is *not* equal to {@link #cardinality}
-   */
-  public long size() {
-    return capacity();
-  }
-
-  public int length() {
-    return buf.capacity() << 6;
-  }
-
-  /** Returns true if there are no set bits */
-  public boolean isEmpty() {
-    return cardinality() == 0;
-  }
-
-  // /** Expert: returns the long[] storing the bits */
-  // public long[] getBits() { return bits; }
-  //
-  // /** Expert: sets a new long[] to use as the bit storage */
-  // public void setBits(long[] bits) { this.bits = bits; }
-
-  /** Expert: gets the number of longs in the array that are in use */
-  public int getNumWords() {
-    return wlen;
-  }
-
-  /** Expert: sets the number of longs in the array that are in use */
-  public void setNumWords(int nWords) {
-    this.wlen = nWords;
-  }
-
-  /** Returns true or false for the specified bit index. */
-  public boolean get(int index) {
-    int i = index >> 6; // div 64
-    // signed shift will keep a negative index and force an
-    // array-index-out-of-bounds-exception, removing the need for an explicit check.
-    if (i >= buf.capacity()) return false;
-
-    int bit = index & 0x3f; // mod 64
-    long bitmask = 1L << bit;
-    return (buf.getLong(i) & bitmask) != 0;
-  }
-
-  /**
-   * Returns true or false for the specified bit index. The index should be less than the BufBitSet size
-   */
-  public boolean fastGet(int index) {
-    assert index >= 0 && index < numBits;
-    int i = index >> 6; // div 64
-    // signed shift will keep a negative index and force an
-    // array-index-out-of-bounds-exception, removing the need for an explicit check.
-    int bit = index & 0x3f; // mod 64
-    long bitmask = 1L << bit;
-    return (buf.getLong(i) & bitmask) != 0;
-  }
-
-  /**
-   * Returns true or false for the specified bit index
-   */
-  public boolean get(long index) {
-    int i = (int) (index >> 6); // div 64
-    if (i >= buf.capacity()) return false;
-    int bit = (int) index & 0x3f; // mod 64
-    long bitmask = 1L << bit;
-    return (buf.getLong(i) & bitmask) != 0;
-  }
-
-  /**
-   * Returns true or false for the specified bit index. The index should be less than the BufBitSet size.
-   */
-  public boolean fastGet(long index) {
-    assert index >= 0 && index < numBits;
-    int i = (int) (index >> 6); // div 64
-    int bit = (int) index & 0x3f; // mod 64
-    long bitmask = 1L << bit;
-    return (buf.getLong(i) & bitmask) != 0;
-  }
-
-  /*
-   * // alternate implementation of get() public boolean get1(int index) { int i = index >> 6; // div 64 int bit = index
-   * & 0x3f; // mod 64 return ((buf.getLong(i)>>>bit) & 0x01) != 0; // this does a long shift and a bittest (on x86) vs
-   * // a long shift, and a long AND, (the test for zero is prob a no-op) // testing on a P4 indicates this is slower
-   * than (buf.getLong(i) & bitmask) != 0; }
-   */
-
-  /**
-   * returns 1 if the bit is set, 0 if not. The index should be less than the BufBitSet size
-   */
-  public int getBit(int index) {
-    assert index >= 0 && index < numBits;
-    int i = index >> 6; // div 64
-    int bit = index & 0x3f; // mod 64
-    return ((int) (buf.getLong(i) >>> bit)) & 0x01;
-  }
-
-  /*
-   * public boolean get2(int index) { int word = index >> 6; // div 64 int bit = index & 0x0000003f; // mod 64 return
-   * (buf.getLong(word) << bit) < 0; // hmmm, this would work if bit order were reversed // we could right shift and
-   * check for parity bit, if it was available to us. }
-   */
-
-  /** sets a bit, expanding the set size if necessary */
-  public void set(long index) {
-    int wordNum = expandingWordNum(index);
-    int bit = (int) index & 0x3f;
-    long bitmask = 1L << bit;
-    buf.setLong(wordNum, buf.getLong(wordNum) | bitmask);
-    buf.setLong(wordNum, buf.getLong(wordNum) | bitmask);
-  }
-
-  /**
-   * Sets the bit at the specified index. The index should be less than the BufBitSet size.
-   */
-  public void fastSet(int index) {
-    assert index >= 0 && index < numBits;
-    int wordNum = index >> 6; // div 64
-    int bit = index & 0x3f; // mod 64
-    long bitmask = 1L << bit;
-    buf.setLong(wordNum, buf.getLong(wordNum) | bitmask);
-  }
-
-  /**
-   * Sets the bit at the specified index. The index should be less than the BufBitSet size.
-   */
-  public void fastSet(long index) {
-    assert index >= 0 && index < numBits;
-    int wordNum = (int) (index >> 6);
-    int bit = (int) index & 0x3f;
-    long bitmask = 1L << bit;
-    buf.setLong(wordNum, buf.getLong(wordNum) | bitmask);
-  }
-
-  /**
-   * Sets a range of bits, expanding the set size if necessary
-   * 
-   * @param startIndex
-   *          lower index
-   * @param endIndex
-   *          one-past the last bit to set
-   */
-  public void set(long startIndex, long endIndex) {
-    if (endIndex <= startIndex) return;
-
-    int startWord = (int) (startIndex >> 6);
-
-    // since endIndex is one past the end, this is index of the last
-    // word to be changed.
-    int endWord = expandingWordNum(endIndex - 1);
-
-    long startmask = -1L << startIndex;
-    long endmask = -1L >>> -endIndex; // 64-(endIndex&0x3f) is the same as -endIndex due to wrap
-
-    if (startWord == endWord) {
-      buf.setLong(startWord, buf.getLong(startWord) | (startmask & endmask));
-      return;
-    }
-    buf.setLong(startWord, buf.getLong(startWord) | startmask);
-
-    fill(buf, startWord + 1, endWord, -1L);
-    buf.setLong(endWord, buf.getLong(endWord) | endmask);
-  }
-
-  private void fill(ByteBuf buf, int start, int end, long val) {
-    for (int i = 0; i < buf.capacity(); i += 8) {
-      buf.setLong(i, val);
-    }
-  }
-
-  private final void setLongWord(int pos, long value) {
-    buf.setLong(pos * 8, value);
-  }
-
-  private final long getLongWord(int pos) {
-    return buf.getLong(pos * 8);
-  }
-
-  protected int expandingWordNum(long index) {
-    int wordNum = (int) (index >> 6);
-    if (wordNum >= wlen) {
-      ensureCapacity(index + 1);
-      wlen = wordNum + 1;
-    }
-    assert (numBits = Math.max(numBits, index + 1)) >= 0;
-    return wordNum;
-  }
-
-  /**
-   * clears a bit. The index should be less than the BufBitSet size.
-   */
-  public void fastClear(int index) {
-    assert index >= 0 && index < numBits;
-    int wordNum = index >> 6;
-    int bit = index & 0x03f;
-    long bitmask = 1L << bit;
-    buf.setLong(wordNum, buf.getLong(wordNum) & ~bitmask);
-    // hmmm, it takes one more instruction to clear than it does to set... any
-    // way to work around this? If there were only 63 bits per word, we could
-    // use a right shift of 10111111...111 in binary to position the 0 in the
-    // correct place (using sign extension).
-    // Could also use Long.rotateRight() or rotateLeft() *if* they were converted
-    // by the JVM into a native instruction.
-    // buf.getLong(word) &= Long.rotateLeft(0xfffffffe,bit);
-  }
-
-  /**
-   * clears a bit. The index should be less than the BufBitSet size.
-   */
-  public void fastClear(long index) {
-    assert index >= 0 && index < numBits;
-    int wordNum = (int) (index >> 6); // div 64
-    int bit = (int) index & 0x3f; // mod 64
-    long bitmask = 1L << bit;
-    buf.setLong(wordNum, buf.getLong(wordNum) & ~bitmask);
-  }
-
-  /** clears a bit, allowing access beyond the current set size without changing the size. */
-  public void clear(long index) {
-    int wordNum = (int) (index >> 6); // div 64
-    if (wordNum >= wlen) return;
-    int bit = (int) index & 0x3f; // mod 64
-    long bitmask = 1L << bit;
-    buf.setLong(wordNum, buf.getLong(wordNum) & ~bitmask);
-  }
-
-  /**
-   * Clears a range of bits. Clearing past the end does not change the size of the set.
-   * 
-   * @param startIndex
-   *          lower index
-   * @param endIndex
-   *          one-past the last bit to clear
-   */
-  public void clear(int startIndex, int endIndex) {
-    if (endIndex <= startIndex) return;
-
-    int startWord = (startIndex >> 6);
-    if (startWord >= wlen) return;
-
-    // since endIndex is one past the end, this is index of the last
-    // word to be changed.
-    int endWord = ((endIndex - 1) >> 6);
-
-    long startmask = -1L << startIndex;
-    long endmask = -1L >>> -endIndex; // 64-(endIndex&0x3f) is the same as -endIndex due to wrap
-
-    // invert masks since we are clearing
-    startmask = ~startmask;
-    endmask = ~endmask;
-
-    if (startWord == endWord) {
-      buf.setLong(startWord, buf.getLong(startWord) & (startmask | endmask));
-      return;
-    }
-
-    buf.setLong(startWord, buf.getLong(startWord) & startmask);
-
-    int middle = Math.min(wlen, endWord);
-    fill(buf, startWord + 1, middle, 0L);
-    if (endWord < wlen) {
-      buf.setLong(endWord, buf.getLong(endWord) & endmask);
-    }
-  }
-
-  /**
-   * Clears a range of bits. Clearing past the end does not change the size of the set.
-   * 
-   * @param startIndex
-   *          lower index
-   * @param endIndex
-   *          one-past the last bit to clear
-   */
-  public void clear(long startIndex, long endIndex) {
-    if (endIndex <= startIndex) return;
-
-    int startWord = (int) (startIndex >> 6);
-    if (startWord >= wlen) return;
-
-    // since endIndex is one past the end, this is index of the last
-    // word to be changed.
-    int endWord = (int) ((endIndex - 1) >> 6);
-
-    long startmask = -1L << startIndex;
-    long endmask = -1L >>> -endIndex; // 64-(endIndex&0x3f) is the same as -endIndex due to wrap
-
-    // invert masks since we are clearing
-    startmask = ~startmask;
-    endmask = ~endmask;
-
-    if (startWord == endWord) {
-      buf.setLong(startWord, buf.getLong(startWord) & (startmask | endmask));
-      return;
-    }
-
-    buf.setLong(startWord, buf.getLong(startWord) & startmask);
-
-    int middle = Math.min(wlen, endWord);
-    fill(buf, startWord + 1, middle, 0L);
-    if (endWord < wlen) {
-      buf.setLong(endWord, buf.getLong(endWord) & endmask);
-    }
-  }
-
-  /**
-   * Sets a bit and returns the previous value. The index should be less than the BufBitSet size.
-   */
-  public boolean getAndSet(int index) {
-    assert index >= 0 && index < numBits;
-    int wordNum = index >> 6; // div 64
-    int bit = index & 0x3f; // mod 64
-    long bitmask = 1L << bit;
-    long longVal = buf.getLong(wordNum);
-    boolean val = (longVal & bitmask) != 0;
-    buf.setLong(wordNum, longVal | bitmask);
-    return val;
-  }
-
-  /**
-   * flips a bit. The index should be less than the BufBitSet size.
-   */
-  public void fastFlip(int index) {
-    assert index >= 0 && index < numBits;
-    int wordNum = index >> 6; // div 64
-    int bit = index & 0x3f; // mod 64
-    long bitmask = 1L << bit;
-    buf.setLong(wordNum, (buf.getLong(wordNum) ^ bitmask));
-  }
-
-  /**
-   * flips a bit. The index should be less than the BufBitSet size.
-   */
-  public void fastFlip(long index) {
-    assert index >= 0 && index < numBits;
-    int wordNum = (int) (index >> 6); // div 64
-    int bit = (int) index & 0x3f; // mod 64
-    long bitmask = 1L << bit;
-    buf.setLong(wordNum, (buf.getLong(wordNum) ^ bitmask));
-  }
-
-  /** flips a bit, expanding the set size if necessary */
-  public void flip(long index) {
-    int wordNum = expandingWordNum(index);
-    int bit = (int) index & 0x3f; // mod 64
-    long bitmask = 1L << bit;
-    buf.setLong(wordNum, (buf.getLong(wordNum) ^ bitmask));
-  }
-
-  /**
-   * flips a bit and returns the resulting bit value. The index should be less than the BufBitSet size.
-   */
-  public boolean flipAndGet(int index) {
-    assert index >= 0 && index < numBits;
-    int wordNum = index >> 6; // div 64
-    int bit = index & 0x3f; // mod 64
-    long bitmask = 1L << bit;
-    long longVal = buf.getLong(wordNum);
-    buf.setLong(wordNum, longVal ^ bitmask);
-    return (longVal & bitmask) != 0;
-  }
-
-  /**
-   * flips a bit and returns the resulting bit value. The index should be less than the BufBitSet size.
-   */
-  public boolean flipAndGet(long index) {
-    assert index >= 0 && index < numBits;
-    int wordNum = (int) (index >> 6); // div 64
-    int bit = (int) index & 0x3f; // mod 64
-    long bitmask = 1L << bit;
-    long longVal = buf.getLong(wordNum);
-    buf.setLong(wordNum, longVal ^ bitmask);
-    return (longVal & bitmask) != 0;
-  }
-
-  /**
-   * Flips a range of bits, expanding the set size if necessary
-   * 
-   * @param startIndex
-   *          lower index
-   * @param endIndex
-   *          one-past the last bit to flip
-   */
-  public void flip(long startIndex, long endIndex) {
-    if (endIndex <= startIndex) return;
-    int startWord = (int) (startIndex >> 6);
-
-    // since endIndex is one past the end, this is index of the last
-    // word to be changed.
-    int endWord = expandingWordNum(endIndex - 1);
-
-    /***
-     * Grrr, java shifting wraps around so -1L>>>64 == -1 for that reason, make sure not to use endmask if the bits to
-     * flip will be zero in the last word (redefine endWord to be the last changed...) long startmask = -1L <<
-     * (startIndex & 0x3f); // example: 11111...111000 long endmask = -1L >>> (64-(endIndex & 0x3f)); // example:
-     * 00111...111111
-     ***/
-
-    long startmask = -1L << startIndex;
-    long endmask = -1L >>> -endIndex; // 64-(endIndex&0x3f) is the same as -endIndex due to wrap
-
-    if (startWord == endWord) {
-      buf.setLong(startWord, buf.getLong(startWord) ^ (startmask & endmask));
-      return;
-    }
-
-    buf.setLong(startWord, buf.getLong(startWord) ^ startmask);
-
-    for (int i = startWord + 1; i < endWord; i++) {
-      buf.setLong(i, ~buf.getLong(i));
-    }
-
-    buf.setLong(endWord, buf.getLong(endWord) ^ endmask);
-  }
-
-  /*
-   * public static int pop(long v0, long v1, long v2, long v3) { // derived from pop_array by setting last four elems to
-   * 0. // exchanges one pop() call for 10 elementary operations // saving about 7 instructions... is there a better
-   * way? long twosA=v0 & v1; long ones=v0^v1;
-   * 
-   * long u2=ones^v2; long twosB =(ones&v2)|(u2&v3); ones=u2^v3;
-   * 
-   * long fours=(twosA&twosB); long twos=twosA^twosB;
-   * 
-   * return (pop(fours)<<2) + (pop(twos)<<1) + pop(ones);
-   * 
-   * }
-   */
-
-  /** @return the number of set bits */
-  public long cardinality() {
-    return BitUtil.pop_array(buf, 0, wlen);
-  }
-
-  /**
-   * Returns the popcount or cardinality of the intersection of the two sets. Neither set is modified.
-   */
-  public static long intersectionCount(BufBitSet a, BufBitSet b) {
-    return BitUtil.pop_intersect(a.buf, b.buf, 0, Math.min(a.wlen, b.wlen));
-  }
-
-  /**
-   * Returns the popcount or cardinality of the union of the two sets. Neither set is modified.
-   */
-  public static long unionCount(BufBitSet a, BufBitSet b) {
-    long tot = BitUtil.pop_union(a.buf, b.buf, 0, Math.min(a.wlen, b.wlen));
-    if (a.wlen < b.wlen) {
-      tot += BitUtil.pop_array(b.buf, a.wlen, b.wlen - a.wlen);
-    } else if (a.wlen > b.wlen) {
-      tot += BitUtil.pop_array(a.buf, b.wlen, a.wlen - b.wlen);
-    }
-    return tot;
-  }
-
-  /**
-   * Returns the popcount or cardinality of "a and not b" or "intersection(a, not(b))". Neither set is modified.
-   */
-  public static long andNotCount(BufBitSet a, BufBitSet b) {
-    long tot = BitUtil.pop_andnot(a.buf, b.buf, 0, Math.min(a.wlen, b.wlen));
-    if (a.wlen > b.wlen) {
-      tot += BitUtil.pop_array(a.buf, b.wlen, a.wlen - b.wlen);
-    }
-    return tot;
-  }
-
-  /**
-   * Returns the popcount or cardinality of the exclusive-or of the two sets. Neither set is modified.
-   */
-  public static long xorCount(BufBitSet a, BufBitSet b) {
-    long tot = BitUtil.pop_xor(a.buf, b.buf, 0, Math.min(a.wlen, b.wlen));
-    if (a.wlen < b.wlen) {
-      tot += BitUtil.pop_array(b.buf, a.wlen, b.wlen - a.wlen);
-    } else if (a.wlen > b.wlen) {
-      tot += BitUtil.pop_array(a.buf, b.wlen, a.wlen - b.wlen);
-    }
-    return tot;
-  }
-
-  /**
-   * Returns the index of the first set bit starting at the index specified. -1 is returned if there are no more set
-   * bits.
-   */
-  public int nextSetBit(int index) {
-    int i = index >> 6;
-    if (i >= wlen) return -1;
-    int subIndex = index & 0x3f; // index within the word
-    long word = buf.getLong(i) >> subIndex; // skip all the bits to the right of index
-
-    if (word != 0) {
-      return (i << 6) + subIndex + Long.numberOfTrailingZeros(word);
-    }
-
-    while (++i < wlen) {
-      word = buf.getLong(i);
-      if (word != 0) return (i << 6) + Long.numberOfTrailingZeros(word);
-    }
-
-    return -1;
-  }
-
-  /**
-   * Returns the index of the first set bit starting at the index specified. -1 is returned if there are no more set
-   * bits.
-   */
-  public long nextSetBit(long index) {
-    int i = (int) (index >>> 6);
-    if (i >= wlen) return -1;
-    int subIndex = (int) index & 0x3f; // index within the word
-    long word = buf.getLong(i) >>> subIndex; // skip all the bits to the right of index
-
-    if (word != 0) {
-      return (((long) i) << 6) + (subIndex + Long.numberOfTrailingZeros(word));
-    }
-
-    while (++i < wlen) {
-      word = buf.getLong(i);
-      if (word != 0) return (((long) i) << 6) + Long.numberOfTrailingZeros(word);
-    }
-
-    return -1;
-  }
-
-  /**
-   * Returns the index of the first set bit starting downwards at the index specified. -1 is returned if there are no
-   * more set bits.
-   */
-  public int prevSetBit(int index) {
-    int i = index >> 6;
-    final int subIndex;
-    long word;
-    if (i >= wlen) {
-      i = wlen - 1;
-      if (i < 0) return -1;
-      subIndex = 63; // last possible bit
-      word = buf.getLong(i);
-    } else {
-      if (i < 0) return -1;
-      subIndex = index & 0x3f; // index within the word
-      word = (buf.getLong(i) << (63 - subIndex)); // skip all the bits to the left of index
-    }
-
-    if (word != 0) {
-      return (i << 6) + subIndex - Long.numberOfLeadingZeros(word); // See LUCENE-3197
-    }
-
-    while (--i >= 0) {
-      word = buf.getLong(i);
-      if (word != 0) {
-        return (i << 6) + 63 - Long.numberOfLeadingZeros(word);
-      }
-    }
-
-    return -1;
-  }
-
-  /**
-   * Returns the index of the first set bit starting downwards at the index specified. -1 is returned if there are no
-   * more set bits.
-   */
-  public long prevSetBit(long index) {
-    int i = (int) (index >> 6);
-    final int subIndex;
-    long word;
-    if (i >= wlen) {
-      i = wlen - 1;
-      if (i < 0) return -1;
-      subIndex = 63; // last possible bit
-      word = buf.getLong(i);
-    } else {
-      if (i < 0) return -1;
-      subIndex = (int) index & 0x3f; // index within the word
-      word = (buf.getLong(i) << (63 - subIndex)); // skip all the bits to the left of index
-    }
-
-    if (word != 0) {
-      return (((long) i) << 6) + subIndex - Long.numberOfLeadingZeros(word); // See LUCENE-3197
-    }
-
-    while (--i >= 0) {
-      word = buf.getLong(i);
-      if (word != 0) {
-        return (((long) i) << 6) + 63 - Long.numberOfLeadingZeros(word);
-      }
-    }
-
-    return -1;
-  }
-
-  BufBitSet cloneTest() {
-    BufBitSet obs = new BufBitSet(allocator, buf.copy());
-    return obs;
-  }
-
-  /** this = this AND other */
-  public void intersect(BufBitSet other) {
-    int newLen = Math.min(this.wlen, other.wlen);
-    ByteBuf thisArr = this.buf;
-    ByteBuf otherArr = other.buf;
-    // testing against zero can be more efficient
-    int pos = newLen;
-    while (--pos >= 0) {
-      thisArr.setLong(pos, thisArr.getLong(pos) & otherArr.getLong(pos));
-    }
-    if (this.wlen > newLen) {
-      // fill zeros from the new shorter length to the old length
-      fill(buf, newLen, this.wlen, 0);
-    }
-    this.wlen = newLen;
-  }
-
-  /** this = this OR other */
-  public void union(BufBitSet other) {
-    int newLen = Math.max(wlen, other.wlen);
-    ensureCapacityWords(newLen);
-    assert (numBits = Math.max(other.numBits, numBits)) >= 0;
-
-    ByteBuf thisArr = this.buf;
-    ByteBuf otherArr = other.buf;
-
-    int pos = Math.min(wlen, other.wlen);
-    while (--pos >= 0) {
-      thisArr.setLong(pos, thisArr.getLong(pos) | otherArr.getLong(pos));
-    }
-    if (this.wlen < newLen) {
-      System.arraycopy(otherArr, this.wlen, thisArr, this.wlen, newLen - this.wlen);
-    }
-    this.wlen = newLen;
-  }
-
-  /** Remove all elements set in other. this = this AND_NOT other */
-  public void remove(BufBitSet other) {
-    int idx = Math.min(wlen, other.wlen);
-    ByteBuf thisArr = this.buf;
-    ByteBuf otherArr = other.buf;
-    while (--idx >= 0) {
-      thisArr.setLong(idx, thisArr.getLong(idx) & ~otherArr.getLong(idx));
-    }
-  }
-
-  /** this = this XOR other */
-  public void xor(BufBitSet other) {
-    int newLen = Math.max(wlen, other.wlen);
-    ensureCapacityWords(newLen);
-    assert (numBits = Math.max(other.numBits, numBits)) >= 0;
-
-    ByteBuf thisArr = this.buf;
-    ByteBuf otherArr = other.buf;
-    int pos = Math.min(wlen, other.wlen);
-    while (--pos >= 0) {
-      thisArr.setLong(pos, thisArr.getLong(pos) ^ otherArr.getLong(pos));
-    }
-    if (this.wlen < newLen) {
-      otherArr.readerIndex(wlen);
-      otherArr.writeBytes(thisArr);
-    }
-    this.wlen = newLen;
-
-  }
-
-  // some BitSet compatability methods
-
-  // ** see {@link intersect} */
-  public void and(BufBitSet other) {
-    intersect(other);
-  }
-
-  // ** see {@link union} */
-  public void or(BufBitSet other) {
-    union(other);
-  }
-
-  // ** see {@link andNot} */
-  public void andNot(BufBitSet other) {
-    remove(other);
-  }
-
-  /** returns true if the sets have any elements in common */
-  public boolean intersects(BufBitSet other) {
-    int pos = Math.min(this.wlen, other.wlen);
-    ByteBuf thisArr = this.buf;
-    ByteBuf otherArr = other.buf;
-    while (--pos >= 0) {
-      if ((thisArr.getLong(pos) & otherArr.getLong(pos)) != 0) return true;
-    }
-    return false;
-  }
-
-  public void ensureCapacityWords(int numWords) {
-    if (buf.capacity() < numWords) {
-      ByteBuf newBuf = allocator.buffer(numWords * 8);
-      buf.writeBytes(newBuf);
-      buf.release();
-      buf = newBuf;
-      this.numBits = numWords * 64;
-    }
-  }
-
-  /**
-   * Ensure that the long[] is big enough to hold numBits, expanding it if necessary. getNumWords() is unchanged by this
-   * call.
-   */
-  public void ensureCapacity(long numBits) {
-    ensureCapacityWords(bits2words(numBits));
-  }
-
-  /**
-   * Lowers numWords, the number of words in use, by checking for trailing zero words.
-   */
-  public void trimTrailingZeros() {
-    int idx = wlen - 1;
-    while (idx >= 0 && buf.getLong(idx) == 0)
-      idx--;
-    wlen = idx + 1;
-  }
-
-  /** returns the number of 64 bit words it would take to hold numBits */
-  public static int bits2words(long numBits) {
-    return (int) (((numBits - 1) >>> 6) + 1);
-  }
-
-  /** returns true if both sets have the same bits set */
-  @Override
-  public boolean equals(Object o) {
-    if (this == o) return true;
-    if (!(o instanceof BufBitSet)) return false;
-    BufBitSet a;
-    BufBitSet b = (BufBitSet) o;
-    // make a the larger set.
-    if (b.wlen > this.wlen) {
-      a = b;
-      b = this;
-    } else {
-      a = this;
-    }
-
-    // check for any set bits out of the range of b
-    for (int i = a.wlen - 1; i >= b.wlen; i--) {
-      if (a.buf.getLong(i) != 0) return false;
-    }
-
-    for (int i = b.wlen - 1; i >= 0; i--) {
-      if (a.buf.getLong(i) != b.buf.getLong(i)) return false;
-    }
-
-    return true;
-  }
-
-  @Override
-  public int hashCode() {
-    // Start with a zero hash and use a mix that results in zero if the input is zero.
-    // This effectively truncates trailing zeros without an explicit check.
-    long h = 0;
-    for (int i = buf.capacity(); --i >= 0;) {
-      h ^= buf.getLong(i);
-      h = (h << 1) | (h >>> 63); // rotate left
-    }
-    // fold leftmost bits into right and add a constant to prevent
-    // empty sets from returning 0, which is too common.
-    return (int) ((h >> 32) ^ h) + 0x98761234;
-  }
-
-  public void release() {
-    this.buf.release();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/3d41be41/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/NullValueException.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/NullValueException.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/NullValueException.java
deleted file mode 100644
index 027b698..0000000
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/NullValueException.java
+++ /dev/null
@@ -1,9 +0,0 @@
-package org.apache.drill.exec.record.vector;
-
-import org.apache.drill.common.exceptions.DrillRuntimeException;
-
-public class NullValueException extends DrillRuntimeException {
-  public NullValueException(int index) {
-    super("Element at index position: " + index + " is null");
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/3d41be41/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/NullableFixed8.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/NullableFixed8.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/NullableFixed8.java
deleted file mode 100644
index 48aa0c2..0000000
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/NullableFixed8.java
+++ /dev/null
@@ -1,43 +0,0 @@
-/*******************************************************************************
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- ******************************************************************************/
-package org.apache.drill.exec.record.vector;
-
-import org.apache.drill.exec.memory.BufferAllocator;
-import org.apache.drill.exec.record.MaterializedField;
-
-public final class NullableFixed8 extends NullableValueVector<NullableFixed8, Fixed8>{
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(NullableFixed8.class);
-
-  public NullableFixed8(MaterializedField field, BufferAllocator allocator) {
-    super(field, allocator);
-  }
-
-  @Override
-  protected Fixed8 getNewValueVector(BufferAllocator allocator) {
-    return new Fixed8(this.field, allocator);
-  }
-
-  public long get(int index){
-    return 1l;
-  }
-  
-  public void set(int index, long value){
-    
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/3d41be41/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/SelectionVector.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/SelectionVector.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/SelectionVector.java
deleted file mode 100644
index 3546bd8..0000000
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/SelectionVector.java
+++ /dev/null
@@ -1,49 +0,0 @@
-/*******************************************************************************
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- ******************************************************************************/
-package org.apache.drill.exec.record.vector;
-
-import org.apache.drill.exec.memory.BufferAllocator;
-import org.apache.drill.exec.record.MaterializedField;
-
-/**
- * Convenience/Clarification Fixed2 wrapper.
- */
-public class SelectionVector {
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(SelectionVector.class);
-
-  public SelectionVector(MaterializedField field, BufferAllocator allocator) {
-
-  }
-
-  public int capacity() {
-    return -1;
-  }
-
-  public void allocateNew(int count) {
-
-  }
-<<<<<<< HEAD:sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/SelectionVector.java
-=======
-  
-  public final int getInt(int index){
-    index*=4;
-    return data.getInt(index);
-  }
->>>>>>> Build working:sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/NullableFixed4.java
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/3d41be41/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/SelectionVector2.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/SelectionVector2.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/SelectionVector2.java
deleted file mode 100644
index b808dc6..0000000
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/SelectionVector2.java
+++ /dev/null
@@ -1,49 +0,0 @@
-/*******************************************************************************
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- ******************************************************************************/
-package org.apache.drill.exec.record.vector;
-
-import io.netty.buffer.ByteBuf;
-
-import org.apache.drill.exec.memory.BufferAllocator;
-import org.apache.drill.exec.record.DeadBuf;
-
-/**
- * A selection vector that fronts, at most, a
- */
-public class SelectionVector2{
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(SelectionVector2.class);
-
-  private final BufferAllocator allocator;
-  private ByteBuf buffer = DeadBuf.DEAD_BUFFER;
-
-  public SelectionVector2(BufferAllocator allocator) {
-    this.allocator = allocator;
-  }
-
-  public int getCount(){
-    return -1;
-  }
-
-  public int getIndex(int directIndex){
-    return buffer.getChar(directIndex);
-  }
-
-  public void setIndex(int directIndex, char value){
-    buffer.setChar(directIndex, value);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/3d41be41/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/SelectionVector4.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/SelectionVector4.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/SelectionVector4.java
deleted file mode 100644
index d857146..0000000
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/SelectionVector4.java
+++ /dev/null
@@ -1,41 +0,0 @@
-/*******************************************************************************
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- ******************************************************************************/
-
-package org.apache.drill.exec.record.vector;
-
-import io.netty.buffer.ByteBuf;
-
-import org.apache.drill.exec.memory.BufferAllocator;
-import org.apache.drill.exec.record.DeadBuf;
-
-public class SelectionVector4 {
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(SelectionVector4.class);
-
-  private final BufferAllocator allocator;
-  private ByteBuf buffer = DeadBuf.DEAD_BUFFER;
-
-  public SelectionVector4(BufferAllocator allocator) {
-    this.allocator = allocator;
-  }
-
-  public int getCount(){
-    return -1;
-  }
-
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/3d41be41/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/JSONRecordReader.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/JSONRecordReader.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/JSONRecordReader.java
index 8c31aa4..8513dfe 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/JSONRecordReader.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/JSONRecordReader.java
@@ -22,14 +22,8 @@ import org.apache.drill.exec.memory.BufferAllocator;
 import org.apache.drill.exec.ops.FragmentContext;
 import org.apache.drill.exec.physical.impl.OutputMutator;
 import org.apache.drill.exec.record.MaterializedField;
-import org.apache.drill.exec.record.vector.NullableBit;
-import org.apache.drill.exec.record.vector.NullableFixed4;
-import org.apache.drill.exec.record.vector.NullableVarLen4;
-import org.apache.drill.exec.record.vector.TypeHelper;
-import org.apache.drill.exec.record.vector.ValueVector;
 import org.apache.drill.exec.schema.DiffSchema;
 import org.apache.drill.exec.schema.Field;
-import org.apache.drill.exec.schema.IdGenerator;
 import org.apache.drill.exec.schema.ListSchema;
 import org.apache.drill.exec.schema.NamedField;
 import org.apache.drill.exec.schema.ObjectSchema;
@@ -37,10 +31,15 @@ import org.apache.drill.exec.schema.OrderedField;
 import org.apache.drill.exec.schema.RecordSchema;
 import org.apache.drill.exec.schema.SchemaIdGenerator;
 import org.apache.drill.exec.schema.json.jackson.JacksonHelper;
+import org.apache.drill.exec.vector.AllocationHelper;
+import org.apache.drill.exec.vector.NullableBitVector;
+import org.apache.drill.exec.vector.NullableFloat4Vector;
+import org.apache.drill.exec.vector.NullableIntVector;
+import org.apache.drill.exec.vector.NullableVarChar4Vector;
+import org.apache.drill.exec.vector.TypeHelper;
+import org.apache.drill.exec.vector.ValueVector;
 
 import com.beust.jcommander.internal.Maps;
-import com.carrotsearch.hppc.IntObjectOpenHashMap;
-import com.carrotsearch.hppc.cursors.ObjectCursor;
 import com.fasterxml.jackson.core.JsonFactory;
 import com.fasterxml.jackson.core.JsonParser;
 import com.fasterxml.jackson.core.JsonToken;
@@ -416,8 +415,8 @@ public class JSONRecordReader implements RecordReader {
         if (holder == null) {
             MajorType type = field.getFieldType();
             MaterializedField f = MaterializedField.create(new SchemaPath(field.getFieldName(), ExpressionPosition.UNKNOWN), type);
-            ValueVector<?> v = TypeHelper.getNewVector(f, allocator);
-            v.allocateNew(batchSize);
+            ValueVector v = TypeHelper.getNewVector(f, allocator);
+            AllocationHelper.allocate(v, batchSize, 50);
             holder = new VectorHolder(batchSize, v);
             valueVectorMap.put(field, holder);
             outputMutator.addField(v);

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/3d41be41/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/AllocationHelper.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/AllocationHelper.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/AllocationHelper.java
new file mode 100644
index 0000000..1631e70
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/AllocationHelper.java
@@ -0,0 +1,15 @@
+package org.apache.drill.exec.vector;
+
+public class AllocationHelper {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(AllocationHelper.class);
+  
+  public static void allocate(ValueVector v, int valueCount, int bytesPerValue){
+    if(v instanceof FixedWidthVector){
+      ((FixedWidthVector) v).allocateNew(valueCount);
+    }else if(v instanceof VariableWidthVector){
+      ((VariableWidthVector) v).allocateNew(valueCount * bytesPerValue, valueCount);
+    }else{
+      throw new UnsupportedOperationException();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/3d41be41/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/BaseValueVector.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/BaseValueVector.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/BaseValueVector.java
index a8678f5..e87e132 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/BaseValueVector.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/BaseValueVector.java
@@ -25,8 +25,6 @@ abstract class BaseValueVector implements ValueVector{
   }
   
   abstract class BaseAccessor implements ValueVector.Accessor{
-
-    
     public void reset(){}
   }
   

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/3d41be41/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/BitVector.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/BitVector.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/BitVector.java
index 9d247f5..9a9f438 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/BitVector.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/BitVector.java
@@ -8,6 +8,7 @@ import org.apache.drill.exec.memory.BufferAllocator;
 import org.apache.drill.exec.proto.UserBitShared.FieldMetadata;
 import org.apache.drill.exec.record.DeadBuf;
 import org.apache.drill.exec.record.MaterializedField;
+import org.apache.drill.exec.record.TransferPair;
 /**
  * Bit implements a vector of bit-width values.  Elements in the vector are accessed
  * by position from the logical start of the vector.
@@ -74,6 +75,32 @@ public final class BitVector extends BaseDataValueVector implements FixedWidthVe
     return new Accessor();
   }
   
+  public TransferPair getTransferPair(){
+    return new TransferImpl();
+  }
+  
+  public void transferTo(BitVector target){
+    target.data = data;
+    target.data.retain();
+    target.recordCount = recordCount;
+    clear();
+  }
+  
+  private class TransferImpl implements TransferPair{
+    BitVector to;
+    
+    public TransferImpl(){
+      this.to = new BitVector(getField(), allocator);
+    }
+    
+    public BitVector getTo(){
+      return to;
+    }
+    
+    public void transfer(){
+      transferTo(to);
+    }
+  }
   
   public class Accessor extends BaseAccessor{
 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/3d41be41/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/ValueVector.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/ValueVector.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/ValueVector.java
index 328182b..27089ac 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/ValueVector.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/ValueVector.java
@@ -21,9 +21,9 @@ import io.netty.buffer.ByteBuf;
 
 import java.io.Closeable;
 
-import org.apache.drill.exec.proto.SchemaDefProtos.FieldDef;
 import org.apache.drill.exec.proto.UserBitShared.FieldMetadata;
 import org.apache.drill.exec.record.MaterializedField;
+import org.apache.drill.exec.record.TransferPair;
 
 /**
  * ValueVectorTypes defines a set of template-generated classes which implement type-specific
@@ -50,6 +50,9 @@ public interface ValueVector extends Closeable {
   public void clear();
   
   
+  public TransferPair getTransferPair();
+
+  
   /**
    * Return the underlying buffers associated with this vector. Note that this doesn't impact the
    * reference counts for this buffer so it only should be used for in-context access. Also note

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/3d41be41/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/expr/ExpressionTest.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/expr/ExpressionTest.java b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/expr/ExpressionTest.java
index 007abb3..623af0e 100644
--- a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/expr/ExpressionTest.java
+++ b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/expr/ExpressionTest.java
@@ -21,18 +21,13 @@ import org.apache.drill.common.expression.parser.ExprParser.parse_return;
 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.exec.expr.CodeGenerator.HoldingContainer;
 import org.apache.drill.exec.expr.fn.FunctionImplementationRegistry;
 import org.apache.drill.exec.physical.impl.project.Projector;
 import org.apache.drill.exec.record.RecordBatch;
 import org.apache.drill.exec.record.RecordBatch.TypedFieldId;
-import org.apache.drill.exec.record.vector.Fixed4;
+import org.apache.drill.exec.vector.IntVector;
 import org.junit.After;
 import org.junit.Test;
-import org.slf4j.ILoggerFactory;
-import org.slf4j.LoggerFactory;
-
-import ch.qos.logback.classic.LoggerContext;
 
 public class ExpressionTest {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ExpressionTest.class);
@@ -51,8 +46,8 @@ public class ExpressionTest {
       {
         batch.getValueVectorId(new SchemaPath("alpha", ExpressionPosition.UNKNOWN));
         result = tfid;
-        batch.getValueVectorById(tfid.getFieldId(), Fixed4.class);
-        result = new Fixed4(null, null);
+        batch.getValueVectorById(tfid.getFieldId(), IntVector.class);
+        result = new IntVector(null, null);
       }
 
     };

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/3d41be41/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/SimpleRootExec.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/SimpleRootExec.java b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/SimpleRootExec.java
index 093c58f..d125ec0 100644
--- a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/SimpleRootExec.java
+++ b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/SimpleRootExec.java
@@ -5,7 +5,7 @@ import org.apache.drill.exec.physical.impl.ScreenCreator.ScreenRoot;
 import org.apache.drill.exec.record.RecordBatch;
 import org.apache.drill.exec.record.RecordBatch.IterOutcome;
 import org.apache.drill.exec.record.RecordBatch.TypedFieldId;
-import org.apache.drill.exec.record.vector.ValueVector;
+import org.apache.drill.exec.vector.ValueVector;
 
 public class SimpleRootExec implements RootExec{
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(SimpleRootExec.class);
@@ -22,7 +22,7 @@ public class SimpleRootExec implements RootExec{
   }
 
 
-  public <T extends ValueVector<T>> T getValueVectorById(SchemaPath path, Class<?> vvClass){
+  public <T extends ValueVector> T getValueVectorById(SchemaPath path, Class<?> vvClass){
     TypedFieldId tfid = incoming.getValueVectorId(path);
     return incoming.getValueVectorById(tfid.getFieldId(), vvClass);
   }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/3d41be41/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 3dc961b..7b002ea 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
@@ -17,7 +17,7 @@
  ******************************************************************************/
 package org.apache.drill.exec.physical.impl;
 
-import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.*;
 
 import java.util.List;
 
@@ -29,9 +29,9 @@ import org.apache.drill.exec.record.RecordBatchLoader;
 import org.apache.drill.exec.rpc.user.QueryResultBatch;
 import org.apache.drill.exec.server.Drillbit;
 import org.apache.drill.exec.server.RemoteServiceSet;
+import org.apache.drill.exec.vector.ValueVector;
 import org.junit.Test;
 
-import com.carrotsearch.hppc.cursors.IntObjectCursor;
 import com.google.common.base.Charsets;
 import com.google.common.io.Files;
 
@@ -57,42 +57,40 @@ public class TestSimpleFragmentRun extends PopUnitTestBase {
       boolean schemaChanged = batchLoader.load(batch.getHeader().getDef(), batch.getData());
       boolean firstColumn = true;
 
-          // print headers.
-          if (schemaChanged) {
-            System.out.println("\n\n========NEW SCHEMA=========\n\n");
-            for (ValueVector<?> value : batchLoader) {
+      // print headers.
+      if (schemaChanged) {
+        System.out.println("\n\n========NEW SCHEMA=========\n\n");
+        for (ValueVector value : batchLoader) {
 
-              if (firstColumn) {
-                firstColumn = false;
-              } else {
-                System.out.print("\t");
-              }
-              System.out.print(value.getField().getName());
-              System.out.print("[");
-              System.out.print(value.getField().getType().getMinorType());
-              System.out.print("]");
-            }
-            System.out.println();
+          if (firstColumn) {
+            firstColumn = false;
+          } else {
+            System.out.print("\t");
           }
+          System.out.print(value.getField().getName());
+          System.out.print("[");
+          System.out.print(value.getField().getType().getMinorType());
+          System.out.print("]");
+        }
+        System.out.println();
+      }
 
 
-          for (int i = 0; i < batchLoader.getRecordCount(); i++) {
-            boolean first = true;
-            recordCount++;
-            for (ValueVector<?> value : batchLoader) {
-              if (first) {
-                first = false;
-              } else {
-                System.out.print("\t");
-              }
-              System.out.print(value.getObject(i));
-            }
-            if(!first) System.out.println();
+      for (int i = 0; i < batchLoader.getRecordCount(); i++) {
+        boolean first = true;
+        recordCount++;
+        for (ValueVector value : batchLoader) {
+          if (first) {
+            first = false;
+          } else {
+            System.out.print("\t");
           }
-          System.out.print(v.value.getAccessor().getObject(i));
+          System.out.print(value.getAccessor().getObject(i));
         }
         if(!first) System.out.println();
       }
+    
+  
 
     }
     logger.debug("Received results {}", results);

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/3d41be41/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/project/TestSimpleProjection.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/project/TestSimpleProjection.java b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/project/TestSimpleProjection.java
index 4144a54..e3f03f2 100644
--- a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/project/TestSimpleProjection.java
+++ b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/project/TestSimpleProjection.java
@@ -18,9 +18,9 @@ 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.FragmentHandle;
-import org.apache.drill.exec.record.vector.Fixed8;
 import org.apache.drill.exec.rpc.user.UserServer.UserClientConnection;
 import org.apache.drill.exec.server.DrillbitContext;
+import org.apache.drill.exec.vector.BigIntVector;
 import org.junit.After;
 import org.junit.Test;
 
@@ -50,13 +50,18 @@ public class TestSimpleProjection {
     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()){
-      Fixed8 c1 = exec.getValueVectorById(new SchemaPath("col1", ExpressionPosition.UNKNOWN), Fixed8.class);
-      Fixed8 c2 = exec.getValueVectorById(new SchemaPath("col2", ExpressionPosition.UNKNOWN), Fixed8.class);
+      BigIntVector c1 = exec.getValueVectorById(new SchemaPath("col1", ExpressionPosition.UNKNOWN), BigIntVector.class);
+      BigIntVector c2 = exec.getValueVectorById(new SchemaPath("col2", ExpressionPosition.UNKNOWN), BigIntVector.class);
       int x = 0;
-      for(int i =0; i < c1.getRecordCount(); i++){
-        assertEquals(c1.get(i)+1, c2.get(i));
-        x += c1.get(i);
+      BigIntVector.Accessor a1, a2;
+      a1 = c1.getAccessor();
+      a2 = c2.getAccessor();
+      
+      for(int i =0; i < c1.getAccessor().getRecordCount(); i++){
+        assertEquals(a1.get(i)+1, a2.get(i));
+        x += a1.get(i);
       }
+      
       System.out.println(x);
     }
   }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/3d41be41/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/record/vector/TestOpenBitSet.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/record/vector/TestOpenBitSet.java b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/record/vector/TestOpenBitSet.java
deleted file mode 100644
index 66f69de..0000000
--- a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/record/vector/TestOpenBitSet.java
+++ /dev/null
@@ -1,361 +0,0 @@
-/*******************************************************************************
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- ******************************************************************************/
-package org.apache.drill.exec.record.vector;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-import io.netty.buffer.ByteBufAllocator;
-import io.netty.buffer.UnpooledByteBufAllocator;
-
-import java.util.BitSet;
-import java.util.Random;
-
-import org.junit.Ignore;
-import org.junit.Test;
-
-@Ignore
-public class TestOpenBitSet {
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestOpenBitSet.class);
-
-  Random random = new Random();
-  ByteBufAllocator allocator = UnpooledByteBufAllocator.DEFAULT;
-  
-  public int atLeast(int val){
-    return val + random.nextInt(val);
-  }
-  
-  
-  public Random random() {
-    return new Random();
-  }
-
-  void doGet(BitSet a, BufBitSet b) {
-    int max = a.size();
-    for (int i = 0; i < max; i++) {
-      if (a.get(i) != b.get(i)) {
-        fail("mismatch: BitSet=[" + i + "]=" + a.get(i));
-      }
-      if (a.get(i) != b.get((long) i)) {
-        fail("mismatch: BitSet=[" + i + "]=" + a.get(i));
-      }
-    }
-  }
-
-  void doGetFast(BitSet a, BufBitSet b, int max) {
-    for (int i = 0; i < max; i++) {
-      if (a.get(i) != b.fastGet(i)) {
-        fail("mismatch: BitSet=[" + i + "]=" + a.get(i));
-      }
-      if (a.get(i) != b.fastGet((long) i)) {
-        fail("mismatch: BitSet=[" + i + "]=" + a.get(i));
-      }
-    }
-  }
-
-  void doNextSetBit(BitSet a, BufBitSet b) {
-    int aa = -1, bb = -1;
-    do {
-      aa = a.nextSetBit(aa + 1);
-      bb = b.nextSetBit(bb + 1);
-      assertEquals(aa, bb);
-    } while (aa >= 0);
-  }
-
-  void doNextSetBitLong(BitSet a, BufBitSet b) {
-    int aa = -1, bb = -1;
-    do {
-      aa = a.nextSetBit(aa + 1);
-      bb = (int) b.nextSetBit((long) (bb + 1));
-      assertEquals(aa, bb);
-    } while (aa >= 0);
-  }
-
-  void doPrevSetBit(BitSet a, BufBitSet b) {
-    int aa = a.size() + random().nextInt(100);
-    int bb = aa;
-    do {
-      // aa = a.prevSetBit(aa-1);
-      aa--;
-      while ((aa >= 0) && (!a.get(aa))) {
-        aa--;
-      }
-      bb = b.prevSetBit(bb - 1);
-      assertEquals(aa, bb);
-    } while (aa >= 0);
-  }
-
-  void doPrevSetBitLong(BitSet a, BufBitSet b) {
-    int aa = a.size() + random().nextInt(100);
-    int bb = aa;
-    do {
-      // aa = a.prevSetBit(aa-1);
-      aa--;
-      while ((aa >= 0) && (!a.get(aa))) {
-        aa--;
-      }
-      bb = (int) b.prevSetBit((long) (bb - 1));
-      assertEquals(aa, bb);
-    } while (aa >= 0);
-  }
-
-  // test interleaving different OpenBitSetIterator.next()/skipTo()
-  void doIterate(BitSet a, BufBitSet b, int mode) {
-    // if (mode == 1) doIterate1(a, b);
-    // if (mode == 2) doIterate2(a, b);
-  }
-
-  //
-  // void doIterate1(BitSet a, OpenBitSet b) {
-  // int aa = -1, bb = -1;
-  // OpenBitSetIterator iterator = new OpenBitSetIterator(b);
-  // do {
-  // aa = a.nextSetBit(aa + 1);
-  // bb = random().nextBoolean() ? iterator.nextDoc() : iterator.advance(bb + 1);
-  // assertEquals(aa == -1 ? DocIdSetIterator.NO_MORE_DOCS : aa, bb);
-  // } while (aa >= 0);
-  // }
-  //
-  // void doIterate2(BitSet a, OpenBitSet b) {
-  // int aa = -1, bb = -1;
-  // OpenBitSetIterator iterator = new OpenBitSetIterator(b);
-  // do {
-  // aa = a.nextSetBit(aa + 1);
-  // bb = random().nextBoolean() ? iterator.nextDoc() : iterator.advance(bb + 1);
-  // assertEquals(aa == -1 ? DocIdSetIterator.NO_MORE_DOCS : aa, bb);
-  // } while (aa >= 0);
-  // }
-
-  void doRandomSets(int maxSize, int iter, int mode) {
-    BitSet a0 = null;
-    BufBitSet b0 = null;
-
-    for (int i = 0; i < iter; i++) {
-      int sz = random().nextInt(maxSize);
-      BitSet a = new BitSet(sz);
-      BufBitSet b = new BufBitSet(sz, allocator);
-
-      // test the various ways of setting bits
-      if (sz > 0) {
-        int nOper = random().nextInt(sz);
-        for (int j = 0; j < nOper; j++) {
-          int idx;
-
-          idx = random().nextInt(sz);
-          a.set(idx);
-          b.fastSet(idx);
-
-          idx = random().nextInt(sz);
-          a.set(idx);
-          b.fastSet((long) idx);
-
-          idx = random().nextInt(sz);
-          a.clear(idx);
-          b.fastClear(idx);
-
-          idx = random().nextInt(sz);
-          a.clear(idx);
-          b.fastClear((long) idx);
-
-          idx = random().nextInt(sz);
-          a.flip(idx);
-          b.fastFlip(idx);
-
-          boolean val = b.flipAndGet(idx);
-          boolean val2 = b.flipAndGet(idx);
-          assertTrue(val != val2);
-
-          idx = random().nextInt(sz);
-          a.flip(idx);
-          b.fastFlip((long) idx);
-
-          val = b.flipAndGet((long) idx);
-          val2 = b.flipAndGet((long) idx);
-          assertTrue(val != val2);
-
-          val = b.getAndSet(idx);
-          assertTrue(val2 == val);
-          assertTrue(b.get(idx));
-
-          if (!val) b.fastClear(idx);
-          assertTrue(b.get(idx) == val);
-        }
-      }
-
-      // test that the various ways of accessing the bits are equivalent
-      doGet(a, b);
-      doGetFast(a, b, sz);
-
-      // test ranges, including possible extension
-      int fromIndex, toIndex;
-      fromIndex = random().nextInt(sz + 80);
-      toIndex = fromIndex + random().nextInt((sz >> 1) + 1);
-      BitSet aa = (BitSet) a.clone();
-      aa.flip(fromIndex, toIndex);
-      BufBitSet bb = b.cloneTest();
-      bb.flip(fromIndex, toIndex);
-
-      doIterate(aa, bb, mode); // a problem here is from flip or doIterate
-
-      fromIndex = random().nextInt(sz + 80);
-      toIndex = fromIndex + random().nextInt((sz >> 1) + 1);
-      aa = (BitSet) a.clone();
-      aa.clear(fromIndex, toIndex);
-      bb = b.cloneTest();
-      bb.clear(fromIndex, toIndex);
-
-      doNextSetBit(aa, bb); // a problem here is from clear() or nextSetBit
-      doNextSetBitLong(aa, bb);
-
-      doPrevSetBit(aa, bb);
-      doPrevSetBitLong(aa, bb);
-
-      fromIndex = random().nextInt(sz + 80);
-      toIndex = fromIndex + random().nextInt((sz >> 1) + 1);
-      aa = (BitSet) a.clone();
-      aa.set(fromIndex, toIndex);
-      bb = b.cloneTest();
-      bb.set(fromIndex, toIndex);
-
-      doNextSetBit(aa, bb); // a problem here is from set() or nextSetBit
-      doNextSetBitLong(aa, bb);
-
-      doPrevSetBit(aa, bb);
-      doPrevSetBitLong(aa, bb);
-
-      if (a0 != null) {
-        assertEquals(a.equals(a0), b.equals(b0));
-
-        assertEquals(a.cardinality(), b.cardinality());
-
-        BitSet a_and = (BitSet) a.clone();
-        a_and.and(a0);
-        BitSet a_or = (BitSet) a.clone();
-        a_or.or(a0);
-        BitSet a_xor = (BitSet) a.clone();
-        a_xor.xor(a0);
-        BitSet a_andn = (BitSet) a.clone();
-        a_andn.andNot(a0);
-
-        BufBitSet b_and = b.cloneTest();
-        assertEquals(b, b_and);
-        b_and.and(b0);
-        BufBitSet b_or = b.cloneTest();
-        b_or.or(b0);
-        BufBitSet b_xor = b.cloneTest();
-        b_xor.xor(b0);
-        BufBitSet b_andn = b.cloneTest();
-        b_andn.andNot(b0);
-
-        doIterate(a_and, b_and, mode);
-        doIterate(a_or, b_or, mode);
-        doIterate(a_xor, b_xor, mode);
-        doIterate(a_andn, b_andn, mode);
-
-        assertEquals(a_and.cardinality(), b_and.cardinality());
-        assertEquals(a_or.cardinality(), b_or.cardinality());
-        assertEquals(a_xor.cardinality(), b_xor.cardinality());
-        assertEquals(a_andn.cardinality(), b_andn.cardinality());
-
-        // test non-mutating popcounts
-        assertEquals(b_and.cardinality(), BufBitSet.intersectionCount(b, b0));
-        assertEquals(b_or.cardinality(), BufBitSet.unionCount(b, b0));
-        assertEquals(b_xor.cardinality(), BufBitSet.xorCount(b, b0));
-        assertEquals(b_andn.cardinality(), BufBitSet.andNotCount(b, b0));
-      }
-
-      a0 = a;
-      b0 = b;
-    }
-  }
-
-  // large enough to flush obvious bugs, small enough to run in <.5 sec as part of a
-  // larger testsuite.
-  @Test
-  public void testSmall() {
-    doRandomSets(atLeast(1200), atLeast(1000), 1);
-    doRandomSets(atLeast(1200), atLeast(1000), 2);
-  }
-
-  // uncomment to run a bigger test (~2 minutes).
-  /*
-   * public void testBig() { doRandomSets(2000,200000, 1); doRandomSets(2000,200000, 2); }
-   */
-
-  @Test
-  public void testEquals() {
-    BufBitSet b1 = new BufBitSet(1111, allocator);
-    BufBitSet b2 = new BufBitSet(2222, allocator);
-    assertTrue(b1.equals(b2));
-    assertTrue(b2.equals(b1));
-    b1.set(10);
-    assertFalse(b1.equals(b2));
-    assertFalse(b2.equals(b1));
-    b2.set(10);
-    assertTrue(b1.equals(b2));
-    assertTrue(b2.equals(b1));
-    b2.set(2221);
-    assertFalse(b1.equals(b2));
-    assertFalse(b2.equals(b1));
-    b1.set(2221);
-    assertTrue(b1.equals(b2));
-    assertTrue(b2.equals(b1));
-
-    // try different type of object
-    assertFalse(b1.equals(new Object()));
-  }
-
-  @Test
-  public void testHashCodeEquals() {
-    BufBitSet bs1 = new BufBitSet(200, allocator);
-    BufBitSet bs2 = new BufBitSet(64, allocator);
-    bs1.set(3);
-    bs2.set(3);
-    assertEquals(bs1, bs2);
-    assertEquals(bs1.hashCode(), bs2.hashCode());
-  }
-
-  private BufBitSet makeOpenBitSet(int[] a) {
-    BufBitSet bs = new BufBitSet(64, allocator);
-    for (int e : a) {
-      bs.set(e);
-    }
-    return bs;
-  }
-
-  private BitSet makeBitSet(int[] a) {
-    BitSet bs = new BitSet();
-    for (int e : a) {
-      bs.set(e);
-    }
-    return bs;
-  }
-
-  private void checkPrevSetBitArray(int[] a) {
-    BufBitSet obs = makeOpenBitSet(a);
-    BitSet bs = makeBitSet(a);
-    doPrevSetBit(bs, obs);
-  }
-
-  public void testPrevSetBit() {
-    checkPrevSetBitArray(new int[] {});
-    checkPrevSetBitArray(new int[] { 0 });
-    checkPrevSetBitArray(new int[] { 0, 2 });
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/3d41be41/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/record/vector/TestValueVector.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/record/vector/TestValueVector.java b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/record/vector/TestValueVector.java
index 6a1f3ad..0fed756 100644
--- a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/record/vector/TestValueVector.java
+++ b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/record/vector/TestValueVector.java
@@ -4,8 +4,11 @@ import static org.junit.Assert.assertEquals;
 
 import java.nio.charset.Charset;
 
+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.exec.memory.DirectBufferAllocator;
-import org.apache.drill.exec.proto.SchemaDefProtos;
+import org.apache.drill.exec.proto.SchemaDefProtos.FieldDef;
 import org.apache.drill.exec.record.MaterializedField;
 import org.apache.drill.exec.vector.BitVector;
 import org.apache.drill.exec.vector.NullableFloat4Vector;
@@ -22,15 +25,13 @@ public class TestValueVector {
   @Test
   public void testFixedType() {
     // Build a required uint field definition
-    SchemaDefProtos.MajorType.Builder typeBuilder = SchemaDefProtos.MajorType.newBuilder();
-    SchemaDefProtos.FieldDef.Builder defBuilder = SchemaDefProtos.FieldDef.newBuilder();
+    MajorType.Builder typeBuilder = MajorType.newBuilder();
+    FieldDef.Builder defBuilder = FieldDef.newBuilder();
     typeBuilder
-        .setMinorType(SchemaDefProtos.MinorType.UINT4)
-        .setMode(SchemaDefProtos.DataMode.REQUIRED)
+        .setMinorType(MinorType.UINT4)
+        .setMode(DataMode.REQUIRED)
         .setWidth(4);
     defBuilder
-        .setFieldId(1)
-        .setParentId(0)
         .setMajorType(typeBuilder.build());
         MaterializedField field = MaterializedField.create(defBuilder.build());
 
@@ -58,15 +59,13 @@ public class TestValueVector {
   @Test
   public void testNullableVarLen2() {
     // Build an optional varchar field definition
-    SchemaDefProtos.MajorType.Builder typeBuilder = SchemaDefProtos.MajorType.newBuilder();
-    SchemaDefProtos.FieldDef.Builder defBuilder = SchemaDefProtos.FieldDef.newBuilder();
+    MajorType.Builder typeBuilder = MajorType.newBuilder();
+    FieldDef.Builder defBuilder = FieldDef.newBuilder();
     typeBuilder
-        .setMinorType(SchemaDefProtos.MinorType.VARCHAR2)
-        .setMode(SchemaDefProtos.DataMode.OPTIONAL)
+        .setMinorType(MinorType.VARCHAR2)
+        .setMode(DataMode.OPTIONAL)
         .setWidth(2);
     defBuilder
-        .setFieldId(1)
-        .setParentId(0)
         .setMajorType(typeBuilder.build());
     MaterializedField field = MaterializedField.create(defBuilder.build());
 
@@ -106,15 +105,13 @@ public class TestValueVector {
   @Test
   public void testNullableFixedType() {
     // Build an optional uint field definition
-    SchemaDefProtos.MajorType.Builder typeBuilder = SchemaDefProtos.MajorType.newBuilder();
-    SchemaDefProtos.FieldDef.Builder defBuilder = SchemaDefProtos.FieldDef.newBuilder();
+    MajorType.Builder typeBuilder = MajorType.newBuilder();
+    FieldDef.Builder defBuilder = FieldDef.newBuilder();
     typeBuilder
-        .setMinorType(SchemaDefProtos.MinorType.UINT4)
-        .setMode(SchemaDefProtos.DataMode.OPTIONAL)
+        .setMinorType(MinorType.UINT4)
+        .setMode(DataMode.OPTIONAL)
         .setWidth(4);
     defBuilder
-        .setFieldId(1)
-        .setParentId(0)
         .setMajorType(typeBuilder.build());
     MaterializedField field = MaterializedField.create(defBuilder.build());
 
@@ -195,15 +192,13 @@ public class TestValueVector {
   @Test
   public void testNullableFloat() {
     // Build an optional float field definition
-    SchemaDefProtos.MajorType.Builder typeBuilder = SchemaDefProtos.MajorType.newBuilder();
-    SchemaDefProtos.FieldDef.Builder defBuilder = SchemaDefProtos.FieldDef.newBuilder();
+    MajorType.Builder typeBuilder = MajorType.newBuilder();
+    FieldDef.Builder defBuilder = FieldDef.newBuilder();
     typeBuilder
-        .setMinorType(SchemaDefProtos.MinorType.FLOAT4)
-        .setMode(SchemaDefProtos.DataMode.OPTIONAL)
+        .setMinorType(MinorType.FLOAT4)
+        .setMode(DataMode.OPTIONAL)
         .setWidth(4);
     defBuilder
-        .setFieldId(1)
-        .setParentId(0)
         .setMajorType(typeBuilder.build());
     MaterializedField field = MaterializedField.create(defBuilder.build());
 
@@ -256,15 +251,13 @@ public class TestValueVector {
   @Test
   public void testBitVector() {
     // Build a required boolean field definition
-    SchemaDefProtos.MajorType.Builder typeBuilder = SchemaDefProtos.MajorType.newBuilder();
-    SchemaDefProtos.FieldDef.Builder defBuilder = SchemaDefProtos.FieldDef.newBuilder();
+    MajorType.Builder typeBuilder = MajorType.newBuilder();
+    FieldDef.Builder defBuilder = FieldDef.newBuilder();
     typeBuilder
-        .setMinorType(SchemaDefProtos.MinorType.BOOLEAN)
-        .setMode(SchemaDefProtos.DataMode.REQUIRED)
+        .setMinorType(MinorType.BOOLEAN)
+        .setMode(DataMode.REQUIRED)
         .setWidth(4);
     defBuilder
-        .setFieldId(1)
-        .setParentId(0)
         .setMajorType(typeBuilder.build());
     MaterializedField field = MaterializedField.create(defBuilder.build());
 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/3d41be41/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/JSONRecordReaderTest.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/JSONRecordReaderTest.java b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/JSONRecordReaderTest.java
index d8966ae..7c9e8f4 100644
--- a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/JSONRecordReaderTest.java
+++ b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/JSONRecordReaderTest.java
@@ -19,6 +19,7 @@ import org.apache.drill.exec.ops.FragmentContext;
 import org.apache.drill.exec.physical.impl.OutputMutator;
 import org.apache.drill.exec.proto.SchemaDefProtos;
 import org.apache.drill.exec.proto.UserBitShared;
+import org.apache.drill.exec.record.MaterializedField;
 import org.apache.drill.exec.vector.ValueVector;
 import org.junit.Ignore;
 import org.junit.Test;
@@ -34,7 +35,7 @@ public class JSONRecordReaderTest {
 
   class MockOutputMutator implements OutputMutator {
     List<MaterializedField> removedFields = Lists.newArrayList();
-    List<ValueVector<?>> addFields = Lists.newArrayList();
+    List<ValueVector> addFields = Lists.newArrayList();
 
     @Override
     public void removeField(MaterializedField field) throws SchemaChangeException {
@@ -42,7 +43,7 @@ public class JSONRecordReaderTest {
     }
 
     @Override
-    public void addField(ValueVector<?> vector) throws SchemaChangeException {
+    public void addField(ValueVector vector) throws SchemaChangeException {
       addFields.add(vector);
     }
 
@@ -54,7 +55,7 @@ public class JSONRecordReaderTest {
       return removedFields;
     }
 
-    List<ValueVector<?>> getAddFields() {
+    List<ValueVector> getAddFields() {
       return addFields;
     }
   }
@@ -69,7 +70,7 @@ public class JSONRecordReaderTest {
       return;
     }
 
-    T val = (T) valueVector.getObject(index);
+    T val = (T) valueVector.getAccessor().getObject(index);
     if (val instanceof byte[]) {
       assertTrue(Arrays.equals((byte[]) value, (byte[]) val));
     } else {
@@ -89,15 +90,15 @@ public class JSONRecordReaderTest {
     JSONRecordReader jr = new JSONRecordReader(context, getResource("scan_json_test_1.json"));
 
     MockOutputMutator mutator = new MockOutputMutator();
-    List<ValueVector<?>> addFields = mutator.getAddFields();
+    List<ValueVector> addFields = mutator.getAddFields();
     jr.setup(mutator);
     assertEquals(2, jr.next());
     assertEquals(3, addFields.size());
     assertField(addFields.get(0), 0, MinorType.INT, 123, "test");
-    assertField(addFields.get(1), 0, MinorType.BOOLEAN, 1, "b");
+    assertField(addFields.get(1), 0, MinorType.BOOLEAN, true, "b");
     assertField(addFields.get(2), 0, MinorType.VARCHAR4, "hi!".getBytes(UTF_8), "c");
     assertField(addFields.get(0), 1, MinorType.INT, 1234, "test");
-    assertField(addFields.get(1), 1, MinorType.BOOLEAN, 0, "b");
+    assertField(addFields.get(1), 1, MinorType.BOOLEAN, false, "b");
     assertField(addFields.get(2), 1, MinorType.VARCHAR4, "drill!".getBytes(UTF_8), "c");
 
     assertEquals(0, jr.next());
@@ -116,7 +117,7 @@ public class JSONRecordReaderTest {
 
     JSONRecordReader jr = new JSONRecordReader(context, getResource("scan_json_test_2.json"));
     MockOutputMutator mutator = new MockOutputMutator();
-    List<ValueVector<?>> addFields = mutator.getAddFields();
+    List<ValueVector> addFields = mutator.getAddFields();
 
     jr.setup(mutator);
     assertEquals(3, jr.next());
@@ -124,25 +125,25 @@ public class JSONRecordReaderTest {
     assertField(addFields.get(0), 0, MinorType.INT, 123, "test");
     assertField(addFields.get(1), 0, MinorType.INT, 1, "b");
     assertField(addFields.get(2), 0, MinorType.FLOAT4, (float) 2.15, "c");
-    assertField(addFields.get(3), 0, MinorType.BOOLEAN, 1, "bool");
+    assertField(addFields.get(3), 0, MinorType.BOOLEAN, true, "bool");
     assertField(addFields.get(4), 0, MinorType.VARCHAR4, "test1".getBytes(UTF_8), "str1");
 
     assertField(addFields.get(0), 1, MinorType.INT, 1234, "test");
     assertField(addFields.get(1), 1, MinorType.INT, 3, "b");
-    assertField(addFields.get(3), 1, MinorType.BOOLEAN, 0, "bool");
+    assertField(addFields.get(3), 1, MinorType.BOOLEAN, false, "bool");
     assertField(addFields.get(4), 1, MinorType.VARCHAR4, "test2".getBytes(UTF_8), "str1");
     assertField(addFields.get(5), 1, MinorType.INT, 4, "d");
 
     assertField(addFields.get(0), 2, MinorType.INT, 12345, "test");
     assertField(addFields.get(2), 2, MinorType.FLOAT4, (float) 5.16, "c");
-    assertField(addFields.get(3), 2, MinorType.BOOLEAN, 1, "bool");
+    assertField(addFields.get(3), 2, MinorType.BOOLEAN, true, "bool");
     assertField(addFields.get(5), 2, MinorType.INT, 6, "d");
     assertField(addFields.get(6), 2, MinorType.VARCHAR4, "test3".getBytes(UTF_8), "str2");
     assertTrue(mutator.getRemovedFields().isEmpty());
     assertEquals(0, jr.next());
   }
 
-  @Test
+  @Test @Ignore
   public void testChangedSchemaInTwoBatches(@Injectable final FragmentContext context) throws IOException,
       ExecutionSetupException {
     new Expectations() {
@@ -155,7 +156,7 @@ public class JSONRecordReaderTest {
     JSONRecordReader jr = new JSONRecordReader(context, getResource("scan_json_test_2.json"), 64); // batch only fits 1
                                                                                                    // int
     MockOutputMutator mutator = new MockOutputMutator();
-    List<ValueVector<?>> addFields = mutator.getAddFields();
+    List<ValueVector> addFields = mutator.getAddFields();
     List<MaterializedField> removedFields = mutator.getRemovedFields();
 
     jr.setup(mutator);
@@ -164,14 +165,14 @@ public class JSONRecordReaderTest {
     assertField(addFields.get(0), 0, MinorType.INT, 123, "test");
     assertField(addFields.get(1), 0, MinorType.INT, 1, "b");
     assertField(addFields.get(2), 0, MinorType.FLOAT4, (float) 2.15, "c");
-    assertField(addFields.get(3), 0, MinorType.BOOLEAN, 1, "bool");
+    assertField(addFields.get(3), 0, MinorType.BOOLEAN, true, "bool");
     assertField(addFields.get(4), 0, MinorType.VARCHAR4, "test1".getBytes(UTF_8), "str1");
     assertTrue(removedFields.isEmpty());
     assertEquals(1, jr.next());
     assertEquals(6, addFields.size());
     assertField(addFields.get(0), 0, MinorType.INT, 1234, "test");
     assertField(addFields.get(1), 0, MinorType.INT, 3, "b");
-    assertField(addFields.get(3), 0, MinorType.BOOLEAN, 0, "bool");
+    assertField(addFields.get(3), 0, MinorType.BOOLEAN, false, "bool");
     assertField(addFields.get(4), 0, MinorType.VARCHAR4, "test2".getBytes(UTF_8), "str1");
     assertField(addFields.get(5), 0, MinorType.INT, 4, "d");
     assertEquals(1, removedFields.size());
@@ -180,7 +181,7 @@ public class JSONRecordReaderTest {
     assertEquals(1, jr.next());
     assertEquals(8, addFields.size()); // The reappearing of field 'c' is also included
     assertField(addFields.get(0), 0, MinorType.INT, 12345, "test");
-    assertField(addFields.get(3), 0, MinorType.BOOLEAN, 1, "bool");
+    assertField(addFields.get(3), 0, MinorType.BOOLEAN, true, "bool");
     assertField(addFields.get(5), 0, MinorType.INT, 6, "d");
     assertField(addFields.get(6), 0, MinorType.FLOAT4, (float) 5.16, "c");
     assertField(addFields.get(7), 0, MinorType.VARCHAR4, "test3".getBytes(UTF_8), "str2");
@@ -190,7 +191,7 @@ public class JSONRecordReaderTest {
     assertEquals(0, jr.next());
   }
 
-  @Test
+  @Test @Ignore
   public void testNestedFieldInSameBatch(@Injectable final FragmentContext context) throws ExecutionSetupException {
     new Expectations() {
       {
@@ -202,7 +203,7 @@ public class JSONRecordReaderTest {
     JSONRecordReader jr = new JSONRecordReader(context, getResource("scan_json_test_3.json"));
 
     MockOutputMutator mutator = new MockOutputMutator();
-    List<ValueVector<?>> addFields = mutator.getAddFields();
+    List<ValueVector> addFields = mutator.getAddFields();
     jr.setup(mutator);
     assertEquals(2, jr.next());
     assertEquals(5, addFields.size());
@@ -210,12 +211,12 @@ public class JSONRecordReaderTest {
     assertField(addFields.get(1), 0, MinorType.MAP, null, "a");
     assertField(addFields.get(2), 0, MinorType.VARCHAR4, "test".getBytes(UTF_8), "b");
     assertField(addFields.get(3), 0, MinorType.MAP, null, "a");
-    assertField(addFields.get(4), 0, MinorType.BOOLEAN, 1, "d");
+    assertField(addFields.get(4), 0, MinorType.BOOLEAN, true, "d");
     assertField(addFields.get(0), 1, MinorType.INT, 1234, "test");
     assertField(addFields.get(1), 1, MinorType.MAP, null, "a");
     assertField(addFields.get(2), 1, MinorType.VARCHAR4, "test2".getBytes(UTF_8), "b");
     assertField(addFields.get(3), 1, MinorType.MAP, null, "a");
-    assertField(addFields.get(4), 1, MinorType.BOOLEAN, 0, "d");
+    assertField(addFields.get(4), 1, MinorType.BOOLEAN, true, "d");
 
     assertEquals(0, jr.next());
     assertTrue(mutator.getRemovedFields().isEmpty());


[24/53] [abbrv] git commit: Various test cleanups: Rename tests so that they are picked up by Surefire. Fix breaking tests, including sql parser. Remove old, unused classes. Update sqlparser/pom.xml to use ${project.version} instead of specific versio

Posted by ja...@apache.org.
Various test cleanups:  Rename tests so that they are picked up by Surefire. Fix breaking tests, including sql parser.  Remove old, unused classes.  Update sqlparser/pom.xml to use ${project.version} instead of specific version.  Add test-jar generation to pom for common and exec/ref


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

Branch: refs/heads/master
Commit: ea52111117ce924c30512628eefdc17b5071ed3c
Parents: 87ab072
Author: Jacques Nadeau <ja...@apache.org>
Authored: Wed May 29 13:26:35 2013 -0700
Committer: Jacques Nadeau <ja...@apache.org>
Committed: Wed May 29 13:26:35 2013 -0700

----------------------------------------------------------------------
 sandbox/prototype/common/pom.xml                | 155 +++++++++----------
 .../drill/exec/server/BootStrapContext.java     |   2 +-
 .../exec/physical/config/ParsePhysicalPlan.java |  42 -----
 .../physical/config/TestParsePhysicalPlan.java  |  45 ++++++
 .../physical/impl/DistributedFragmentRun.java   |  58 -------
 .../exec/physical/impl/SimpleFragmentRun.java   | 100 ------------
 .../impl/TestDistributedFragmentRun.java        |  58 +++++++
 .../physical/impl/TestSimpleFragmentRun.java    | 100 ++++++++++++
 .../apache/drill/exec/pop/CheckFragmenter.java  |  71 ---------
 .../drill/exec/pop/CheckInjectionValue.java     |  61 --------
 .../apache/drill/exec/pop/FragmentChecker.java  |  85 ----------
 .../drill/exec/pop/TestFragmentChecker.java     |  85 ++++++++++
 .../apache/drill/exec/pop/TestFragmenter.java   |  71 +++++++++
 .../drill/exec/pop/TestInjectionValue.java      |  59 +++++++
 .../drill/exec/record/column/SimpleExec.java    |  30 ----
 .../apache/drill/exec/server/StartDrillbit.java |  31 ----
 .../src/test/resources/physical_screen.json     |  37 +++--
 sandbox/prototype/exec/ref/pom.xml              |  17 ++
 .../exec/ref/src/test/resources/donuts.json     |   2 +-
 sandbox/prototype/sqlparser/pom.xml             |   4 +-
 20 files changed, 538 insertions(+), 575 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ea521111/sandbox/prototype/common/pom.xml
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/pom.xml b/sandbox/prototype/common/pom.xml
index beb2d28..4a8c5d8 100644
--- a/sandbox/prototype/common/pom.xml
+++ b/sandbox/prototype/common/pom.xml
@@ -1,106 +1,103 @@
 <?xml version="1.0"?>
 <project
-	xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"
-	xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance">
-	<modelVersion>4.0.0</modelVersion>
-	<parent>
-		<artifactId>prototype-parent</artifactId>
-		<groupId>org.apache.drill</groupId>
-		<version>1.0-SNAPSHOT</version>
-	</parent>
+        xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"
+        xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance">
+    <modelVersion>4.0.0</modelVersion>
+    <parent>
+        <artifactId>prototype-parent</artifactId>
+        <groupId>org.apache.drill</groupId>
+        <version>1.0-SNAPSHOT</version>
+    </parent>
 
-	<artifactId>common</artifactId>
-	<packaging>jar</packaging>
-	<name>common</name>
+    <artifactId>common</artifactId>
+    <packaging>jar</packaging>
+    <name>common</name>
 
-	<dependencies>
+    <dependencies>
 
-		<dependency>
-			<groupId>com.google.protobuf</groupId>
-			<artifactId>protobuf-java</artifactId>
-			<version>2.5.0</version>
-		</dependency>
+        <dependency>
+            <groupId>com.google.protobuf</groupId>
+            <artifactId>protobuf-java</artifactId>
+            <version>2.5.0</version>
+        </dependency>
 
-		<dependency>
-			<groupId>com.typesafe</groupId>
-			<artifactId>config</artifactId>
-			<version>1.0.0</version>
-		</dependency>
+        <dependency>
+            <groupId>com.typesafe</groupId>
+            <artifactId>config</artifactId>
+            <version>1.0.0</version>
+        </dependency>
 
-		<dependency>
-			<groupId>org.apache.commons</groupId>
-			<artifactId>commons-lang3</artifactId>
-			<version>3.1</version>
-		</dependency>
+        <dependency>
+            <groupId>org.apache.commons</groupId>
+            <artifactId>commons-lang3</artifactId>
+            <version>3.1</version>
+        </dependency>
 
-		<dependency>
-			<groupId>org.msgpack</groupId>
-			<artifactId>msgpack</artifactId>
-			<version>0.6.6</version>
-		</dependency>
+        <dependency>
+            <groupId>org.msgpack</groupId>
+            <artifactId>msgpack</artifactId>
+            <version>0.6.6</version>
+        </dependency>
 
-		<dependency>
-			<groupId>org.reflections</groupId>
-			<artifactId>reflections</artifactId>
-			<version>0.9.8</version>
-		</dependency>
+        <dependency>
+            <groupId>org.reflections</groupId>
+            <artifactId>reflections</artifactId>
+            <version>0.9.8</version>
+        </dependency>
 
-		<dependency>
-			<groupId>com.fasterxml.jackson.core</groupId>
-			<artifactId>jackson-annotations</artifactId>
-			<version>2.1.1</version>
-		</dependency>
-		<dependency>
-			<groupId>org.hibernate</groupId>
-			<artifactId>hibernate-validator</artifactId>
-			<version>4.3.1.Final</version>
-		</dependency>
-		<dependency>
-			<groupId>com.fasterxml.jackson.core</groupId>
-			<artifactId>jackson-databind</artifactId>
-			<version>2.1.1</version>
-		</dependency>
-		<dependency>
-			<groupId>org.antlr</groupId>
-			<artifactId>antlr-runtime</artifactId>
-			<version>3.4</version>
-		</dependency>
+        <dependency>
+            <groupId>com.fasterxml.jackson.core</groupId>
+            <artifactId>jackson-annotations</artifactId>
+            <version>2.1.1</version>
+        </dependency>
+        <dependency>
+            <groupId>org.hibernate</groupId>
+            <artifactId>hibernate-validator</artifactId>
+            <version>4.3.1.Final</version>
+        </dependency>
+        <dependency>
+            <groupId>com.fasterxml.jackson.core</groupId>
+            <artifactId>jackson-databind</artifactId>
+            <version>2.1.1</version>
+        </dependency>
+        <dependency>
+            <groupId>org.antlr</groupId>
+            <artifactId>antlr-runtime</artifactId>
+            <version>3.4</version>
+        </dependency>
 
 
+    </dependencies>
 
-	</dependencies>
 
-
-
-	<build>
-		<plugins>
+    <build>
+        <plugins>
             <plugin>
                 <artifactId>maven-jar-plugin</artifactId>
                 <executions>
                     <execution>
                         <id>test-jar</id>
-                        <phase>package</phase>
                         <goals>
                             <goal>test-jar</goal>
                         </goals>
                     </execution>
                 </executions>
             </plugin>
-			<plugin>
-				<groupId>org.antlr</groupId>
-				<artifactId>antlr3-maven-plugin</artifactId>
-				<version>3.4</version>
-				<configuration>
+            <plugin>
+                <groupId>org.antlr</groupId>
+                <artifactId>antlr3-maven-plugin</artifactId>
+                <version>3.4</version>
+                <configuration>
 
-				</configuration>
-				<executions>
-					<execution>
-						<goals>
-							<goal>antlr</goal>
-						</goals>
-					</execution>
-				</executions>
-			</plugin>
-		</plugins>
-	</build>
+                </configuration>
+                <executions>
+                    <execution>
+                        <goals>
+                            <goal>antlr</goal>
+                        </goals>
+                    </execution>
+                </executions>
+            </plugin>
+        </plugins>
+    </build>
 </project>

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ea521111/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/server/BootStrapContext.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/server/BootStrapContext.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/server/BootStrapContext.java
index ed13748..5ca90a7 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/server/BootStrapContext.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/server/BootStrapContext.java
@@ -61,7 +61,7 @@ public class BootStrapContext implements Closeable{
   }
 
   public void close(){
-    loop.shutdown();
+    loop.shutdownGracefully();
     allocator.close();
   }
   

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ea521111/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/config/ParsePhysicalPlan.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/config/ParsePhysicalPlan.java b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/config/ParsePhysicalPlan.java
deleted file mode 100644
index 3b6bf6a..0000000
--- a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/config/ParsePhysicalPlan.java
+++ /dev/null
@@ -1,42 +0,0 @@
-/*******************************************************************************
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- ******************************************************************************/
-package org.apache.drill.exec.physical.config;
-
-import org.apache.drill.common.config.DrillConfig;
-import org.apache.drill.common.util.FileUtils;
-import org.apache.drill.exec.physical.PhysicalPlan;
-import org.junit.Test;
-
-import com.fasterxml.jackson.databind.ObjectReader;
-import com.fasterxml.jackson.databind.ObjectWriter;
-import com.google.common.base.Charsets;
-import com.google.common.io.Files;
-
-public class ParsePhysicalPlan {
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ParsePhysicalPlan.class);
-  
-  
-  @Test 
-  public void parseSimplePlan() throws Exception{
-    DrillConfig c = DrillConfig.create();
-    ObjectReader r = c.getMapper().reader(PhysicalPlan.class);
-    ObjectWriter writer = c.getMapper().writer();
-    PhysicalPlan plan = PhysicalPlan.parse(r, Files.toString(FileUtils.getResourceAsFile("/physical_test1.json"), Charsets.UTF_8));
-    System.out.println(plan.unparse(writer));
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ea521111/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/config/TestParsePhysicalPlan.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/config/TestParsePhysicalPlan.java b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/config/TestParsePhysicalPlan.java
new file mode 100644
index 0000000..dff17ce
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/config/TestParsePhysicalPlan.java
@@ -0,0 +1,45 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.physical.config;
+
+import org.apache.drill.common.config.DrillConfig;
+import org.apache.drill.common.util.FileUtils;
+import org.apache.drill.exec.physical.PhysicalPlan;
+import org.apache.drill.exec.planner.PhysicalPlanReader;
+import org.apache.drill.exec.proto.CoordinationProtos;
+import org.junit.Test;
+
+import com.fasterxml.jackson.databind.ObjectReader;
+import com.fasterxml.jackson.databind.ObjectWriter;
+import com.google.common.base.Charsets;
+import com.google.common.io.Files;
+
+public class TestParsePhysicalPlan {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestParsePhysicalPlan.class);
+  
+  
+  @Test 
+  public void parseSimplePlan() throws Exception{
+    DrillConfig c = DrillConfig.create();
+    PhysicalPlanReader reader = new PhysicalPlanReader(c, c.getMapper(), CoordinationProtos.DrillbitEndpoint.getDefaultInstance());
+    ObjectReader r = c.getMapper().reader(PhysicalPlan.class);
+    ObjectWriter writer = c.getMapper().writer();
+    PhysicalPlan plan = reader.readPhysicalPlan(Files.toString(FileUtils.getResourceAsFile("/physical_test1.json"), Charsets.UTF_8));
+    System.out.println(plan.unparse(writer));
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ea521111/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/DistributedFragmentRun.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/DistributedFragmentRun.java b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/DistributedFragmentRun.java
deleted file mode 100644
index 586ccf6..0000000
--- a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/DistributedFragmentRun.java
+++ /dev/null
@@ -1,58 +0,0 @@
-/*******************************************************************************
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- ******************************************************************************/
-package org.apache.drill.exec.physical.impl;
-
-import static org.junit.Assert.*;
-
-import java.util.List;
-
-import org.apache.drill.common.util.FileUtils;
-import org.apache.drill.exec.client.DrillClient;
-import org.apache.drill.exec.pop.PopUnitTestBase;
-import org.apache.drill.exec.proto.UserProtos.QueryType;
-import org.apache.drill.exec.rpc.user.QueryResultBatch;
-import org.apache.drill.exec.server.Drillbit;
-import org.apache.drill.exec.server.RemoteServiceSet;
-import org.junit.Test;
-
-import com.google.common.base.Charsets;
-import com.google.common.io.Files;
-
-//@Ignore
-public class DistributedFragmentRun extends PopUnitTestBase{
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DistributedFragmentRun.class);
-  
-  
-  @Test 
-  public void oneBitOneExchangeRun() throws Exception{
-    RemoteServiceSet serviceSet = RemoteServiceSet.getLocalServiceSet();
-
-    try(Drillbit bit1 = new Drillbit(CONFIG, serviceSet); DrillClient client = new DrillClient(CONFIG, serviceSet.getCoordinator());){
-      bit1.run();
-      client.connect();
-      List<QueryResultBatch> results = client.runQuery(QueryType.PHYSICAL, Files.toString(FileUtils.getResourceAsFile("/physical_single_exchange.json"), Charsets.UTF_8));
-      int count = 0;
-      for(QueryResultBatch b : results){
-        count += b.getHeader().getRowCount();
-      }
-      assertEquals(100, count);
-    }
-    
-
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ea521111/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/SimpleFragmentRun.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/SimpleFragmentRun.java b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/SimpleFragmentRun.java
deleted file mode 100644
index 6755bb6..0000000
--- a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/SimpleFragmentRun.java
+++ /dev/null
@@ -1,100 +0,0 @@
-/*******************************************************************************
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- ******************************************************************************/
-package org.apache.drill.exec.physical.impl;
-
-import 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;
-import org.apache.drill.exec.proto.UserProtos.QueryType;
-import org.apache.drill.exec.record.RecordBatchLoader;
-import org.apache.drill.exec.record.vector.ValueVector;
-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.Test;
-
-import com.carrotsearch.hppc.cursors.IntObjectCursor;
-import com.google.common.base.Charsets;
-import com.google.common.io.Files;
-
-public class SimpleFragmentRun extends PopUnitTestBase {
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(SimpleFragmentRun.class);
-
-  @Test
-  public void runNoExchangeFragment() throws Exception {
-    try(RemoteServiceSet serviceSet = RemoteServiceSet.getLocalServiceSet(); 
-        Drillbit bit = new Drillbit(CONFIG, serviceSet); 
-        DrillClient client = new DrillClient(CONFIG, serviceSet.getCoordinator());){
-    
-    // run query.
-    bit.run();
-    client.connect();
-    List<QueryResultBatch> results = client.runQuery(QueryType.PHYSICAL, Files.toString(FileUtils.getResourceAsFile("/physical_test2.json"), Charsets.UTF_8));
-    
-    // look at records
-    RecordBatchLoader batchLoader = new RecordBatchLoader(bit.getContext().getAllocator());
-    int recordCount = 0;
-    for (QueryResultBatch batch : results) {
-      if(!batch.hasData()) continue;
-      boolean schemaChanged = batchLoader.load(batch.getHeader().getDef(), batch.getData());
-      boolean firstColumn = true;
-
-      // print headers.
-      if (schemaChanged) {
-        System.out.println("\n\n========NEW SCHEMA=========\n\n");
-        for (IntObjectCursor<ValueVector<?>> v : batchLoader) {
-
-          if (firstColumn) {
-            firstColumn = false;
-          } else {
-            System.out.print("\t");
-          }
-          System.out.print(v.value.getField().getName());
-          System.out.print("[");
-          System.out.print(v.value.getField().getType().getMinorType());
-          System.out.print("]");
-        }
-        System.out.println();
-      }
-
-
-      for (int i = 0; i < batchLoader.getRecordCount(); i++) {
-        boolean first = true;
-        recordCount++;
-        for (IntObjectCursor<ValueVector<?>> v : batchLoader) {
-          if (first) {
-            first = false;
-          } else {
-            System.out.print("\t");
-          }
-          System.out.print(v.value.getObject(i));
-        }
-        if(!first) System.out.println();
-      }
-
-    }
-    logger.debug("Received results {}", results);
-    assertEquals(recordCount, 200);
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ea521111/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestDistributedFragmentRun.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestDistributedFragmentRun.java b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestDistributedFragmentRun.java
new file mode 100644
index 0000000..d4147ee
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestDistributedFragmentRun.java
@@ -0,0 +1,58 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.physical.impl;
+
+import static org.junit.Assert.*;
+
+import java.util.List;
+
+import org.apache.drill.common.util.FileUtils;
+import org.apache.drill.exec.client.DrillClient;
+import org.apache.drill.exec.pop.PopUnitTestBase;
+import org.apache.drill.exec.proto.UserProtos.QueryType;
+import org.apache.drill.exec.rpc.user.QueryResultBatch;
+import org.apache.drill.exec.server.Drillbit;
+import org.apache.drill.exec.server.RemoteServiceSet;
+import org.junit.Test;
+
+import com.google.common.base.Charsets;
+import com.google.common.io.Files;
+
+
+public class TestDistributedFragmentRun extends PopUnitTestBase{
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestDistributedFragmentRun.class);
+  
+  
+  @Test 
+  public void oneBitOneExchangeRun() throws Exception{
+    RemoteServiceSet serviceSet = RemoteServiceSet.getLocalServiceSet();
+
+    try(Drillbit bit1 = new Drillbit(CONFIG, serviceSet); DrillClient client = new DrillClient(CONFIG, serviceSet.getCoordinator());){
+      bit1.run();
+      client.connect();
+      List<QueryResultBatch> results = client.runQuery(QueryType.PHYSICAL, Files.toString(FileUtils.getResourceAsFile("/physical_single_exchange.json"), Charsets.UTF_8));
+      int count = 0;
+      for(QueryResultBatch b : results){
+        count += b.getHeader().getRowCount();
+      }
+      assertEquals(100, count);
+    }
+    
+
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ea521111/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
new file mode 100644
index 0000000..9f9d502
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestSimpleFragmentRun.java
@@ -0,0 +1,100 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.physical.impl;
+
+import 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;
+import org.apache.drill.exec.proto.UserProtos.QueryType;
+import org.apache.drill.exec.record.RecordBatchLoader;
+import org.apache.drill.exec.record.vector.ValueVector;
+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.Test;
+
+import com.carrotsearch.hppc.cursors.IntObjectCursor;
+import com.google.common.base.Charsets;
+import com.google.common.io.Files;
+
+public class TestSimpleFragmentRun extends PopUnitTestBase {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestSimpleFragmentRun.class);
+
+  @Test
+  public void runNoExchangeFragment() throws Exception {
+    try(RemoteServiceSet serviceSet = RemoteServiceSet.getLocalServiceSet(); 
+        Drillbit bit = new Drillbit(CONFIG, serviceSet); 
+        DrillClient client = new DrillClient(CONFIG, serviceSet.getCoordinator());){
+    
+    // run query.
+    bit.run();
+    client.connect();
+    List<QueryResultBatch> results = client.runQuery(QueryType.PHYSICAL, Files.toString(FileUtils.getResourceAsFile("/physical_test2.json"), Charsets.UTF_8));
+    
+    // look at records
+    RecordBatchLoader batchLoader = new RecordBatchLoader(bit.getContext().getAllocator());
+    int recordCount = 0;
+    for (QueryResultBatch batch : results) {
+      if(!batch.hasData()) continue;
+      boolean schemaChanged = batchLoader.load(batch.getHeader().getDef(), batch.getData());
+      boolean firstColumn = true;
+
+      // print headers.
+      if (schemaChanged) {
+        System.out.println("\n\n========NEW SCHEMA=========\n\n");
+        for (IntObjectCursor<ValueVector<?>> v : batchLoader) {
+
+          if (firstColumn) {
+            firstColumn = false;
+          } else {
+            System.out.print("\t");
+          }
+          System.out.print(v.value.getField().getName());
+          System.out.print("[");
+          System.out.print(v.value.getField().getType().getMinorType());
+          System.out.print("]");
+        }
+        System.out.println();
+      }
+
+
+      for (int i = 0; i < batchLoader.getRecordCount(); i++) {
+        boolean first = true;
+        recordCount++;
+        for (IntObjectCursor<ValueVector<?>> v : batchLoader) {
+          if (first) {
+            first = false;
+          } else {
+            System.out.print("\t");
+          }
+          System.out.print(v.value.getObject(i));
+        }
+        if(!first) System.out.println();
+      }
+
+    }
+    logger.debug("Received results {}", results);
+    assertEquals(recordCount, 200);
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ea521111/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/pop/CheckFragmenter.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/pop/CheckFragmenter.java b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/pop/CheckFragmenter.java
deleted file mode 100644
index 1e0c5b6..0000000
--- a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/pop/CheckFragmenter.java
+++ /dev/null
@@ -1,71 +0,0 @@
-/*******************************************************************************
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- ******************************************************************************/
-package org.apache.drill.exec.pop;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertNull;
-
-import java.io.IOException;
-
-import org.apache.drill.exec.exception.FragmentSetupException;
-import org.apache.drill.exec.planner.PhysicalPlanReader;
-import org.apache.drill.exec.planner.fragment.Fragment;
-import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
-import org.junit.Test;
-
-public class CheckFragmenter extends PopUnitTestBase {
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(CheckFragmenter.class);
-
-
-  @Test
-  public void ensureOneFragment() throws FragmentSetupException, IOException {
-    PhysicalPlanReader ppr = new PhysicalPlanReader(CONFIG, CONFIG.getMapper(), DrillbitEndpoint.getDefaultInstance());
-    Fragment b = getRootFragment(ppr, "/physical_test1.json");
-    assertEquals(1, getFragmentCount(b));
-    assertEquals(0, b.getReceivingExchangePairs().size());
-    assertNull(b.getSendingExchange());
-  }
-
-  @Test
-  public void ensureThreeFragments() throws FragmentSetupException, IOException {
-    PhysicalPlanReader ppr = new PhysicalPlanReader(CONFIG, CONFIG.getMapper(), DrillbitEndpoint.getDefaultInstance());
-    Fragment b = getRootFragment(ppr, "/physical_simpleexchange.json");
-    logger.debug("Fragment Node {}", b);
-    assertEquals(3, getFragmentCount(b));
-    assertEquals(1, b.getReceivingExchangePairs().size());
-    assertNull(b.getSendingExchange());
-
-    // get first child.
-    b = b.iterator().next().getNode();
-    assertEquals(1, b.getReceivingExchangePairs().size());
-    assertNotNull(b.getSendingExchange());
-
-    b = b.iterator().next().getNode();
-    assertEquals(0, b.getReceivingExchangePairs().size());
-    assertNotNull(b.getSendingExchange());
-  }
-
-
-
-  
-
-  
-  
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ea521111/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/pop/CheckInjectionValue.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/pop/CheckInjectionValue.java b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/pop/CheckInjectionValue.java
deleted file mode 100644
index 1d0fb91..0000000
--- a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/pop/CheckInjectionValue.java
+++ /dev/null
@@ -1,61 +0,0 @@
-/*******************************************************************************
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- ******************************************************************************/
-package org.apache.drill.exec.pop;
-
-import static org.junit.Assert.*;
-
-import java.util.List;
-
-import org.apache.drill.common.config.DrillConfig;
-import org.apache.drill.common.util.FileUtils;
-import org.apache.drill.exec.physical.PhysicalPlan;
-import org.apache.drill.exec.physical.base.PhysicalOperator;
-import org.apache.drill.exec.physical.config.Screen;
-import org.apache.drill.exec.planner.PhysicalPlanReader;
-import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
-import org.apache.drill.exec.server.DrillbitContext;
-import org.apache.drill.exec.store.StorageEngineRegistry;
-import org.junit.BeforeClass;
-import org.junit.Test;
-
-import com.google.common.base.Charsets;
-import com.google.common.io.Files;
-
-public class CheckInjectionValue {
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(CheckInjectionValue.class);
-  
-  static DrillConfig config;
-  
-  @BeforeClass
-  public static void setup(){
-    config = DrillConfig.create();
-  }
-  
-  @Test
-  public void testInjected() throws Exception{
-    PhysicalPlanReader r = new PhysicalPlanReader(config, config.getMapper(), DrillbitEndpoint.getDefaultInstance());
-    PhysicalPlan p = r.readPhysicalPlan(Files.toString(FileUtils.getResourceAsFile("/physical_screen.json"), Charsets.UTF_8));
-    
-    List<PhysicalOperator> o = p.getSortedOperators(false);
-    
-    PhysicalOperator op = o.iterator().next();
-    assertEquals(Screen.class, op.getClass());
-    Screen s = (Screen) op;
-    assertEquals(DrillbitEndpoint.getDefaultInstance(), s.getEndpoint());
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ea521111/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/pop/FragmentChecker.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/pop/FragmentChecker.java b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/pop/FragmentChecker.java
deleted file mode 100644
index e1db639..0000000
--- a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/pop/FragmentChecker.java
+++ /dev/null
@@ -1,85 +0,0 @@
-/*******************************************************************************
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- ******************************************************************************/
-package org.apache.drill.exec.pop;
-
-import static org.junit.Assert.assertEquals;
-
-import java.util.List;
-
-import org.apache.drill.exec.planner.PhysicalPlanReader;
-import org.apache.drill.exec.planner.fragment.Fragment;
-import org.apache.drill.exec.planner.fragment.PlanningSet;
-import org.apache.drill.exec.planner.fragment.SimpleParallelizer;
-import org.apache.drill.exec.planner.fragment.StatsCollector;
-import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
-import org.apache.drill.exec.proto.ExecProtos.PlanFragment;
-import org.apache.drill.exec.proto.UserBitShared.QueryId;
-import org.apache.drill.exec.work.QueryWorkUnit;
-import org.junit.Test;
-
-import com.google.common.collect.Lists;
-
-public class FragmentChecker extends PopUnitTestBase{
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(FragmentChecker.class);
-  
-  
-  @Test
-  public void checkSimpleExchangePlan() throws Exception{
-    print("/physical_simpleexchange.json", 2, 3);
-
-  }
-  
-  
-  private void print(String fragmentFile, int bitCount, int exepectedFragmentCount) throws Exception{
-    
-    System.out.println(String.format("=================Building plan fragments for [%s].  Allowing %d total Drillbits.==================", fragmentFile, bitCount));
-    PhysicalPlanReader ppr = new PhysicalPlanReader(CONFIG, CONFIG.getMapper(), DrillbitEndpoint.getDefaultInstance());
-    Fragment fragmentRoot = getRootFragment(ppr, fragmentFile);
-    PlanningSet planningSet = StatsCollector.collectStats(fragmentRoot);
-    SimpleParallelizer par = new SimpleParallelizer();
-    List<DrillbitEndpoint> endpoints = Lists.newArrayList();
-    DrillbitEndpoint localBit = null;
-    for(int i =0; i < bitCount; i++){
-      DrillbitEndpoint b1 = DrillbitEndpoint.newBuilder().setAddress("localhost").setBitPort(1234+i).build();
-      if(i ==0) localBit = b1; 
-      endpoints.add(b1);
-    }
-    
-    
-    QueryWorkUnit qwu = par.getFragments(localBit, QueryId.getDefaultInstance(), endpoints, ppr, fragmentRoot, planningSet, 10);
-    System.out.println(String.format("=========ROOT FRAGMENT [%d:%d] =========", qwu.getRootFragment().getHandle().getMajorFragmentId(), qwu.getRootFragment().getHandle().getMinorFragmentId()));
-    
-    System.out.print(qwu.getRootFragment().getFragmentJson());
-    
-    
-    for(PlanFragment f : qwu.getFragments()){
-      System.out.println(String.format("=========Fragment [%d:%d]=====", f.getHandle().getMajorFragmentId(), f.getHandle().getMinorFragmentId()));
-      System.out.print(f.getFragmentJson());
-    }
-    
-    //assertEquals(exepectedFragmentCount, qwu.getFragments().size());
-
-    logger.debug("Planning Set {}", planningSet);
-  }
-  
-  @Test
-  public void validateSingleExchangeFragment() throws Exception{
-    print("/physical_single_exchange.json", 1, 2);
-
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ea521111/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/pop/TestFragmentChecker.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/pop/TestFragmentChecker.java b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/pop/TestFragmentChecker.java
new file mode 100644
index 0000000..710326e
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/pop/TestFragmentChecker.java
@@ -0,0 +1,85 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.pop;
+
+import static org.junit.Assert.assertEquals;
+
+import java.util.List;
+
+import org.apache.drill.exec.planner.PhysicalPlanReader;
+import org.apache.drill.exec.planner.fragment.Fragment;
+import org.apache.drill.exec.planner.fragment.PlanningSet;
+import org.apache.drill.exec.planner.fragment.SimpleParallelizer;
+import org.apache.drill.exec.planner.fragment.StatsCollector;
+import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
+import org.apache.drill.exec.proto.ExecProtos.PlanFragment;
+import org.apache.drill.exec.proto.UserBitShared.QueryId;
+import org.apache.drill.exec.work.QueryWorkUnit;
+import org.junit.Test;
+
+import com.google.common.collect.Lists;
+
+public class TestFragmentChecker extends PopUnitTestBase{
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestFragmentChecker.class);
+  
+  
+  @Test
+  public void checkSimpleExchangePlan() throws Exception{
+    print("/physical_simpleexchange.json", 2, 3);
+
+  }
+  
+  
+  private void print(String fragmentFile, int bitCount, int exepectedFragmentCount) throws Exception{
+    
+    System.out.println(String.format("=================Building plan fragments for [%s].  Allowing %d total Drillbits.==================", fragmentFile, bitCount));
+    PhysicalPlanReader ppr = new PhysicalPlanReader(CONFIG, CONFIG.getMapper(), DrillbitEndpoint.getDefaultInstance());
+    Fragment fragmentRoot = getRootFragment(ppr, fragmentFile);
+    PlanningSet planningSet = StatsCollector.collectStats(fragmentRoot);
+    SimpleParallelizer par = new SimpleParallelizer();
+    List<DrillbitEndpoint> endpoints = Lists.newArrayList();
+    DrillbitEndpoint localBit = null;
+    for(int i =0; i < bitCount; i++){
+      DrillbitEndpoint b1 = DrillbitEndpoint.newBuilder().setAddress("localhost").setBitPort(1234+i).build();
+      if(i ==0) localBit = b1; 
+      endpoints.add(b1);
+    }
+    
+    
+    QueryWorkUnit qwu = par.getFragments(localBit, QueryId.getDefaultInstance(), endpoints, ppr, fragmentRoot, planningSet, 10);
+    System.out.println(String.format("=========ROOT FRAGMENT [%d:%d] =========", qwu.getRootFragment().getHandle().getMajorFragmentId(), qwu.getRootFragment().getHandle().getMinorFragmentId()));
+    
+    System.out.print(qwu.getRootFragment().getFragmentJson());
+    
+    
+    for(PlanFragment f : qwu.getFragments()){
+      System.out.println(String.format("=========Fragment [%d:%d]=====", f.getHandle().getMajorFragmentId(), f.getHandle().getMinorFragmentId()));
+      System.out.print(f.getFragmentJson());
+    }
+    
+    //assertEquals(exepectedFragmentCount, qwu.getFragments().size());
+
+    logger.debug("Planning Set {}", planningSet);
+  }
+  
+  @Test
+  public void validateSingleExchangeFragment() throws Exception{
+    print("/physical_single_exchange.json", 1, 2);
+
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ea521111/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/pop/TestFragmenter.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/pop/TestFragmenter.java b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/pop/TestFragmenter.java
new file mode 100644
index 0000000..27bee40
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/pop/TestFragmenter.java
@@ -0,0 +1,71 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.pop;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+
+import java.io.IOException;
+
+import org.apache.drill.exec.exception.FragmentSetupException;
+import org.apache.drill.exec.planner.PhysicalPlanReader;
+import org.apache.drill.exec.planner.fragment.Fragment;
+import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
+import org.junit.Test;
+
+public class TestFragmenter extends PopUnitTestBase {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestFragmenter.class);
+
+
+  @Test
+  public void ensureOneFragment() throws FragmentSetupException, IOException {
+    PhysicalPlanReader ppr = new PhysicalPlanReader(CONFIG, CONFIG.getMapper(), DrillbitEndpoint.getDefaultInstance());
+    Fragment b = getRootFragment(ppr, "/physical_test1.json");
+    assertEquals(1, getFragmentCount(b));
+    assertEquals(0, b.getReceivingExchangePairs().size());
+    assertNull(b.getSendingExchange());
+  }
+
+  @Test
+  public void ensureThreeFragments() throws FragmentSetupException, IOException {
+    PhysicalPlanReader ppr = new PhysicalPlanReader(CONFIG, CONFIG.getMapper(), DrillbitEndpoint.getDefaultInstance());
+    Fragment b = getRootFragment(ppr, "/physical_simpleexchange.json");
+    logger.debug("Fragment Node {}", b);
+    assertEquals(3, getFragmentCount(b));
+    assertEquals(1, b.getReceivingExchangePairs().size());
+    assertNull(b.getSendingExchange());
+
+    // get first child.
+    b = b.iterator().next().getNode();
+    assertEquals(1, b.getReceivingExchangePairs().size());
+    assertNotNull(b.getSendingExchange());
+
+    b = b.iterator().next().getNode();
+    assertEquals(0, b.getReceivingExchangePairs().size());
+    assertNotNull(b.getSendingExchange());
+  }
+
+
+
+  
+
+  
+  
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ea521111/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/pop/TestInjectionValue.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/pop/TestInjectionValue.java b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/pop/TestInjectionValue.java
new file mode 100644
index 0000000..606c531
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/pop/TestInjectionValue.java
@@ -0,0 +1,59 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.pop;
+
+import static org.junit.Assert.*;
+
+import java.util.List;
+
+import org.apache.drill.common.config.DrillConfig;
+import org.apache.drill.common.util.FileUtils;
+import org.apache.drill.exec.physical.PhysicalPlan;
+import org.apache.drill.exec.physical.base.PhysicalOperator;
+import org.apache.drill.exec.physical.config.Screen;
+import org.apache.drill.exec.planner.PhysicalPlanReader;
+import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import com.google.common.base.Charsets;
+import com.google.common.io.Files;
+
+public class TestInjectionValue {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestInjectionValue.class);
+  
+  static DrillConfig config;
+  
+  @BeforeClass
+  public static void setup(){
+    config = DrillConfig.create();
+  }
+  
+  @Test
+  public void testInjected() throws Exception{
+    PhysicalPlanReader r = new PhysicalPlanReader(config, config.getMapper(), DrillbitEndpoint.getDefaultInstance());
+    PhysicalPlan p = r.readPhysicalPlan(Files.toString(FileUtils.getResourceAsFile("/physical_screen.json"), Charsets.UTF_8));
+    
+    List<PhysicalOperator> o = p.getSortedOperators(false);
+    
+    PhysicalOperator op = o.iterator().next();
+    assertEquals(Screen.class, op.getClass());
+    Screen s = (Screen) op;
+    assertEquals(DrillbitEndpoint.getDefaultInstance(), s.getEndpoint());
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ea521111/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/record/column/SimpleExec.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/record/column/SimpleExec.java b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/record/column/SimpleExec.java
deleted file mode 100644
index e3747e1..0000000
--- a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/record/column/SimpleExec.java
+++ /dev/null
@@ -1,30 +0,0 @@
-/*******************************************************************************
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- ******************************************************************************/
-package org.apache.drill.exec.record.column;
-
-import org.junit.Test;
-
-public class SimpleExec {
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(SimpleExec.class);
-  
-  @Test
-  public void columnarAnd() throws Exception{
-   
-
-  }
-}

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

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ea521111/sandbox/prototype/exec/java-exec/src/test/resources/physical_screen.json
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/resources/physical_screen.json b/sandbox/prototype/exec/java-exec/src/test/resources/physical_screen.json
index 8bb97db..99a66c5 100644
--- a/sandbox/prototype/exec/java-exec/src/test/resources/physical_screen.json
+++ b/sandbox/prototype/exec/java-exec/src/test/resources/physical_screen.json
@@ -1,24 +1,33 @@
 {
-    head:{
-        type:"APACHE_DRILL_PHYSICAL",
-        version:"1",
-        generator:{
-            type:"manual"
+    "head": {
+        "type": "APACHE_DRILL_PHYSICAL",
+        "version": "1",
+        "generator": {
+            "type": "manual"
         }
     },
-    graph:[
+    "graph": [
         {
-            @id:1,
-            pop:"mock-scan",
-            url: "http://apache.org",
-            entries:[
-            	{id:1, records: 100, size: 100}
+            "@id": 1,
+            "pop": "mock-scan",
+            "url": "http://apache.org",
+            "entries": [
+                {
+                    "records": 100,
+                    "types": [
+                        {
+                            "name": "blue",
+                            "type": "INT",
+                            "mode": "REQUIRED"
+                        }
+                    ]
+                }
             ]
         },
         {
-            @id:2,
-            pop: "screen",
-            child: 1
+            "@id": 2,
+            "pop": "screen",
+            "child": 1
         }
     ]
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ea521111/sandbox/prototype/exec/ref/pom.xml
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/ref/pom.xml b/sandbox/prototype/exec/ref/pom.xml
index 045a36b..956f095 100644
--- a/sandbox/prototype/exec/ref/pom.xml
+++ b/sandbox/prototype/exec/ref/pom.xml
@@ -52,5 +52,22 @@
 		</dependency>
 	</dependencies>
 
+  <build>
+    <plugins>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-jar-plugin</artifactId>
+        <version>2.4</version>
+
+        <executions>
+          <execution>
+            <goals>
+              <goal>test-jar</goal>
+            </goals>
+          </execution>
+        </executions>
+      </plugin>
+    </plugins>
+  </build>
 
 </project>

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ea521111/sandbox/prototype/exec/ref/src/test/resources/donuts.json
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/ref/src/test/resources/donuts.json b/sandbox/prototype/exec/ref/src/test/resources/donuts.json
index 9fd6e3e..2d98b59 100644
--- a/sandbox/prototype/exec/ref/src/test/resources/donuts.json
+++ b/sandbox/prototype/exec/ref/src/test/resources/donuts.json
@@ -20,7 +20,7 @@
 			[
 				{ "id": "5001", "type": "None" },
 				{ "id": "5002", "type": "Glazed" },
-				{ "id": "5005", "type": "Sugar", color: "White"},
+				{ "id": "5005", "type": "Sugar" },
 				{ "id": "5007", "type": "Powdered Sugar" },
 				{ "id": "5006", "type": "Chocolate with Sprinkles" },
 				{ "id": "5003", "type": "Chocolate" },

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ea521111/sandbox/prototype/sqlparser/pom.xml
----------------------------------------------------------------------
diff --git a/sandbox/prototype/sqlparser/pom.xml b/sandbox/prototype/sqlparser/pom.xml
index 4bc576b..0eb0944 100644
--- a/sandbox/prototype/sqlparser/pom.xml
+++ b/sandbox/prototype/sqlparser/pom.xml
@@ -39,12 +39,12 @@
 		<dependency>
 			<groupId>org.apache.drill.exec</groupId>
 			<artifactId>ref</artifactId>
-			<version>1.0-SNAPSHOT</version>
+      <version>${project.version}</version>
 		</dependency>
 		<dependency>
 			<groupId>org.apache.drill.exec</groupId>
 			<artifactId>ref</artifactId>
-			<version>1.0-SNAPSHOT</version>
+      <version>${project.version}</version>
 			<classifier>tests</classifier>
 			<scope>test</scope>
 		</dependency>


[15/53] [abbrv] Update typing system. Update RPC system. Add Fragmenting Implementation. Working single node. Distributed failing due to threading issues.

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockStorageEngine.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockStorageEngine.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockStorageEngine.java
new file mode 100644
index 0000000..0044628
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockStorageEngine.java
@@ -0,0 +1,58 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.physical.config;
+
+import java.io.IOException;
+import java.util.Collection;
+
+import org.apache.drill.common.logical.data.Scan;
+import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
+import org.apache.drill.exec.store.AbstractStorageEngine;
+import org.apache.drill.exec.store.RecordReader;
+import org.apache.drill.exec.store.StorageEngine;
+import org.apache.drill.exec.store.StorageEngine.ReadEntry;
+
+import com.google.common.collect.ListMultimap;
+
+public class MockStorageEngine extends AbstractStorageEngine{
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(MockStorageEngine.class);
+
+  @Override
+  public boolean supportsRead() {
+    return true;
+  }
+
+  @Override
+  public Collection<ReadEntry> getReadEntries(Scan scan) throws IOException {
+    return null;
+  }
+
+  @Override
+  public ListMultimap<ReadEntry, DrillbitEndpoint> getReadLocations(Collection<ReadEntry> entries) {
+    return null;
+  }
+
+  @Override
+  public RecordReader getReader(FragmentContext context, ReadEntry readEntry) throws IOException {
+    return null;
+  }
+
+  
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockStorePOP.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockStorePOP.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockStorePOP.java
new file mode 100644
index 0000000..639d0d2
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockStorePOP.java
@@ -0,0 +1,75 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.physical.config;
+
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.drill.exec.physical.EndpointAffinity;
+import org.apache.drill.exec.physical.OperatorCost;
+import org.apache.drill.exec.physical.base.AbstractStore;
+import org.apache.drill.exec.physical.base.PhysicalOperator;
+import org.apache.drill.exec.physical.base.Store;
+import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.annotation.JsonTypeName;
+
+@JsonTypeName("mock-store")
+public class MockStorePOP extends AbstractStore {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(MockStorePOP.class);
+
+  @JsonCreator
+  public MockStorePOP(@JsonProperty("child") PhysicalOperator child) {
+    super(child);
+  }
+
+  public int getMaxWidth() {
+    return 1;
+  }
+
+  @Override
+  public List<EndpointAffinity> getOperatorAffinity() {
+    return Collections.emptyList();
+  }
+
+  @Override
+  public void applyAssignments(List<DrillbitEndpoint> endpoints) {
+    
+  }
+
+  @Override
+  public Store getSpecificStore(PhysicalOperator child, int minorFragmentId) {
+    return new MockStorePOP(child);
+  }
+
+  @Override
+  public OperatorCost getCost() {
+    return new OperatorCost(1,getSize().getRecordCount()*getSize().getRecordSize(),1,1);
+  }
+
+  @Override
+  protected PhysicalOperator getNewWithChild(PhysicalOperator child) {
+    return new MockStorePOP(child);
+  }
+
+
+  
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/PartitionRange.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/PartitionRange.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/PartitionRange.java
new file mode 100644
index 0000000..eb77eeb
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/PartitionRange.java
@@ -0,0 +1,47 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.physical.config;
+
+import org.apache.drill.common.expression.LogicalExpression;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+
+public class PartitionRange {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(PartitionRange.class);
+  
+  private LogicalExpression start;
+  private LogicalExpression finish;
+  
+  @JsonCreator
+  public PartitionRange(@JsonProperty("start") LogicalExpression start, @JsonProperty("finish") LogicalExpression finish) {
+    super();
+    this.start = start;
+    this.finish = finish;
+  }
+
+  public LogicalExpression getStart() {
+    return start;
+  }
+
+  public LogicalExpression getFinish() {
+    return finish;
+  }
+  
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/Project.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/Project.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/Project.java
new file mode 100644
index 0000000..e869393
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/Project.java
@@ -0,0 +1,72 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.physical.config;
+
+import java.util.Iterator;
+import java.util.List;
+
+import org.apache.drill.common.logical.data.NamedExpression;
+import org.apache.drill.exec.physical.OperatorCost;
+import org.apache.drill.exec.physical.base.AbstractSingle;
+import org.apache.drill.exec.physical.base.PhysicalOperator;
+import org.apache.drill.exec.physical.base.PhysicalVisitor;
+import org.apache.drill.exec.physical.base.Size;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.annotation.JsonTypeName;
+
+@JsonTypeName("project")
+public class Project extends AbstractSingle{
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(Project.class);
+
+  private final List<NamedExpression> exprs;
+  
+  @JsonCreator
+  public Project(@JsonProperty("exprs") List<NamedExpression> exprs, @JsonProperty("child") PhysicalOperator child) {
+    super(child);
+    this.exprs = exprs;
+  }
+
+  public List<NamedExpression> getExprs() {
+    return exprs;
+  }
+
+  @Override
+  public <T, X, E extends Throwable> T accept(PhysicalVisitor<T, X, E> physicalVisitor, X value) throws E{
+    return physicalVisitor.visitProject(this, value);
+  }
+
+  @Override
+  public OperatorCost getCost() {
+    return new OperatorCost(0, 0, 1000, child.getSize().getRecordCount());
+  }
+  
+  @Override
+  public Size getSize() {
+    //TODO: This should really change the row width...
+    return child.getSize();
+  }
+
+  @Override
+  protected PhysicalOperator getNewWithChild(PhysicalOperator child) {
+    return new Project(exprs, child);
+  }
+  
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/RandomReceiver.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/RandomReceiver.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/RandomReceiver.java
new file mode 100644
index 0000000..ed41586
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/RandomReceiver.java
@@ -0,0 +1,83 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.physical.config;
+
+import java.util.List;
+
+import org.apache.drill.exec.physical.OperatorCost;
+import org.apache.drill.exec.physical.base.AbstractReceiver;
+import org.apache.drill.exec.physical.base.PhysicalOperator;
+import org.apache.drill.exec.physical.base.PhysicalVisitor;
+import org.apache.drill.exec.physical.base.Size;
+import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonIgnore;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.annotation.JsonTypeName;
+import com.google.common.base.Preconditions;
+
+@JsonTypeName("random-receiver")
+public class RandomReceiver extends AbstractReceiver{
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(RandomReceiver.class);
+
+  private List<DrillbitEndpoint> senders;
+  
+  @JsonCreator
+  public RandomReceiver(@JsonProperty("sender-major-fragment") int oppositeMajorFragmentId, @JsonProperty("senders") List<DrillbitEndpoint> senders){
+    super(oppositeMajorFragmentId);
+    this.senders = senders;
+  }
+  
+  @Override
+  public List<DrillbitEndpoint> getProvidingEndpoints() {
+    return senders;
+  }
+
+  @Override
+  public boolean supportsOutOfOrderExchange() {
+    return true;
+  }
+
+  @Override
+  public OperatorCost getCost() {
+    //TODO: deal with receiver cost through exchange.
+    return new OperatorCost(1,1,1,1);
+  }
+
+  
+  @Override
+  public <T, X, E extends Throwable> T accept(PhysicalVisitor<T, X, E> physicalVisitor, X value) throws E {
+    return physicalVisitor.visitRandomReceiver(this, value);
+  }
+
+  @Override
+  public Size getSize() {
+    //TODO: deal with size info through exchange.
+    return new Size(1,1);
+  }
+
+  @Override
+  public int getOppositeMajorFragmentId() {
+    return 0;
+  }
+
+  
+
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/RangeSender.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/RangeSender.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/RangeSender.java
new file mode 100644
index 0000000..7d64dba
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/RangeSender.java
@@ -0,0 +1,72 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.physical.config;
+
+import java.util.List;
+
+import org.apache.drill.exec.physical.base.AbstractSender;
+import org.apache.drill.exec.physical.base.PhysicalOperator;
+import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.annotation.JsonTypeName;
+
+@JsonTypeName("range-sender")
+public class RangeSender extends AbstractSender{
+
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(RangeSender.class);
+
+  List<EndpointPartition> partitions;
+  
+  @JsonCreator
+  public RangeSender(@JsonProperty("receiver-major-fragment") int oppositeMajorFragmentId, @JsonProperty("child") PhysicalOperator child, @JsonProperty("partitions") List<EndpointPartition> partitions) {
+    super(oppositeMajorFragmentId, child);
+    this.partitions = partitions;
+  }
+
+  @Override
+  public List<DrillbitEndpoint> getDestinations() {
+    return null;
+  }
+
+
+  @Override
+  protected PhysicalOperator getNewWithChild(PhysicalOperator child) {
+    return new RangeSender(oppositeMajorFragmentId, child, partitions);
+  }
+
+
+  public static class EndpointPartition{
+    private final PartitionRange range;
+    private final DrillbitEndpoint endpoint;
+    
+    @JsonCreator
+    public EndpointPartition(@JsonProperty("range") PartitionRange range, @JsonProperty("endpoint") DrillbitEndpoint endpoint) {
+      super();
+      this.range = range;
+      this.endpoint = endpoint;
+    }
+    public PartitionRange getRange() {
+      return range;
+    }
+    public DrillbitEndpoint getEndpoint() {
+      return endpoint;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/Screen.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/Screen.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/Screen.java
new file mode 100644
index 0000000..86a201d
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/Screen.java
@@ -0,0 +1,106 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.physical.config;
+
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.drill.common.exceptions.PhysicalOperatorSetupException;
+import org.apache.drill.exec.physical.EndpointAffinity;
+import org.apache.drill.exec.physical.OperatorCost;
+import org.apache.drill.exec.physical.base.AbstractStore;
+import org.apache.drill.exec.physical.base.PhysicalOperator;
+import org.apache.drill.exec.physical.base.PhysicalVisitor;
+import org.apache.drill.exec.physical.base.Root;
+import org.apache.drill.exec.physical.base.Size;
+import org.apache.drill.exec.physical.base.Store;
+import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
+
+import com.fasterxml.jackson.annotation.JacksonInject;
+import com.fasterxml.jackson.annotation.JsonIgnore;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.annotation.JsonTypeName;
+import com.google.common.base.Preconditions;
+
+@JsonTypeName("screen")
+public class Screen extends AbstractStore implements Root{
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(Screen.class);
+
+  private final DrillbitEndpoint endpoint;
+
+  public Screen(@JsonProperty("child") PhysicalOperator child, @JacksonInject DrillbitEndpoint endpoint) {
+    super(child);
+    this.endpoint = endpoint;
+  }
+
+  @Override
+  public List<EndpointAffinity> getOperatorAffinity() {
+    return Collections.singletonList(new EndpointAffinity(endpoint, 1000000000000l));
+  }
+
+  @Override
+  public int getMaxWidth() {
+    return 1;
+  }
+
+  @Override
+  public void applyAssignments(List<DrillbitEndpoint> endpoints) throws PhysicalOperatorSetupException {
+    // we actually don't have to do anything since nothing should have changed. we'll check just check that things
+    // didn't get screwed up.
+    if (endpoints.size() != 1) throw new PhysicalOperatorSetupException("A Screen operator can only be assigned to a single node.");
+    DrillbitEndpoint endpoint = endpoints.iterator().next();
+    logger.debug("Endpoint this: {}, assignment: {}", this.endpoint, endpoint);
+    if (!endpoint.equals(this.endpoint)) {
+      throw new PhysicalOperatorSetupException(String.format(
+          "A Screen operator can only be assigned to its home node.  Expected endpoint %s, Actual endpoint: %s",
+          this.endpoint, endpoint));
+    }
+  }
+
+  @Override
+  public Store getSpecificStore(PhysicalOperator child, int minorFragmentId) {
+    return new Screen(child, endpoint);
+  }
+
+  @JsonIgnore
+  public DrillbitEndpoint getEndpoint() {
+    return endpoint;
+  }
+
+  @Override
+  public String toString() {
+    return "Screen [endpoint=" + endpoint + ", getChild()=" + getChild() + "]";
+  }
+
+  @Override
+  public OperatorCost getCost() {
+    return new OperatorCost(0, 0, 1000, child.getSize().getRecordCount());
+  }
+
+  @Override
+  protected PhysicalOperator getNewWithChild(PhysicalOperator child) {
+    return new Screen(child, endpoint);
+  }
+
+  @Override
+  public <T, X, E extends Throwable> T accept(PhysicalVisitor<T, X, E> physicalVisitor, X value) throws E {
+    return physicalVisitor.visitScreen(this, value);
+  }
+
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/SingleSender.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/SingleSender.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/SingleSender.java
new file mode 100644
index 0000000..79d937a
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/SingleSender.java
@@ -0,0 +1,78 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.physical.config;
+
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.drill.common.graph.GraphVisitor;
+import org.apache.drill.exec.physical.OperatorCost;
+import org.apache.drill.exec.physical.base.AbstractSender;
+import org.apache.drill.exec.physical.base.PhysicalOperator;
+import org.apache.drill.exec.physical.base.PhysicalVisitor;
+import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonIgnore;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.annotation.JsonTypeName;
+
+/**
+ * Sender that pushes all data to a single destination node.
+ */
+@JsonTypeName("single-sender")
+public class SingleSender extends AbstractSender {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(SingleSender.class);
+
+  private final DrillbitEndpoint destination;
+  
+  @JsonCreator
+  public SingleSender(@JsonProperty("receiver-major-fragment") int oppositeMajorFragmentId, @JsonProperty("child") PhysicalOperator child, @JsonProperty("destination") DrillbitEndpoint destination) {
+    super(oppositeMajorFragmentId, child);
+    this.destination = destination;
+  }
+
+  @Override
+  @JsonIgnore
+  public List<DrillbitEndpoint> getDestinations() {
+    return Collections.singletonList(destination);
+  }
+
+  @Override
+  public OperatorCost getCost() {
+    long recordSize = child.getSize().getRecordSize() * child.getSize().getRecordCount();
+    return new OperatorCost((float) recordSize, recordSize, 0, child.getSize().getRecordCount()/(1<<16));
+  }
+
+  @Override
+  protected PhysicalOperator getNewWithChild(PhysicalOperator child) {
+    return new SingleSender(oppositeMajorFragmentId, child, destination);
+  }
+
+  @Override
+  public <T, X, E extends Throwable> T accept(PhysicalVisitor<T, X, E> physicalVisitor, X value) throws E {
+    return physicalVisitor.visitSingleSender(this, value);
+  }
+ 
+
+  public DrillbitEndpoint getDestination() {
+    return destination;
+  }
+ 
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/Sort.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/Sort.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/Sort.java
new file mode 100644
index 0000000..e4ece6b
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/Sort.java
@@ -0,0 +1,86 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.physical.config;
+
+import java.util.List;
+
+import org.apache.drill.common.defs.OrderDef;
+import org.apache.drill.exec.physical.OperatorCost;
+import org.apache.drill.exec.physical.base.AbstractSingle;
+import org.apache.drill.exec.physical.base.PhysicalOperator;
+import org.apache.drill.exec.physical.base.PhysicalVisitor;
+import org.apache.drill.exec.physical.base.Size;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonIgnore;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.annotation.JsonTypeName;
+
+@JsonTypeName("sort")
+public class Sort extends AbstractSingle{
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(Sort.class);
+  
+  private final List<OrderDef> orderings;
+  private boolean reverse = false;
+  
+  @JsonCreator
+  public Sort(@JsonProperty("child") PhysicalOperator child, @JsonProperty("orderings") List<OrderDef> orderings, @JsonProperty("reverse") boolean reverse) {
+    super(child);
+    this.orderings = orderings;
+    this.reverse = reverse;
+  }
+
+  public List<OrderDef> getOrderings() {
+    return orderings;
+  }
+
+  public boolean getReverse() {
+    return reverse;
+  }
+
+  @Override
+  public <T, X, E extends Throwable> T accept(PhysicalVisitor<T, X, E> physicalVisitor, X value) throws E{
+    return physicalVisitor.visitSort(this, value);
+  }
+
+  @Override
+  public OperatorCost getCost() {
+    Size childSize = child.getSize();
+    long n = childSize.getRecordCount();
+    long width = childSize.getRecordSize();
+
+    //TODO: Magic Number, let's assume 1/10 of data can fit in memory. 
+    int k = 10;
+    long n2 = n/k;
+    double cpuCost = 
+        k * n2 * (Math.log(n2)/Math.log(2)) + // 
+        n * (Math.log(k)/Math.log(2));
+    double diskCost = n*width*2;
+    
+    return new OperatorCost(0, (float) diskCost, (float) n2*width, (float) cpuCost);
+  }
+
+  @Override
+  protected PhysicalOperator getNewWithChild(PhysicalOperator child) {
+    return new Sort(child, orderings, reverse);
+  }
+
+    
+  
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/UnionExchange.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/UnionExchange.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/UnionExchange.java
new file mode 100644
index 0000000..56467ce
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/UnionExchange.java
@@ -0,0 +1,79 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.physical.config;
+
+import java.util.List;
+
+import org.apache.drill.common.exceptions.PhysicalOperatorSetupException;
+import org.apache.drill.exec.physical.base.AbstractExchange;
+import org.apache.drill.exec.physical.base.ExchangeCost;
+import org.apache.drill.exec.physical.base.PhysicalOperator;
+import org.apache.drill.exec.physical.base.PhysicalVisitor;
+import org.apache.drill.exec.physical.base.Receiver;
+import org.apache.drill.exec.physical.base.Sender;
+import org.apache.drill.exec.physical.base.Size;
+import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
+
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.annotation.JsonTypeName;
+
+@JsonTypeName("union-exchange")
+public class UnionExchange extends AbstractExchange{
+
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(UnionExchange.class);
+
+  private List<DrillbitEndpoint> senderLocations;
+  private DrillbitEndpoint destinationLocation;
+  
+  public UnionExchange(@JsonProperty("child") PhysicalOperator child) {
+    super(child);
+  }
+  
+  @Override
+  public void setupSenders(List<DrillbitEndpoint> senderLocations) {
+    this.senderLocations = senderLocations;
+  }
+
+  @Override
+  protected void setupReceivers(List<DrillbitEndpoint> receiverLocations) throws PhysicalOperatorSetupException {
+    if(receiverLocations.size() != 1) throw new PhysicalOperatorSetupException("A Union Exchange only supports a single receiver endpoint.");
+    this.destinationLocation = receiverLocations.iterator().next();
+  }
+
+  @Override
+  public Sender getSender(int minorFragmentId, PhysicalOperator child) {
+    return new SingleSender(this.receiverMajorFragmentId, child, destinationLocation);
+  }
+
+  @Override
+  public Receiver getReceiver(int minorFragmentId) {
+    return new RandomReceiver(this.senderMajorFragmentId, senderLocations);
+  }
+
+  @Override
+  public int getMaxSendWidth() {
+    return Integer.MAX_VALUE;
+  }
+
+  @Override
+  protected PhysicalOperator getNewWithChild(PhysicalOperator child) {
+    return new UnionExchange(child);
+  }
+  
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/BatchCreator.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/BatchCreator.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/BatchCreator.java
new file mode 100644
index 0000000..9a7df56
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/BatchCreator.java
@@ -0,0 +1,31 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.physical.impl;
+
+import java.util.List;
+
+import org.apache.drill.common.exceptions.ExecutionSetupException;
+import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.physical.base.PhysicalOperator;
+import org.apache.drill.exec.record.RecordBatch;
+
+public interface BatchCreator<T extends PhysicalOperator> {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(BatchCreator.class);
+  
+  public RecordBatch getBatch(FragmentContext context, T config, List<RecordBatch> children) throws ExecutionSetupException;
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/FilterRecordBatch.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/FilterRecordBatch.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/FilterRecordBatch.java
new file mode 100644
index 0000000..6592ca1
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/FilterRecordBatch.java
@@ -0,0 +1,108 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.physical.impl;
+
+import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.record.BatchSchema;
+import org.apache.drill.exec.record.InvalidValueAccessor;
+import org.apache.drill.exec.record.RecordBatch;
+import org.apache.drill.exec.record.vector.SelectionVector;
+import org.apache.drill.exec.record.vector.ValueVector;
+
+public abstract class FilterRecordBatch implements RecordBatch {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(FilterRecordBatch.class);
+
+  private RecordBatch incoming;
+  private SelectionVector selectionVector;
+  private BatchSchema schema;
+  private FilteringRecordBatchTransformer transformer;
+  private int outstanding;
+
+  public FilterRecordBatch(RecordBatch batch) {
+    this.incoming = batch;
+  }
+
+  @Override
+  public FragmentContext getContext() {
+    return incoming.getContext();
+  }
+
+  @Override
+  public BatchSchema getSchema() {
+    return schema;
+  }
+
+  @Override
+  public int getRecordCount() {
+    return 0;
+  }
+
+  @Override
+  public void kill() {
+    incoming.kill();
+  }
+
+  @Override
+  public <T extends ValueVector<T>> T getValueVector(int fieldId, Class<T> clazz) throws InvalidValueAccessor {
+    return null;
+  }
+
+  abstract int applyFilter(SelectionVector vector, int count);
+
+  /**
+   * Release all assets.
+   */
+  private void close() {
+
+  }
+
+  @Override
+  public IterOutcome next() {
+    while (true) {
+      IterOutcome o = incoming.next();
+      switch (o) {
+      case OK_NEW_SCHEMA:
+        transformer = incoming.getContext().getFilteringExpression(null);
+        schema = transformer.getSchema();
+        // fall through to ok.
+      case OK:
+
+      case NONE:
+      case STOP:
+        close();
+        return IterOutcome.STOP;
+      }
+
+      if (outstanding > 0) {
+        // move data to output location.
+
+        for (int i = incoming.getRecordCount() - outstanding; i < incoming.getRecordCount(); i++) {
+
+        }
+      }
+
+      // make sure the bit vector is as large as the current record batch.
+      if (selectionVector.capacity() < incoming.getRecordCount()) {
+        selectionVector.allocateNew(incoming.getRecordCount());
+      }
+
+      return null;
+    }
+
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/FilteringRecordBatchTransformer.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/FilteringRecordBatchTransformer.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/FilteringRecordBatchTransformer.java
new file mode 100644
index 0000000..191521a
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/FilteringRecordBatchTransformer.java
@@ -0,0 +1,58 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.physical.impl;
+
+import org.apache.drill.exec.record.BatchSchema;
+import org.apache.drill.exec.record.RecordBatch;
+import org.apache.drill.exec.record.vector.SelectionVector;
+
+public abstract class FilteringRecordBatchTransformer {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(FilteringRecordBatchTransformer.class);
+  
+  final RecordBatch incoming;
+  final SelectionVector selectionVector;
+  final BatchSchema schema;
+  
+  public FilteringRecordBatchTransformer(RecordBatch incoming, OutputMutator output, SelectionVector selectionVector) {
+    super();
+    this.incoming = incoming;
+    this.selectionVector = selectionVector;
+    this.schema = innerSetup();
+  }
+
+  public abstract BatchSchema innerSetup();
+  
+  /**
+   * Applies the filter to the selection index.  Ignores any values in the selection vector, instead creating a.
+   * @return
+   */
+  public abstract int apply();
+  
+  /**
+   * Applies the filter to the selection index.  Utilizes the existing selection index and only evaluates on those records.
+   * @return
+   */
+  public abstract int applyWithSelection();
+
+  public BatchSchema getSchema() {
+    return schema;
+  }
+  
+  
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/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
new file mode 100644
index 0000000..d98c107
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ImplCreator.java
@@ -0,0 +1,102 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.physical.impl;
+
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.drill.common.exceptions.ExecutionSetupException;
+import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.physical.base.AbstractPhysicalVisitor;
+import org.apache.drill.exec.physical.base.FragmentRoot;
+import org.apache.drill.exec.physical.base.PhysicalOperator;
+import org.apache.drill.exec.physical.base.Scan;
+import org.apache.drill.exec.physical.config.MockScanBatchCreator;
+import org.apache.drill.exec.physical.config.MockScanPOP;
+import org.apache.drill.exec.physical.config.RandomReceiver;
+import org.apache.drill.exec.physical.config.Screen;
+import org.apache.drill.exec.physical.config.SingleSender;
+import org.apache.drill.exec.record.RecordBatch;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+
+public class ImplCreator extends AbstractPhysicalVisitor<RecordBatch, FragmentContext, ExecutionSetupException>{
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ImplCreator.class);
+
+  private MockScanBatchCreator msc = new MockScanBatchCreator();
+  private ScreenCreator sc = new ScreenCreator();
+  private RandomReceiverCreator rrc = new RandomReceiverCreator();
+  private SingleSenderCreator ssc = new SingleSenderCreator();
+  private RootExec root = null;
+  
+  private ImplCreator(){}
+  
+  public RootExec getRoot(){
+    return root;
+  }
+  
+  
+  @Override
+  public RecordBatch visitScan(Scan<?> scan, FragmentContext context) throws ExecutionSetupException {
+    Preconditions.checkNotNull(scan);
+    Preconditions.checkNotNull(context);
+    
+    if(scan instanceof MockScanPOP){
+      return msc.getBatch(context, (MockScanPOP) scan, Collections.<RecordBatch> emptyList());
+    }else{
+      return super.visitScan(scan, context);  
+    }
+    
+  }
+
+  @Override
+  public RecordBatch visitScreen(Screen op, FragmentContext context) throws ExecutionSetupException {
+    Preconditions.checkArgument(root == null);
+    root = sc.getRoot(context, op, getChildren(op, context));
+    return null;
+  }
+
+  
+  
+  @Override
+  public RecordBatch visitSingleSender(SingleSender op, FragmentContext context) throws ExecutionSetupException {
+    root = ssc.getRoot(context, op, getChildren(op, context));
+    return null;
+  }
+
+  @Override
+  public RecordBatch visitRandomReceiver(RandomReceiver op, FragmentContext context) throws ExecutionSetupException {
+    return rrc.getBatch(context, op, null);
+  }
+
+  private List<RecordBatch> getChildren(PhysicalOperator op, FragmentContext context) throws ExecutionSetupException{
+    List<RecordBatch> children = Lists.newArrayList();
+    for(PhysicalOperator child : op){
+      children.add(child.accept(this, context));
+    }
+    return children;
+  }
+  
+  public static RootExec getExec(FragmentContext context, FragmentRoot root) throws ExecutionSetupException{
+    ImplCreator i = new ImplCreator();
+    root.accept(i, context);
+    if(i.root == null) throw new ExecutionSetupException("The provided fragment did not have a root node that correctly created a RootExec value.");
+    return i.getRoot();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/OutputMutator.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/OutputMutator.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/OutputMutator.java
new file mode 100644
index 0000000..ce0cf66
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/OutputMutator.java
@@ -0,0 +1,28 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.physical.impl;
+
+import org.apache.drill.exec.exception.SchemaChangeException;
+import org.apache.drill.exec.record.BatchSchema;
+import org.apache.drill.exec.record.vector.ValueVector;
+
+public interface OutputMutator {
+  public void removeField(int fieldId) throws SchemaChangeException;
+  public void addField(int fieldId, ValueVector<?> vector) throws SchemaChangeException ;
+  public void setNewSchema() throws SchemaChangeException ;
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/PhysicalConfig.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/PhysicalConfig.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/PhysicalConfig.java
new file mode 100644
index 0000000..9995bc2
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/PhysicalConfig.java
@@ -0,0 +1,29 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.physical.impl;
+
+import java.lang.annotation.ElementType;
+import java.lang.annotation.Retention;
+import java.lang.annotation.RetentionPolicy;
+import java.lang.annotation.Target;
+
+@Target({ElementType.TYPE})
+@Retention(RetentionPolicy.RUNTIME)
+public @interface PhysicalConfig {
+  Class<?> value();
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/RandomReceiverCreator.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/RandomReceiverCreator.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/RandomReceiverCreator.java
new file mode 100644
index 0000000..4b991f8
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/RandomReceiverCreator.java
@@ -0,0 +1,46 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.physical.impl;
+
+import java.util.List;
+
+import org.apache.drill.common.exceptions.ExecutionSetupException;
+import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.physical.config.RandomReceiver;
+import org.apache.drill.exec.record.RecordBatch;
+import org.apache.drill.exec.work.batch.IncomingBuffers;
+import org.apache.drill.exec.work.batch.RawBatchBuffer;
+
+public class RandomReceiverCreator implements BatchCreator<RandomReceiver>{
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(RandomReceiverCreator.class);
+
+  @Override
+  public RecordBatch getBatch(FragmentContext context, RandomReceiver receiver, List<RecordBatch> children)
+      throws ExecutionSetupException {
+    assert children == null || children.isEmpty();
+    IncomingBuffers bufHolder = context.getBuffers();
+    assert bufHolder != null : "IncomingBuffers must be defined for any place a receiver is declared.";
+    
+    RawBatchBuffer[] buffers = bufHolder.getBuffers(receiver.getOppositeMajorFragmentId());
+    assert buffers.length == 1;
+    RawBatchBuffer buffer = buffers[0];
+    return new WireRecordBatch(context, buffer);
+  }
+  
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/RootCreator.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/RootCreator.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/RootCreator.java
new file mode 100644
index 0000000..80def05
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/RootCreator.java
@@ -0,0 +1,31 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.physical.impl;
+
+import java.util.List;
+
+import org.apache.drill.common.exceptions.ExecutionSetupException;
+import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.physical.base.PhysicalOperator;
+import org.apache.drill.exec.record.RecordBatch;
+
+public interface RootCreator<T extends PhysicalOperator> {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(RootCreator.class);
+  
+  public RootExec getRoot(FragmentContext context, T config, List<RecordBatch> children) throws ExecutionSetupException;
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/RootExec.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/RootExec.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/RootExec.java
new file mode 100644
index 0000000..3f8aac7
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/RootExec.java
@@ -0,0 +1,40 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.physical.impl;
+
+import org.apache.drill.exec.exception.FragmentSetupException;
+
+/**
+ * A FragmentRoot is a node which is the last processing node in a query plan. FragmentTerminals include Exchange
+ * output nodes and storage nodes.  They are there driving force behind the completion of a query.
+ */
+public interface RootExec {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(RootExec.class);
+  
+  /**
+   * Do the next batch of work.  
+   * @return Whether or not additional batches of work are necessary.  False means that this fragment is done.
+   */
+  public boolean next();
+  
+  /**
+   * Inform all children to clean up and go away.
+   */
+  public void stop();
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScanBatch.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScanBatch.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScanBatch.java
new file mode 100644
index 0000000..33c1e29
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScanBatch.java
@@ -0,0 +1,172 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.physical.impl;
+
+import io.netty.buffer.ByteBuf;
+
+import java.util.Iterator;
+import java.util.List;
+
+import org.apache.drill.common.exceptions.ExecutionSetupException;
+import org.apache.drill.exec.exception.SchemaChangeException;
+import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.proto.UserBitShared.FieldMetadata;
+import org.apache.drill.exec.proto.UserBitShared.RecordBatchDef;
+import org.apache.drill.exec.record.BatchSchema;
+import org.apache.drill.exec.record.InvalidValueAccessor;
+import org.apache.drill.exec.record.RecordBatch;
+import org.apache.drill.exec.record.SchemaBuilder;
+import org.apache.drill.exec.record.WritableBatch;
+import org.apache.drill.exec.record.vector.ValueVector;
+import org.apache.drill.exec.store.RecordReader;
+
+import com.carrotsearch.hppc.IntObjectOpenHashMap;
+import com.carrotsearch.hppc.procedures.IntObjectProcedure;
+import com.google.common.collect.Lists;
+
+/**
+ * Record batch used for a particular scan. Operators against one or more
+ */
+public class ScanBatch implements RecordBatch {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ScanBatch.class);
+
+  private IntObjectOpenHashMap<ValueVector<?>> fields = new IntObjectOpenHashMap<ValueVector<?>>();
+  private BatchSchema schema;
+  private int recordCount;
+  private boolean schemaChanged = true;
+  private final FragmentContext context;
+  private Iterator<RecordReader> readers;
+  private RecordReader currentReader;
+  private final Mutator mutator = new Mutator();
+
+  public ScanBatch(FragmentContext context, Iterator<RecordReader> readers)
+      throws ExecutionSetupException {
+    this.context = context;
+    this.readers = readers;
+    if (!readers.hasNext()) throw new ExecutionSetupException("A scan batch must contain at least one reader.");
+    this.currentReader = readers.next();
+    this.currentReader.setup(mutator);
+  }
+
+  private void schemaChanged() {
+    schema = null;
+    schemaChanged = true;
+  }
+
+  @Override
+  public FragmentContext getContext() {
+    return context;
+  }
+
+  @Override
+  public BatchSchema getSchema() {
+    return schema;
+  }
+
+  @Override
+  public int getRecordCount() {
+    return recordCount;
+  }
+
+  @Override
+  public void kill() {
+    releaseAssets();
+  }
+
+  private void releaseAssets() {
+    fields.forEach(new IntObjectProcedure<ValueVector<?>>() {
+      @Override
+      public void apply(int key, ValueVector<?> value) {
+        value.close();
+      }
+    });
+  }
+
+  @SuppressWarnings("unchecked")
+  @Override
+  public <T extends ValueVector<T>> T getValueVector(int fieldId, Class<T> clazz) throws InvalidValueAccessor {
+    if (fields.containsKey(fieldId)) throw new InvalidValueAccessor(String.format("Unknown value accesor for field id %d."));
+    ValueVector<?> vector = this.fields.lget();
+    if (vector.getClass().isAssignableFrom(clazz)) {
+      return (T) vector;
+    } else {
+      throw new InvalidValueAccessor(String.format(
+          "You requested a field accessor of type %s for field id %d but the actual type was %s.",
+          clazz.getCanonicalName(), fieldId, vector.getClass().getCanonicalName()));
+    }
+  }
+
+  @Override
+  public IterOutcome next() {
+    while ((recordCount = currentReader.next()) == 0) {
+      try {
+        if (!readers.hasNext()) {
+          currentReader.cleanup();
+          releaseAssets();
+          return IterOutcome.NONE;
+        }
+        currentReader.cleanup();
+        currentReader = readers.next();
+        currentReader.setup(mutator);
+      } catch (ExecutionSetupException e) {
+        this.context.fail(e);
+        releaseAssets();
+        return IterOutcome.STOP;
+      }
+    }
+
+    if (schemaChanged) {
+      schemaChanged = false;
+      return IterOutcome.OK_NEW_SCHEMA;
+    } else {
+      return IterOutcome.OK;
+    }
+  }
+
+  private class Mutator implements OutputMutator {
+    private SchemaBuilder builder = BatchSchema.newBuilder();
+    
+    public void removeField(int fieldId) throws SchemaChangeException {
+      schemaChanged();
+      ValueVector<?> v = fields.remove(fieldId);
+      if (v == null) throw new SchemaChangeException("Failure attempting to remove an unknown field.");
+      v.close();
+    }
+
+    public void addField(int fieldId, ValueVector<?> vector) {
+      schemaChanged();
+      ValueVector<?> v = fields.put(fieldId, vector);
+      vector.getField();
+      builder.addField(vector.getField());
+      if (v != null) v.close();
+    }
+
+    @Override
+    public void setNewSchema() throws SchemaChangeException {
+      ScanBatch.this.schema = this.builder.build();
+      ScanBatch.this.schemaChanged = true;
+    }
+
+  }
+
+  @Override
+  public WritableBatch getWritableBatch() {
+    return WritableBatch.get(this.getRecordCount(), fields);
+  }
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/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
new file mode 100644
index 0000000..c0711db
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScreenCreator.java
@@ -0,0 +1,90 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.physical.impl;
+
+import java.util.List;
+
+import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.physical.config.Screen;
+import org.apache.drill.exec.physical.impl.materialize.RecordMaterializer;
+import org.apache.drill.exec.physical.impl.materialize.VectorRecordMaterializer;
+import org.apache.drill.exec.record.RecordBatch;
+import org.apache.drill.exec.record.RecordBatch.IterOutcome;
+import org.apache.drill.exec.rpc.user.UserServer.UserClientConnection;
+
+import com.google.common.base.Preconditions;
+
+public class ScreenCreator implements RootCreator<Screen>{
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ScreenCreator.class);
+
+  @Override
+  public RootExec getRoot(FragmentContext context, Screen config, List<RecordBatch> children) {
+    Preconditions.checkArgument(children.size() == 1);
+    return new ScreenRoot(context, children.iterator().next());
+  }
+  
+  
+  private static class ScreenRoot implements RootExec{
+
+    final RecordBatch incoming;
+    final FragmentContext context;
+    final UserClientConnection connection;
+    private RecordMaterializer materializer;
+    
+    public ScreenRoot(FragmentContext context, RecordBatch incoming){
+      assert context.getConnection() != null : "A screen root should only be run on the driving node which is connected directly to the client.  As such, this should always be true.";
+
+      this.context = context;
+      this.incoming = incoming;
+      this.connection = context.getConnection();
+    }
+    
+    @Override
+    public boolean next() {
+      IterOutcome outcome = incoming.next();
+      boolean isLast = false;
+      switch(outcome){
+      case NONE:
+      case STOP:
+        connection.sendResult(materializer.convertNext(true));
+        context.batchesCompleted.inc(1);
+        context.recordsCompleted.inc(incoming.getRecordCount());
+        return false;
+        
+      case OK_NEW_SCHEMA:
+        materializer = new VectorRecordMaterializer(context, incoming);
+        // fall through.
+        // fall through
+      case OK:
+        connection.sendResult(materializer.convertNext(false));
+        context.batchesCompleted.inc(1);
+        context.recordsCompleted.inc(incoming.getRecordCount());
+        return !isLast;
+      default:
+        throw new UnsupportedOperationException();
+      }
+    }
+
+    @Override
+    public void stop() {
+      incoming.kill();
+    }
+
+    
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/SingleSenderCreator.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/SingleSenderCreator.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/SingleSenderCreator.java
new file mode 100644
index 0000000..60c2d78
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/SingleSenderCreator.java
@@ -0,0 +1,89 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.physical.impl;
+
+import java.util.List;
+
+import org.apache.drill.common.exceptions.ExecutionSetupException;
+import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.physical.config.SingleSender;
+import org.apache.drill.exec.proto.ExecProtos.FragmentHandle;
+import org.apache.drill.exec.record.FragmentWritableBatch;
+import org.apache.drill.exec.record.RecordBatch;
+import org.apache.drill.exec.record.RecordBatch.IterOutcome;
+import org.apache.drill.exec.rpc.bit.BitTunnel;
+
+public class SingleSenderCreator implements RootCreator<SingleSender>{
+
+  @Override
+  public RootExec getRoot(FragmentContext context, SingleSender config, List<RecordBatch> children)
+      throws ExecutionSetupException {
+    assert children != null && children.size() == 1;
+    return new SingleSenderRootExec(context, children.iterator().next(), config);
+  }
+  
+  
+  private static class SingleSenderRootExec implements RootExec{
+    static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(SingleSenderRootExec.class);
+    private RecordBatch incoming;
+    private BitTunnel tunnel;
+    private FragmentHandle handle;
+    private int recMajor;
+    private FragmentContext context;
+    
+    public SingleSenderRootExec(FragmentContext context, RecordBatch batch, SingleSender config){
+      logger.debug("Creating single sender root exec base on config: {}", config);
+      this.incoming = batch;
+      this.handle = context.getHandle();
+      this.recMajor = config.getOppositeMajorFragmentId();
+      this.tunnel = context.getCommunicator().getTunnel(config.getDestination());
+      this.context = context;
+    }
+    
+    @Override
+    public boolean next() {
+      IterOutcome out = incoming.next();
+      logger.debug("Outcome of sender next {}", out);
+      switch(out){
+      case STOP:
+      case NONE:
+        FragmentWritableBatch b2 = new FragmentWritableBatch(false, handle.getQueryId(), handle.getMajorFragmentId(), handle.getMinorFragmentId(), recMajor, 0, incoming.getWritableBatch());
+        tunnel.sendRecordBatch(context, b2);
+        return false;
+        
+
+      case OK:
+      case OK_NEW_SCHEMA:
+        FragmentWritableBatch batch = new FragmentWritableBatch(true, handle.getQueryId(), handle.getMajorFragmentId(), handle.getMinorFragmentId(), recMajor, 0, incoming.getWritableBatch());
+        tunnel.sendRecordBatch(context, batch);
+        return true;
+
+      case NOT_YET:
+      default:
+        throw new IllegalStateException();
+      }
+    }
+
+    @Override
+    public void stop() {
+    }
+    
+    
+    
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/WireRecordBatch.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/WireRecordBatch.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/WireRecordBatch.java
new file mode 100644
index 0000000..fc7f833
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/WireRecordBatch.java
@@ -0,0 +1,99 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.physical.impl;
+
+import java.util.Iterator;
+
+import org.apache.drill.exec.exception.SchemaChangeException;
+import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.proto.UserBitShared.RecordBatchDef;
+import org.apache.drill.exec.record.BatchSchema;
+import org.apache.drill.exec.record.InvalidValueAccessor;
+import org.apache.drill.exec.record.RawFragmentBatch;
+import org.apache.drill.exec.record.RawFragmentBatchProvider;
+import org.apache.drill.exec.record.RecordBatch;
+import org.apache.drill.exec.record.RecordBatchLoader;
+import org.apache.drill.exec.record.WritableBatch;
+import org.apache.drill.exec.record.RecordBatch.IterOutcome;
+import org.apache.drill.exec.record.vector.ValueVector;
+
+public class WireRecordBatch implements RecordBatch{
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(WireRecordBatch.class);
+
+  private RecordBatchLoader batchLoader;
+  private RawFragmentBatchProvider fragProvider;
+  private FragmentContext context;
+
+  
+  public WireRecordBatch(FragmentContext context, RawFragmentBatchProvider fragProvider) {
+    this.fragProvider = fragProvider;
+    this.context = context;
+    this.batchLoader = new RecordBatchLoader(context.getAllocator());
+  }
+
+  @Override
+  public FragmentContext getContext() {
+    return context;
+  }
+
+  @Override
+  public BatchSchema getSchema() {
+    return null;
+  }
+
+  @Override
+  public int getRecordCount() {
+    return batchLoader.getRecordCount();
+  }
+
+  @Override
+  public void kill() {
+    fragProvider.kill(context);
+  }
+
+  @Override
+  public <T extends ValueVector<T>> T getValueVector(int fieldId, Class<T> clazz) throws InvalidValueAccessor {
+    return batchLoader.getValueVector(fieldId, clazz);
+  }
+
+  @Override
+  public IterOutcome next() {
+    RawFragmentBatch batch = this.fragProvider.getNext();
+    try{
+      if(batch == null) return IterOutcome.NONE;
+
+      RecordBatchDef rbd = batch.getHeader().getDef();
+      boolean schemaChanged = batchLoader.load(rbd, batch.getBody());
+      if(schemaChanged){
+        return IterOutcome.OK_NEW_SCHEMA;
+      }else{
+        return IterOutcome.OK;
+      }
+    }catch(SchemaChangeException ex){
+      context.fail(ex);
+      return IterOutcome.STOP;
+    }
+  }
+
+  @Override
+  public WritableBatch getWritableBatch() {
+    return batchLoader.getWritableBatch();
+  }
+  
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/materialize/QueryWritableBatch.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/materialize/QueryWritableBatch.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/materialize/QueryWritableBatch.java
new file mode 100644
index 0000000..187e6e9
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/materialize/QueryWritableBatch.java
@@ -0,0 +1,46 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.physical.impl.materialize;
+
+import io.netty.buffer.ByteBuf;
+
+import org.apache.drill.exec.proto.UserProtos.QueryResult;
+import org.apache.drill.exec.record.WritableBatch;
+
+public class QueryWritableBatch {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(QueryWritableBatch.class);
+  
+  private final QueryResult header;
+  private final ByteBuf[] buffers;
+  
+  
+  public QueryWritableBatch(QueryResult header, ByteBuf... buffers) {
+    super();
+    this.header = header;
+    this.buffers = buffers;
+  }
+
+  public ByteBuf[] getBuffers(){
+    return buffers;
+  }
+
+  public QueryResult getHeader() {
+    return header;
+  }
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/materialize/RecordMaterializer.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/materialize/RecordMaterializer.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/materialize/RecordMaterializer.java
new file mode 100644
index 0000000..17c65e9
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/materialize/RecordMaterializer.java
@@ -0,0 +1,31 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.physical.impl.materialize;
+
+import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.proto.UserBitShared.QueryId;
+import org.apache.drill.exec.proto.UserProtos.QueryResult;
+import org.apache.drill.exec.record.MaterializedField;
+import org.apache.drill.exec.record.RecordBatch;
+import org.apache.drill.exec.record.WritableBatch;
+
+public interface RecordMaterializer {
+  
+  public QueryWritableBatch convertNext(boolean isLast);
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/materialize/VectorRecordMaterializer.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/materialize/VectorRecordMaterializer.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/materialize/VectorRecordMaterializer.java
new file mode 100644
index 0000000..e2d2eb9
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/materialize/VectorRecordMaterializer.java
@@ -0,0 +1,52 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.physical.impl.materialize;
+
+import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.proto.UserBitShared.QueryId;
+import org.apache.drill.exec.proto.UserProtos.QueryResult;
+import org.apache.drill.exec.record.MaterializedField;
+import org.apache.drill.exec.record.RecordBatch;
+import org.apache.drill.exec.record.WritableBatch;
+
+public class VectorRecordMaterializer implements RecordMaterializer{
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(VectorRecordMaterializer.class);
+
+  private QueryId queryId;
+  private RecordBatch batch;
+
+  public VectorRecordMaterializer(FragmentContext context, RecordBatch batch) {
+    this.queryId = context.getHandle().getQueryId();
+    this.batch = batch;
+
+    for (MaterializedField f : batch.getSchema()) {
+      logger.debug("New Field: {}", f);
+    }
+  }
+
+  public QueryWritableBatch convertNext(boolean isLast) {
+    WritableBatch w = batch.getWritableBatch();
+
+    QueryResult header = QueryResult.newBuilder() //
+        .setQueryId(queryId) //
+        .setRowCount(batch.getRecordCount()) //
+        .setDef(w.getDef()).setIsLastChunk(isLast).build();
+    QueryWritableBatch batch = new QueryWritableBatch(header, w.getBuffers());
+    return batch;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/AbstractOpWrapperVisitor.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/AbstractOpWrapperVisitor.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/AbstractOpWrapperVisitor.java
new file mode 100644
index 0000000..9b2cb85
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/AbstractOpWrapperVisitor.java
@@ -0,0 +1,45 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.planner;
+
+import org.apache.drill.exec.physical.base.AbstractPhysicalVisitor;
+import org.apache.drill.exec.physical.base.Exchange;
+import org.apache.drill.exec.planner.fragment.Wrapper;
+
+public abstract class AbstractOpWrapperVisitor<RET, EXCEP extends Throwable> extends
+    AbstractPhysicalVisitor<RET, Wrapper, EXCEP> {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(AbstractOpWrapperVisitor.class);
+
+  @Override
+  public final RET visitExchange(Exchange exchange, Wrapper wrapper) throws EXCEP {
+    if (wrapper.getNode().getSendingExchange() == exchange) {
+      return visitSendingExchange(exchange, wrapper);
+    } else {
+      return visitReceivingExchange(exchange, wrapper);
+    }
+  }
+
+  public RET visitSendingExchange(Exchange exchange, Wrapper wrapper) throws EXCEP {
+    return visitOp(exchange, wrapper);
+  }
+
+  public RET visitReceivingExchange(Exchange exchange, Wrapper wrapper) throws EXCEP {
+    return visitOp(exchange, wrapper);
+  }
+
+}


[19/53] [abbrv] git commit: Update typing system. Update RPC system. Add Fragmenting Implementation. Working single node. Distributed failing due to threading issues.

Posted by ja...@apache.org.
Update typing system.  Update RPC system.  Add Fragmenting Implementation.  Working single node. Distributed failing due to threading issues.


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

Branch: refs/heads/master
Commit: e57a8d6d4ae282a79bc6e0a7354de992c391300f
Parents: f0be80d
Author: Jacques Nadeau <ja...@apache.org>
Authored: Sun May 5 20:18:32 2013 -0700
Committer: Jacques Nadeau <ja...@apache.org>
Committed: Mon May 13 18:50:31 2013 -0700

----------------------------------------------------------------------
 sandbox/prototype/common/pom.xml                |  12 +
 .../apache/drill/common/config/DrillConfig.java |   9 +-
 .../exceptions/ExecutionSetupException.java     |  44 +++
 .../PhysicalOperatorSetupException.java         |  45 +++
 .../drill/common/expression/SchemaPath.java     |   2 +-
 .../drill/common/expression/types/DataType.java |  11 +
 .../apache/drill/common/optimize/Optimizer.java |  45 ---
 .../common/physical/DataValidationMode.java     |  24 --
 .../drill/common/physical/EndpointAffinity.java |  60 ----
 .../apache/drill/common/physical/FieldSet.java  |  80 ------
 .../drill/common/physical/OperatorCost.java     |  61 -----
 .../drill/common/physical/PhysicalPlan.java     |  95 -------
 .../apache/drill/common/physical/ReadEntry.java |  25 --
 .../drill/common/physical/RecordField.java      |  60 ----
 .../drill/common/physical/WriteEntry.java       |  22 --
 .../drill/common/physical/pop/Filter.java       |  52 ----
 .../physical/pop/PartitionToRandomExchange.java |  92 -------
 .../drill/common/physical/pop/Project.java      |  53 ----
 .../drill/common/physical/pop/Screen.java       |  77 ------
 .../apache/drill/common/physical/pop/Sort.java  |  57 ----
 .../common/physical/pop/base/AbstractBase.java  |  56 ----
 .../physical/pop/base/AbstractExchange.java     |  68 -----
 .../pop/base/AbstractPhysicalVisitor.java       |  80 ------
 .../physical/pop/base/AbstractReceiver.java     |  32 ---
 .../common/physical/pop/base/AbstractScan.java  |  62 -----
 .../physical/pop/base/AbstractSender.java       |  29 --
 .../physical/pop/base/AbstractSingle.java       |  48 ----
 .../common/physical/pop/base/AbstractStore.java |  42 ---
 .../common/physical/pop/base/Exchange.java      |  69 -----
 .../common/physical/pop/base/ExchangeCost.java  |  55 ----
 .../common/physical/pop/base/FragmentLeaf.java  |  25 --
 .../common/physical/pop/base/FragmentRoot.java  |  25 --
 .../common/physical/pop/base/HasAffinity.java   |  26 --
 .../drill/common/physical/pop/base/Leaf.java    |  21 --
 .../physical/pop/base/PhysicalOperator.java     |  59 ----
 .../physical/pop/base/PhysicalOperatorUtil.java |  34 ---
 .../physical/pop/base/PhysicalVisitor.java      |  43 ---
 .../common/physical/pop/base/Receiver.java      |  38 ---
 .../drill/common/physical/pop/base/Root.java    |  24 --
 .../drill/common/physical/pop/base/Scan.java    |  36 ---
 .../drill/common/physical/pop/base/Sender.java  |  29 --
 .../drill/common/physical/pop/base/Store.java   |  30 --
 .../common/src/main/protobuf/Coordination.proto |  26 --
 .../drill/common/physical/MockScanPOP.java      |  69 -----
 .../drill/common/physical/MockStorePOP.java     |  62 -----
 .../common/physical/ParsePhysicalPlan.java      |  41 ---
 .../common/src/test/resources/drill-module.conf |   1 -
 .../src/test/resources/physical_test1.json      |  33 ---
 sandbox/prototype/exec/java-exec/pom.xml        |   4 +-
 .../java/org/apache/drill/exec/ByteReorder.java |  54 ----
 .../drill/exec/cache/DistributedCache.java      |  14 +-
 .../org/apache/drill/exec/cache/HazelCache.java |  87 ++----
 .../org/apache/drill/exec/cache/LocalCache.java |  55 ++++
 .../apache/drill/exec/cache/ProtoBufImpl.java   |  50 ++++
 .../apache/drill/exec/cache/ProtoBufWrap.java   |  69 +++++
 .../org/apache/drill/exec/cache/ProtoMap.java   |  52 ++++
 .../exec/cache/TemplatizedLogicalPlan.java      |  22 --
 .../exec/cache/TemplatizedPhysicalPlan.java     |  22 --
 .../apache/drill/exec/client/DrillClient.java   |  73 ++++-
 .../drill/exec/coord/ClusterCoordinator.java    |   9 +-
 .../exec/coord/DrillServiceInstanceHelper.java  |   4 +-
 .../drill/exec/coord/DrillbitEndpointSerDe.java |  65 +++++
 .../exec/coord/LocalClusterCoordinator.java     |  95 +++++++
 .../drill/exec/coord/ZKClusterCoordinator.java  |  30 +-
 .../drill/exec/exception/BitComException.java   |  45 +++
 .../exec/exception/ExecutionSetupException.java |  45 ---
 .../exec/exception/FragmentSetupException.java  |   2 +
 .../drill/exec/foreman/CancelableQuery.java     |  22 --
 .../drill/exec/foreman/ExecutionPlanner.java    |  24 --
 .../org/apache/drill/exec/foreman/Foreman.java  |  39 ---
 .../drill/exec/foreman/QueryWorkUnit.java       |  54 ----
 .../drill/exec/foreman/ResourceRequest.java     |  30 --
 .../drill/exec/foreman/StatusProvider.java      |  24 --
 .../drill/exec/memory/BufferAllocator.java      |   5 +-
 .../exec/metrics/SingleThreadNestedCounter.java |  22 +-
 .../ops/FilteringRecordBatchTransformer.java    |  58 ----
 .../apache/drill/exec/ops/FragmentContext.java  |  54 +++-
 .../drill/exec/ops/FragmentConverter.java       |  30 --
 .../org/apache/drill/exec/ops/FragmentRoot.java |  37 ---
 .../apache/drill/exec/ops/OperatorFactory.java  |  22 --
 .../apache/drill/exec/ops/OutputMutator.java    |  28 --
 .../org/apache/drill/exec/ops/QueryContext.java |  27 +-
 .../org/apache/drill/exec/ops/ScanBatch.java    | 157 -----------
 .../drill/exec/ops/StreamingRecordBatch.java    |  25 --
 .../exec/ops/exchange/ExchangeRecordBatch.java  |  22 --
 .../exec/ops/exchange/RecordBatchSender.java    |  24 --
 .../exec/ops/filter/FilterRecordBatch.java      | 109 --------
 .../exec/ops/filter/SelectionVectorUpdater.java |  80 ------
 .../drill/exec/opt/IdentityOptimizer.java       |   3 +-
 .../org/apache/drill/exec/opt/Optimizer.java    |  45 +++
 .../drill/exec/physical/DataValidationMode.java |  24 ++
 .../drill/exec/physical/EndpointAffinity.java   |  60 ++++
 .../drill/exec/physical/OperatorCost.java       |  66 +++++
 .../drill/exec/physical/PhysicalPlan.java       |  94 +++++++
 .../apache/drill/exec/physical/ReadEntry.java   |  33 +++
 .../apache/drill/exec/physical/RecordField.java |  60 ++++
 .../apache/drill/exec/physical/WriteEntry.java  |  22 ++
 .../drill/exec/physical/base/AbstractBase.java  |  43 +++
 .../exec/physical/base/AbstractExchange.java    |  90 ++++++
 .../physical/base/AbstractPhysicalVisitor.java  | 124 +++++++++
 .../exec/physical/base/AbstractReceiver.java    |  63 +++++
 .../drill/exec/physical/base/AbstractScan.java  |  84 ++++++
 .../exec/physical/base/AbstractSender.java      |  53 ++++
 .../exec/physical/base/AbstractSingle.java      |  68 +++++
 .../drill/exec/physical/base/AbstractStore.java |  36 +++
 .../drill/exec/physical/base/Exchange.java      |  92 +++++++
 .../drill/exec/physical/base/ExchangeCost.java  |  68 +++++
 .../drill/exec/physical/base/FragmentLeaf.java  |  25 ++
 .../drill/exec/physical/base/FragmentRoot.java  |  25 ++
 .../drill/exec/physical/base/HasAffinity.java   |  37 +++
 .../apache/drill/exec/physical/base/Leaf.java   |  25 ++
 .../exec/physical/base/PhysicalOperator.java    |  80 ++++++
 .../physical/base/PhysicalOperatorUtil.java     |  34 +++
 .../exec/physical/base/PhysicalVisitor.java     |  61 +++++
 .../drill/exec/physical/base/Receiver.java      |  51 ++++
 .../apache/drill/exec/physical/base/Root.java   |  24 ++
 .../apache/drill/exec/physical/base/Scan.java   |  36 +++
 .../apache/drill/exec/physical/base/Sender.java |  44 +++
 .../apache/drill/exec/physical/base/Size.java   |  48 ++++
 .../apache/drill/exec/physical/base/Store.java  |  74 +++++
 .../drill/exec/physical/config/Filter.java      |  75 +++++
 .../physical/config/HashPartitionSender.java    |  58 ++++
 .../physical/config/HashToRandomExchange.java   |  86 ++++++
 .../exec/physical/config/MockRecordReader.java  | 113 ++++++++
 .../physical/config/MockScanBatchCreator.java   |  49 ++++
 .../drill/exec/physical/config/MockScanPOP.java | 193 +++++++++++++
 .../exec/physical/config/MockStorageEngine.java |  58 ++++
 .../exec/physical/config/MockStorePOP.java      |  75 +++++
 .../exec/physical/config/PartitionRange.java    |  47 ++++
 .../drill/exec/physical/config/Project.java     |  72 +++++
 .../exec/physical/config/RandomReceiver.java    |  83 ++++++
 .../drill/exec/physical/config/RangeSender.java |  72 +++++
 .../drill/exec/physical/config/Screen.java      | 106 ++++++++
 .../exec/physical/config/SingleSender.java      |  78 ++++++
 .../apache/drill/exec/physical/config/Sort.java |  86 ++++++
 .../exec/physical/config/UnionExchange.java     |  79 ++++++
 .../drill/exec/physical/impl/BatchCreator.java  |  31 +++
 .../exec/physical/impl/FilterRecordBatch.java   | 108 ++++++++
 .../impl/FilteringRecordBatchTransformer.java   |  58 ++++
 .../drill/exec/physical/impl/ImplCreator.java   | 102 +++++++
 .../drill/exec/physical/impl/OutputMutator.java |  28 ++
 .../exec/physical/impl/PhysicalConfig.java      |  29 ++
 .../physical/impl/RandomReceiverCreator.java    |  46 ++++
 .../drill/exec/physical/impl/RootCreator.java   |  31 +++
 .../drill/exec/physical/impl/RootExec.java      |  40 +++
 .../drill/exec/physical/impl/ScanBatch.java     | 172 ++++++++++++
 .../drill/exec/physical/impl/ScreenCreator.java |  90 ++++++
 .../exec/physical/impl/SingleSenderCreator.java |  89 ++++++
 .../exec/physical/impl/WireRecordBatch.java     |  99 +++++++
 .../impl/materialize/QueryWritableBatch.java    |  46 ++++
 .../impl/materialize/RecordMaterializer.java    |  31 +++
 .../materialize/VectorRecordMaterializer.java   |  52 ++++
 .../exec/planner/AbstractOpWrapperVisitor.java  |  45 +++
 .../apache/drill/exec/planner/ExecPlanner.java  |   8 +-
 .../exec/planner/FragmentMaterializer.java      |  86 ------
 .../apache/drill/exec/planner/FragmentNode.java | 138 ----------
 .../drill/exec/planner/FragmentPlanningSet.java |  61 -----
 .../drill/exec/planner/FragmentRunnable.java    | 124 ---------
 .../drill/exec/planner/FragmentScheduler.java   |  32 ---
 .../drill/exec/planner/FragmentStats.java       |  63 -----
 .../exec/planner/FragmentStatsCollector.java    | 109 --------
 .../drill/exec/planner/FragmentVisitor.java     |  22 --
 .../drill/exec/planner/FragmentWrapper.java     | 127 ---------
 .../planner/FragmentingPhysicalVisitor.java     |  71 -----
 .../exec/planner/MaterializedFragment.java      |  69 -----
 .../drill/exec/planner/PhysicalPlanReader.java  |  63 ++++-
 .../apache/drill/exec/planner/ScanFinder.java   |  54 ----
 .../drill/exec/planner/SimpleExecPlanner.java   |  28 +-
 .../drill/exec/planner/SimpleParallelizer.java  | 147 ----------
 .../drill/exec/planner/fragment/Fragment.java   | 150 ++++++++++
 .../exec/planner/fragment/FragmentVisitor.java  |  23 ++
 .../planner/fragment/MakeFragmentsVisitor.java  |  69 +++++
 .../planner/fragment/MaterializedFragment.java  |  69 +++++
 .../exec/planner/fragment/Materializer.java     | 107 ++++++++
 .../exec/planner/fragment/PlanningSet.java      |  66 +++++
 .../planner/fragment/SimpleParallelizer.java    | 163 +++++++++++
 .../drill/exec/planner/fragment/Stats.java      |  70 +++++
 .../exec/planner/fragment/StatsCollector.java   | 106 ++++++++
 .../drill/exec/planner/fragment/Wrapper.java    | 186 +++++++++++++
 .../exec/pop/receiver/NWayOrderingReceiver.java |  52 ----
 .../drill/exec/pop/receiver/RandomReceiver.java |  55 ----
 .../exec/pop/sender/HashPartitionSender.java    |  49 ----
 .../apache/drill/exec/record/BatchSchema.java   | 102 +------
 .../exec/record/FragmentWritableBatch.java      |  59 ++++
 .../drill/exec/record/InvalidValueAccessor.java |   2 +-
 .../drill/exec/record/MajorTypeSerDe.java       | 116 ++++++++
 .../drill/exec/record/MaterializedField.java    | 167 +++++++++---
 .../drill/exec/record/RawFragmentBatch.java     |  44 +++
 .../exec/record/RawFragmentBatchProvider.java   |  27 ++
 .../apache/drill/exec/record/RecordBatch.java   |   9 +-
 .../drill/exec/record/RecordBatchLoader.java    | 143 ++++++++++
 .../apache/drill/exec/record/SchemaBuilder.java | 127 +++++++++
 .../apache/drill/exec/record/WritableBatch.java | 108 ++++++++
 .../record/vector/AbstractFixedValueVector.java |  21 +-
 .../exec/record/vector/BaseValueVector.java     |  86 ++++--
 .../apache/drill/exec/record/vector/Bit.java    | 168 ++++++++++++
 .../drill/exec/record/vector/BitVector.java     | 166 -----------
 .../drill/exec/record/vector/ByteVector.java    |  48 ----
 .../apache/drill/exec/record/vector/Fixed1.java |  43 +++
 .../drill/exec/record/vector/Fixed12.java       |  35 +++
 .../drill/exec/record/vector/Fixed16.java       |  37 +++
 .../apache/drill/exec/record/vector/Fixed2.java |  53 ++++
 .../apache/drill/exec/record/vector/Fixed4.java |  55 ++++
 .../apache/drill/exec/record/vector/Fixed8.java |  58 ++++
 .../drill/exec/record/vector/FixedLen.java      |  45 +++
 .../drill/exec/record/vector/Int16Vector.java   |  52 ----
 .../drill/exec/record/vector/Int32Vector.java   |  52 ----
 .../exec/record/vector/NullableFixed4.java      |  37 +++
 .../exec/record/vector/NullableInt32Vector.java |  47 ----
 .../exec/record/vector/NullableValueVector.java |  36 ++-
 .../drill/exec/record/vector/RepeatMap.java     |  57 ++++
 .../exec/record/vector/SelectionVector.java     |  10 +-
 .../drill/exec/record/vector/TypeHelper.java    | 250 +++++++++++++++++
 .../drill/exec/record/vector/UInt16Vector.java  |  51 ----
 .../drill/exec/record/vector/ValueVector.java   |  46 +++-
 .../drill/exec/record/vector/VarLen1.java       |  36 +++
 .../drill/exec/record/vector/VarLen2.java       |  36 +++
 .../drill/exec/record/vector/VarLen4.java       |  36 +++
 .../exec/record/vector/VariableVector.java      |  35 ++-
 .../exec/rpc/AbstractHandshakeHandler.java      |  57 ++++
 .../java/org/apache/drill/exec/rpc/Acks.java    |  27 ++
 .../org/apache/drill/exec/rpc/BasicClient.java  | 137 ++++++++--
 .../exec/rpc/BasicClientWithConnection.java     |  64 +++++
 .../org/apache/drill/exec/rpc/BasicServer.java  |  61 ++++-
 .../drill/exec/rpc/CoordinationQueue.java       |  32 ++-
 .../apache/drill/exec/rpc/DrillRpcFuture.java   |  78 +-----
 .../drill/exec/rpc/DrillRpcFutureImpl.java      | 118 ++++++++
 .../drill/exec/rpc/InboundRpcMessage.java       |  13 +-
 .../drill/exec/rpc/OutboundRpcMessage.java      |  41 ++-
 .../apache/drill/exec/rpc/RemoteConnection.java |  45 +++
 .../org/apache/drill/exec/rpc/Response.java     |   8 +-
 .../java/org/apache/drill/exec/rpc/RpcBus.java  | 136 ++++++----
 .../org/apache/drill/exec/rpc/RpcConfig.java    | 150 ++++++++++
 .../org/apache/drill/exec/rpc/RpcDecoder.java   |  14 +-
 .../org/apache/drill/exec/rpc/RpcEncoder.java   |  15 +-
 .../drill/exec/rpc/RpcExceptionHandler.java     |  21 +-
 .../org/apache/drill/exec/rpc/RpcMessage.java   |   9 +-
 .../drill/exec/rpc/RpcOutcomeListener.java      |  28 ++
 .../exec/rpc/ZeroCopyProtobufLengthDecoder.java |  16 +-
 .../exec/rpc/bit/AvailabilityListener.java      |  22 ++
 .../apache/drill/exec/rpc/bit/BitClient.java    |  77 ++++--
 .../org/apache/drill/exec/rpc/bit/BitCom.java   |  75 ++---
 .../rpc/bit/BitComDefaultInstanceHandler.java   |  51 ++++
 .../drill/exec/rpc/bit/BitComHandler.java       | 136 ----------
 .../apache/drill/exec/rpc/bit/BitComImpl.java   | 194 +++++++------
 .../drill/exec/rpc/bit/BitConnection.java       | 168 ++++++++++++
 .../exec/rpc/bit/BitConnectionManager.java      |  80 ++++++
 .../apache/drill/exec/rpc/bit/BitRpcConfig.java |  46 ++++
 .../apache/drill/exec/rpc/bit/BitServer.java    |  61 +++--
 .../apache/drill/exec/rpc/bit/BitTunnel.java    | 215 +++++++++++++--
 .../apache/drill/exec/rpc/bit/ListenerPool.java |  56 ++++
 .../drill/exec/rpc/user/QueryResultBatch.java   |  49 ++++
 .../apache/drill/exec/rpc/user/UserClient.java  | 169 ++++++++++--
 .../exec/rpc/user/UserResultsListener.java      |  41 +++
 .../drill/exec/rpc/user/UserRpcConfig.java      |  39 +++
 .../apache/drill/exec/rpc/user/UserServer.java  |  93 +++++--
 .../drill/exec/server/BootStrapContext.java     |  68 +++++
 .../org/apache/drill/exec/server/Drillbit.java  |  65 +++--
 .../drill/exec/server/DrillbitContext.java      |  58 ++--
 .../drill/exec/server/RemoteServiceSet.java     |  59 ++++
 .../drill/exec/service/ServiceEngine.java       |  45 ++-
 .../drill/exec/store/AbstractStorageEngine.java |   2 +-
 .../apache/drill/exec/store/RecordReader.java   |   7 +-
 .../apache/drill/exec/store/StorageEngine.java  |   2 +-
 .../org/apache/drill/exec/util/AtomicState.java |  58 ++++
 .../work/AbstractFragmentRunnerListener.java    | 109 ++++++++
 .../apache/drill/exec/work/CancelableQuery.java |  22 ++
 .../drill/exec/work/EndpointListener.java       |  45 +++
 .../apache/drill/exec/work/FragmentRunner.java  | 124 +++++++++
 .../drill/exec/work/FragmentRunnerListener.java |  28 ++
 .../apache/drill/exec/work/QueryWorkUnit.java   |  64 +++++
 .../drill/exec/work/RecordOutputStream.java     |  22 ++
 .../work/RemotingFragmentRunnerListener.java    |  46 ++++
 .../apache/drill/exec/work/ResourceRequest.java |  30 ++
 .../apache/drill/exec/work/RootNodeDriver.java  |  25 ++
 .../apache/drill/exec/work/StatusProvider.java  |  24 ++
 .../org/apache/drill/exec/work/WorkManager.java | 168 ++++++++++++
 .../work/batch/AbstractFragmentCollector.java   |  84 ++++++
 .../drill/exec/work/batch/BatchCollector.java   |  32 +++
 .../drill/exec/work/batch/BitComHandler.java    |  41 +++
 .../exec/work/batch/BitComHandlerImpl.java      | 205 ++++++++++++++
 .../drill/exec/work/batch/IncomingBuffers.java  | 108 ++++++++
 .../drill/exec/work/batch/MergingCollector.java |  45 +++
 .../exec/work/batch/PartitionedCollector.java   |  42 +++
 .../drill/exec/work/batch/RawBatchBuffer.java   |  33 +++
 .../exec/work/batch/UnlmitedRawBatchBuffer.java |  73 +++++
 .../drill/exec/work/foreman/ErrorHelper.java    |  47 ++++
 .../apache/drill/exec/work/foreman/Foreman.java | 272 +++++++++++++++++++
 .../work/foreman/FragmentStatusListener.java    |  26 ++
 .../work/foreman/RunningFragmentManager.java    | 266 ++++++++++++++++++
 .../drill/exec/work/foreman/TunnelManager.java  |  53 ++++
 .../work/fragment/IncomingFragmentHandler.java  |  49 ++++
 .../work/fragment/LocalFragmentHandler.java     |  69 +++++
 .../work/fragment/RemoteFragmentHandler.java    | 123 +++++++++
 .../apache/drill/exec/work/user/UserWorker.java |  72 +++++
 .../src/main/protobuf/Coordination.proto        |  26 ++
 .../src/main/protobuf/ExecutionProtos.proto     |  55 ++--
 .../src/main/protobuf/GeneralRPC.proto          |   6 +-
 .../java-exec/src/main/protobuf/SchemaDef.proto |  86 ++++--
 .../exec/java-exec/src/main/protobuf/User.proto |  69 +++--
 .../src/main/protobuf/UserBitShared.proto       |  46 ++++
 .../apache/drill/exec/DrillSystemTestBase.java  |   7 +
 .../exec/client/DrillClientSystemTest.java      |  25 +-
 .../exec/compile/TestClassCompilationTypes.java |   5 +-
 .../exec/physical/config/ParsePhysicalPlan.java |  42 +++
 .../physical/impl/DistributedFragmentRun.java   |  53 ++++
 .../exec/physical/impl/SimpleFragmentRun.java   | 100 +++++++
 .../apache/drill/exec/pop/CheckFragmenter.java  |  70 ++---
 .../drill/exec/pop/CheckInjectionValue.java     |  12 +-
 .../apache/drill/exec/pop/FragmentChecker.java  |  66 +++++
 .../apache/drill/exec/pop/PopUnitTestBase.java  |  71 +++++
 .../drill/exec/rpc/user/RunRemoteQuery.java     |  41 ---
 .../apache/drill/exec/rpc/user/UserRpcTest.java | 107 --------
 .../apache/drill/exec/server/TestBitRpc.java    |  84 ++++++
 .../drill/exec/store/MockRecordConfig.java      |  46 ----
 .../drill/exec/store/MockRecordReader.java      | 108 --------
 .../drill/exec/store/MockStorageEngine.java     |  54 ----
 .../src/test/resources/drill-module.conf        |   5 +-
 .../java-exec/src/test/resources/logback.xml    |  10 +-
 .../src/test/resources/physical_screen.json     |   5 +-
 .../test/resources/physical_simpleexchange.json |  36 ++-
 .../resources/physical_single_exchange.json     |  34 +++
 .../src/test/resources/physical_test1.json      |  40 +++
 .../src/test/resources/physical_test2.json      |  34 +++
 .../exec/ref/src/test/resources/donuts.json     |   2 +-
 sandbox/prototype/pom.xml                       |   2 +-
 326 files changed, 13371 insertions(+), 6094 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/common/pom.xml
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/pom.xml b/sandbox/prototype/common/pom.xml
index 3a15c05..beb2d28 100644
--- a/sandbox/prototype/common/pom.xml
+++ b/sandbox/prototype/common/pom.xml
@@ -74,6 +74,18 @@
 
 	<build>
 		<plugins>
+            <plugin>
+                <artifactId>maven-jar-plugin</artifactId>
+                <executions>
+                    <execution>
+                        <id>test-jar</id>
+                        <phase>package</phase>
+                        <goals>
+                            <goal>test-jar</goal>
+                        </goals>
+                    </execution>
+                </executions>
+            </plugin>
 			<plugin>
 				<groupId>org.antlr</groupId>
 				<artifactId>antlr3-maven-plugin</artifactId>

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/common/src/main/java/org/apache/drill/common/config/DrillConfig.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/config/DrillConfig.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/config/DrillConfig.java
index 2b8f45d..18d5e51 100644
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/config/DrillConfig.java
+++ b/sandbox/prototype/common/src/main/java/org/apache/drill/common/config/DrillConfig.java
@@ -27,9 +27,9 @@ import org.apache.drill.common.exceptions.DrillConfigurationException;
 import org.apache.drill.common.expression.LogicalExpression;
 import org.apache.drill.common.logical.StorageEngineConfigBase;
 import org.apache.drill.common.logical.data.LogicalOperatorBase;
-import org.apache.drill.common.physical.pop.base.PhysicalOperatorUtil;
 import org.apache.drill.common.util.PathScanner;
 
+import com.fasterxml.jackson.core.JsonGenerator;
 import com.fasterxml.jackson.core.JsonParser.Feature;
 import com.fasterxml.jackson.databind.ObjectMapper;
 import com.fasterxml.jackson.databind.SerializationFeature;
@@ -50,15 +50,18 @@ public final class DrillConfig extends NestedConfig{
   public DrillConfig(Config config) {
     super(config);
     mapper = new ObjectMapper();
-    SimpleModule deserModule = new SimpleModule("LogicalExpressionDeserializationModule").addDeserializer(LogicalExpression.class, new LogicalExpression.De(this));
+    SimpleModule deserModule = new SimpleModule("LogicalExpressionDeserializationModule")
+      .addDeserializer(LogicalExpression.class, new LogicalExpression.De(this));
+    
     mapper.registerModule(deserModule);
     mapper.enable(SerializationFeature.INDENT_OUTPUT);
     mapper.configure(Feature.ALLOW_UNQUOTED_FIELD_NAMES, true);
+    mapper.configure(JsonGenerator.Feature.QUOTE_FIELD_NAMES, false);
     mapper.configure(Feature.ALLOW_COMMENTS, true);
     mapper.registerSubtypes(LogicalOperatorBase.getSubTypes(this));
-    mapper.registerSubtypes(PhysicalOperatorUtil.getSubTypes(this));
     mapper.registerSubtypes(StorageEngineConfigBase.getSubTypes(this));
     
+    
   };
   
   /**

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/common/src/main/java/org/apache/drill/common/exceptions/ExecutionSetupException.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/exceptions/ExecutionSetupException.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/exceptions/ExecutionSetupException.java
new file mode 100644
index 0000000..9096d89
--- /dev/null
+++ b/sandbox/prototype/common/src/main/java/org/apache/drill/common/exceptions/ExecutionSetupException.java
@@ -0,0 +1,44 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.common.exceptions;
+
+
+public class ExecutionSetupException extends DrillException{
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ExecutionSetupException.class);
+  
+  public ExecutionSetupException() {
+    super();
+  }
+
+  public ExecutionSetupException(String message, Throwable cause, boolean enableSuppression, boolean writableStackTrace) {
+    super(message, cause, enableSuppression, writableStackTrace);
+  }
+
+  public ExecutionSetupException(String message, Throwable cause) {
+    super(message, cause);
+  }
+
+  public ExecutionSetupException(String message) {
+    super(message);
+  }
+
+  public ExecutionSetupException(Throwable cause) {
+    super(cause);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/common/src/main/java/org/apache/drill/common/exceptions/PhysicalOperatorSetupException.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/exceptions/PhysicalOperatorSetupException.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/exceptions/PhysicalOperatorSetupException.java
new file mode 100644
index 0000000..f22cb16
--- /dev/null
+++ b/sandbox/prototype/common/src/main/java/org/apache/drill/common/exceptions/PhysicalOperatorSetupException.java
@@ -0,0 +1,45 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.common.exceptions;
+
+public class PhysicalOperatorSetupException extends ExecutionSetupException{
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(PhysicalOperatorSetupException.class);
+
+  public PhysicalOperatorSetupException() {
+    super();
+  }
+
+  public PhysicalOperatorSetupException(String message, Throwable cause, boolean enableSuppression,
+      boolean writableStackTrace) {
+    super(message, cause, enableSuppression, writableStackTrace);
+  }
+
+  public PhysicalOperatorSetupException(String message, Throwable cause) {
+    super(message, cause);
+  }
+
+  public PhysicalOperatorSetupException(String message) {
+    super(message);
+  }
+
+  public PhysicalOperatorSetupException(Throwable cause) {
+    super(cause);
+  }
+  
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/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 b3675a8..004d812 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
@@ -104,7 +104,7 @@ public class SchemaPath extends LogicalExpressionBase{
 
 	}
 		
-	
+
 	
 	@Override
   public <T> T accept(ExprVisitor<T> visitor) {

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/types/DataType.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/types/DataType.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/types/DataType.java
index 25b82a7..56e2485 100644
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/types/DataType.java
+++ b/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/types/DataType.java
@@ -74,6 +74,17 @@ public abstract class DataType {
   public static final DataType ARRAY = new AtomType("ARRAY", Comparability.NONE, false);
   public static final DataType NULL = new AtomType("NULL", Comparability.NONE, false);
   
+  //TODO: Hack to get some size data, needs to be fixed so that each type reveals it's size.
+  public int size(){
+    if(this == BOOLEAN){
+      return 1;
+    }else if(this == INT32){
+      return 4;
+    }else if(this == INT16){
+      return 4;
+    }
+    return 2;
+  }
   
   static final Map<String, DataType> TYPES;
   static {

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/common/src/main/java/org/apache/drill/common/optimize/Optimizer.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/optimize/Optimizer.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/optimize/Optimizer.java
deleted file mode 100644
index 4b2037c..0000000
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/optimize/Optimizer.java
+++ /dev/null
@@ -1,45 +0,0 @@
-/*******************************************************************************
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- ******************************************************************************/
-package org.apache.drill.common.optimize;
-
-import java.io.Closeable;
-
-import org.apache.drill.common.config.DrillConfig;
-import org.apache.drill.common.exceptions.DrillConfigurationException;
-import org.apache.drill.common.logical.LogicalPlan;
-import org.apache.drill.common.physical.PhysicalPlan;
-
-public abstract class Optimizer implements Closeable{
-  
-  public static String OPTIMIZER_IMPL_KEY = "drill.exec.optimizer.implementation";
-  
-  public abstract void init(DrillConfig config);
-  
-  public abstract PhysicalPlan optimize(OptimizationContext context, LogicalPlan plan);
-  public abstract void close();
-  
-  public static Optimizer getOptimizer(DrillConfig config) throws DrillConfigurationException{
-    Optimizer o = config.getInstanceOf(OPTIMIZER_IMPL_KEY, Optimizer.class);
-    o.init(config);
-    return o;
-  }
-  
-  public interface OptimizationContext{
-    public int getPriority();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/DataValidationMode.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/DataValidationMode.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/DataValidationMode.java
deleted file mode 100644
index 6de2cfd..0000000
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/DataValidationMode.java
+++ /dev/null
@@ -1,24 +0,0 @@
-/*******************************************************************************
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- ******************************************************************************/
-package org.apache.drill.common.physical;
-
-public enum DataValidationMode {
-  TERMINATE, // terminate the query if the data doesn't match expected.
-  DROP_RECORD, // drop the record that doesn't match the expected situation.
-  SINK_RECORD // record the failed record along with the rule violation in a secondary location.
-}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/EndpointAffinity.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/EndpointAffinity.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/EndpointAffinity.java
deleted file mode 100644
index 9ccf430..0000000
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/EndpointAffinity.java
+++ /dev/null
@@ -1,60 +0,0 @@
-/*******************************************************************************
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- ******************************************************************************/
-package org.apache.drill.common.physical;
-
-import org.apache.drill.common.proto.CoordinationProtos.DrillbitEndpoint;
-
-
-public class EndpointAffinity implements Comparable<EndpointAffinity>{
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(EndpointAffinity.class);
-  
-  private DrillbitEndpoint endpoint;
-  private float affinity = 0.0f;
-  
-  public EndpointAffinity(DrillbitEndpoint endpoint) {
-    super();
-    this.endpoint = endpoint;
-  }
-  
-  public EndpointAffinity(DrillbitEndpoint endpoint, float affinity) {
-    super();
-    this.endpoint = endpoint;
-    this.affinity = affinity;
-  }
-
-  public DrillbitEndpoint getEndpoint() {
-    return endpoint;
-  }
-  public void setEndpoint(DrillbitEndpoint endpoint) {
-    this.endpoint = endpoint;
-  }
-  public float getAffinity() {
-    return affinity;
-  }
-  
-  @Override
-  public int compareTo(EndpointAffinity o) {
-    return Float.compare(affinity, o.affinity);
-  }
-  
-  public void addAffinity(float f){
-    affinity += f;
-  }
-  
-  
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/FieldSet.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/FieldSet.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/FieldSet.java
deleted file mode 100644
index c76098d..0000000
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/FieldSet.java
+++ /dev/null
@@ -1,80 +0,0 @@
-/*******************************************************************************
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- ******************************************************************************/
-package org.apache.drill.common.physical;
-
-import java.io.IOException;
-import java.util.List;
-
-import org.apache.drill.common.physical.FieldSet.De;
-import org.apache.drill.common.physical.FieldSet.Se;
-
-import com.fasterxml.jackson.core.JsonGenerationException;
-import com.fasterxml.jackson.core.JsonGenerator;
-import com.fasterxml.jackson.core.JsonParser;
-import com.fasterxml.jackson.core.JsonProcessingException;
-import com.fasterxml.jackson.core.type.TypeReference;
-import com.fasterxml.jackson.databind.DeserializationContext;
-import com.fasterxml.jackson.databind.SerializerProvider;
-import com.fasterxml.jackson.databind.annotation.JsonDeserialize;
-import com.fasterxml.jackson.databind.annotation.JsonSerialize;
-import com.fasterxml.jackson.databind.deser.std.StdDeserializer;
-import com.fasterxml.jackson.databind.ser.std.StdSerializer;
-import com.google.common.collect.Lists;
-
-@JsonSerialize(using = Se.class)
-@JsonDeserialize(using = De.class)
-public class FieldSet {
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(FieldSet.class);
-  
-  private List<RecordField> fields;
-  
-  public FieldSet(Iterable<RecordField> fields){
-    this.fields = Lists.newArrayList(fields);
-  }
-  
-
-  public static class De extends StdDeserializer<FieldSet> {
-    
-    public De() {
-      super(FieldSet.class);
-    }
-
-    @Override
-    public FieldSet deserialize(JsonParser jp, DeserializationContext ctxt) throws IOException,
-        JsonProcessingException {
-      Iterable<RecordField> fields = jp.readValueAs(new TypeReference<List<RecordField>>(){});
-      logger.debug("Fields {}", fields);
-      return new FieldSet(fields);
-    }
-
-  }
-
-  public static class Se extends StdSerializer<FieldSet> {
-
-    public Se() {
-      super(FieldSet.class);
-    }
-
-    @Override
-    public void serialize(FieldSet value, JsonGenerator jgen, SerializerProvider provider) throws IOException,
-        JsonGenerationException {
-      jgen.writeObject(value.fields);
-    }
-
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/OperatorCost.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/OperatorCost.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/OperatorCost.java
deleted file mode 100644
index fadfff0..0000000
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/OperatorCost.java
+++ /dev/null
@@ -1,61 +0,0 @@
-/*******************************************************************************
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- ******************************************************************************/
-package org.apache.drill.common.physical;
-
-import com.fasterxml.jackson.annotation.JsonCreator;
-import com.fasterxml.jackson.annotation.JsonProperty;
-
-public class OperatorCost {
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(OperatorCost.class);
-  
-  private final float network; 
-  private final float disk;
-  private final float memory;
-  private final float cpu;
-  
-  @JsonCreator
-  public OperatorCost(@JsonProperty("network") float network, @JsonProperty("disk") float disk, @JsonProperty("memory") float memory, @JsonProperty("cpu") float cpu) {
-    super();
-    this.network = network;
-    this.disk = disk;
-    this.memory = memory;
-    this.cpu = cpu;
-  }
-
-  public float getNetwork() {
-    return network;
-  }
-
-  public float getDisk() {
-    return disk;
-  }
-
-  public float getMemory() {
-    return memory;
-  }
-
-  public float getCpu() {
-    return cpu;
-  }
-  
-  public static OperatorCost combine(OperatorCost c1, OperatorCost c2){
-    return new OperatorCost(c1.network + c2.network, c1.disk + c2.disk, c1.memory + c2.memory, c1.cpu + c2.cpu);
-  }
-
-  
-}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/PhysicalPlan.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/PhysicalPlan.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/PhysicalPlan.java
deleted file mode 100644
index e83dac7..0000000
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/PhysicalPlan.java
+++ /dev/null
@@ -1,95 +0,0 @@
-/*******************************************************************************
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- ******************************************************************************/
-package org.apache.drill.common.physical;
-
-import java.io.IOException;
-import java.util.List;
-
-import org.apache.drill.common.PlanProperties;
-import org.apache.drill.common.config.DrillConfig;
-import org.apache.drill.common.graph.Graph;
-import org.apache.drill.common.graph.GraphAlgos;
-import org.apache.drill.common.physical.pop.base.Leaf;
-import org.apache.drill.common.physical.pop.base.PhysicalOperator;
-import org.apache.drill.common.physical.pop.base.Root;
-
-import com.fasterxml.jackson.annotation.JsonCreator;
-import com.fasterxml.jackson.annotation.JsonProperty;
-import com.fasterxml.jackson.annotation.JsonPropertyOrder;
-import com.fasterxml.jackson.core.JsonProcessingException;
-import com.fasterxml.jackson.databind.ObjectMapper;
-import com.fasterxml.jackson.databind.ObjectReader;
-import com.fasterxml.jackson.databind.ObjectWriter;
-import com.google.common.collect.Lists;
-
-@JsonPropertyOrder({ "head", "graph" })
-public class PhysicalPlan {
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(PhysicalPlan.class);
-  
-  PlanProperties properties;
-  
-  Graph<PhysicalOperator, Root, Leaf> graph;
-  
-  @JsonCreator
-  public PhysicalPlan(@JsonProperty("head") PlanProperties properties, @JsonProperty("graph") List<PhysicalOperator> operators){
-    this.properties = properties;
-    this.graph = Graph.newGraph(operators, Root.class, Leaf.class);
-  }
-  
-  @JsonProperty("graph")
-  public List<PhysicalOperator> getSortedOperators(){
-    // reverse the list so that nested references are flattened rather than nested.
-    return getSortedOperators(true);
-  }
-  
-  public List<PhysicalOperator> getSortedOperators(boolean reverse){
-    List<PhysicalOperator> list = GraphAlgos.TopoSorter.sort(graph);
-    if(reverse){
-      return Lists.reverse(list);
-    }else{
-      return list;
-    }
-    
-  }
-
-
-  @JsonProperty("head")
-  public PlanProperties getProperties() {
-    return properties;
-  }
-
-  /** Parses a physical plan. */
-  public static PhysicalPlan parse(ObjectReader reader, String planString) {
-    try {
-      PhysicalPlan plan = reader.readValue(planString);
-      return plan;
-    } catch (IOException e) {
-      throw new RuntimeException(e);
-    }
-  }
-
-  /** Converts a physical plan to a string. (Opposite of {@link #parse}.) */
-  public String unparse(ObjectWriter writer) {
-    try {
-      return writer.writeValueAsString(this);
-    } catch (JsonProcessingException e) {
-      throw new RuntimeException(e);
-    }
-  }
-  
-}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/ReadEntry.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/ReadEntry.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/ReadEntry.java
deleted file mode 100644
index 7c23cf5..0000000
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/ReadEntry.java
+++ /dev/null
@@ -1,25 +0,0 @@
-/*******************************************************************************
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- ******************************************************************************/
-package org.apache.drill.common.physical;
-
-/** 
- * Describes a chunk of read work that will be done.
- */
-public interface ReadEntry {
-   
-}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/RecordField.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/RecordField.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/RecordField.java
deleted file mode 100644
index 8d0072a..0000000
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/RecordField.java
+++ /dev/null
@@ -1,60 +0,0 @@
-/*******************************************************************************
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- ******************************************************************************/
-package org.apache.drill.common.physical;
-
-import org.apache.drill.common.expression.types.DataType;
-
-import com.fasterxml.jackson.annotation.JsonCreator;
-import com.fasterxml.jackson.annotation.JsonProperty;
-
-public class RecordField {
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(RecordField.class);
-
-  
-  private DataType type;
-  private ValueMode mode;
-  
-  @JsonCreator
-  public RecordField(@JsonProperty("type") DataType type, @JsonProperty("mode") ValueMode mode) {
-    super();
-    this.type = type;
-    this.mode = mode;
-  }
-
-  public DataType getType() {
-    return type;
-  }
-
-  public ValueMode getMode() {
-    return mode;
-  }
-  
-  public static enum ValueMode {
-    VECTOR,
-    DICT,
-    RLE
-  }
-  
-  public static enum ValueType {
-    OPTIONAL,
-    REQUIRED, 
-    REPEATED
-  }
-  
-  
-}

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

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/Filter.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/Filter.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/Filter.java
deleted file mode 100644
index 2c86d99..0000000
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/Filter.java
+++ /dev/null
@@ -1,52 +0,0 @@
-/*******************************************************************************
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- ******************************************************************************/
-package org.apache.drill.common.physical.pop;
-
-import org.apache.drill.common.expression.LogicalExpression;
-import org.apache.drill.common.physical.pop.base.AbstractSingle;
-import org.apache.drill.common.physical.pop.base.PhysicalOperator;
-import org.apache.drill.common.physical.pop.base.PhysicalVisitor;
-
-import com.fasterxml.jackson.annotation.JsonCreator;
-import com.fasterxml.jackson.annotation.JsonProperty;
-import com.fasterxml.jackson.annotation.JsonTypeName;
-
-@JsonTypeName("filter")
-public class Filter extends AbstractSingle {
-
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(Filter.class);
-
-  private final LogicalExpression expr;
-  
-  @JsonCreator
-  public Filter(@JsonProperty("child") PhysicalOperator child, @JsonProperty("expr") LogicalExpression expr) {
-    super(child);
-    this.expr = expr;
-  }
-
-  public LogicalExpression getExpr() {
-    return expr;
-  }
-
-  @Override
-  public <T, X, E extends Throwable> T accept(PhysicalVisitor<T, X, E> physicalVisitor, X value) throws E{
-    return physicalVisitor.visitFilter(this, value);
-  }
-
-  
-}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/PartitionToRandomExchange.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/PartitionToRandomExchange.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/PartitionToRandomExchange.java
deleted file mode 100644
index 0289780..0000000
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/PartitionToRandomExchange.java
+++ /dev/null
@@ -1,92 +0,0 @@
-/*******************************************************************************
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- ******************************************************************************/
-package org.apache.drill.common.physical.pop;
-
-import java.util.List;
-
-import org.apache.drill.common.defs.PartitionDef;
-import org.apache.drill.common.expression.LogicalExpression;
-import org.apache.drill.common.physical.OperatorCost;
-import org.apache.drill.common.physical.pop.base.AbstractExchange;
-import org.apache.drill.common.physical.pop.base.ExchangeCost;
-import org.apache.drill.common.physical.pop.base.PhysicalOperator;
-import org.apache.drill.common.physical.pop.base.PhysicalVisitor;
-import org.apache.drill.common.physical.pop.base.Receiver;
-import org.apache.drill.common.physical.pop.base.Sender;
-import org.apache.drill.common.proto.CoordinationProtos.DrillbitEndpoint;
-
-import com.fasterxml.jackson.annotation.JsonCreator;
-import com.fasterxml.jackson.annotation.JsonProperty;
-import com.fasterxml.jackson.annotation.JsonTypeName;
-
-@JsonTypeName("partition-to-random-exchange")
-public class PartitionToRandomExchange extends AbstractExchange{
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(PartitionToRandomExchange.class);
-
-  private final PartitionDef partition;
-  private final int maxWidth;
-  
-  @JsonCreator
-  public PartitionToRandomExchange(@JsonProperty("child") PhysicalOperator child, @JsonProperty("partition") PartitionDef partition, @JsonProperty("cost") ExchangeCost cost) {
-    super(child, cost);
-    this.partition = partition;
-    
-    LogicalExpression[] parts = partition.getStarts();
-    if(parts != null && parts.length > 0){
-      this.maxWidth = parts.length+1;
-    }else{
-      this.maxWidth = Integer.MAX_VALUE;
-    }
-  }
-
-  public PartitionDef getPartition() {
-    return partition;
-  }
-
-  @Override
-  public <T, X, E extends Throwable> T accept(PhysicalVisitor<T, X, E> physicalVisitor, X value) throws E{
-    return physicalVisitor.visitExchange(this,  value);
-  }
-
-  @Override
-  public int getMaxSendWidth() {
-    return maxWidth;
-  }
-
-  @Override
-  public void setupSenders(List<DrillbitEndpoint> senderLocations) {
-  }
-
-  @Override
-  public void setupReceivers(List<DrillbitEndpoint> receiverLocations) {
-  }
-
-  @Override
-  public Sender getSender(int minorFragmentId, PhysicalOperator child) {
-    return null;
-  }
-
-  @Override
-  public Receiver getReceiver(int minorFragmentId) {
-    return null;
-  }
-
-
-  
-  
-}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/Project.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/Project.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/Project.java
deleted file mode 100644
index 7cff28d..0000000
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/Project.java
+++ /dev/null
@@ -1,53 +0,0 @@
-/*******************************************************************************
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- ******************************************************************************/
-package org.apache.drill.common.physical.pop;
-
-import java.util.Iterator;
-import java.util.List;
-
-import org.apache.drill.common.logical.data.NamedExpression;
-import org.apache.drill.common.physical.pop.base.AbstractSingle;
-import org.apache.drill.common.physical.pop.base.PhysicalOperator;
-import org.apache.drill.common.physical.pop.base.PhysicalVisitor;
-
-import com.fasterxml.jackson.annotation.JsonCreator;
-import com.fasterxml.jackson.annotation.JsonProperty;
-import com.fasterxml.jackson.annotation.JsonTypeName;
-
-@JsonTypeName("project")
-public class Project extends AbstractSingle{
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(Project.class);
-
-  private final List<NamedExpression> exprs;
-  
-  @JsonCreator
-  public Project(@JsonProperty("exprs") List<NamedExpression> exprs, @JsonProperty("child") PhysicalOperator child) {
-    super(child);
-    this.exprs = exprs;
-  }
-
-  public List<NamedExpression> getExprs() {
-    return exprs;
-  }
-
-  
-  @Override
-  public <T, X, E extends Throwable> T accept(PhysicalVisitor<T, X, E> physicalVisitor, X value) throws E{
-    return physicalVisitor.visitProject(this, value);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/Screen.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/Screen.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/Screen.java
deleted file mode 100644
index fdbd8f1..0000000
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/Screen.java
+++ /dev/null
@@ -1,77 +0,0 @@
-/*******************************************************************************
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- ******************************************************************************/
-package org.apache.drill.common.physical.pop;
-
-import java.util.Collections;
-import java.util.List;
-
-import org.apache.drill.common.physical.EndpointAffinity;
-import org.apache.drill.common.physical.pop.base.AbstractStore;
-import org.apache.drill.common.physical.pop.base.PhysicalOperator;
-import org.apache.drill.common.physical.pop.base.Store;
-import org.apache.drill.common.proto.CoordinationProtos.DrillbitEndpoint;
-
-import com.fasterxml.jackson.annotation.JacksonInject;
-import com.fasterxml.jackson.annotation.JsonIgnore;
-import com.fasterxml.jackson.annotation.JsonProperty;
-import com.fasterxml.jackson.annotation.JsonTypeName;
-
-@JsonTypeName("screen")
-public class Screen extends AbstractStore {
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(Screen.class);
-
-  private final DrillbitEndpoint endpoint;
-
-  public Screen(@JsonProperty("child") PhysicalOperator child, @JacksonInject DrillbitEndpoint endpoint) {
-    super(child);
-    this.endpoint = endpoint;
-  }
-
-  @Override
-  public List<EndpointAffinity> getOperatorAffinity() {
-    return Collections.singletonList(new EndpointAffinity(endpoint, 1000));
-  }
-
-  @Override
-  public int getMaxWidth() {
-    return 1;
-  }
-
-  @Override
-  public void applyAssignments(List<DrillbitEndpoint> endpoints) {
-    // we actually don't have to do anything since nothing should have changed. we'll check just check that things
-    // didn't get screwed up.
-    if (endpoints.size() != 1)
-      throw new UnsupportedOperationException("A Screen operator can only be assigned to a single node.");
-    DrillbitEndpoint endpoint = endpoints.iterator().next();
-    if (this.endpoint != endpoint)
-      throw new UnsupportedOperationException("A Screen operator can only be assigned to its home node.");
-
-  }
-
-  @Override
-  public Store getSpecificStore(PhysicalOperator child, int minorFragmentId) {
-    return new Screen(child, endpoint);
-  }
-
-  @JsonIgnore
-  public DrillbitEndpoint getEndpoint() {
-    return endpoint;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/Sort.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/Sort.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/Sort.java
deleted file mode 100644
index b4d802d..0000000
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/Sort.java
+++ /dev/null
@@ -1,57 +0,0 @@
-/*******************************************************************************
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- ******************************************************************************/
-package org.apache.drill.common.physical.pop;
-
-import org.apache.drill.common.expression.LogicalExpression;
-import org.apache.drill.common.physical.pop.base.AbstractSingle;
-import org.apache.drill.common.physical.pop.base.PhysicalOperator;
-import org.apache.drill.common.physical.pop.base.PhysicalVisitor;
-
-import com.fasterxml.jackson.annotation.JsonCreator;
-import com.fasterxml.jackson.annotation.JsonProperty;
-import com.fasterxml.jackson.annotation.JsonTypeName;
-
-@JsonTypeName("sort")
-public class Sort extends AbstractSingle{
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(Sort.class);
-  
-  private final LogicalExpression expr;
-  private boolean reverse = false;
-  
-  @JsonCreator
-  public Sort(@JsonProperty("child") PhysicalOperator child, @JsonProperty("expr") LogicalExpression expr, @JsonProperty("reverse") boolean reverse) {
-    super(child);
-    this.expr = expr;
-    this.reverse = reverse;
-  }
-  
-  public LogicalExpression getExpr() {
-    return expr;
-  }
-
-  public boolean getReverse() {
-    return reverse;
-  }
-
-  @Override
-  public <T, X, E extends Throwable> T accept(PhysicalVisitor<T, X, E> physicalVisitor, X value) throws E{
-    return physicalVisitor.visitSort(this, value);
-  }
-    
-  
-}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/AbstractBase.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/AbstractBase.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/AbstractBase.java
deleted file mode 100644
index 5d3584c..0000000
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/AbstractBase.java
+++ /dev/null
@@ -1,56 +0,0 @@
-/*******************************************************************************
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- ******************************************************************************/
-package org.apache.drill.common.physical.pop.base;
-
-import org.apache.drill.common.graph.GraphVisitor;
-import org.apache.drill.common.physical.OperatorCost;
-
-public abstract class AbstractBase implements PhysicalOperator{
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(AbstractBase.class);
-
-  private OperatorCost cost;
-  
-  @Override
-  public void accept(GraphVisitor<PhysicalOperator> visitor) {
-    visitor.enter(this);
-    if(this.iterator() == null) throw new IllegalArgumentException("Null iterator for pop." + this);
-    for(PhysicalOperator o : this){
-      o.accept(visitor);  
-    }
-    visitor.leave(this);
-  }
-  
-  @Override
-  public boolean isExecutable() {
-    return true;
-  }
-
-  @Override
-  public OperatorCost getCost() {
-    return cost;
-  }
-  
-  // should be used only for the purposes of json...
-  void setCost(OperatorCost cost){
-    this.cost = cost;
-  }
-
-
-  
-  
-}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/AbstractExchange.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/AbstractExchange.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/AbstractExchange.java
deleted file mode 100644
index 1f60c53..0000000
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/AbstractExchange.java
+++ /dev/null
@@ -1,68 +0,0 @@
-/*******************************************************************************
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- ******************************************************************************/
-package org.apache.drill.common.physical.pop.base;
-
-import java.util.List;
-
-import org.apache.drill.common.physical.OperatorCost;
-import org.apache.drill.common.proto.CoordinationProtos.DrillbitEndpoint;
-
-import com.fasterxml.jackson.annotation.JsonIgnore;
-import com.fasterxml.jackson.annotation.JsonProperty;
-
-public abstract class AbstractExchange extends AbstractSingle implements Exchange {
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(AbstractExchange.class);
-
-  private final ExchangeCost cost;
-
-  public AbstractExchange(PhysicalOperator child, ExchangeCost cost) {
-    super(child);
-    this.cost = cost;
-  }
-
-  /**
-   * Exchanges are not executable. The Execution layer first has to set their parallelization and convert them into
-   * something executable
-   */
-  @Override
-  public boolean isExecutable() {
-    return false;
-  }
-
-  @Override
-  public OperatorCost getAggregateSendCost() {
-    return cost.getSend();
-  }
-
-  @Override
-  public OperatorCost getAggregateReceiveCost() {
-    return cost.getReceive();
-  }
-
-  @Override
-  public ExchangeCost getExchangeCost() {
-    return cost;
-  }
-
-  @JsonIgnore
-  @Override
-  public OperatorCost getCost() {
-    return cost.getCombinedCost();
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/AbstractPhysicalVisitor.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/AbstractPhysicalVisitor.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/AbstractPhysicalVisitor.java
deleted file mode 100644
index 49f7bda..0000000
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/AbstractPhysicalVisitor.java
+++ /dev/null
@@ -1,80 +0,0 @@
-/*******************************************************************************
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- ******************************************************************************/
-package org.apache.drill.common.physical.pop.base;
-
-import org.apache.drill.common.physical.pop.Filter;
-import org.apache.drill.common.physical.pop.PartitionToRandomExchange;
-import org.apache.drill.common.physical.pop.Project;
-import org.apache.drill.common.physical.pop.Sort;
-
-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);
-
-  @Override
-  public T visitExchange(Exchange exchange, X value) throws E{
-    return visitUnknown(exchange, value);
-  }
-
-  @Override
-  public T visitFilter(Filter filter, X value) throws E{
-    return visitUnknown(filter, value);
-  }
-
-  @Override
-  public T visitProject(Project project, X value) throws E{
-    return visitUnknown(project, value);
-  }
-
-  @Override
-  public T visitSort(Sort sort, X value) throws E{
-    return visitUnknown(sort, value);
-  }
-
-  @Override
-  public T visitSender(Sender sender, X value) throws E {
-    return visitUnknown(sender, value);
-  }
-
-  @Override
-  public T visitReceiver(Receiver receiver, X value) throws E {
-    return visitUnknown(receiver, value);
-  }
-
-  @Override
-  public T visitScan(Scan<?> scan, X value) throws E{
-    return visitUnknown(scan, value);
-  }
-
-  @Override
-  public T visitStore(Store store, X value) throws E{
-    return visitUnknown(store, value);
-  }
-
-  @Override
-  public T visitPartitionToRandomExchange(PartitionToRandomExchange partitionToRandom, X value) throws E{
-    return visitExchange(partitionToRandom, value);
-  }
-
-  @Override
-  public T visitUnknown(PhysicalOperator op, X value) throws E{
-    throw new UnsupportedOperationException(String.format(
-        "The PhysicalVisitor of type %s does not currently support visiting the PhysicalOperator type %s.", this
-            .getClass().getCanonicalName(), op.getClass().getCanonicalName()));
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/AbstractReceiver.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/AbstractReceiver.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/AbstractReceiver.java
deleted file mode 100644
index fd9d93c..0000000
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/AbstractReceiver.java
+++ /dev/null
@@ -1,32 +0,0 @@
-/*******************************************************************************
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- ******************************************************************************/
-package org.apache.drill.common.physical.pop.base;
-
-import java.util.Iterator;
-
-import com.google.common.collect.Iterators;
-
-public abstract class AbstractReceiver extends AbstractBase implements Receiver{
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(AbstractReceiver.class);
-
-  @Override
-  public Iterator<PhysicalOperator> iterator() {
-    return Iterators.emptyIterator();
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/AbstractScan.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/AbstractScan.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/AbstractScan.java
deleted file mode 100644
index 3727139..0000000
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/AbstractScan.java
+++ /dev/null
@@ -1,62 +0,0 @@
-/*******************************************************************************
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- ******************************************************************************/
-package org.apache.drill.common.physical.pop.base;
-
-import java.util.Iterator;
-import java.util.List;
-
-import org.apache.drill.common.physical.ReadEntry;
-import org.apache.drill.common.proto.CoordinationProtos.DrillbitEndpoint;
-
-import com.fasterxml.jackson.annotation.JsonProperty;
-import com.google.common.collect.Iterators;
-
-public abstract class AbstractScan<R extends ReadEntry> extends AbstractBase implements Scan<R>{
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(AbstractScan.class);
-  
-  private List<R> readEntries;
-  
-  public AbstractScan(List<R> readEntries) {
-    this.readEntries = readEntries;
-  }
-
-  @Override
-  @JsonProperty("entries")
-  public List<R> getReadEntries() {
-    return readEntries;
-  }
-  
-  @Override
-  public Iterator<PhysicalOperator> iterator() {
-    return Iterators.emptyIterator();
-  }
-
-  @Override
-  public boolean isExecutable() {
-    return true;
-  }
-
-  @Override
-  public <T, X, E extends Throwable> T accept(PhysicalVisitor<T, X, E> physicalVisitor, X value) throws E{
-    return physicalVisitor.visitScan(this, value);
-  }
-  
-  
-  
-  
-}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/AbstractSender.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/AbstractSender.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/AbstractSender.java
deleted file mode 100644
index 8b0608a..0000000
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/AbstractSender.java
+++ /dev/null
@@ -1,29 +0,0 @@
-/*******************************************************************************
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- ******************************************************************************/
-package org.apache.drill.common.physical.pop.base;
-
-
-
-public abstract class AbstractSender extends AbstractSingle implements Sender {
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(AbstractSender.class);
-
-  public AbstractSender(PhysicalOperator child) {
-    super(child);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/AbstractSingle.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/AbstractSingle.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/AbstractSingle.java
deleted file mode 100644
index 49358df..0000000
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/AbstractSingle.java
+++ /dev/null
@@ -1,48 +0,0 @@
-/*******************************************************************************
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- ******************************************************************************/
-package org.apache.drill.common.physical.pop.base;
-
-import java.util.Iterator;
-
-
-import com.google.common.collect.Iterators;
-
-/**
- * Describes an operator that expects a single child operator as its input.
- * @param <T> The type of Exec model supported.
- */
-public abstract class AbstractSingle extends AbstractBase{
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(AbstractSingle.class);
-  
-  private final PhysicalOperator child;
-
-  public AbstractSingle(PhysicalOperator child) {
-    super();
-    this.child = child;
-  }
-
-  @Override
-  public Iterator<PhysicalOperator> iterator() {
-    return Iterators.singletonIterator(child);
-  }
-
-  public PhysicalOperator getChild(){
-    return child;
-  }
-  
-}