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/05/01 00:05:37 UTC

[1/6] WIP fragmentation, physical plan, byte compiling, some vector work

Updated Branches:
  refs/heads/execwork 5ede21ffc -> f0be80dcd


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>


[5/6] 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
  */


[6/6] 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/execwork
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 +-
 .../org/apache/drill/common/defs/PartitionDef.java |   14 +-
 .../drill/common/expression/types/DataType.java    |    4 +
 .../drill/common/physical/EndpointAffinity.java    |   60 ++++
 .../apache/drill/common/physical/OperatorCost.java |   61 ++++
 .../apache/drill/common/physical/POPConfig.java    |   24 --
 .../org/apache/drill/common/physical/POPCost.java  |   34 --
 .../apache/drill/common/physical/PhysicalPlan.java |   37 ++-
 .../org/apache/drill/common/physical/SetSpec.java  |   36 --
 .../apache/drill/common/physical/StitchDef.java    |   48 ---
 .../drill/common/physical/pop/ExchangePOP.java     |   54 ----
 .../drill/common/physical/pop/FieldCombinePOP.java |   28 --
 .../common/physical/pop/FieldSubdividePOP.java     |   22 --
 .../apache/drill/common/physical/pop/Filter.java   |   52 +++
 .../apache/drill/common/physical/pop/POPBase.java  |   65 ----
 .../physical/pop/PartitionToRandomExchange.java    |   92 ++++++
 .../common/physical/pop/PhysicalOperator.java      |   35 --
 .../apache/drill/common/physical/pop/Project.java  |   53 +++
 .../drill/common/physical/pop/ProjectPOP.java      |   53 ---
 .../common/physical/pop/QuickNWaySortPOP.java      |   50 ---
 .../apache/drill/common/physical/pop/ScanPOP.java  |   55 ----
 .../apache/drill/common/physical/pop/Screen.java   |   77 +++++
 .../drill/common/physical/pop/SingleChildPOP.java  |   41 ---
 .../apache/drill/common/physical/pop/SinkPOP.java  |   22 --
 .../org/apache/drill/common/physical/pop/Sort.java |   57 ++++
 .../apache/drill/common/physical/pop/SortPOP.java  |   54 ----
 .../drill/common/physical/pop/SourcePOP.java       |   22 --
 .../apache/drill/common/physical/pop/StorePOP.java |   61 ----
 .../common/physical/pop/base/AbstractBase.java     |   56 ++++
 .../common/physical/pop/base/AbstractExchange.java |   68 ++++
 .../physical/pop/base/AbstractPhysicalVisitor.java |   80 +++++
 .../common/physical/pop/base/AbstractReceiver.java |   32 ++
 .../common/physical/pop/base/AbstractScan.java     |   62 ++++
 .../common/physical/pop/base/AbstractSender.java   |   29 ++
 .../common/physical/pop/base/AbstractSingle.java   |   48 +++
 .../common/physical/pop/base/AbstractStore.java    |   42 +++
 .../drill/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 ++
 .../common/physical/pop/base/PhysicalOperator.java |   59 ++++
 .../physical/pop/base/PhysicalOperatorUtil.java    |   34 ++
 .../common/physical/pop/base/PhysicalVisitor.java  |   43 +++
 .../drill/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 ---
 .../drill/common/physical/props/PartitionProp.java |   36 --
 .../drill/common/physical/props/PhysicalProp.java  |   24 --
 .../drill/common/physical/props/SegmentProp.java   |   42 ---
 .../common/src/main/protobuf/Coordination.proto    |   26 ++
 .../apache/drill/common/physical/MockScanPOP.java  |   30 ++-
 .../apache/drill/common/physical/MockStorePOP.java |   46 ++--
 .../drill/common/physical/ParsePhysicalPlan.java   |    8 +-
 .../apache/drill/common/physical/ParsePlan.java    |   36 --
 .../common/src/test/resources/drill-module.conf    |    2 +-
 .../common/src/test/resources/dsort-physical.json  |   66 ----
 .../common/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 ++++
 .../apache/drill/exec/DirectBufferAllocator.java   |   47 ---
 .../java/org/apache/drill/exec/ExecConstants.java  |    1 +
 .../apache/drill/exec/cache/DistributedCache.java  |    8 +-
 .../org/apache/drill/exec/cache/HazelCache.java    |   15 +-
 .../org/apache/drill/exec/client/DrillClient.java  |   20 +-
 .../drill/exec/compile/ClassBodyBuilder.java       |  247 +++++++++++++++
 .../apache/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 +++
 .../apache/drill/exec/foreman/CancelableQuery.java |   22 ++
 .../drill/exec/foreman/ExecutionPlanner.java       |   24 ++
 .../org/apache/drill/exec/foreman/Foreman.java     |   39 +++
 .../apache/drill/exec/foreman/QueryWorkUnit.java   |   54 ++++
 .../apache/drill/exec/foreman/ResourceRequest.java |   30 ++
 .../apache/drill/exec/foreman/StatusProvider.java  |   24 ++
 .../apache/drill/exec/memory/BufferAllocator.java  |   58 ++++
 .../drill/exec/memory/DirectBufferAllocator.java   |   58 ++++
 .../exec/metrics/SingleThreadNestedCounter.java    |   55 ++++
 .../org/apache/drill/exec/ops/BatchIterator.java   |   32 --
 .../exec/ops/FilteringRecordBatchTransformer.java  |   58 ++++
 .../org/apache/drill/exec/ops/FragmentContext.java |   47 +++-
 .../apache/drill/exec/ops/FragmentConverter.java   |   30 ++
 .../org/apache/drill/exec/ops/FragmentRoot.java    |   37 +++
 .../org/apache/drill/exec/ops/OperatorFactory.java |   22 ++
 .../org/apache/drill/exec/ops/QueryContext.java    |   51 +++
 .../org/apache/drill/exec/ops/QueryOutcome.java    |   22 --
 .../java/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 --
 .../drill/exec/ops/filter/FilterRecordBatch.java   |  109 +++++++
 .../exec/ops/filter/SelectionVectorUpdater.java    |   80 +++++
 .../org/apache/drill/exec/planner/ExecPlanner.java |    9 +-
 .../drill/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 ++
 .../apache/drill/exec/planner/FragmentStats.java   |   63 ++++
 .../drill/exec/planner/FragmentStatsCollector.java |  109 +++++++
 .../apache/drill/exec/planner/FragmentVisitor.java |   22 ++
 .../apache/drill/exec/planner/FragmentWrapper.java |  127 ++++++++
 .../exec/planner/FragmentingPhysicalVisitor.java   |   71 ++++
 .../drill/exec/planner/MaterializedFragment.java   |   69 ++++
 .../drill/exec/planner/PhysicalPlanReader.java     |   47 +++
 .../org/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 ++++
 .../drill/exec/pop/sender/HashPartitionSender.java |   49 +++
 .../org/apache/drill/exec/record/BatchSchema.java  |   29 ++-
 .../drill/exec/record/MaterializedField.java       |    4 +
 .../record/vector/AbstractFixedValueVector.java    |    2 +-
 .../drill/exec/record/vector/BaseValueVector.java  |    6 +-
 .../apache/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 +-
 .../drill/exec/record/vector/SelectionVector.java  |   31 ++
 .../drill/exec/record/vector/UInt16Vector.java     |   51 +++
 .../drill/exec/record/vector/ValueVector.java      |    1 +
 .../drill/exec/record/vector/VariableVector.java   |    2 +-
 .../java/org/apache/drill/exec/rpc/bit/BitCom.java |   26 ++-
 .../apache/drill/exec/rpc/bit/BitComHandler.java   |    2 +-
 .../org/apache/drill/exec/rpc/bit/BitComImpl.java  |   24 +-
 .../apache/drill/exec/rpc/bit/SendProgress.java    |   22 ++
 .../org/apache/drill/exec/rpc/user/UserClient.java |    4 +
 .../org/apache/drill/exec/rpc/user/UserServer.java |    2 +-
 .../org/apache/drill/exec/server/Drillbit.java     |   11 +-
 .../apache/drill/exec/server/DrillbitContext.java  |   27 ++-
 .../drill/exec/store/AbstractStorageEngine.java    |   83 +++++
 .../org/apache/drill/exec/store/StorageEngine.java |   32 ++-
 .../drill/exec/store/StorageEngineRegistry.java    |    4 +-
 .../java-exec/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 ---
 .../exec/java-exec/src/test/java/ExternalSort.java |   21 --
 .../src/test/java/GenerateExternalSortData.java    |  124 --------
 .../exec/compile/ExampleExternalInterface.java     |   23 ++
 .../exec/compile/ExampleInternalInterface.java     |   24 ++
 .../apache/drill/exec/compile/ExampleTemplate.java |   30 ++
 .../exec/compile/TestClassCompilationTypes.java    |   67 ++++
 .../exec/compile/TestClassTransformation.java      |   53 +++
 .../org/apache/drill/exec/pop/CheckFragmenter.java |   86 +++++
 .../apache/drill/exec/pop/CheckInjectionValue.java |   61 ++++
 .../apache/drill/exec/rpc/user/RunRemoteQuery.java |   41 +++
 .../apache/drill/exec/server/StartDrillbit.java    |    4 +-
 .../apache/drill/exec/store/MockRecordConfig.java  |   46 +++
 .../apache/drill/exec/store/MockRecordReader.java  |  108 +++++++
 .../apache/drill/exec/store/MockStorageEngine.java |   54 ++++
 .../java-exec/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();
+  }
+
+}


[3/6] 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);


[2/6] 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;
+  }
+  
+  
+}


[4/6] 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<?>>();