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/14 03:52:41 UTC

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

Updated Branches:
  refs/heads/execwork f0be80dcd -> e57a8d6d4


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


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

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

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

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

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

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

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

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

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

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

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

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

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

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/RpcExceptionHandler.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/RpcExceptionHandler.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/RpcExceptionHandler.java
index ef1b88f..a0aed94 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/RpcExceptionHandler.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/RpcExceptionHandler.java
@@ -25,28 +25,23 @@ public class RpcExceptionHandler implements ChannelHandler{
   
   public RpcExceptionHandler(){
   }
-  
-  @Override
-  public void beforeAdd(ChannelHandlerContext ctx) throws Exception {
-  }
 
-  @Override
-  public void afterAdd(ChannelHandlerContext ctx) throws Exception {
-  }
 
   @Override
-  public void beforeRemove(ChannelHandlerContext ctx) throws Exception {
+  public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) throws Exception {
+    if(!ctx.channel().isOpen()) return;
+    logger.info("Exception in pipeline.  Closing channel between local " + ctx.channel().localAddress() + " and remote " + ctx.channel().remoteAddress(), cause);
+    ctx.close();
   }
 
+
   @Override
-  public void afterRemove(ChannelHandlerContext ctx) throws Exception {
+  public void handlerAdded(ChannelHandlerContext ctx) throws Exception {
   }
 
+
   @Override
-  public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) throws Exception {
-    if(!ctx.channel().isOpen()) return;
-    logger.info("Exception in pipeline.  Closing channel between local " + ctx.channel().localAddress() + " and remote " + ctx.channel().remoteAddress(), cause);
-    ctx.close();
+  public void handlerRemoved(ChannelHandlerContext ctx) throws Exception {
   }
   
 }

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

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

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

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

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

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

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

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

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


Re: [13/13] git commit: Update typing system. Update RPC system. Add Fragmenting Implementation. Working single node. Distributed failing due to threading issues.

Posted by Ted Dunning <te...@gmail.com>.

Sent from my iPhone

On May 13, 2013, at 18:52, jacques@apache.org wrote:

> Update typing system.  Update RPC system.  Add Fragmenting Implementation.  Working single node. Distributed failing due to threading issues.
> 
> 
> Project: http://git-wip-us.apache.org/repos/asf/incubator-drill/repo
> Commit: http://git-wip-us.apache.org/repos/asf/incubator-drill/commit/e57a8d6d
> Tree: http://git-wip-us.apache.org/repos/asf/incubator-drill/tree/e57a8d6d
> Diff: http://git-wip-us.apache.org/repos/asf/incubator-drill/diff/e57a8d6d
> 
> Branch: refs/heads/execwork
> Commit: e57a8d6d4ae282a79bc6e0a7354de992c391300f
> Parents: f0be80d
> Author: Jacques Nadeau <ja...@apache.org>
> Authored: Sun May 5 20:18:32 2013 -0700
> Committer: Jacques Nadeau <ja...@apache.org>
> Committed: Mon May 13 18:50:31 2013 -0700
> 
> ----------------------------------------------------------------------
> sandbox/prototype/common/pom.xml                   |   12 +
> .../apache/drill/common/config/DrillConfig.java    |    9 +-
> .../common/exceptions/ExecutionSetupException.java |   44 +++
> .../exceptions/PhysicalOperatorSetupException.java |   45 +++
> .../apache/drill/common/expression/SchemaPath.java |    2 +-
> .../drill/common/expression/types/DataType.java    |   11 +
> .../apache/drill/common/optimize/Optimizer.java    |   45 ---
> .../drill/common/physical/DataValidationMode.java  |   24 --
> .../drill/common/physical/EndpointAffinity.java    |   60 ----
> .../org/apache/drill/common/physical/FieldSet.java |   80 -----
> .../apache/drill/common/physical/OperatorCost.java |   61 ----
> .../apache/drill/common/physical/PhysicalPlan.java |   95 -----
> .../apache/drill/common/physical/ReadEntry.java    |   25 --
> .../apache/drill/common/physical/RecordField.java  |   60 ----
> .../apache/drill/common/physical/WriteEntry.java   |   22 --
> .../apache/drill/common/physical/pop/Filter.java   |   52 ---
> .../physical/pop/PartitionToRandomExchange.java    |   92 -----
> .../apache/drill/common/physical/pop/Project.java  |   53 ---
> .../apache/drill/common/physical/pop/Screen.java   |   77 ----
> .../org/apache/drill/common/physical/pop/Sort.java |   57 ---
> .../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 --
> .../common/src/main/protobuf/Coordination.proto    |   26 --
> .../apache/drill/common/physical/MockScanPOP.java  |   69 ----
> .../apache/drill/common/physical/MockStorePOP.java |   62 ----
> .../drill/common/physical/ParsePhysicalPlan.java   |   41 ---
> .../common/src/test/resources/drill-module.conf    |    1 -
> .../common/src/test/resources/physical_test1.json  |   33 --
> sandbox/prototype/exec/java-exec/pom.xml           |    4 +-
> .../java/org/apache/drill/exec/ByteReorder.java    |   54 ---
> .../apache/drill/exec/cache/DistributedCache.java  |   14 +-
> .../org/apache/drill/exec/cache/HazelCache.java    |   87 ++----
> .../org/apache/drill/exec/cache/LocalCache.java    |   55 +++
> .../org/apache/drill/exec/cache/ProtoBufImpl.java  |   50 +++
> .../org/apache/drill/exec/cache/ProtoBufWrap.java  |   69 ++++
> .../java/org/apache/drill/exec/cache/ProtoMap.java |   52 +++
> .../drill/exec/cache/TemplatizedLogicalPlan.java   |   22 --
> .../drill/exec/cache/TemplatizedPhysicalPlan.java  |   22 --
> .../org/apache/drill/exec/client/DrillClient.java  |   73 ++++-
> .../drill/exec/coord/ClusterCoordinator.java       |    9 +-
> .../exec/coord/DrillServiceInstanceHelper.java     |    4 +-
> .../drill/exec/coord/DrillbitEndpointSerDe.java    |   65 ++++
> .../drill/exec/coord/LocalClusterCoordinator.java  |   95 +++++
> .../drill/exec/coord/ZKClusterCoordinator.java     |   30 ++-
> .../drill/exec/exception/BitComException.java      |   45 +++
> .../exec/exception/ExecutionSetupException.java    |   45 ---
> .../exec/exception/FragmentSetupException.java     |    2 +
> .../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  |    5 +-
> .../exec/metrics/SingleThreadNestedCounter.java    |   22 +-
> .../exec/ops/FilteringRecordBatchTransformer.java  |   58 ---
> .../org/apache/drill/exec/ops/FragmentContext.java |   54 +++-
> .../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/OutputMutator.java   |   28 --
> .../org/apache/drill/exec/ops/QueryContext.java    |   27 +-
> .../java/org/apache/drill/exec/ops/ScanBatch.java  |  157 ---------
> .../drill/exec/ops/StreamingRecordBatch.java       |   25 --
> .../exec/ops/exchange/ExchangeRecordBatch.java     |   22 --
> .../drill/exec/ops/exchange/RecordBatchSender.java |   24 --
> .../drill/exec/ops/filter/FilterRecordBatch.java   |  109 ------
> .../exec/ops/filter/SelectionVectorUpdater.java    |   80 -----
> .../apache/drill/exec/opt/IdentityOptimizer.java   |    3 +-
> .../java/org/apache/drill/exec/opt/Optimizer.java  |   45 +++
> .../drill/exec/physical/DataValidationMode.java    |   24 ++
> .../drill/exec/physical/EndpointAffinity.java      |   60 ++++
> .../apache/drill/exec/physical/OperatorCost.java   |   66 ++++
> .../apache/drill/exec/physical/PhysicalPlan.java   |   94 +++++
> .../org/apache/drill/exec/physical/ReadEntry.java  |   33 ++
> .../apache/drill/exec/physical/RecordField.java    |   60 ++++
> .../org/apache/drill/exec/physical/WriteEntry.java |   22 ++
> .../drill/exec/physical/base/AbstractBase.java     |   43 +++
> .../drill/exec/physical/base/AbstractExchange.java |   90 +++++
> .../physical/base/AbstractPhysicalVisitor.java     |  124 +++++++
> .../drill/exec/physical/base/AbstractReceiver.java |   63 ++++
> .../drill/exec/physical/base/AbstractScan.java     |   84 +++++
> .../drill/exec/physical/base/AbstractSender.java   |   53 +++
> .../drill/exec/physical/base/AbstractSingle.java   |   68 ++++
> .../drill/exec/physical/base/AbstractStore.java    |   36 ++
> .../apache/drill/exec/physical/base/Exchange.java  |   92 +++++
> .../drill/exec/physical/base/ExchangeCost.java     |   68 ++++
> .../drill/exec/physical/base/FragmentLeaf.java     |   25 ++
> .../drill/exec/physical/base/FragmentRoot.java     |   25 ++
> .../drill/exec/physical/base/HasAffinity.java      |   37 ++
> .../org/apache/drill/exec/physical/base/Leaf.java  |   25 ++
> .../drill/exec/physical/base/PhysicalOperator.java |   80 +++++
> .../exec/physical/base/PhysicalOperatorUtil.java   |   34 ++
> .../drill/exec/physical/base/PhysicalVisitor.java  |   61 ++++
> .../apache/drill/exec/physical/base/Receiver.java  |   51 +++
> .../org/apache/drill/exec/physical/base/Root.java  |   24 ++
> .../org/apache/drill/exec/physical/base/Scan.java  |   36 ++
> .../apache/drill/exec/physical/base/Sender.java    |   44 +++
> .../org/apache/drill/exec/physical/base/Size.java  |   48 +++
> .../org/apache/drill/exec/physical/base/Store.java |   74 ++++
> .../apache/drill/exec/physical/config/Filter.java  |   75 ++++
> .../exec/physical/config/HashPartitionSender.java  |   58 +++
> .../exec/physical/config/HashToRandomExchange.java |   86 +++++
> .../exec/physical/config/MockRecordReader.java     |  113 ++++++
> .../exec/physical/config/MockScanBatchCreator.java |   49 +++
> .../drill/exec/physical/config/MockScanPOP.java    |  193 ++++++++++
> .../exec/physical/config/MockStorageEngine.java    |   58 +++
> .../drill/exec/physical/config/MockStorePOP.java   |   75 ++++
> .../drill/exec/physical/config/PartitionRange.java |   47 +++
> .../apache/drill/exec/physical/config/Project.java |   72 ++++
> .../drill/exec/physical/config/RandomReceiver.java |   83 +++++
> .../drill/exec/physical/config/RangeSender.java    |   72 ++++
> .../apache/drill/exec/physical/config/Screen.java  |  106 ++++++
> .../drill/exec/physical/config/SingleSender.java   |   78 ++++
> .../apache/drill/exec/physical/config/Sort.java    |   86 +++++
> .../drill/exec/physical/config/UnionExchange.java  |   79 +++++
> .../drill/exec/physical/impl/BatchCreator.java     |   31 ++
> .../exec/physical/impl/FilterRecordBatch.java      |  108 ++++++
> .../impl/FilteringRecordBatchTransformer.java      |   58 +++
> .../drill/exec/physical/impl/ImplCreator.java      |  102 ++++++
> .../drill/exec/physical/impl/OutputMutator.java    |   28 ++
> .../drill/exec/physical/impl/PhysicalConfig.java   |   29 ++
> .../exec/physical/impl/RandomReceiverCreator.java  |   46 +++
> .../drill/exec/physical/impl/RootCreator.java      |   31 ++
> .../apache/drill/exec/physical/impl/RootExec.java  |   40 +++
> .../apache/drill/exec/physical/impl/ScanBatch.java |  172 +++++++++
> .../drill/exec/physical/impl/ScreenCreator.java    |   90 +++++
> .../exec/physical/impl/SingleSenderCreator.java    |   89 +++++
> .../drill/exec/physical/impl/WireRecordBatch.java  |   99 ++++++
> .../impl/materialize/QueryWritableBatch.java       |   46 +++
> .../impl/materialize/RecordMaterializer.java       |   31 ++
> .../impl/materialize/VectorRecordMaterializer.java |   52 +++
> .../exec/planner/AbstractOpWrapperVisitor.java     |   45 +++
> .../org/apache/drill/exec/planner/ExecPlanner.java |    8 +-
> .../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     |   63 +++-
> .../org/apache/drill/exec/planner/ScanFinder.java  |   54 ---
> .../drill/exec/planner/SimpleExecPlanner.java      |   28 +-
> .../drill/exec/planner/SimpleParallelizer.java     |  147 --------
> .../drill/exec/planner/fragment/Fragment.java      |  150 ++++++++
> .../exec/planner/fragment/FragmentVisitor.java     |   23 ++
> .../planner/fragment/MakeFragmentsVisitor.java     |   69 ++++
> .../planner/fragment/MaterializedFragment.java     |   69 ++++
> .../drill/exec/planner/fragment/Materializer.java  |  107 ++++++
> .../drill/exec/planner/fragment/PlanningSet.java   |   66 ++++
> .../exec/planner/fragment/SimpleParallelizer.java  |  163 +++++++++
> .../apache/drill/exec/planner/fragment/Stats.java  |   70 ++++
> .../exec/planner/fragment/StatsCollector.java      |  106 ++++++
> .../drill/exec/planner/fragment/Wrapper.java       |  186 ++++++++++
> .../exec/pop/receiver/NWayOrderingReceiver.java    |   52 ---
> .../drill/exec/pop/receiver/RandomReceiver.java    |   55 ---
> .../drill/exec/pop/sender/HashPartitionSender.java |   49 ---
> .../org/apache/drill/exec/record/BatchSchema.java  |  102 +-----
> .../drill/exec/record/FragmentWritableBatch.java   |   59 +++
> .../drill/exec/record/InvalidValueAccessor.java    |    2 +-
> .../apache/drill/exec/record/MajorTypeSerDe.java   |  116 ++++++
> .../drill/exec/record/MaterializedField.java       |  167 +++++++---
> .../apache/drill/exec/record/RawFragmentBatch.java |   44 +++
> .../exec/record/RawFragmentBatchProvider.java      |   27 ++
> .../org/apache/drill/exec/record/RecordBatch.java  |    9 +-
> .../drill/exec/record/RecordBatchLoader.java       |  143 ++++++++
> .../apache/drill/exec/record/SchemaBuilder.java    |  127 +++++++
> .../apache/drill/exec/record/WritableBatch.java    |  108 ++++++
> .../record/vector/AbstractFixedValueVector.java    |   21 +-
> .../drill/exec/record/vector/BaseValueVector.java  |   86 ++++-
> .../org/apache/drill/exec/record/vector/Bit.java   |  168 +++++++++
> .../apache/drill/exec/record/vector/BitVector.java |  166 ---------
> .../drill/exec/record/vector/ByteVector.java       |   48 ---
> .../apache/drill/exec/record/vector/Fixed1.java    |   43 +++
> .../apache/drill/exec/record/vector/Fixed12.java   |   35 ++
> .../apache/drill/exec/record/vector/Fixed16.java   |   37 ++
> .../apache/drill/exec/record/vector/Fixed2.java    |   53 +++
> .../apache/drill/exec/record/vector/Fixed4.java    |   55 +++
> .../apache/drill/exec/record/vector/Fixed8.java    |   58 +++
> .../apache/drill/exec/record/vector/FixedLen.java  |   45 +++
> .../drill/exec/record/vector/Int16Vector.java      |   52 ---
> .../drill/exec/record/vector/Int32Vector.java      |   52 ---
> .../drill/exec/record/vector/NullableFixed4.java   |   37 ++
> .../exec/record/vector/NullableInt32Vector.java    |   47 ---
> .../exec/record/vector/NullableValueVector.java    |   36 ++-
> .../apache/drill/exec/record/vector/RepeatMap.java |   57 +++
> .../drill/exec/record/vector/SelectionVector.java  |   10 +-
> .../drill/exec/record/vector/TypeHelper.java       |  250 +++++++++++++
> .../drill/exec/record/vector/UInt16Vector.java     |   51 ---
> .../drill/exec/record/vector/ValueVector.java      |   46 +++-
> .../apache/drill/exec/record/vector/VarLen1.java   |   36 ++
> .../apache/drill/exec/record/vector/VarLen2.java   |   36 ++
> .../apache/drill/exec/record/vector/VarLen4.java   |   36 ++
> .../drill/exec/record/vector/VariableVector.java   |   35 ++-
> .../drill/exec/rpc/AbstractHandshakeHandler.java   |   57 +++
> .../main/java/org/apache/drill/exec/rpc/Acks.java  |   27 ++
> .../org/apache/drill/exec/rpc/BasicClient.java     |  137 +++++++-
> .../drill/exec/rpc/BasicClientWithConnection.java  |   64 ++++
> .../org/apache/drill/exec/rpc/BasicServer.java     |   61 +++-
> .../apache/drill/exec/rpc/CoordinationQueue.java   |   32 +-
> .../org/apache/drill/exec/rpc/DrillRpcFuture.java  |   78 +----
> .../apache/drill/exec/rpc/DrillRpcFutureImpl.java  |  118 +++++++
> .../apache/drill/exec/rpc/InboundRpcMessage.java   |   13 +-
> .../apache/drill/exec/rpc/OutboundRpcMessage.java  |   41 ++-
> .../apache/drill/exec/rpc/RemoteConnection.java    |   45 +++
> .../java/org/apache/drill/exec/rpc/Response.java   |    8 +-
> .../java/org/apache/drill/exec/rpc/RpcBus.java     |  136 +++++---
> .../java/org/apache/drill/exec/rpc/RpcConfig.java  |  150 ++++++++
> .../java/org/apache/drill/exec/rpc/RpcDecoder.java |   14 +-
> .../java/org/apache/drill/exec/rpc/RpcEncoder.java |   15 +-
> .../apache/drill/exec/rpc/RpcExceptionHandler.java |   21 +-
> .../java/org/apache/drill/exec/rpc/RpcMessage.java |    9 +-
> .../apache/drill/exec/rpc/RpcOutcomeListener.java  |   28 ++
> .../exec/rpc/ZeroCopyProtobufLengthDecoder.java    |   16 +-
> .../drill/exec/rpc/bit/AvailabilityListener.java   |   22 ++
> .../org/apache/drill/exec/rpc/bit/BitClient.java   |   77 ++++-
> .../java/org/apa

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

Posted by ja...@apache.org.
Update typing system.  Update RPC system.  Add Fragmenting Implementation.  Working single node. Distributed failing due to threading issues.


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

Branch: refs/heads/execwork
Commit: e57a8d6d4ae282a79bc6e0a7354de992c391300f
Parents: f0be80d
Author: Jacques Nadeau <ja...@apache.org>
Authored: Sun May 5 20:18:32 2013 -0700
Committer: Jacques Nadeau <ja...@apache.org>
Committed: Mon May 13 18:50:31 2013 -0700

----------------------------------------------------------------------
 sandbox/prototype/common/pom.xml                   |   12 +
 .../apache/drill/common/config/DrillConfig.java    |    9 +-
 .../common/exceptions/ExecutionSetupException.java |   44 +++
 .../exceptions/PhysicalOperatorSetupException.java |   45 +++
 .../apache/drill/common/expression/SchemaPath.java |    2 +-
 .../drill/common/expression/types/DataType.java    |   11 +
 .../apache/drill/common/optimize/Optimizer.java    |   45 ---
 .../drill/common/physical/DataValidationMode.java  |   24 --
 .../drill/common/physical/EndpointAffinity.java    |   60 ----
 .../org/apache/drill/common/physical/FieldSet.java |   80 -----
 .../apache/drill/common/physical/OperatorCost.java |   61 ----
 .../apache/drill/common/physical/PhysicalPlan.java |   95 -----
 .../apache/drill/common/physical/ReadEntry.java    |   25 --
 .../apache/drill/common/physical/RecordField.java  |   60 ----
 .../apache/drill/common/physical/WriteEntry.java   |   22 --
 .../apache/drill/common/physical/pop/Filter.java   |   52 ---
 .../physical/pop/PartitionToRandomExchange.java    |   92 -----
 .../apache/drill/common/physical/pop/Project.java  |   53 ---
 .../apache/drill/common/physical/pop/Screen.java   |   77 ----
 .../org/apache/drill/common/physical/pop/Sort.java |   57 ---
 .../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 --
 .../common/src/main/protobuf/Coordination.proto    |   26 --
 .../apache/drill/common/physical/MockScanPOP.java  |   69 ----
 .../apache/drill/common/physical/MockStorePOP.java |   62 ----
 .../drill/common/physical/ParsePhysicalPlan.java   |   41 ---
 .../common/src/test/resources/drill-module.conf    |    1 -
 .../common/src/test/resources/physical_test1.json  |   33 --
 sandbox/prototype/exec/java-exec/pom.xml           |    4 +-
 .../java/org/apache/drill/exec/ByteReorder.java    |   54 ---
 .../apache/drill/exec/cache/DistributedCache.java  |   14 +-
 .../org/apache/drill/exec/cache/HazelCache.java    |   87 ++----
 .../org/apache/drill/exec/cache/LocalCache.java    |   55 +++
 .../org/apache/drill/exec/cache/ProtoBufImpl.java  |   50 +++
 .../org/apache/drill/exec/cache/ProtoBufWrap.java  |   69 ++++
 .../java/org/apache/drill/exec/cache/ProtoMap.java |   52 +++
 .../drill/exec/cache/TemplatizedLogicalPlan.java   |   22 --
 .../drill/exec/cache/TemplatizedPhysicalPlan.java  |   22 --
 .../org/apache/drill/exec/client/DrillClient.java  |   73 ++++-
 .../drill/exec/coord/ClusterCoordinator.java       |    9 +-
 .../exec/coord/DrillServiceInstanceHelper.java     |    4 +-
 .../drill/exec/coord/DrillbitEndpointSerDe.java    |   65 ++++
 .../drill/exec/coord/LocalClusterCoordinator.java  |   95 +++++
 .../drill/exec/coord/ZKClusterCoordinator.java     |   30 ++-
 .../drill/exec/exception/BitComException.java      |   45 +++
 .../exec/exception/ExecutionSetupException.java    |   45 ---
 .../exec/exception/FragmentSetupException.java     |    2 +
 .../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  |    5 +-
 .../exec/metrics/SingleThreadNestedCounter.java    |   22 +-
 .../exec/ops/FilteringRecordBatchTransformer.java  |   58 ---
 .../org/apache/drill/exec/ops/FragmentContext.java |   54 +++-
 .../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/OutputMutator.java   |   28 --
 .../org/apache/drill/exec/ops/QueryContext.java    |   27 +-
 .../java/org/apache/drill/exec/ops/ScanBatch.java  |  157 ---------
 .../drill/exec/ops/StreamingRecordBatch.java       |   25 --
 .../exec/ops/exchange/ExchangeRecordBatch.java     |   22 --
 .../drill/exec/ops/exchange/RecordBatchSender.java |   24 --
 .../drill/exec/ops/filter/FilterRecordBatch.java   |  109 ------
 .../exec/ops/filter/SelectionVectorUpdater.java    |   80 -----
 .../apache/drill/exec/opt/IdentityOptimizer.java   |    3 +-
 .../java/org/apache/drill/exec/opt/Optimizer.java  |   45 +++
 .../drill/exec/physical/DataValidationMode.java    |   24 ++
 .../drill/exec/physical/EndpointAffinity.java      |   60 ++++
 .../apache/drill/exec/physical/OperatorCost.java   |   66 ++++
 .../apache/drill/exec/physical/PhysicalPlan.java   |   94 +++++
 .../org/apache/drill/exec/physical/ReadEntry.java  |   33 ++
 .../apache/drill/exec/physical/RecordField.java    |   60 ++++
 .../org/apache/drill/exec/physical/WriteEntry.java |   22 ++
 .../drill/exec/physical/base/AbstractBase.java     |   43 +++
 .../drill/exec/physical/base/AbstractExchange.java |   90 +++++
 .../physical/base/AbstractPhysicalVisitor.java     |  124 +++++++
 .../drill/exec/physical/base/AbstractReceiver.java |   63 ++++
 .../drill/exec/physical/base/AbstractScan.java     |   84 +++++
 .../drill/exec/physical/base/AbstractSender.java   |   53 +++
 .../drill/exec/physical/base/AbstractSingle.java   |   68 ++++
 .../drill/exec/physical/base/AbstractStore.java    |   36 ++
 .../apache/drill/exec/physical/base/Exchange.java  |   92 +++++
 .../drill/exec/physical/base/ExchangeCost.java     |   68 ++++
 .../drill/exec/physical/base/FragmentLeaf.java     |   25 ++
 .../drill/exec/physical/base/FragmentRoot.java     |   25 ++
 .../drill/exec/physical/base/HasAffinity.java      |   37 ++
 .../org/apache/drill/exec/physical/base/Leaf.java  |   25 ++
 .../drill/exec/physical/base/PhysicalOperator.java |   80 +++++
 .../exec/physical/base/PhysicalOperatorUtil.java   |   34 ++
 .../drill/exec/physical/base/PhysicalVisitor.java  |   61 ++++
 .../apache/drill/exec/physical/base/Receiver.java  |   51 +++
 .../org/apache/drill/exec/physical/base/Root.java  |   24 ++
 .../org/apache/drill/exec/physical/base/Scan.java  |   36 ++
 .../apache/drill/exec/physical/base/Sender.java    |   44 +++
 .../org/apache/drill/exec/physical/base/Size.java  |   48 +++
 .../org/apache/drill/exec/physical/base/Store.java |   74 ++++
 .../apache/drill/exec/physical/config/Filter.java  |   75 ++++
 .../exec/physical/config/HashPartitionSender.java  |   58 +++
 .../exec/physical/config/HashToRandomExchange.java |   86 +++++
 .../exec/physical/config/MockRecordReader.java     |  113 ++++++
 .../exec/physical/config/MockScanBatchCreator.java |   49 +++
 .../drill/exec/physical/config/MockScanPOP.java    |  193 ++++++++++
 .../exec/physical/config/MockStorageEngine.java    |   58 +++
 .../drill/exec/physical/config/MockStorePOP.java   |   75 ++++
 .../drill/exec/physical/config/PartitionRange.java |   47 +++
 .../apache/drill/exec/physical/config/Project.java |   72 ++++
 .../drill/exec/physical/config/RandomReceiver.java |   83 +++++
 .../drill/exec/physical/config/RangeSender.java    |   72 ++++
 .../apache/drill/exec/physical/config/Screen.java  |  106 ++++++
 .../drill/exec/physical/config/SingleSender.java   |   78 ++++
 .../apache/drill/exec/physical/config/Sort.java    |   86 +++++
 .../drill/exec/physical/config/UnionExchange.java  |   79 +++++
 .../drill/exec/physical/impl/BatchCreator.java     |   31 ++
 .../exec/physical/impl/FilterRecordBatch.java      |  108 ++++++
 .../impl/FilteringRecordBatchTransformer.java      |   58 +++
 .../drill/exec/physical/impl/ImplCreator.java      |  102 ++++++
 .../drill/exec/physical/impl/OutputMutator.java    |   28 ++
 .../drill/exec/physical/impl/PhysicalConfig.java   |   29 ++
 .../exec/physical/impl/RandomReceiverCreator.java  |   46 +++
 .../drill/exec/physical/impl/RootCreator.java      |   31 ++
 .../apache/drill/exec/physical/impl/RootExec.java  |   40 +++
 .../apache/drill/exec/physical/impl/ScanBatch.java |  172 +++++++++
 .../drill/exec/physical/impl/ScreenCreator.java    |   90 +++++
 .../exec/physical/impl/SingleSenderCreator.java    |   89 +++++
 .../drill/exec/physical/impl/WireRecordBatch.java  |   99 ++++++
 .../impl/materialize/QueryWritableBatch.java       |   46 +++
 .../impl/materialize/RecordMaterializer.java       |   31 ++
 .../impl/materialize/VectorRecordMaterializer.java |   52 +++
 .../exec/planner/AbstractOpWrapperVisitor.java     |   45 +++
 .../org/apache/drill/exec/planner/ExecPlanner.java |    8 +-
 .../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     |   63 +++-
 .../org/apache/drill/exec/planner/ScanFinder.java  |   54 ---
 .../drill/exec/planner/SimpleExecPlanner.java      |   28 +-
 .../drill/exec/planner/SimpleParallelizer.java     |  147 --------
 .../drill/exec/planner/fragment/Fragment.java      |  150 ++++++++
 .../exec/planner/fragment/FragmentVisitor.java     |   23 ++
 .../planner/fragment/MakeFragmentsVisitor.java     |   69 ++++
 .../planner/fragment/MaterializedFragment.java     |   69 ++++
 .../drill/exec/planner/fragment/Materializer.java  |  107 ++++++
 .../drill/exec/planner/fragment/PlanningSet.java   |   66 ++++
 .../exec/planner/fragment/SimpleParallelizer.java  |  163 +++++++++
 .../apache/drill/exec/planner/fragment/Stats.java  |   70 ++++
 .../exec/planner/fragment/StatsCollector.java      |  106 ++++++
 .../drill/exec/planner/fragment/Wrapper.java       |  186 ++++++++++
 .../exec/pop/receiver/NWayOrderingReceiver.java    |   52 ---
 .../drill/exec/pop/receiver/RandomReceiver.java    |   55 ---
 .../drill/exec/pop/sender/HashPartitionSender.java |   49 ---
 .../org/apache/drill/exec/record/BatchSchema.java  |  102 +-----
 .../drill/exec/record/FragmentWritableBatch.java   |   59 +++
 .../drill/exec/record/InvalidValueAccessor.java    |    2 +-
 .../apache/drill/exec/record/MajorTypeSerDe.java   |  116 ++++++
 .../drill/exec/record/MaterializedField.java       |  167 +++++++---
 .../apache/drill/exec/record/RawFragmentBatch.java |   44 +++
 .../exec/record/RawFragmentBatchProvider.java      |   27 ++
 .../org/apache/drill/exec/record/RecordBatch.java  |    9 +-
 .../drill/exec/record/RecordBatchLoader.java       |  143 ++++++++
 .../apache/drill/exec/record/SchemaBuilder.java    |  127 +++++++
 .../apache/drill/exec/record/WritableBatch.java    |  108 ++++++
 .../record/vector/AbstractFixedValueVector.java    |   21 +-
 .../drill/exec/record/vector/BaseValueVector.java  |   86 ++++-
 .../org/apache/drill/exec/record/vector/Bit.java   |  168 +++++++++
 .../apache/drill/exec/record/vector/BitVector.java |  166 ---------
 .../drill/exec/record/vector/ByteVector.java       |   48 ---
 .../apache/drill/exec/record/vector/Fixed1.java    |   43 +++
 .../apache/drill/exec/record/vector/Fixed12.java   |   35 ++
 .../apache/drill/exec/record/vector/Fixed16.java   |   37 ++
 .../apache/drill/exec/record/vector/Fixed2.java    |   53 +++
 .../apache/drill/exec/record/vector/Fixed4.java    |   55 +++
 .../apache/drill/exec/record/vector/Fixed8.java    |   58 +++
 .../apache/drill/exec/record/vector/FixedLen.java  |   45 +++
 .../drill/exec/record/vector/Int16Vector.java      |   52 ---
 .../drill/exec/record/vector/Int32Vector.java      |   52 ---
 .../drill/exec/record/vector/NullableFixed4.java   |   37 ++
 .../exec/record/vector/NullableInt32Vector.java    |   47 ---
 .../exec/record/vector/NullableValueVector.java    |   36 ++-
 .../apache/drill/exec/record/vector/RepeatMap.java |   57 +++
 .../drill/exec/record/vector/SelectionVector.java  |   10 +-
 .../drill/exec/record/vector/TypeHelper.java       |  250 +++++++++++++
 .../drill/exec/record/vector/UInt16Vector.java     |   51 ---
 .../drill/exec/record/vector/ValueVector.java      |   46 +++-
 .../apache/drill/exec/record/vector/VarLen1.java   |   36 ++
 .../apache/drill/exec/record/vector/VarLen2.java   |   36 ++
 .../apache/drill/exec/record/vector/VarLen4.java   |   36 ++
 .../drill/exec/record/vector/VariableVector.java   |   35 ++-
 .../drill/exec/rpc/AbstractHandshakeHandler.java   |   57 +++
 .../main/java/org/apache/drill/exec/rpc/Acks.java  |   27 ++
 .../org/apache/drill/exec/rpc/BasicClient.java     |  137 +++++++-
 .../drill/exec/rpc/BasicClientWithConnection.java  |   64 ++++
 .../org/apache/drill/exec/rpc/BasicServer.java     |   61 +++-
 .../apache/drill/exec/rpc/CoordinationQueue.java   |   32 +-
 .../org/apache/drill/exec/rpc/DrillRpcFuture.java  |   78 +----
 .../apache/drill/exec/rpc/DrillRpcFutureImpl.java  |  118 +++++++
 .../apache/drill/exec/rpc/InboundRpcMessage.java   |   13 +-
 .../apache/drill/exec/rpc/OutboundRpcMessage.java  |   41 ++-
 .../apache/drill/exec/rpc/RemoteConnection.java    |   45 +++
 .../java/org/apache/drill/exec/rpc/Response.java   |    8 +-
 .../java/org/apache/drill/exec/rpc/RpcBus.java     |  136 +++++---
 .../java/org/apache/drill/exec/rpc/RpcConfig.java  |  150 ++++++++
 .../java/org/apache/drill/exec/rpc/RpcDecoder.java |   14 +-
 .../java/org/apache/drill/exec/rpc/RpcEncoder.java |   15 +-
 .../apache/drill/exec/rpc/RpcExceptionHandler.java |   21 +-
 .../java/org/apache/drill/exec/rpc/RpcMessage.java |    9 +-
 .../apache/drill/exec/rpc/RpcOutcomeListener.java  |   28 ++
 .../exec/rpc/ZeroCopyProtobufLengthDecoder.java    |   16 +-
 .../drill/exec/rpc/bit/AvailabilityListener.java   |   22 ++
 .../org/apache/drill/exec/rpc/bit/BitClient.java   |   77 ++++-
 .../java/org/apache/drill/exec/rpc/bit/BitCom.java |   75 +---
 .../exec/rpc/bit/BitComDefaultInstanceHandler.java |   51 +++
 .../apache/drill/exec/rpc/bit/BitComHandler.java   |  136 -------
 .../org/apache/drill/exec/rpc/bit/BitComImpl.java  |  194 ++++++-----
 .../apache/drill/exec/rpc/bit/BitConnection.java   |  168 +++++++++
 .../drill/exec/rpc/bit/BitConnectionManager.java   |   80 +++++
 .../apache/drill/exec/rpc/bit/BitRpcConfig.java    |   46 +++
 .../org/apache/drill/exec/rpc/bit/BitServer.java   |   61 +++-
 .../org/apache/drill/exec/rpc/bit/BitTunnel.java   |  215 +++++++++++-
 .../apache/drill/exec/rpc/bit/ListenerPool.java    |   56 +++
 .../drill/exec/rpc/user/QueryResultBatch.java      |   49 +++
 .../org/apache/drill/exec/rpc/user/UserClient.java |  169 ++++++++-
 .../drill/exec/rpc/user/UserResultsListener.java   |   41 +++
 .../apache/drill/exec/rpc/user/UserRpcConfig.java  |   39 ++
 .../org/apache/drill/exec/rpc/user/UserServer.java |   93 ++++--
 .../apache/drill/exec/server/BootStrapContext.java |   68 ++++
 .../org/apache/drill/exec/server/Drillbit.java     |   65 ++--
 .../apache/drill/exec/server/DrillbitContext.java  |   58 +++-
 .../apache/drill/exec/server/RemoteServiceSet.java |   59 +++
 .../apache/drill/exec/service/ServiceEngine.java   |   45 ++--
 .../drill/exec/store/AbstractStorageEngine.java    |    2 +-
 .../org/apache/drill/exec/store/RecordReader.java  |    7 +-
 .../org/apache/drill/exec/store/StorageEngine.java |    2 +-
 .../org/apache/drill/exec/util/AtomicState.java    |   58 +++
 .../exec/work/AbstractFragmentRunnerListener.java  |  109 ++++++
 .../apache/drill/exec/work/CancelableQuery.java    |   22 ++
 .../apache/drill/exec/work/EndpointListener.java   |   45 +++
 .../org/apache/drill/exec/work/FragmentRunner.java |  124 +++++++
 .../drill/exec/work/FragmentRunnerListener.java    |   28 ++
 .../org/apache/drill/exec/work/QueryWorkUnit.java  |   64 ++++
 .../apache/drill/exec/work/RecordOutputStream.java |   22 ++
 .../exec/work/RemotingFragmentRunnerListener.java  |   46 +++
 .../apache/drill/exec/work/ResourceRequest.java    |   30 ++
 .../org/apache/drill/exec/work/RootNodeDriver.java |   25 ++
 .../org/apache/drill/exec/work/StatusProvider.java |   24 ++
 .../org/apache/drill/exec/work/WorkManager.java    |  168 +++++++++
 .../exec/work/batch/AbstractFragmentCollector.java |   84 +++++
 .../drill/exec/work/batch/BatchCollector.java      |   32 ++
 .../drill/exec/work/batch/BitComHandler.java       |   41 +++
 .../drill/exec/work/batch/BitComHandlerImpl.java   |  205 +++++++++++
 .../drill/exec/work/batch/IncomingBuffers.java     |  108 ++++++
 .../drill/exec/work/batch/MergingCollector.java    |   45 +++
 .../exec/work/batch/PartitionedCollector.java      |   42 +++
 .../drill/exec/work/batch/RawBatchBuffer.java      |   33 ++
 .../exec/work/batch/UnlmitedRawBatchBuffer.java    |   73 ++++
 .../drill/exec/work/foreman/ErrorHelper.java       |   47 +++
 .../apache/drill/exec/work/foreman/Foreman.java    |  272 +++++++++++++++
 .../exec/work/foreman/FragmentStatusListener.java  |   26 ++
 .../exec/work/foreman/RunningFragmentManager.java  |  266 ++++++++++++++
 .../drill/exec/work/foreman/TunnelManager.java     |   53 +++
 .../work/fragment/IncomingFragmentHandler.java     |   49 +++
 .../exec/work/fragment/LocalFragmentHandler.java   |   69 ++++
 .../exec/work/fragment/RemoteFragmentHandler.java  |  123 +++++++
 .../apache/drill/exec/work/user/UserWorker.java    |   72 ++++
 .../java-exec/src/main/protobuf/Coordination.proto |   26 ++
 .../src/main/protobuf/ExecutionProtos.proto        |   55 ++--
 .../java-exec/src/main/protobuf/GeneralRPC.proto   |    6 +-
 .../java-exec/src/main/protobuf/SchemaDef.proto    |   86 ++++--
 .../exec/java-exec/src/main/protobuf/User.proto    |   69 ++--
 .../src/main/protobuf/UserBitShared.proto          |   46 +++
 .../org/apache/drill/exec/DrillSystemTestBase.java |    7 +
 .../drill/exec/client/DrillClientSystemTest.java   |   25 +-
 .../exec/compile/TestClassCompilationTypes.java    |    5 +-
 .../exec/physical/config/ParsePhysicalPlan.java    |   42 +++
 .../exec/physical/impl/DistributedFragmentRun.java |   53 +++
 .../exec/physical/impl/SimpleFragmentRun.java      |  100 ++++++
 .../org/apache/drill/exec/pop/CheckFragmenter.java |   70 ++--
 .../apache/drill/exec/pop/CheckInjectionValue.java |   12 +-
 .../org/apache/drill/exec/pop/FragmentChecker.java |   66 ++++
 .../org/apache/drill/exec/pop/PopUnitTestBase.java |   71 ++++
 .../apache/drill/exec/rpc/user/RunRemoteQuery.java |   41 ---
 .../apache/drill/exec/rpc/user/UserRpcTest.java    |  107 ------
 .../org/apache/drill/exec/server/TestBitRpc.java   |   84 +++++
 .../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 |    5 +-
 .../exec/java-exec/src/test/resources/logback.xml  |   10 +-
 .../src/test/resources/physical_screen.json        |    5 +-
 .../test/resources/physical_simpleexchange.json    |   36 ++-
 .../test/resources/physical_single_exchange.json   |   34 ++
 .../src/test/resources/physical_test1.json         |   40 +++
 .../src/test/resources/physical_test2.json         |   34 ++
 .../exec/ref/src/test/resources/donuts.json        |    2 +-
 sandbox/prototype/pom.xml                          |    2 +-
 326 files changed, 13371 insertions(+), 6094 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/common/pom.xml
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/pom.xml b/sandbox/prototype/common/pom.xml
index 3a15c05..beb2d28 100644
--- a/sandbox/prototype/common/pom.xml
+++ b/sandbox/prototype/common/pom.xml
@@ -74,6 +74,18 @@
 
 	<build>
 		<plugins>
+            <plugin>
+                <artifactId>maven-jar-plugin</artifactId>
+                <executions>
+                    <execution>
+                        <id>test-jar</id>
+                        <phase>package</phase>
+                        <goals>
+                            <goal>test-jar</goal>
+                        </goals>
+                    </execution>
+                </executions>
+            </plugin>
 			<plugin>
 				<groupId>org.antlr</groupId>
 				<artifactId>antlr3-maven-plugin</artifactId>

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/common/src/main/java/org/apache/drill/common/config/DrillConfig.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/config/DrillConfig.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/config/DrillConfig.java
index 2b8f45d..18d5e51 100644
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/config/DrillConfig.java
+++ b/sandbox/prototype/common/src/main/java/org/apache/drill/common/config/DrillConfig.java
@@ -27,9 +27,9 @@ import org.apache.drill.common.exceptions.DrillConfigurationException;
 import org.apache.drill.common.expression.LogicalExpression;
 import org.apache.drill.common.logical.StorageEngineConfigBase;
 import org.apache.drill.common.logical.data.LogicalOperatorBase;
-import org.apache.drill.common.physical.pop.base.PhysicalOperatorUtil;
 import org.apache.drill.common.util.PathScanner;
 
+import com.fasterxml.jackson.core.JsonGenerator;
 import com.fasterxml.jackson.core.JsonParser.Feature;
 import com.fasterxml.jackson.databind.ObjectMapper;
 import com.fasterxml.jackson.databind.SerializationFeature;
@@ -50,15 +50,18 @@ public final class DrillConfig extends NestedConfig{
   public DrillConfig(Config config) {
     super(config);
     mapper = new ObjectMapper();
-    SimpleModule deserModule = new SimpleModule("LogicalExpressionDeserializationModule").addDeserializer(LogicalExpression.class, new LogicalExpression.De(this));
+    SimpleModule deserModule = new SimpleModule("LogicalExpressionDeserializationModule")
+      .addDeserializer(LogicalExpression.class, new LogicalExpression.De(this));
+    
     mapper.registerModule(deserModule);
     mapper.enable(SerializationFeature.INDENT_OUTPUT);
     mapper.configure(Feature.ALLOW_UNQUOTED_FIELD_NAMES, true);
+    mapper.configure(JsonGenerator.Feature.QUOTE_FIELD_NAMES, false);
     mapper.configure(Feature.ALLOW_COMMENTS, true);
     mapper.registerSubtypes(LogicalOperatorBase.getSubTypes(this));
-    mapper.registerSubtypes(PhysicalOperatorUtil.getSubTypes(this));
     mapper.registerSubtypes(StorageEngineConfigBase.getSubTypes(this));
     
+    
   };
   
   /**

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

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

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/SchemaPath.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/SchemaPath.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/SchemaPath.java
index b3675a8..004d812 100644
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/SchemaPath.java
+++ b/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/SchemaPath.java
@@ -104,7 +104,7 @@ public class SchemaPath extends LogicalExpressionBase{
 
 	}
 		
-	
+
 	
 	@Override
   public <T> T accept(ExprVisitor<T> visitor) {

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/types/DataType.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/types/DataType.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/types/DataType.java
index 25b82a7..56e2485 100644
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/types/DataType.java
+++ b/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/types/DataType.java
@@ -74,6 +74,17 @@ public abstract class DataType {
   public static final DataType ARRAY = new AtomType("ARRAY", Comparability.NONE, false);
   public static final DataType NULL = new AtomType("NULL", Comparability.NONE, false);
   
+  //TODO: Hack to get some size data, needs to be fixed so that each type reveals it's size.
+  public int size(){
+    if(this == BOOLEAN){
+      return 1;
+    }else if(this == INT32){
+      return 4;
+    }else if(this == INT16){
+      return 4;
+    }
+    return 2;
+  }
   
   static final Map<String, DataType> TYPES;
   static {

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/common/src/main/java/org/apache/drill/common/optimize/Optimizer.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/optimize/Optimizer.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/optimize/Optimizer.java
deleted file mode 100644
index 4b2037c..0000000
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/optimize/Optimizer.java
+++ /dev/null
@@ -1,45 +0,0 @@
-/*******************************************************************************
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- ******************************************************************************/
-package org.apache.drill.common.optimize;
-
-import java.io.Closeable;
-
-import org.apache.drill.common.config.DrillConfig;
-import org.apache.drill.common.exceptions.DrillConfigurationException;
-import org.apache.drill.common.logical.LogicalPlan;
-import org.apache.drill.common.physical.PhysicalPlan;
-
-public abstract class Optimizer implements Closeable{
-  
-  public static String OPTIMIZER_IMPL_KEY = "drill.exec.optimizer.implementation";
-  
-  public abstract void init(DrillConfig config);
-  
-  public abstract PhysicalPlan optimize(OptimizationContext context, LogicalPlan plan);
-  public abstract void close();
-  
-  public static Optimizer getOptimizer(DrillConfig config) throws DrillConfigurationException{
-    Optimizer o = config.getInstanceOf(OPTIMIZER_IMPL_KEY, Optimizer.class);
-    o.init(config);
-    return o;
-  }
-  
-  public interface OptimizationContext{
-    public int getPriority();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/DataValidationMode.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/DataValidationMode.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/DataValidationMode.java
deleted file mode 100644
index 6de2cfd..0000000
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/DataValidationMode.java
+++ /dev/null
@@ -1,24 +0,0 @@
-/*******************************************************************************
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- ******************************************************************************/
-package org.apache.drill.common.physical;
-
-public enum DataValidationMode {
-  TERMINATE, // terminate the query if the data doesn't match expected.
-  DROP_RECORD, // drop the record that doesn't match the expected situation.
-  SINK_RECORD // record the failed record along with the rule violation in a secondary location.
-}

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

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

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

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/PhysicalPlan.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/PhysicalPlan.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/PhysicalPlan.java
deleted file mode 100644
index e83dac7..0000000
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/PhysicalPlan.java
+++ /dev/null
@@ -1,95 +0,0 @@
-/*******************************************************************************
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- ******************************************************************************/
-package org.apache.drill.common.physical;
-
-import java.io.IOException;
-import java.util.List;
-
-import org.apache.drill.common.PlanProperties;
-import org.apache.drill.common.config.DrillConfig;
-import org.apache.drill.common.graph.Graph;
-import org.apache.drill.common.graph.GraphAlgos;
-import org.apache.drill.common.physical.pop.base.Leaf;
-import org.apache.drill.common.physical.pop.base.PhysicalOperator;
-import org.apache.drill.common.physical.pop.base.Root;
-
-import com.fasterxml.jackson.annotation.JsonCreator;
-import com.fasterxml.jackson.annotation.JsonProperty;
-import com.fasterxml.jackson.annotation.JsonPropertyOrder;
-import com.fasterxml.jackson.core.JsonProcessingException;
-import com.fasterxml.jackson.databind.ObjectMapper;
-import com.fasterxml.jackson.databind.ObjectReader;
-import com.fasterxml.jackson.databind.ObjectWriter;
-import com.google.common.collect.Lists;
-
-@JsonPropertyOrder({ "head", "graph" })
-public class PhysicalPlan {
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(PhysicalPlan.class);
-  
-  PlanProperties properties;
-  
-  Graph<PhysicalOperator, Root, Leaf> graph;
-  
-  @JsonCreator
-  public PhysicalPlan(@JsonProperty("head") PlanProperties properties, @JsonProperty("graph") List<PhysicalOperator> operators){
-    this.properties = properties;
-    this.graph = Graph.newGraph(operators, Root.class, Leaf.class);
-  }
-  
-  @JsonProperty("graph")
-  public List<PhysicalOperator> getSortedOperators(){
-    // reverse the list so that nested references are flattened rather than nested.
-    return getSortedOperators(true);
-  }
-  
-  public List<PhysicalOperator> getSortedOperators(boolean reverse){
-    List<PhysicalOperator> list = GraphAlgos.TopoSorter.sort(graph);
-    if(reverse){
-      return Lists.reverse(list);
-    }else{
-      return list;
-    }
-    
-  }
-
-
-  @JsonProperty("head")
-  public PlanProperties getProperties() {
-    return properties;
-  }
-
-  /** Parses a physical plan. */
-  public static PhysicalPlan parse(ObjectReader reader, String planString) {
-    try {
-      PhysicalPlan plan = reader.readValue(planString);
-      return plan;
-    } catch (IOException e) {
-      throw new RuntimeException(e);
-    }
-  }
-
-  /** Converts a physical plan to a string. (Opposite of {@link #parse}.) */
-  public String unparse(ObjectWriter writer) {
-    try {
-      return writer.writeValueAsString(this);
-    } catch (JsonProcessingException e) {
-      throw new RuntimeException(e);
-    }
-  }
-  
-}

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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


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

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/coord/ZKClusterCoordinator.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/coord/ZKClusterCoordinator.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/coord/ZKClusterCoordinator.java
index 85c573d..75dce2c 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/coord/ZKClusterCoordinator.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/coord/ZKClusterCoordinator.java
@@ -23,16 +23,19 @@ import static com.google.common.collect.Collections2.transform;
 import java.io.IOException;
 import java.util.Collection;
 import java.util.Collections;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
 
 import org.apache.drill.common.config.DrillConfig;
-import org.apache.drill.common.proto.CoordinationProtos.DrillbitEndpoint;
 import org.apache.drill.exec.ExecConstants;
+import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
 
 import com.google.common.base.Function;
 import com.netflix.curator.RetryPolicy;
 import com.netflix.curator.framework.CuratorFramework;
 import com.netflix.curator.framework.CuratorFrameworkFactory;
 import com.netflix.curator.framework.state.ConnectionState;
+import com.netflix.curator.framework.state.ConnectionStateListener;
 import com.netflix.curator.retry.RetryNTimes;
 import com.netflix.curator.x.discovery.ServiceDiscovery;
 import com.netflix.curator.x.discovery.ServiceDiscoveryBuilder;
@@ -52,6 +55,7 @@ public class ZKClusterCoordinator extends ClusterCoordinator {
   private ServiceCache<DrillbitEndpoint> serviceCache;
   private volatile Collection<DrillbitEndpoint> endpoints = Collections.emptyList();
   private final String serviceName;
+  private final CountDownLatch initialConnection = new CountDownLatch(1);
 
   public ZKClusterCoordinator(DrillConfig config) throws IOException {
 
@@ -64,6 +68,7 @@ public class ZKClusterCoordinator extends ClusterCoordinator {
       .retryPolicy(rp)
       .connectString(config.getString(ExecConstants.ZK_CONNECTION))
       .build();
+    curator.getConnectionStateListenable().addListener(new InitialConnectionListener());
     discovery = getDiscovery();
     serviceCache = discovery.
       serviceCacheBuilder()
@@ -72,15 +77,36 @@ public class ZKClusterCoordinator extends ClusterCoordinator {
       .build();
   }
 
-  public void start() throws Exception {
+  public void start(long millisToWait) throws Exception {
     logger.debug("Starting ZKClusterCoordination.");
     curator.start();
     discovery.start();
     serviceCache.start();
     serviceCache.addListener(new ZKListener());
+    
+    if(millisToWait != 0){
+      boolean success = this.initialConnection.await(millisToWait, TimeUnit.MILLISECONDS);
+      if(!success) throw new IOException(String.format("Failure to connect to the zookeeper cluster service within the allotted time of %d milliseconds.", millisToWait));
+    }else{
+      this.initialConnection.await();
+    }
+    
+    
     updateEndpoints();
   }
+  
+  private class InitialConnectionListener implements ConnectionStateListener{
 
+    @Override
+    public void stateChanged(CuratorFramework client, ConnectionState newState) {
+      if(newState == ConnectionState.CONNECTED){
+        ZKClusterCoordinator.this.initialConnection.countDown();
+        client.getConnectionStateListenable().removeListener(this);
+      }
+    }
+    
+  }
+  
   private class ZKListener implements ServiceCacheListener {
 
     @Override

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

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

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

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

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

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

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

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

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

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/memory/BufferAllocator.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/memory/BufferAllocator.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/memory/BufferAllocator.java
index 2b3f574..6bddab7 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/memory/BufferAllocator.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/memory/BufferAllocator.java
@@ -19,11 +19,10 @@ package org.apache.drill.exec.memory;
 
 import io.netty.buffer.ByteBuf;
 import io.netty.buffer.ByteBufAllocator;
-import io.netty.buffer.PooledByteBufAllocator;
 
 import java.io.Closeable;
 
-import org.apache.drill.exec.server.DrillbitContext;
+import org.apache.drill.common.config.DrillConfig;
 
 /**
  * Wrapper class to deal with byte buffer allocation. Ensures users only use designated methods.  Also allows inser 
@@ -48,7 +47,7 @@ public abstract class BufferAllocator implements Closeable{
   @Override
   public abstract void close(); 
   
-  public static BufferAllocator getAllocator(DrillbitContext context){
+  public static BufferAllocator getAllocator(DrillConfig config){
     // TODO: support alternative allocators (including a debugging allocator that records all allocation locations for each buffer).
     return new DirectBufferAllocator();
   }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/metrics/SingleThreadNestedCounter.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/metrics/SingleThreadNestedCounter.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/metrics/SingleThreadNestedCounter.java
index 6b89c12..1f47041 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/metrics/SingleThreadNestedCounter.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/metrics/SingleThreadNestedCounter.java
@@ -22,34 +22,34 @@ import org.apache.drill.exec.server.DrillbitContext;
 import com.yammer.metrics.Counter;
 
 /**
- * Wraps a parent counter so that local in thread metrics can be collected while collecting for a global counter.
+ * Wraps a parent counter so that local in-thread metrics can be collected while collecting for a global counter. Note
+ * that this one writer, many reader safe.
  */
 public class SingleThreadNestedCounter {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(SingleThreadNestedCounter.class);
-  
+
   private volatile long count;
   private final Counter counter;
-  
-  
+
   public SingleThreadNestedCounter(DrillbitContext context, String name) {
     super();
     this.counter = context.getMetrics().counter(name);
   }
 
-  public long inc(long n){
+  public long inc(long n) {
     counter.inc(n);
-    count+= n;
+    count += n;
     return count;
   }
-  
-  public long dec(long n){
+
+  public long dec(long n) {
     counter.dec(n);
     count -= n;
     return count;
   }
-  
-  public long get(){
+
+  public long get() {
     return count;
   }
-  
+
 }

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

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/FragmentContext.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/FragmentContext.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/FragmentContext.java
index 0cf17e9..e64453c 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/FragmentContext.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/FragmentContext.java
@@ -20,10 +20,15 @@ package org.apache.drill.exec.ops;
 import org.apache.drill.common.expression.LogicalExpression;
 import org.apache.drill.exec.memory.BufferAllocator;
 import org.apache.drill.exec.metrics.SingleThreadNestedCounter;
-import org.apache.drill.exec.planner.FragmentRunnable;
-import org.apache.drill.exec.proto.ExecProtos.PlanFragment;
+import org.apache.drill.exec.physical.impl.FilteringRecordBatchTransformer;
+import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
+import org.apache.drill.exec.proto.ExecProtos.FragmentHandle;
+import org.apache.drill.exec.proto.ExecProtos.FragmentStatus;
 import org.apache.drill.exec.rpc.bit.BitCom;
+import org.apache.drill.exec.rpc.user.UserServer.UserClientConnection;
 import org.apache.drill.exec.server.DrillbitContext;
+import org.apache.drill.exec.work.FragmentRunner;
+import org.apache.drill.exec.work.batch.IncomingBuffers;
 
 import com.yammer.metrics.MetricRegistry;
 import com.yammer.metrics.Timer;
@@ -34,51 +39,72 @@ import com.yammer.metrics.Timer;
 public class FragmentContext {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(FragmentContext.class);
 
-  private final static String METRIC_TIMER_FRAGMENT_TIME = MetricRegistry.name(FragmentRunnable.class, "completionTimes");
-  private final static String METRIC_BATCHES_COMPLETED = MetricRegistry.name(FragmentRunnable.class, "batchesCompleted");
-  private final static String METRIC_RECORDS_COMPLETED = MetricRegistry.name(FragmentRunnable.class, "recordsCompleted");
-  private final static String METRIC_DATA_PROCESSED = MetricRegistry.name(FragmentRunnable.class, "dataProcessed");
+  private final static String METRIC_TIMER_FRAGMENT_TIME = MetricRegistry.name(FragmentRunner.class, "completionTimes");
+  private final static String METRIC_BATCHES_COMPLETED = MetricRegistry.name(FragmentRunner.class, "batchesCompleted");
+  private final static String METRIC_RECORDS_COMPLETED = MetricRegistry.name(FragmentRunner.class, "recordsCompleted");
+  private final static String METRIC_DATA_PROCESSED = MetricRegistry.name(FragmentRunner.class, "dataProcessed");
 
   private final DrillbitContext context;
-  private final PlanFragment fragment;
   public final SingleThreadNestedCounter batchesCompleted;
   public final SingleThreadNestedCounter recordsCompleted;
   public final SingleThreadNestedCounter dataProcessed;
   public final Timer fragmentTime;
+  private final FragmentHandle handle;
+  private final UserClientConnection connection;
+  private final IncomingBuffers buffers;
 
-  public FragmentContext(DrillbitContext dbContext, PlanFragment fragment) {
+  public FragmentContext(DrillbitContext dbContext, FragmentHandle handle, UserClientConnection connection, IncomingBuffers buffers) {
     this.fragmentTime = dbContext.getMetrics().timer(METRIC_TIMER_FRAGMENT_TIME);
     this.batchesCompleted = new SingleThreadNestedCounter(dbContext, METRIC_BATCHES_COMPLETED);
     this.recordsCompleted = new SingleThreadNestedCounter(dbContext, METRIC_RECORDS_COMPLETED);
     this.dataProcessed = new SingleThreadNestedCounter(dbContext, METRIC_DATA_PROCESSED);
     this.context = dbContext;
-    this.fragment = fragment;
+    this.connection = connection;
+    this.handle = handle;
+    this.buffers = buffers;
   }
 
   public void fail(Throwable cause) {
 
   }
 
+  
   public DrillbitContext getDrillbitContext(){
     return context;
   }
-  
-  public PlanFragment getFragment() {
-    return fragment;
+
+  public DrillbitEndpoint getIdentity(){
+    return context.getEndpoint();
   }
   
+  public FragmentHandle getHandle() {
+    return handle;
+  }
+
   public BufferAllocator getAllocator(){
     // TODO: A local query allocator to ensure memory limits and accurately gauge memory usage.
     return context.getAllocator();
   }
 
-  
   public FilteringRecordBatchTransformer getFilteringExpression(LogicalExpression expr){
     return null;
   }
   
+  public void addMetricsToStatus(FragmentStatus.Builder stats){
+    stats.setBatchesCompleted(batchesCompleted.get());
+    stats.setDataProcessed(dataProcessed.get());
+    stats.setRecordsCompleted(recordsCompleted.get());
+  }
   
+  public UserClientConnection getConnection() {
+    return connection;
+  }
+
   public BitCom getCommunicator(){
-    return null;
+    return context.getBitCom();
+  }
+  
+  public IncomingBuffers getBuffers(){
+    return buffers;
   }
 }

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

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

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

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

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/QueryContext.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/QueryContext.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/QueryContext.java
index fe37e70..fd24deb 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/QueryContext.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/QueryContext.java
@@ -19,33 +19,42 @@ package org.apache.drill.exec.ops;
 
 import java.util.Collection;
 
-import org.apache.drill.common.proto.CoordinationProtos.DrillbitEndpoint;
+import org.apache.drill.exec.cache.DistributedCache;
+import org.apache.drill.exec.planner.PhysicalPlanReader;
+import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
+import org.apache.drill.exec.proto.UserBitShared.QueryId;
 import org.apache.drill.exec.server.DrillbitContext;
 
-import com.fasterxml.jackson.databind.ObjectMapper;
-
 public class QueryContext {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(QueryContext.class);
   
-  private long queryId;
+  private QueryId queryId;
   private DrillbitContext drillbitContext;
   
-  public QueryContext(long queryId, DrillbitContext drllbitContext) {
+  public QueryContext(QueryId queryId, DrillbitContext drllbitContext) {
     super();
     this.queryId = queryId;
     this.drillbitContext = drllbitContext;
   }
   
-  public long getQueryId() {
-    return queryId;
+  public DrillbitEndpoint getCurrentEndpoint(){
+    return drillbitContext.getEndpoint();
   }
   
-  public ObjectMapper getMapper(){
-    return drillbitContext.getConfig().getMapper();
+  public QueryId getQueryId() {
+    return queryId;
+  }
+
+  public DistributedCache getCache(){
+    return drillbitContext.getCache();
   }
   
   public Collection<DrillbitEndpoint> getActiveEndpoints(){
     return drillbitContext.getBits();
   }
   
+  public PhysicalPlanReader getPlanReader(){
+    return drillbitContext.getPlanReader();
+  }
+  
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/ScanBatch.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/ScanBatch.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/ScanBatch.java
deleted file mode 100644
index b46804f..0000000
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/ScanBatch.java
+++ /dev/null
@@ -1,157 +0,0 @@
-/*******************************************************************************
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- ******************************************************************************/
-package org.apache.drill.exec.ops;
-
-import java.util.Iterator;
-
-import org.apache.drill.exec.exception.ExecutionSetupException;
-import org.apache.drill.exec.exception.SchemaChangeException;
-import org.apache.drill.exec.record.BatchSchema;
-import org.apache.drill.exec.record.InvalidValueAccessor;
-import org.apache.drill.exec.record.RecordBatch;
-import org.apache.drill.exec.record.vector.ValueVector;
-import org.apache.drill.exec.store.RecordReader;
-
-import com.carrotsearch.hppc.IntObjectOpenHashMap;
-import com.carrotsearch.hppc.procedures.IntObjectProcedure;
-
-/**
- * Record batch used for a particular scan. Operators against one or more
- */
-public abstract class ScanBatch implements RecordBatch {
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ScanBatch.class);
-
-  private IntObjectOpenHashMap<ValueVector<?>> fields = new IntObjectOpenHashMap<ValueVector<?>>();
-  private BatchSchema schema;
-  private int recordCount;
-  private boolean schemaChanged = true;
-  private final FragmentContext context;
-  private Iterator<RecordReader> readers;
-  private RecordReader currentReader;
-  private final BatchSchema expectedSchema;
-  private final Mutator mutator = new Mutator();
-
-  public ScanBatch(BatchSchema expectedSchema, Iterator<RecordReader> readers, FragmentContext context)
-      throws ExecutionSetupException {
-    this.expectedSchema = expectedSchema;
-    this.context = context;
-    this.readers = readers;
-    if (!readers.hasNext()) throw new ExecutionSetupException("A scan batch must contain at least one reader.");
-    this.currentReader = readers.next();
-    this.currentReader.setup(expectedSchema, mutator);
-  }
-
-  private void schemaChanged() {
-    schema = null;
-    schemaChanged = true;
-  }
-
-  @Override
-  public FragmentContext getContext() {
-    return context;
-  }
-
-  @Override
-  public BatchSchema getSchema() {
-    return schema;
-  }
-
-  @Override
-  public int getRecordCount() {
-    return recordCount;
-  }
-
-  @Override
-  public void kill() {
-    releaseAssets();
-  }
-
-  private void releaseAssets() {
-    fields.forEach(new IntObjectProcedure<ValueVector<?>>() {
-      @Override
-      public void apply(int key, ValueVector<?> value) {
-        value.close();
-      }
-    });
-  }
-
-  @SuppressWarnings("unchecked")
-  @Override
-  public <T extends ValueVector<T>> T getValueVector(int fieldId, Class<T> clazz) throws InvalidValueAccessor {
-    if (fields.containsKey(fieldId))
-      throw new InvalidValueAccessor(String.format("Unknown value accesor for field id %d."));
-    ValueVector<?> vector = this.fields.lget();
-    if (vector.getClass().isAssignableFrom(clazz)) {
-      return (T) vector;
-    } else {
-      throw new InvalidValueAccessor(String.format(
-          "You requested a field accessor of type %s for field id %d but the actual type was %s.",
-          clazz.getCanonicalName(), fieldId, vector.getClass().getCanonicalName()));
-    }
-  }
-
-  @Override
-  public IterOutcome next() {
-    while ((recordCount = currentReader.next()) == 0) {
-      try {
-        if (!readers.hasNext()) {
-          currentReader.cleanup();
-          releaseAssets();
-          return IterOutcome.NONE;
-        }
-        currentReader.cleanup();
-        currentReader = readers.next();
-        currentReader.setup(expectedSchema, mutator);
-      } catch (ExecutionSetupException e) {
-        this.context.fail(e);
-        releaseAssets();
-        return IterOutcome.STOP;
-      }
-    }
-
-    if (schemaChanged) {
-      schemaChanged = false;
-      return IterOutcome.OK_NEW_SCHEMA;
-    } else {
-      return IterOutcome.OK;
-    }
-  }
-
-  private class Mutator implements OutputMutator {
-
-    public void removeField(int fieldId) throws SchemaChangeException {
-      schemaChanged();
-      ValueVector<?> v = fields.remove(fieldId);
-      if (v == null) throw new SchemaChangeException("Failure attempting to remove an unknown field.");
-      v.close();
-    }
-
-    public void addField(int fieldId, ValueVector<?> vector) {
-      schemaChanged();
-      ValueVector<?> v = fields.put(fieldId, vector);
-      if (v != null) v.close();
-    }
-
-    @Override
-    public void setNewSchema(BatchSchema schema) {
-      ScanBatch.this.schema = schema;
-      ScanBatch.this.schemaChanged = true;
-    }
-
-  }
-}

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

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

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

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

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

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/opt/IdentityOptimizer.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/opt/IdentityOptimizer.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/opt/IdentityOptimizer.java
index 70a42be..d2aaca3 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/opt/IdentityOptimizer.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/opt/IdentityOptimizer.java
@@ -19,8 +19,7 @@ package org.apache.drill.exec.opt;
 
 import org.apache.drill.common.config.DrillConfig;
 import org.apache.drill.common.logical.LogicalPlan;
-import org.apache.drill.common.optimize.Optimizer;
-import org.apache.drill.common.physical.PhysicalPlan;
+import org.apache.drill.exec.physical.PhysicalPlan;
 
 public class IdentityOptimizer extends Optimizer {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(IdentityOptimizer.class);

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/opt/Optimizer.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/opt/Optimizer.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/opt/Optimizer.java
new file mode 100644
index 0000000..9f506c1
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/opt/Optimizer.java
@@ -0,0 +1,45 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.opt;
+
+import java.io.Closeable;
+
+import org.apache.drill.common.config.DrillConfig;
+import org.apache.drill.common.exceptions.DrillConfigurationException;
+import org.apache.drill.common.logical.LogicalPlan;
+import org.apache.drill.exec.physical.PhysicalPlan;
+
+public abstract class Optimizer implements Closeable{
+  
+  public static String OPTIMIZER_IMPL_KEY = "drill.exec.optimizer.implementation";
+  
+  public abstract void init(DrillConfig config);
+  
+  public abstract PhysicalPlan optimize(OptimizationContext context, LogicalPlan plan);
+  public abstract void close();
+  
+  public static Optimizer getOptimizer(DrillConfig config) throws DrillConfigurationException{
+    Optimizer o = config.getInstanceOf(OPTIMIZER_IMPL_KEY, Optimizer.class);
+    o.init(config);
+    return o;
+  }
+  
+  public interface OptimizationContext{
+    public int getPriority();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/DataValidationMode.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/DataValidationMode.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/DataValidationMode.java
new file mode 100644
index 0000000..334119d
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/DataValidationMode.java
@@ -0,0 +1,24 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.physical;
+
+public enum DataValidationMode {
+  TERMINATE, // terminate the query if the data doesn't match expected.
+  DROP_RECORD, // drop the record that doesn't match the expected situation.
+  SINK_RECORD // record the failed record along with the rule violation in a secondary location.
+}

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

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

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/PhysicalPlan.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/PhysicalPlan.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/PhysicalPlan.java
new file mode 100644
index 0000000..84bfc87
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/PhysicalPlan.java
@@ -0,0 +1,94 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.physical;
+
+import java.io.IOException;
+import java.util.List;
+
+import org.apache.drill.common.PlanProperties;
+import org.apache.drill.common.config.DrillConfig;
+import org.apache.drill.common.graph.Graph;
+import org.apache.drill.common.graph.GraphAlgos;
+import org.apache.drill.exec.physical.base.Leaf;
+import org.apache.drill.exec.physical.base.PhysicalOperator;
+import org.apache.drill.exec.physical.base.Root;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.annotation.JsonPropertyOrder;
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.ObjectReader;
+import com.fasterxml.jackson.databind.ObjectWriter;
+import com.google.common.collect.Lists;
+
+@JsonPropertyOrder({ "head", "graph" })
+public class PhysicalPlan {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(PhysicalPlan.class);
+  
+  PlanProperties properties;
+  
+  Graph<PhysicalOperator, Root, Leaf> graph;
+  
+  @JsonCreator
+  public PhysicalPlan(@JsonProperty("head") PlanProperties properties, @JsonProperty("graph") List<PhysicalOperator> operators){
+    this.properties = properties;
+    this.graph = Graph.newGraph(operators, Root.class, Leaf.class);
+  }
+  
+  @JsonProperty("graph")
+  public List<PhysicalOperator> getSortedOperators(){
+    // reverse the list so that nested references are flattened rather than nested.
+    return getSortedOperators(true);
+  }
+  
+  public List<PhysicalOperator> getSortedOperators(boolean reverse){
+    List<PhysicalOperator> list = GraphAlgos.TopoSorter.sort(graph);
+    if(reverse){
+      return Lists.reverse(list);
+    }else{
+      return list;
+    }
+    
+  }
+
+
+  @JsonProperty("head")
+  public PlanProperties getProperties() {
+    return properties;
+  }
+
+  /** Parses a physical plan. */
+  public static PhysicalPlan parse(ObjectReader reader, String planString) {
+    try {
+      PhysicalPlan plan = reader.readValue(planString);
+      return plan;
+    } catch (IOException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  /** Converts a physical plan to a string. (Opposite of {@link #parse}.) */
+  public String unparse(ObjectWriter writer) {
+    try {
+      return writer.writeValueAsString(this);
+    } catch (JsonProcessingException e) {
+      throw new RuntimeException(e);
+    }
+  }
+  
+}

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

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

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

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


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

Posted by Ted Dunning <te...@gmail.com>.

Sent from my iPhone

On May 13, 2013, at 18:52, jacques@apache.org wrote:

> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractExchange.java
> ----------------------------------------------------------------------
> diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractExchange.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractExchange.java
> new file mode 100644
> index 0000000..42a15ae
> --- /dev/null
> +++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractExchange.java
> @@ -0,0 +1,90 @@
> +/*******************************************************************************
> + * Licensed to the Apache Software Foundation (ASF) under one
> + * or more contributor license agreements.  See the NOTICE file
> + * distributed with this work for additional information
> + * regarding copyright ownership.  The ASF licenses this file
> + * to you under the Apache License, Version 2.0 (the
> + * "License"); you may not use this file except in compliance
> + * with the License.  You may obtain a copy of the License at
> + * 
> + * http://www.apache.org/licenses/LICENSE-2.0
> + * 
> + * Unless required by applicable law or agreed to in writing, software
> + * distributed under the License is distributed on an "AS IS" BASIS,
> + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
> + * See the License for the specific language governing permissions and
> + * limitations under the License.
> + ******************************************************************************/
> +package org.apache.drill.exec.physical.base;
> +
> +import java.util.List;
> +
> +import org.apache.drill.common.exceptions.PhysicalOperatorSetupException;
> +import org.apache.drill.exec.physical.OperatorCost;
> +import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
> +
> +import com.fasterxml.jackson.annotation.JsonIgnore;
> +import com.fasterxml.jackson.annotation.JsonProperty;
> +
> +public abstract class AbstractExchange extends AbstractSingle implements Exchange {
> +  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(AbstractExchange.class);
> +
> +  protected int senderMajorFragmentId;
> +  protected int receiverMajorFragmentId;
> +
> +  public AbstractExchange(PhysicalOperator child) {
> +    super(child);
> +  }
> +
> +  /**
> +   * Exchanges are not executable. The Execution layer first has to set their parallelization and convert them into
> +   * something executable
> +   */
> +  @Override
> +  public boolean isExecutable() {
> +    return false;
> +  }
> +
> +  protected abstract void setupSenders(List<DrillbitEndpoint> senderLocations) throws PhysicalOperatorSetupException ;
> +  protected abstract void setupReceivers(List<DrillbitEndpoint> senderLocations) throws PhysicalOperatorSetupException ;
> +  
> +  @Override
> +  public final void setupSenders(int majorFragmentId, List<DrillbitEndpoint> senderLocations) throws PhysicalOperatorSetupException {
> +    this.senderMajorFragmentId = majorFragmentId;
> +    setupSenders(senderLocations);
> +  }
> +  
> +
> +  @Override
> +  public final void setupReceivers(int majorFragmentId, List<DrillbitEndpoint> receiverLocations) throws PhysicalOperatorSetupException {
> +    this.receiverMajorFragmentId = majorFragmentId;
> +    setupReceivers(receiverLocations);
> +  }
> +  
> +  @Override
> +  public OperatorCost getAggregateSendCost() {
> +    return getExchangeCost().getSendCost();
> +  }
> +
> +  @Override
> +  public OperatorCost getAggregateReceiveCost() {
> +    return getExchangeCost().getReceiveCost();
> +  }
> +
> +  @Override
> +  public final <T, X, E extends Throwable> T accept(PhysicalVisitor<T, X, E> physicalVisitor, X value) throws E {
> +    return physicalVisitor.visitExchange(this, value);
> +  }
> +
> +  @Override
> +  public ExchangeCost getExchangeCost(){
> +    return ExchangeCost.getSimpleEstimate(getSize());
> +  }
> +
> +  @JsonIgnore
> +  @Override
> +  public OperatorCost getCost() {
> +    return getExchangeCost().getCombinedCost();
> +  }
> +
> +}
> 
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractPhysicalVisitor.java
> ----------------------------------------------------------------------
> diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractPhysicalVisitor.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractPhysicalVisitor.java
> new file mode 100644
> index 0000000..f782325
> --- /dev/null
> +++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractPhysicalVisitor.java
> @@ -0,0 +1,124 @@
> +/*******************************************************************************
> + * Licensed to the Apache Software Foundation (ASF) under one
> + * or more contributor license agreements.  See the NOTICE file
> + * distributed with this work for additional information
> + * regarding copyright ownership.  The ASF licenses this file
> + * to you under the Apache License, Version 2.0 (the
> + * "License"); you may not use this file except in compliance
> + * with the License.  You may obtain a copy of the License at
> + * 
> + * http://www.apache.org/licenses/LICENSE-2.0
> + * 
> + * Unless required by applicable law or agreed to in writing, software
> + * distributed under the License is distributed on an "AS IS" BASIS,
> + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
> + * See the License for the specific language governing permissions and
> + * limitations under the License.
> + ******************************************************************************/
> +package org.apache.drill.exec.physical.base;
> +
> +import org.apache.drill.exec.physical.config.Filter;
> +import org.apache.drill.exec.physical.config.HashPartitionSender;
> +import org.apache.drill.exec.physical.config.HashToRandomExchange;
> +import org.apache.drill.exec.physical.config.Project;
> +import org.apache.drill.exec.physical.config.RandomReceiver;
> +import org.apache.drill.exec.physical.config.RangeSender;
> +import org.apache.drill.exec.physical.config.Screen;
> +import org.apache.drill.exec.physical.config.SingleSender;
> +import org.apache.drill.exec.physical.config.Sort;
> +import org.apache.drill.exec.physical.config.UnionExchange;
> +
> +public abstract class AbstractPhysicalVisitor<T, X, E extends Throwable> implements PhysicalVisitor<T, X, E> {
> +  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(AbstractPhysicalVisitor.class);
> +
> +  @Override
> +  public T visitExchange(Exchange exchange, X value) throws E{
> +    return visitOp(exchange, value);
> +  }
> +
> +  @Override
> +  public T visitFilter(Filter filter, X value) throws E{
> +    return visitOp(filter, value);
> +  }
> +
> +  @Override
> +  public T visitProject(Project project, X value) throws E{
> +    return visitOp(project, value);
> +  }
> +
> +  @Override
> +  public T visitSort(Sort sort, X value) throws E{
> +    return visitOp(sort, value);
> +  }
> +
> +  @Override
> +  public T visitSender(Sender sender, X value) throws E {
> +    return visitOp(sender, value);
> +  }
> +
> +  @Override
> +  public T visitReceiver(Receiver receiver, X value) throws E {
> +    return visitOp(receiver, value);
> +  }
> +
> +  @Override
> +  public T visitScan(Scan<?> scan, X value) throws E{
> +    return visitOp(scan, value);
> +  }
> +
> +  @Override
> +  public T visitStore(Store store, X value) throws E{
> +    return visitOp(store, value);
> +  }
> +
> +  
> +  public T visitChildren(PhysicalOperator op, X value) throws E{
> +    for(PhysicalOperator child : op){
> +      child.accept(this, value);
> +    }
> +    return null;
> +  }
> +  
> +  @Override
> +  public T visitHashPartitionSender(HashPartitionSender op, X value) throws E {
> +    return visitSender(op, value);
> +  }
> +
> +  @Override
> +  public T visitRandomReceiver(RandomReceiver op, X value) throws E {
> +    return visitReceiver(op, value);
> +  }
> +
> +  @Override
> +  public T visitHashPartitionSender(HashToRandomExchange op, X value) throws E {
> +    return visitExchange(op, value);
> +  }
> +
> +  @Override
> +  public T visitRangeSender(RangeSender op, X value) throws E {
> +    return visitSender(op, value);
> +  }
> +
> +  @Override
> +  public T visitScreen(Screen op, X value) throws E {
> +    return visitStore(op, value);
> +  }
> +
> +  @Override
> +  public T visitSingleSender(SingleSender op, X value) throws E {
> +    return visitSender(op, value);
> +  }
> +
> +  @Override
> +  public T visitUnionExchange(UnionExchange op, X value) throws E {
> +    return visitExchange(op, value);
> +  }
> +
> +  @Override
> +  public T visitOp(PhysicalOperator op, X value) throws E{
> +    throw new UnsupportedOperationException(String.format(
> +        "The PhysicalVisitor of type %s does not currently support visiting the PhysicalOperator type %s.", this
> +            .getClass().getCanonicalName(), op.getClass().getCanonicalName()));
> +  }
> +
> +}
> 
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractReceiver.java
> ----------------------------------------------------------------------
> diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractReceiver.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractReceiver.java
> new file mode 100644
> index 0000000..e8ba19c
> --- /dev/null
> +++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractReceiver.java
> @@ -0,0 +1,63 @@
> +/*******************************************************************************
> + * Licensed to the Apache Software Foundation (ASF) under one
> + * or more contributor license agreements.  See the NOTICE file
> + * distributed with this work for additional information
> + * regarding copyright ownership.  The ASF licenses this file
> + * to you under the Apache License, Version 2.0 (the
> + * "License"); you may not use this file except in compliance
> + * with the License.  You may obtain a copy of the License at
> + * 
> + * http://www.apache.org/licenses/LICENSE-2.0
> + * 
> + * Unless required by applicable law or agreed to in writing, software
> + * distributed under the License is distributed on an "AS IS" BASIS,
> + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
> + * See the License for the specific language governing permissions and
> + * limitations under the License.
> + ******************************************************************************/
> +package org.apache.drill.exec.physical.base;
> +
> +import java.util.Iterator;
> +import java.util.List;
> +
> +import org.apache.drill.exec.physical.OperatorCost;
> +
> +import com.fasterxml.jackson.annotation.JsonIgnore;
> +import com.fasterxml.jackson.annotation.JsonProperty;
> +import com.google.common.base.Preconditions;
> +import com.google.common.collect.Iterators;
> +
> +public abstract class AbstractReceiver extends AbstractBase implements Receiver{
> +
> +  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(AbstractReceiver.class);
> +
> +  private final int oppositeMajorFragmentId; 
> +  
> +  public AbstractReceiver(int oppositeMajorFragmentId){
> +    this.oppositeMajorFragmentId = oppositeMajorFragmentId;
> +  }
> +  
> +  @Override
> +  public Iterator<PhysicalOperator> iterator() {
> +    return Iterators.emptyIterator();
> +  }
> +
> +  @Override
> +  public <T, X, E extends Throwable> T accept(PhysicalVisitor<T, X, E> physicalVisitor, X value) throws E {
> +    return physicalVisitor.visitReceiver(this, value);
> +  }
> +
> +  @Override
> +  public final PhysicalOperator getNewWithChildren(List<PhysicalOperator> children) {
> +    Preconditions.checkArgument(children.isEmpty());
> +    //rewriting is unnecessary since the inputs haven't changed.
> +    return this;
> +  }
> +
> +  @JsonProperty("sender-major-fragment")
> +  public int getOppositeMajorFragmentId() {
> +    return oppositeMajorFragmentId;
> +  }
> +
> +}
> +
> 
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractScan.java
> ----------------------------------------------------------------------
> diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractScan.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractScan.java
> new file mode 100644
> index 0000000..dbde9c5
> --- /dev/null
> +++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractScan.java
> @@ -0,0 +1,84 @@
> +/*******************************************************************************
> + * Licensed to the Apache Software Foundation (ASF) under one
> + * or more contributor license agreements.  See the NOTICE file
> + * distributed with this work for additional information
> + * regarding copyright ownership.  The ASF licenses this file
> + * to you under the Apache License, Version 2.0 (the
> + * "License"); you may not use this file except in compliance
> + * with the License.  You may obtain a copy of the License at
> + * 
> + * http://www.apache.org/licenses/LICENSE-2.0
> + * 
> + * Unless required by applicable law or agreed to in writing, software
> + * distributed under the License is distributed on an "AS IS" BASIS,
> + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
> + * See the License for the specific language governing permissions and
> + * limitations under the License.
> + ******************************************************************************/
> +package org.apache.drill.exec.physical.base;
> +
> +import java.util.Iterator;
> +import java.util.List;
> +
> +import org.apache.drill.exec.physical.OperatorCost;
> +import org.apache.drill.exec.physical.ReadEntry;
> +
> +import com.fasterxml.jackson.annotation.JsonIgnore;
> +import com.fasterxml.jackson.annotation.JsonProperty;
> +import com.google.common.collect.Iterators;
> +
> +public abstract class AbstractScan<R extends ReadEntry> extends AbstractBase implements Scan<R>{
> +  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(AbstractScan.class);
> +  
> +  protected final List<R> readEntries;
> +  private final OperatorCost cost;
> +  private final Size size;
> +  
> +  public AbstractScan(List<R> readEntries) {
> +    this.readEntries = readEntries;
> +    OperatorCost cost = new OperatorCost(0,0,0,0);
> +    Size size = new Size(0,0);
> +    for(R r : readEntries){
> +      cost = cost.add(r.getCost());
> +      size = size.add(r.getSize());
> +    }
> +    this.cost = cost;
> +    this.size = size;
> +  }
> +
> +  @Override
> +  @JsonProperty("entries")
> +  public List<R> getReadEntries() {
> +    return readEntries;
> +  }
> +  
> +  @Override
> +  public Iterator<PhysicalOperator> iterator() {
> +    return Iterators.emptyIterator();
> +  }
> +
> +  @Override
> +  public boolean isExecutable() {
> +    return true;
> +  }
> +
> +  @Override
> +  public <T, X, E extends Throwable> T accept(PhysicalVisitor<T, X, E> physicalVisitor, X value) throws E{
> +    return physicalVisitor.visitScan(this, value);
> +  }
> +
> +  @Override
> +  public OperatorCost getCost() {
> +    return cost;
> +  }
> +
> +  @Override
> +  public Size getSize() {
> +    return size;
> +  }
> +  
> +  
> +  
> +  
> +  
> +}
> 
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractSender.java
> ----------------------------------------------------------------------
> diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractSender.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractSender.java
> new file mode 100644
> index 0000000..f8c22b3
> --- /dev/null
> +++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractSender.java
> @@ -0,0 +1,53 @@
> +/*******************************************************************************
> + * Licensed to the Apache Software Foundation (ASF) under one
> + * or more contributor license agreements.  See

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

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

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractPhysicalVisitor.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractPhysicalVisitor.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractPhysicalVisitor.java
new file mode 100644
index 0000000..f782325
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractPhysicalVisitor.java
@@ -0,0 +1,124 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.physical.base;
+
+import org.apache.drill.exec.physical.config.Filter;
+import org.apache.drill.exec.physical.config.HashPartitionSender;
+import org.apache.drill.exec.physical.config.HashToRandomExchange;
+import org.apache.drill.exec.physical.config.Project;
+import org.apache.drill.exec.physical.config.RandomReceiver;
+import org.apache.drill.exec.physical.config.RangeSender;
+import org.apache.drill.exec.physical.config.Screen;
+import org.apache.drill.exec.physical.config.SingleSender;
+import org.apache.drill.exec.physical.config.Sort;
+import org.apache.drill.exec.physical.config.UnionExchange;
+
+public abstract class AbstractPhysicalVisitor<T, X, E extends Throwable> implements PhysicalVisitor<T, X, E> {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(AbstractPhysicalVisitor.class);
+
+  @Override
+  public T visitExchange(Exchange exchange, X value) throws E{
+    return visitOp(exchange, value);
+  }
+
+  @Override
+  public T visitFilter(Filter filter, X value) throws E{
+    return visitOp(filter, value);
+  }
+
+  @Override
+  public T visitProject(Project project, X value) throws E{
+    return visitOp(project, value);
+  }
+
+  @Override
+  public T visitSort(Sort sort, X value) throws E{
+    return visitOp(sort, value);
+  }
+
+  @Override
+  public T visitSender(Sender sender, X value) throws E {
+    return visitOp(sender, value);
+  }
+
+  @Override
+  public T visitReceiver(Receiver receiver, X value) throws E {
+    return visitOp(receiver, value);
+  }
+
+  @Override
+  public T visitScan(Scan<?> scan, X value) throws E{
+    return visitOp(scan, value);
+  }
+
+  @Override
+  public T visitStore(Store store, X value) throws E{
+    return visitOp(store, value);
+  }
+
+  
+  public T visitChildren(PhysicalOperator op, X value) throws E{
+    for(PhysicalOperator child : op){
+      child.accept(this, value);
+    }
+    return null;
+  }
+  
+  @Override
+  public T visitHashPartitionSender(HashPartitionSender op, X value) throws E {
+    return visitSender(op, value);
+  }
+
+  @Override
+  public T visitRandomReceiver(RandomReceiver op, X value) throws E {
+    return visitReceiver(op, value);
+  }
+
+  @Override
+  public T visitHashPartitionSender(HashToRandomExchange op, X value) throws E {
+    return visitExchange(op, value);
+  }
+
+  @Override
+  public T visitRangeSender(RangeSender op, X value) throws E {
+    return visitSender(op, value);
+  }
+
+  @Override
+  public T visitScreen(Screen op, X value) throws E {
+    return visitStore(op, value);
+  }
+
+  @Override
+  public T visitSingleSender(SingleSender op, X value) throws E {
+    return visitSender(op, value);
+  }
+
+  @Override
+  public T visitUnionExchange(UnionExchange op, X value) throws E {
+    return visitExchange(op, value);
+  }
+
+  @Override
+  public T visitOp(PhysicalOperator op, X value) throws E{
+    throw new UnsupportedOperationException(String.format(
+        "The PhysicalVisitor of type %s does not currently support visiting the PhysicalOperator type %s.", this
+            .getClass().getCanonicalName(), op.getClass().getCanonicalName()));
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractReceiver.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractReceiver.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractReceiver.java
new file mode 100644
index 0000000..e8ba19c
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractReceiver.java
@@ -0,0 +1,63 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.physical.base;
+
+import java.util.Iterator;
+import java.util.List;
+
+import org.apache.drill.exec.physical.OperatorCost;
+
+import com.fasterxml.jackson.annotation.JsonIgnore;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Iterators;
+
+public abstract class AbstractReceiver extends AbstractBase implements Receiver{
+
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(AbstractReceiver.class);
+
+  private final int oppositeMajorFragmentId; 
+  
+  public AbstractReceiver(int oppositeMajorFragmentId){
+    this.oppositeMajorFragmentId = oppositeMajorFragmentId;
+  }
+  
+  @Override
+  public Iterator<PhysicalOperator> iterator() {
+    return Iterators.emptyIterator();
+  }
+
+  @Override
+  public <T, X, E extends Throwable> T accept(PhysicalVisitor<T, X, E> physicalVisitor, X value) throws E {
+    return physicalVisitor.visitReceiver(this, value);
+  }
+
+  @Override
+  public final PhysicalOperator getNewWithChildren(List<PhysicalOperator> children) {
+    Preconditions.checkArgument(children.isEmpty());
+    //rewriting is unnecessary since the inputs haven't changed.
+    return this;
+  }
+
+  @JsonProperty("sender-major-fragment")
+  public int getOppositeMajorFragmentId() {
+    return oppositeMajorFragmentId;
+  }
+
+}
+

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

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

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

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

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

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

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

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

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

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/Leaf.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/Leaf.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/Leaf.java
new file mode 100644
index 0000000..d4ed456
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/Leaf.java
@@ -0,0 +1,25 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.physical.base;
+
+/**
+ * An operator which specifically is a lowest level leaf node of a query plan across all possible fragments. Currently, the only operator that is a Leaf
+ * node are Scan nodes. Ultimately this could include use of Cache scans and other types of atypical data production systems.
+ */
+public interface Leaf extends FragmentLeaf {
+}

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

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

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/PhysicalVisitor.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/PhysicalVisitor.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/PhysicalVisitor.java
new file mode 100644
index 0000000..f36633f
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/PhysicalVisitor.java
@@ -0,0 +1,61 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.physical.base;
+
+import org.apache.drill.exec.physical.config.Filter;
+import org.apache.drill.exec.physical.config.HashPartitionSender;
+import org.apache.drill.exec.physical.config.HashToRandomExchange;
+import org.apache.drill.exec.physical.config.Project;
+import org.apache.drill.exec.physical.config.RandomReceiver;
+import org.apache.drill.exec.physical.config.RangeSender;
+import org.apache.drill.exec.physical.config.Screen;
+import org.apache.drill.exec.physical.config.SingleSender;
+import org.apache.drill.exec.physical.config.Sort;
+import org.apache.drill.exec.physical.config.UnionExchange;
+
+/**
+ * Visitor class designed to traversal of a operator tree.  Basis for a number of operator manipulations including fragmentation and materialization.
+ * @param <RETURN> The class associated with the return of each visit method.
+ * @param <EXTRA> The class object associated with additional data required for a particular operator modification.
+ * @param <EXCEP> An optional exception class that can be thrown when a portion of a modification or traversal fails.  Must extend Throwable.  In the case where the visitor does not throw any caught exception, this can be set as RuntimeException.
+ */
+public interface PhysicalVisitor<RETURN, EXTRA, EXCEP extends Throwable> {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(PhysicalVisitor.class);
+  
+  
+  public RETURN visitExchange(Exchange exchange, EXTRA value) throws EXCEP;
+  public RETURN visitScan(Scan<?> scan, EXTRA value) throws EXCEP;
+  public RETURN visitStore(Store store, EXTRA value) throws EXCEP;
+
+  public RETURN visitFilter(Filter filter, EXTRA value) throws EXCEP;
+  public RETURN visitProject(Project project, EXTRA value) throws EXCEP;
+  public RETURN visitSort(Sort sort, EXTRA value) throws EXCEP;
+  public RETURN visitSender(Sender sender, EXTRA value) throws EXCEP;
+  public RETURN visitReceiver(Receiver receiver, EXTRA value) throws EXCEP;
+  
+  public RETURN visitOp(PhysicalOperator op, EXTRA value) throws EXCEP;
+  
+  
+  public RETURN visitHashPartitionSender(HashPartitionSender op, EXTRA value) throws EXCEP;
+  public RETURN visitRandomReceiver(RandomReceiver op, EXTRA value) throws EXCEP;
+  public RETURN visitHashPartitionSender(HashToRandomExchange op, EXTRA value) throws EXCEP;
+  public RETURN visitRangeSender(RangeSender op, EXTRA value) throws EXCEP;
+  public RETURN visitScreen(Screen op, EXTRA value) throws EXCEP;
+  public RETURN visitSingleSender(SingleSender op, EXTRA value) throws EXCEP;
+  public RETURN visitUnionExchange(UnionExchange op, EXTRA value) throws EXCEP;
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/Receiver.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/Receiver.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/Receiver.java
new file mode 100644
index 0000000..cd4cb4a
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/Receiver.java
@@ -0,0 +1,51 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.physical.base;
+
+import java.util.List;
+
+import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
+
+import com.fasterxml.jackson.annotation.JsonIgnore;
+import com.fasterxml.jackson.annotation.JsonProperty;
+
+/**
+ * A receiver is one half of an exchange operator. The receiver is responsible for taking in one or more streams from
+ * corresponding Senders.  Receivers are a special type of Physical Operator that are typically only expressed within the execution plan.
+ */
+public interface Receiver extends FragmentLeaf {
+  
+  /**
+   * A receiver is expecting streams from one or more providing endpoints.  This method should return a list of the expected sending endpoints.
+   * @return List of counterpart sending DrillbitEndpoints.
+   */
+  public abstract List<DrillbitEndpoint> getProvidingEndpoints();
+
+  /**
+   * Whether or not this receive supports out of order exchange. This provides a hint for the scheduling node on whether
+   * the receiver can start work if only a subset of all sending endpoints are currently providing data. A random
+   * receiver would supports this form of operation. A NWAY receiver would not.
+   * 
+   * @return True if this receiver supports working on a streaming/out of order input.
+   */
+  @JsonIgnore
+  public abstract boolean supportsOutOfOrderExchange();
+
+  @JsonProperty("sender-major-fragment")
+  public int getOppositeMajorFragmentId();
+}

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

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

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/Sender.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/Sender.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/Sender.java
new file mode 100644
index 0000000..71513c7
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/Sender.java
@@ -0,0 +1,44 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.physical.base;
+
+import java.util.List;
+
+import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
+
+import com.fasterxml.jackson.annotation.JsonProperty;
+
+/**
+ * A sender is one half of an exchange node operations. It is responsible for subdividing/cloning and sending a local
+ * record set to a set of destination locations. This is typically only utilized at the level of the execution plan.
+ */
+public interface Sender extends FragmentRoot {
+  
+  /**
+   * Get the list of destination endpoints that this Sender will be communicating with.
+   * @return List of DrillbitEndpoints.
+   */
+  public abstract List<DrillbitEndpoint> getDestinations();
+  
+  /**
+   * Get the receiver major fragment id that is opposite this sender.
+   * @return
+   */
+  @JsonProperty("receiver-major-fragment")
+  public int getOppositeMajorFragmentId();
+}

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

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/Store.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/Store.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/Store.java
new file mode 100644
index 0000000..9b3a812
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/Store.java
@@ -0,0 +1,74 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.physical.base;
+
+import java.util.List;
+
+import org.apache.drill.common.exceptions.PhysicalOperatorSetupException;
+import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
+
+import com.fasterxml.jackson.annotation.JsonIgnore;
+
+/**
+ * An interface which supports storing a record stream. In contrast to the logical layer, in the physical/execution
+ * layers, a Store node is actually an outputting node (rather than a root node) that provides returns one or more
+ * records regarding the completion of the query.
+ */
+public interface Store extends HasAffinity {
+
+  /**
+   * Inform the Store node about the actual decided DrillbitEndpoint assignments desired for storage purposes. This is a
+   * precursor to the execution planner running a set of getSpecificStore() method calls for full Store node
+   * materialization.
+   * 
+   * @param endpoints
+   *          The list of endpoints that this Store node are going to be executed on.
+   * @throws PhysicalOperatorSetupException
+   */
+  public abstract void applyAssignments(List<DrillbitEndpoint> endpoints) throws PhysicalOperatorSetupException;
+
+  /**
+   * Provides full materialized Store operators for execution purposes.
+   * 
+   * @param child
+   *          The child operator that this operator will consume from.
+   * @param minorFragmentId
+   *          The particular minor fragment id associated with this particular fragment materialization.
+   * @return A materialized Store Operator.
+   * @throws PhysicalOperatorSetupException
+   */
+  public abstract Store getSpecificStore(PhysicalOperator child, int minorFragmentId)
+      throws PhysicalOperatorSetupException;
+
+  /**
+   * The maximum allowable width for the Store operation. In some cases, a store operation has a limited number of
+   * parallelizations that it can support. For example, a Screen return cannot be parallelized at all. In this case, a
+   * maxWidth value of 1 will be returned. In the case that there is no limit for parallelization, this method should
+   * return Integer.MAX_VALUE.
+   * 
+   * @return
+   */
+  @JsonIgnore
+  public abstract int getMaxWidth();
+
+  /**
+   * Get the child of this store operator as this will be needed for parallelization materialization purposes.
+   * @return
+   */
+  public abstract PhysicalOperator getChild();
+}
\ No newline at end of file

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

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

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/HashToRandomExchange.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/HashToRandomExchange.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/HashToRandomExchange.java
new file mode 100644
index 0000000..1f158ce
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/HashToRandomExchange.java
@@ -0,0 +1,86 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.physical.config;
+
+import java.beans.Transient;
+import java.util.List;
+
+import org.apache.drill.common.expression.LogicalExpression;
+import org.apache.drill.exec.physical.base.AbstractExchange;
+import org.apache.drill.exec.physical.base.PhysicalOperator;
+import org.apache.drill.exec.physical.base.Receiver;
+import org.apache.drill.exec.physical.base.Sender;
+import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.annotation.JsonTypeName;
+
+@JsonTypeName("hash-to-random-exchange")
+public class HashToRandomExchange extends AbstractExchange{
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(HashToRandomExchange.class);
+
+  
+  private final LogicalExpression expr;
+
+  //ephemeral for setup tasks.
+  private List<DrillbitEndpoint> senderLocations;
+  private List<DrillbitEndpoint> receiverLocations;
+  
+  @JsonCreator
+  public HashToRandomExchange(@JsonProperty("child") PhysicalOperator child, @JsonProperty("expr") LogicalExpression expr) {
+    super(child);
+    this.expr = expr;
+  }
+
+  @Override
+  public int getMaxSendWidth() {
+    return Integer.MAX_VALUE;
+  }
+
+
+  @Override
+  protected void setupSenders(List<DrillbitEndpoint> senderLocations) {
+    this.senderLocations = senderLocations;
+  }
+
+  @Override
+  protected void setupReceivers(List<DrillbitEndpoint> receiverLocations) {
+    this.receiverLocations = receiverLocations;
+  }
+
+  @Override
+  public Sender getSender(int minorFragmentId, PhysicalOperator child) {
+    return new HashPartitionSender(receiverMajorFragmentId, child, expr, receiverLocations);
+  }
+
+  @Override
+  public Receiver getReceiver(int minorFragmentId) {
+    return new RandomReceiver(senderMajorFragmentId, senderLocations);
+  }
+
+  @Override
+  protected PhysicalOperator getNewWithChild(PhysicalOperator child) {
+    return new HashToRandomExchange(child, expr);
+  }
+
+  
+
+  
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockRecordReader.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockRecordReader.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockRecordReader.java
new file mode 100644
index 0000000..eaaeaa3
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockRecordReader.java
@@ -0,0 +1,113 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.physical.config;
+
+import org.apache.drill.common.exceptions.ExecutionSetupException;
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.exec.exception.SchemaChangeException;
+import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.physical.config.MockScanPOP.MockColumn;
+import org.apache.drill.exec.physical.config.MockScanPOP.MockScanEntry;
+import org.apache.drill.exec.physical.impl.OutputMutator;
+import org.apache.drill.exec.proto.SchemaDefProtos.DataMode;
+import org.apache.drill.exec.proto.SchemaDefProtos.MajorType;
+import org.apache.drill.exec.record.MaterializedField;
+import org.apache.drill.exec.record.vector.TypeHelper;
+import org.apache.drill.exec.record.vector.ValueVector;
+import org.apache.drill.exec.store.RecordReader;
+
+public class MockRecordReader implements RecordReader {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(MockRecordReader.class);
+
+  private OutputMutator output;
+  private MockScanEntry config;
+  private FragmentContext context;
+  private ValueVector<?>[] valueVectors;
+  private int recordsRead;
+
+  public MockRecordReader(FragmentContext context, MockScanEntry config) {
+    this.context = context;
+    this.config = config;
+  }
+
+  private int getEstimatedRecordSize(MockColumn[] types) {
+    int x = 0;
+    for (int i = 0; i < types.length; i++) {
+      x += TypeHelper.getSize(types[i].getMajorType());
+    }
+    return x;
+  }
+
+  private ValueVector<?> getVector(int fieldId, String name, MajorType type, int length) {
+    assert context != null : "Context shouldn't be null.";
+    
+    if(type.getMode() != DataMode.REQUIRED) throw new UnsupportedOperationException();
+    
+    MaterializedField f = MaterializedField.create(new SchemaPath(name), fieldId, 0, type);
+    ValueVector<?> v;
+    v = TypeHelper.getNewVector(f, context.getAllocator());
+    v.allocateNew(length);
+    
+    return v;
+
+  }
+
+  @Override
+  public void setup(OutputMutator output) throws ExecutionSetupException {
+    try {
+      this.output = output;
+      int estimateRowSize = getEstimatedRecordSize(config.getTypes());
+      valueVectors = new ValueVector<?>[config.getTypes().length];
+      int batchRecordCount = 250000 / estimateRowSize;
+
+      for (int i = 0; i < config.getTypes().length; i++) {
+        logger.debug("Adding field {} of type {}", i, config.getTypes()[i]);
+        valueVectors[i] = getVector(i, config.getTypes()[i].getName(), config.getTypes()[i].getMajorType(), batchRecordCount);
+        output.addField(i, valueVectors[i]);
+      }
+      output.setNewSchema();
+    } catch (SchemaChangeException e) {
+      throw new ExecutionSetupException("Failure while setting up fields", e);
+    }
+
+  }
+
+  @Override
+  public int next() {
+    int recordSetSize = Math.min(valueVectors[0].capacity(), this.config.getRecords()- recordsRead);
+    recordsRead += recordSetSize;
+    for(ValueVector<?> v : valueVectors){
+      v.randomizeData();
+      v.setRecordCount(recordSetSize);
+    }
+    return recordSetSize;
+  }
+
+  @Override
+  public void cleanup() {
+    for (int i = 0; i < valueVectors.length; i++) {
+      try {
+        output.removeField(valueVectors[i].getField().getFieldId());
+      } catch (SchemaChangeException e) {
+        logger.warn("Failure while trying tremove field.", e);
+      }
+      valueVectors[i].close();
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockScanBatchCreator.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockScanBatchCreator.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockScanBatchCreator.java
new file mode 100644
index 0000000..b821d6e
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockScanBatchCreator.java
@@ -0,0 +1,49 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.physical.config;
+
+import java.util.List;
+
+import org.apache.drill.common.exceptions.ExecutionSetupException;
+import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.physical.config.MockScanPOP.MockScanEntry;
+import org.apache.drill.exec.physical.impl.BatchCreator;
+import org.apache.drill.exec.physical.impl.ScanBatch;
+import org.apache.drill.exec.record.BatchSchema;
+import org.apache.drill.exec.record.InvalidValueAccessor;
+import org.apache.drill.exec.record.RecordBatch;
+import org.apache.drill.exec.record.vector.ValueVector;
+import org.apache.drill.exec.store.RecordReader;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+
+public class MockScanBatchCreator implements BatchCreator<MockScanPOP>{
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(MockScanBatchCreator.class);
+
+  @Override
+  public RecordBatch getBatch(FragmentContext context, MockScanPOP config, List<RecordBatch> children) throws ExecutionSetupException {
+    Preconditions.checkArgument(children.isEmpty());
+    List<MockScanEntry> entries = config.getReadEntries();
+    List<RecordReader> readers = Lists.newArrayList();
+    for(MockScanEntry e : entries){
+      readers.add(new MockRecordReader(context, e));
+    }
+    return new ScanBatch(context, readers.iterator());
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockScanPOP.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockScanPOP.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockScanPOP.java
new file mode 100644
index 0000000..4a3a606
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockScanPOP.java
@@ -0,0 +1,193 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.physical.config;
+
+import java.util.Collections;
+import java.util.LinkedList;
+import java.util.List;
+
+import org.apache.drill.exec.physical.EndpointAffinity;
+import org.apache.drill.exec.physical.OperatorCost;
+import org.apache.drill.exec.physical.ReadEntry;
+import org.apache.drill.exec.physical.base.AbstractScan;
+import org.apache.drill.exec.physical.base.PhysicalOperator;
+import org.apache.drill.exec.physical.base.Scan;
+import org.apache.drill.exec.physical.base.Size;
+import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
+import org.apache.drill.exec.proto.SchemaDefProtos.DataMode;
+import org.apache.drill.exec.proto.SchemaDefProtos.MajorType;
+import org.apache.drill.exec.proto.SchemaDefProtos.MinorType;
+import org.apache.drill.exec.record.vector.TypeHelper;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonIgnore;
+import com.fasterxml.jackson.annotation.JsonInclude;
+import com.fasterxml.jackson.annotation.JsonInclude.Include;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.annotation.JsonTypeName;
+import com.google.common.base.Preconditions;
+
+@JsonTypeName("mock-scan")
+public class MockScanPOP extends AbstractScan<MockScanPOP.MockScanEntry> {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(MockScanPOP.class);
+
+  private final String url;
+  private  LinkedList<MockScanEntry>[] mappings;
+
+  @JsonCreator
+  public MockScanPOP(@JsonProperty("url") String url, @JsonProperty("entries") List<MockScanEntry> readEntries) {
+    super(readEntries);
+    this.url = url;
+  }
+
+  public String getUrl() {
+    return url;
+  }
+
+  
+  public static class MockScanEntry implements ReadEntry {
+
+    private final int records;
+    private final MockColumn[] types;
+    private final int recordSize;
+    
+
+    @JsonCreator
+    public MockScanEntry(@JsonProperty("records") int records, @JsonProperty("types") MockColumn[] types) {
+      this.records = records;
+      this.types = types;
+      int size = 0;
+      for(MockColumn dt : types){
+        size += TypeHelper.getSize(dt.getMajorType());
+      }
+      this.recordSize = size;
+    }
+
+    @Override
+    public OperatorCost getCost() {
+      return new OperatorCost(1, 2, 1, 1);
+    }
+
+    
+    public int getRecords() {
+      return records;
+    }
+
+    public MockColumn[] getTypes() {
+      return types;
+    }
+
+    @Override
+    public Size getSize() {
+      return new Size(records, recordSize);
+    }
+  }
+  
+  @JsonInclude(Include.NON_NULL)
+  public static class MockColumn{
+    @JsonProperty("type") public MinorType minorType;
+    public String name;
+    public DataMode mode;
+    public Integer width;
+    public Integer precision;
+    public Integer scale;
+    
+    
+    @JsonCreator
+    public MockColumn(@JsonProperty("name") String name, @JsonProperty("type") MinorType minorType, @JsonProperty("mode") DataMode mode, @JsonProperty("width") Integer width, @JsonProperty("precision") Integer precision, @JsonProperty("scale") Integer scale) {
+      this.name = name;
+      this.minorType = minorType;
+      this.mode = mode;
+      this.width = width;
+      this.precision = precision;
+      this.scale = scale;
+    }
+    
+    @JsonProperty("type")
+    public MinorType getMinorType() {
+      return minorType;
+    }
+    public String getName() {
+      return name;
+    }
+    public DataMode getMode() {
+      return mode;
+    }
+    public Integer getWidth() {
+      return width;
+    }
+    public Integer getPrecision() {
+      return precision;
+    }
+    public Integer getScale() {
+      return scale;
+    }
+    
+    @JsonIgnore
+    public MajorType getMajorType(){
+      MajorType.Builder b = MajorType.newBuilder();
+      b.setMode(mode);
+      b.setMinorType(minorType);
+      if(precision != null) b.setPrecision(precision);
+      if(width != null) b.setWidth(width);
+      if(scale != null) b.setScale(scale);
+      return b.build();
+    }
+    
+  }
+
+  @Override
+  public List<EndpointAffinity> getOperatorAffinity() {
+    return Collections.emptyList();
+  }
+
+  @SuppressWarnings("unchecked")
+  @Override
+  public void applyAssignments(List<DrillbitEndpoint> endpoints) {
+    Preconditions.checkArgument(endpoints.size() <= getReadEntries().size());
+    
+    mappings = new LinkedList[endpoints.size()];
+
+    int i =0;
+    for(MockScanEntry e : this.getReadEntries()){
+      if(i == endpoints.size()) i -= endpoints.size();
+      LinkedList<MockScanEntry> entries = mappings[i];
+      if(entries == null){
+        entries = new LinkedList<MockScanEntry>();
+        mappings[i] = entries;
+      }
+      entries.add(e);
+      i++;
+    }
+  }
+
+  @Override
+  public Scan<?> getSpecificScan(int minorFragmentId) {
+    assert minorFragmentId < mappings.length : String.format("Mappings length [%d] should be longer than minor fragment id [%d] but it isn't.", mappings.length, minorFragmentId);
+    return new MockScanPOP(url, mappings[minorFragmentId]);
+  }
+
+  @Override
+  @JsonIgnore
+  public PhysicalOperator getNewWithChildren(List<PhysicalOperator> children) {
+    Preconditions.checkArgument(children.isEmpty());
+    return new MockScanPOP(url, readEntries);
+
+  }
+
+}


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

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

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

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

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

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

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

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

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

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

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

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

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

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

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/NullableValueVector.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/NullableValueVector.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/NullableValueVector.java
index 8e714ed..692ab87 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/NullableValueVector.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/NullableValueVector.java
@@ -20,6 +20,7 @@ package org.apache.drill.exec.record.vector;
 import io.netty.buffer.ByteBuf;
 
 import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.proto.UserBitShared.FieldMetadata;
 import org.apache.drill.exec.record.MaterializedField;
 
 /**
@@ -28,18 +29,16 @@ import org.apache.drill.exec.record.MaterializedField;
 abstract class NullableValueVector<T extends NullableValueVector<T, E>, E extends BaseValueVector<E>> extends BaseValueVector<T> {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(NullableValueVector.class);
 
-  protected BitVector bits;
+  protected Bit bits;
   protected E value;
-  private final MaterializedField field;
 
-  public NullableValueVector(int fieldId, BufferAllocator allocator, Class<T> valueClass) {
-    super(fieldId, allocator);
-    bits = new BitVector(fieldId, allocator);
-    value = getNewValueVector(fieldId, allocator);
-    this.field = value.getField().getNullableVersion(valueClass);
+  public NullableValueVector(MaterializedField field, BufferAllocator allocator, Class<T> valueClass) {
+    super(field, allocator);
+    bits = new Bit(null, allocator);
+    value = getNewValueVector(allocator);
   }
   
-  protected abstract E getNewValueVector(int fieldId, BufferAllocator allocator);
+  protected abstract E getNewValueVector(BufferAllocator allocator);
 
   public int isNull(int index){
     return bits.getBit(index);
@@ -76,5 +75,26 @@ abstract class NullableValueVector<T extends NullableValueVector<T, E>, E extend
   }
 
   
+  @Override
+  public ByteBuf[] getBuffers() {
+    return new ByteBuf[]{bits.data, value.data};
+  }
+
+  @Override
+  public void setRecordCount(int recordCount) {
+    super.setRecordCount(recordCount);
+    bits.setRecordCount(recordCount);
+    value.setRecordCount(recordCount);
+  }
+
+  @Override
+  public Object getObject(int index) {
+    if(isNull(index) == 0){
+      return null;
+    }else{
+      return value.getObject(index);
+    }
+  }
+
 }
 

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

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/SelectionVector.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/SelectionVector.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/SelectionVector.java
index e9faa93..323b55f 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/SelectionVector.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/SelectionVector.java
@@ -20,12 +20,16 @@ package org.apache.drill.exec.record.vector;
 import io.netty.buffer.ByteBufAllocator;
 
 import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.record.MaterializedField;
 
-public class SelectionVector extends UInt16Vector{
+/**
+ * Convenience/Clarification Fixed2 wrapper.
+ */
+public class SelectionVector extends Fixed2{
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(SelectionVector.class);
 
-  public SelectionVector(int fieldId, BufferAllocator allocator) {
-    super(fieldId, allocator);
+  public SelectionVector(MaterializedField field, BufferAllocator allocator) {
+    super(field, allocator);
   }
 
 }

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

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

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/ValueVector.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/ValueVector.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/ValueVector.java
index 76b0e90..8a5a822 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/ValueVector.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/ValueVector.java
@@ -21,6 +21,7 @@ import io.netty.buffer.ByteBuf;
 
 import java.io.Closeable;
 
+import org.apache.drill.exec.proto.UserBitShared.FieldMetadata;
 import org.apache.drill.exec.record.MaterializedField;
 
 /**
@@ -44,6 +45,13 @@ public interface ValueVector<T extends ValueVector<T>> extends Closeable {
   public abstract void allocateNew(int valueCount);
 
   /**
+   * Update the value vector to the provided record information.
+   * @param metadata
+   * @param data
+   */
+  public abstract void setTo(FieldMetadata metadata, ByteBuf data);
+  
+  /**
    * Zero copy move of data from this vector to the target vector. Any future access to this vector without being
    * populated by a new vector will cause problems.
    * 
@@ -52,19 +60,19 @@ public interface ValueVector<T extends ValueVector<T>> extends Closeable {
   public abstract void transferTo(T vector);
 
   /**
-   * Return the underlying buffer. Note that this doesn't impact the reference counts for this buffer so it only should be
+   * Return the underlying buffers associated with this vector. Note that this doesn't impact the reference counts for this buffer so it only should be
    * used for in context access. Also note that this buffer changes regularly thus external classes shouldn't hold a
-   * reference to it.
+   * reference to it (unless they change it).
    * 
    * @return The underlying ByteBuf.
    */
-  public abstract ByteBuf getBuffer();
+  public abstract ByteBuf[] getBuffers();
 
   /**
-   * Returns the number of value contained within this vector.
+   * Returns the maximum number of values contained within this vector.
    * @return Vector size
    */
-  public abstract int size();
+  public abstract int capacity();
 
 
   /**
@@ -79,4 +87,32 @@ public interface ValueVector<T extends ValueVector<T>> extends Closeable {
    */
   public abstract MaterializedField getField();
 
+  /**
+   * Define the number of records that are in this value vector.
+   * @param recordCount Number of records active in this vector.  Used for purposes such as getting a writable range of the data.
+   */
+  public abstract void setRecordCount(int recordCount);
+  public abstract int getRecordCount();
+  
+  
+  /**
+   * Get the metadata for this field.
+   * @return
+   */
+  public abstract FieldMetadata getMetadata();
+  
+  /**
+   * Debug interface to get values per record.
+   * @param index The record index.
+   * @return The value in the vector.
+   */
+  public Object getObject(int index);
+  
+  
+  /**
+   * Useful for generating random data.
+   */
+  public void randomizeData();
+    
+  
 }

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

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

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

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/VariableVector.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/VariableVector.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/VariableVector.java
index dd84c94..4247f14 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/VariableVector.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/VariableVector.java
@@ -21,6 +21,7 @@ import io.netty.buffer.ByteBuf;
 
 import org.apache.drill.exec.memory.BufferAllocator;
 import org.apache.drill.exec.record.DeadBuf;
+import org.apache.drill.exec.record.MaterializedField;
 
 /** 
  * A vector of variable length bytes.  Constructed as a vector of lengths or positions and a vector of values.  Random access is only possible if the variable vector stores positions as opposed to lengths.
@@ -29,18 +30,16 @@ public abstract class VariableVector<T extends VariableVector<T, E>, E extends B
 
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(VariableVector.class);
   
-  protected E lengthVector;
+  protected final E lengthVector;
   private ByteBuf values = DeadBuf.DEAD_BUFFER;
   protected int expectedValueLength;
-  private final boolean hasPositions;
   
-  public VariableVector(int fieldId, BufferAllocator allocator, boolean hasPositions) {
-    super(fieldId, allocator);
-    this.lengthVector = getNewLengthVector(fieldId, allocator);
-    this.hasPositions = hasPositions;
+  public VariableVector(MaterializedField field, BufferAllocator allocator) {
+    super(field, allocator);
+    this.lengthVector = getNewLengthVector(allocator);
   }
   
-  protected abstract E getNewLengthVector(int fieldId, BufferAllocator allocator);
+  protected abstract E getNewLengthVector(BufferAllocator allocator);
   
   @Override
   protected int getAllocationSize(int valueCount) {
@@ -67,12 +66,28 @@ public abstract class VariableVector<T extends VariableVector<T, E>, E extends B
       values.release();
       values = DeadBuf.DEAD_BUFFER;
     }
-  }  
+  }
+
   
-  public boolean hasPositions(){
-    return hasPositions;
+  @Override
+  public ByteBuf[] getBuffers() {
+    return new ByteBuf[]{lengthVector.data, values};
   }
+
+  @Override
+  public void setRecordCount(int recordCount) {
+    super.setRecordCount(recordCount);
+    lengthVector.setRecordCount(recordCount);
+  }  
   
+  public void setTotalBytes(int totalBytes){
+    values.writerIndex(totalBytes);
+  }
+
+  @Override
+  public Object getObject(int index) {
+    return null;
+  }
   
   
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/AbstractHandshakeHandler.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/AbstractHandshakeHandler.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/AbstractHandshakeHandler.java
new file mode 100644
index 0000000..859d385
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/AbstractHandshakeHandler.java
@@ -0,0 +1,57 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.rpc;
+
+import io.netty.channel.Channel;
+import io.netty.channel.ChannelHandlerContext;
+import io.netty.channel.ChannelInboundMessageHandlerAdapter;
+
+import com.google.protobuf.Internal.EnumLite;
+import com.google.protobuf.MessageLite;
+import com.google.protobuf.Parser;
+
+public abstract class AbstractHandshakeHandler<T extends MessageLite> extends
+    ChannelInboundMessageHandlerAdapter<InboundRpcMessage> {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(AbstractHandshakeHandler.class);
+
+  protected final EnumLite handshakeType;
+  protected final Parser<T> parser;
+  protected int coordinationId;
+
+  public AbstractHandshakeHandler(EnumLite handshakeType, Parser<T> parser) {
+    super();
+    this.handshakeType = handshakeType;
+    this.parser = parser;
+  }
+
+  @Override
+  public final void messageReceived(ChannelHandlerContext ctx, InboundRpcMessage inbound) throws Exception {
+    coordinationId = inbound.coordinationId;
+    ctx.channel().pipeline().remove(this);
+    if (inbound.rpcType != handshakeType.getNumber())
+      throw new RpcException(String.format("Handshake failure.  Expected %s[%d] but received number [%d]",
+          handshakeType, handshakeType.getNumber(), inbound.rpcType));
+  
+    T msg = parser.parseFrom(inbound.getProtobufBodyAsIS());
+    consumeHandshake(ctx.channel(), msg);
+    
+  }
+
+  protected abstract void consumeHandshake(Channel c, T msg) throws Exception;
+
+}

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

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/BasicClient.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/BasicClient.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/BasicClient.java
index c62d445..0ff2b9d 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/BasicClient.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/BasicClient.java
@@ -18,23 +18,34 @@
 package org.apache.drill.exec.rpc;
 
 import io.netty.bootstrap.Bootstrap;
+import io.netty.buffer.ByteBuf;
 import io.netty.buffer.ByteBufAllocator;
+import io.netty.channel.Channel;
 import io.netty.channel.ChannelFuture;
 import io.netty.channel.ChannelInitializer;
 import io.netty.channel.ChannelOption;
 import io.netty.channel.EventLoopGroup;
 import io.netty.channel.socket.SocketChannel;
 import io.netty.channel.socket.nio.NioSocketChannel;
+import io.netty.util.concurrent.GenericFutureListener;
 
+import com.google.common.util.concurrent.SettableFuture;
 import com.google.protobuf.Internal.EnumLite;
+import com.google.protobuf.MessageLite;
+import com.google.protobuf.Parser;
 
-public abstract class BasicClient<T extends EnumLite> extends RpcBus<T> {
+public abstract class BasicClient<T extends EnumLite, R extends RemoteConnection> extends RpcBus<T, R> {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(BasicClient.class);
 
   private Bootstrap b;
   private volatile boolean connect = false;
+  protected R connection;
+  private EventLoopGroup eventLoop;
 
-  public BasicClient(ByteBufAllocator alloc, EventLoopGroup eventLoopGroup) {
+  public BasicClient(RpcConfig rpcMapping, ByteBufAllocator alloc, EventLoopGroup eventLoopGroup) {
+    super(rpcMapping);
+    this.eventLoop = eventLoopGroup;
+    
     b = new Bootstrap() //
         .group(eventLoopGroup) //
         .channel(NioSocketChannel.class) //
@@ -42,40 +53,132 @@ public abstract class BasicClient<T extends EnumLite> extends RpcBus<T> {
         .option(ChannelOption.SO_RCVBUF, 1 << 17) //
         .option(ChannelOption.SO_SNDBUF, 1 << 17) //
         .handler(new ChannelInitializer<SocketChannel>() {
-          
+
           @Override
           protected void initChannel(SocketChannel ch) throws Exception {
-            ch.closeFuture().addListener(getCloseHandler(ch));
-            
+            logger.debug("initializing client connection.");
+            connection = initRemoteConnection(ch);
+            ch.closeFuture().addListener(getCloseHandler(connection));
+
             ch.pipeline().addLast( //
                 new ZeroCopyProtobufLengthDecoder(), //
-                new RpcDecoder(), //
-                new RpcEncoder(), //
-                new InboundHandler(ch), //
+                new RpcDecoder(rpcConfig.getName()), //
+                new RpcEncoder(rpcConfig.getName()), //
+                getHandshakeHandler(), //
+                new InboundHandler(connection), //
                 new RpcExceptionHandler() //
                 );
-            channel = ch;
             connect = true;
           }
         }) //
-        
-        ;
+
+    ;
+  }
+
+  protected abstract ClientHandshakeHandler<?> getHandshakeHandler();
+
+  protected abstract class ClientHandshakeHandler<T extends MessageLite> extends AbstractHandshakeHandler<T> {
+    private Class<T> responseType;
+
+    public ClientHandshakeHandler(EnumLite handshakeType, Class<T> responseType, Parser<T> parser) {
+      super(handshakeType, parser);
+      this.responseType = responseType;
+    }
+
+    @Override
+    protected final void consumeHandshake(Channel c, T msg) throws Exception {
+      validateHandshake(msg);
+      queue.getFuture(handshakeType.getNumber(), coordinationId, responseType).setValue(msg);
+    }
+
+    protected abstract void validateHandshake(T msg) throws Exception;
+
+  }
+
+  protected GenericFutureListener<ChannelFuture> getCloseHandler(Channel channel) {
+    return new ChannelClosedHandler();
+  }
+
+  protected final <SEND extends MessageLite, RECEIVE extends MessageLite> DrillRpcFutureImpl<RECEIVE> send(
+      T connection, T rpcType, SEND protobufBody, Class<RECEIVE> clazz, ByteBuf... dataBodies) throws RpcException {
+    throw new UnsupportedOperationException(
+        "This shouldn't be used in client mode as a client only has a single connection.");
+  }
+
+  protected <SEND extends MessageLite, RECEIVE extends MessageLite> DrillRpcFuture<RECEIVE> send(T rpcType,
+      SEND protobufBody, Class<RECEIVE> clazz, ByteBuf... dataBodies) throws RpcException {
+    return super.send(connection, rpcType, protobufBody, clazz, dataBodies);
   }
 
   @Override
   public boolean isClient() {
     return true;
   }
-  
-  public ChannelFuture connectAsClient(String host, int port) throws InterruptedException {
-    ChannelFuture f = b.connect(host, port).sync();
-    connect = !connect;
-    return f;
+
+  /**
+   * TODO: This is a horrible hack to manage deadlock caused by creation of BitClient within BitCom.  Should be cleaned up.
+   */
+  private class HandshakeThread<SEND extends MessageLite, RECEIVE extends MessageLite> extends Thread {
+    final SettableFuture<RECEIVE> future;
+    T handshakeType;
+    SEND handshakeValue;
+    String host;
+    int port;
+    Class<RECEIVE> responseClass;
+
+    public HandshakeThread(T handshakeType, SEND handshakeValue, String host, int port, Class<RECEIVE> responseClass) {
+      super();
+      assert host != null && !host.isEmpty();
+      assert port > 0;
+      logger.debug("Creating new handshake thread to connec to {}:{}", host, port);
+      this.setName(String.format("handshake thread for %s", handshakeType.getClass().getCanonicalName()));
+      future = SettableFuture.create();
+      this.handshakeType = handshakeType;
+      this.handshakeValue = handshakeValue;
+      this.host = host;
+      this.port = port;
+      this.responseClass = responseClass;
+    }
+
+    @Override
+    public void run() {
+      try {
+        logger.debug("Starting to get client connection on host {}, port {}.", host, port);
+        
+        ChannelFuture f = b.connect(host, port);
+        f.sync();
+        if (connection == null) throw new RpcException("Failure while attempting to connect to server.");
+        connect = !connect;
+        logger.debug("Client connected, sending handshake.");
+        DrillRpcFuture<RECEIVE> fut = send(handshakeType, handshakeValue, responseClass);
+        future.set(fut.checkedGet());
+        logger.debug("Got bit client connection.");
+      } catch (Exception e) {
+        logger.debug("Failed to get client connection.", e);
+        future.setException(e);
+      }
+    }
+
+  }
+
+  protected <SEND extends MessageLite, RECEIVE extends MessageLite> RECEIVE connectAsClient(T handshakeType,
+      SEND handshakeValue, String host, int port, Class<RECEIVE> responseClass) throws InterruptedException,
+      RpcException {
+    
+    
+    HandshakeThread<SEND, RECEIVE> ht = new HandshakeThread<SEND, RECEIVE>(handshakeType, handshakeValue, host, port, responseClass);
+    ht.start();
+    try{
+      return ht.future.get();  
+    }catch(Exception e){
+      throw new RpcException(e);
+    }
+    
   }
 
   public void close() {
     logger.debug("Closing client");
-    b.shutdown();
+    connection.getChannel().close();
   }
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/BasicClientWithConnection.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/BasicClientWithConnection.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/BasicClientWithConnection.java
new file mode 100644
index 0000000..0e62f14
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/BasicClientWithConnection.java
@@ -0,0 +1,64 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.rpc;
+
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.ByteBufAllocator;
+import io.netty.channel.Channel;
+import io.netty.channel.ChannelFuture;
+import io.netty.channel.EventLoopGroup;
+import io.netty.util.concurrent.GenericFutureListener;
+
+import org.apache.drill.exec.rpc.BasicClientWithConnection.ServerConnection;
+
+import com.google.protobuf.Internal.EnumLite;
+
+public abstract class BasicClientWithConnection<T extends EnumLite> extends BasicClient<T, ServerConnection>{
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(BasicClientWithConnection.class);
+
+  public BasicClientWithConnection(RpcConfig rpcMapping, ByteBufAllocator alloc, EventLoopGroup eventLoopGroup) {
+    super(rpcMapping, alloc, eventLoopGroup);
+  }
+  
+  @Override
+  protected GenericFutureListener<ChannelFuture> getCloseHandler(ServerConnection clientConnection) {
+    return getCloseHandler(clientConnection.getChannel());
+  }
+  
+  @Override
+  protected Response handle(ServerConnection connection, int rpcType, ByteBuf pBody, ByteBuf dBody) throws RpcException {
+    return handle(rpcType, pBody, dBody);
+  }
+  
+  protected abstract Response handle(int rpcType, ByteBuf pBody, ByteBuf dBody) throws RpcException ;
+    
+  @Override
+  public ServerConnection initRemoteConnection(Channel channel) {
+    return new ServerConnection(channel);
+  }
+
+  public static class ServerConnection extends RemoteConnection{
+
+    public ServerConnection(Channel channel) {
+      super(channel);
+    }
+
+  }
+
+  
+}


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

Posted by Ted Dunning <te...@gmail.com>.

Sent from my iPhone

On May 13, 2013, at 18:52, jacques@apache.org wrote:

> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockStorageEngine.java
> ----------------------------------------------------------------------
> diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockStorageEngine.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockStorageEngine.java
> new file mode 100644
> index 0000000..0044628
> --- /dev/null
> +++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockStorageEngine.java
> @@ -0,0 +1,58 @@
> +/*******************************************************************************
> + * Licensed to the Apache Software Foundation (ASF) under one
> + * or more contributor license agreements.  See the NOTICE file
> + * distributed with this work for additional information
> + * regarding copyright ownership.  The ASF licenses this file
> + * to you under the Apache License, Version 2.0 (the
> + * "License"); you may not use this file except in compliance
> + * with the License.  You may obtain a copy of the License at
> + * 
> + * http://www.apache.org/licenses/LICENSE-2.0
> + * 
> + * Unless required by applicable law or agreed to in writing, software
> + * distributed under the License is distributed on an "AS IS" BASIS,
> + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
> + * See the License for the specific language governing permissions and
> + * limitations under the License.
> + ******************************************************************************/
> +package org.apache.drill.exec.physical.config;
> +
> +import java.io.IOException;
> +import java.util.Collection;
> +
> +import org.apache.drill.common.logical.data.Scan;
> +import org.apache.drill.exec.ops.FragmentContext;
> +import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
> +import org.apache.drill.exec.store.AbstractStorageEngine;
> +import org.apache.drill.exec.store.RecordReader;
> +import org.apache.drill.exec.store.StorageEngine;
> +import org.apache.drill.exec.store.StorageEngine.ReadEntry;
> +
> +import com.google.common.collect.ListMultimap;
> +
> +public class MockStorageEngine extends AbstractStorageEngine{
> +  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(MockStorageEngine.class);
> +
> +  @Override
> +  public boolean supportsRead() {
> +    return true;
> +  }
> +
> +  @Override
> +  public Collection<ReadEntry> getReadEntries(Scan scan) throws IOException {
> +    return null;
> +  }
> +
> +  @Override
> +  public ListMultimap<ReadEntry, DrillbitEndpoint> getReadLocations(Collection<ReadEntry> entries) {
> +    return null;
> +  }
> +
> +  @Override
> +  public RecordReader getReader(FragmentContext context, ReadEntry readEntry) throws IOException {
> +    return null;
> +  }
> +
> +  
> +  
> +}
> 
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockStorePOP.java
> ----------------------------------------------------------------------
> diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockStorePOP.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockStorePOP.java
> new file mode 100644
> index 0000000..639d0d2
> --- /dev/null
> +++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockStorePOP.java
> @@ -0,0 +1,75 @@
> +/*******************************************************************************
> + * Licensed to the Apache Software Foundation (ASF) under one
> + * or more contributor license agreements.  See the NOTICE file
> + * distributed with this work for additional information
> + * regarding copyright ownership.  The ASF licenses this file
> + * to you under the Apache License, Version 2.0 (the
> + * "License"); you may not use this file except in compliance
> + * with the License.  You may obtain a copy of the License at
> + * 
> + * http://www.apache.org/licenses/LICENSE-2.0
> + * 
> + * Unless required by applicable law or agreed to in writing, software
> + * distributed under the License is distributed on an "AS IS" BASIS,
> + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
> + * See the License for the specific language governing permissions and
> + * limitations under the License.
> + ******************************************************************************/
> +package org.apache.drill.exec.physical.config;
> +
> +import java.util.Collections;
> +import java.util.List;
> +
> +import org.apache.drill.exec.physical.EndpointAffinity;
> +import org.apache.drill.exec.physical.OperatorCost;
> +import org.apache.drill.exec.physical.base.AbstractStore;
> +import org.apache.drill.exec.physical.base.PhysicalOperator;
> +import org.apache.drill.exec.physical.base.Store;
> +import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
> +
> +import com.fasterxml.jackson.annotation.JsonCreator;
> +import com.fasterxml.jackson.annotation.JsonProperty;
> +import com.fasterxml.jackson.annotation.JsonTypeName;
> +
> +@JsonTypeName("mock-store")
> +public class MockStorePOP extends AbstractStore {
> +  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(MockStorePOP.class);
> +
> +  @JsonCreator
> +  public MockStorePOP(@JsonProperty("child") PhysicalOperator child) {
> +    super(child);
> +  }
> +
> +  public int getMaxWidth() {
> +    return 1;
> +  }
> +
> +  @Override
> +  public List<EndpointAffinity> getOperatorAffinity() {
> +    return Collections.emptyList();
> +  }
> +
> +  @Override
> +  public void applyAssignments(List<DrillbitEndpoint> endpoints) {
> +    
> +  }
> +
> +  @Override
> +  public Store getSpecificStore(PhysicalOperator child, int minorFragmentId) {
> +    return new MockStorePOP(child);
> +  }
> +
> +  @Override
> +  public OperatorCost getCost() {
> +    return new OperatorCost(1,getSize().getRecordCount()*getSize().getRecordSize(),1,1);
> +  }
> +
> +  @Override
> +  protected PhysicalOperator getNewWithChild(PhysicalOperator child) {
> +    return new MockStorePOP(child);
> +  }
> +
> +
> +  
> +
> +}
> 
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/PartitionRange.java
> ----------------------------------------------------------------------
> diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/PartitionRange.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/PartitionRange.java
> new file mode 100644
> index 0000000..eb77eeb
> --- /dev/null
> +++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/PartitionRange.java
> @@ -0,0 +1,47 @@
> +/*******************************************************************************
> + * Licensed to the Apache Software Foundation (ASF) under one
> + * or more contributor license agreements.  See the NOTICE file
> + * distributed with this work for additional information
> + * regarding copyright ownership.  The ASF licenses this file
> + * to you under the Apache License, Version 2.0 (the
> + * "License"); you may not use this file except in compliance
> + * with the License.  You may obtain a copy of the License at
> + * 
> + * http://www.apache.org/licenses/LICENSE-2.0
> + * 
> + * Unless required by applicable law or agreed to in writing, software
> + * distributed under the License is distributed on an "AS IS" BASIS,
> + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
> + * See the License for the specific language governing permissions and
> + * limitations under the License.
> + ******************************************************************************/
> +package org.apache.drill.exec.physical.config;
> +
> +import org.apache.drill.common.expression.LogicalExpression;
> +
> +import com.fasterxml.jackson.annotation.JsonCreator;
> +import com.fasterxml.jackson.annotation.JsonProperty;
> +
> +public class PartitionRange {
> +  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(PartitionRange.class);
> +  
> +  private LogicalExpression start;
> +  private LogicalExpression finish;
> +  
> +  @JsonCreator
> +  public PartitionRange(@JsonProperty("start") LogicalExpression start, @JsonProperty("finish") LogicalExpression finish) {
> +    super();
> +    this.start = start;
> +    this.finish = finish;
> +  }
> +
> +  public LogicalExpression getStart() {
> +    return start;
> +  }
> +
> +  public LogicalExpression getFinish() {
> +    return finish;
> +  }
> +  
> +  
> +}
> 
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/Project.java
> ----------------------------------------------------------------------
> diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/Project.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/Project.java
> new file mode 100644
> index 0000000..e869393
> --- /dev/null
> +++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/Project.java
> @@ -0,0 +1,72 @@
> +/*******************************************************************************
> + * Licensed to the Apache Software Foundation (ASF) under one
> + * or more contributor license agreements.  See the NOTICE file
> + * distributed with this work for additional information
> + * regarding copyright ownership.  The ASF licenses this file
> + * to you under the Apache License, Version 2.0 (the
> + * "License"); you may not use this file except in compliance
> + * with the License.  You may obtain a copy of the License at
> + * 
> + * http://www.apache.org/licenses/LICENSE-2.0
> + * 
> + * Unless required by applicable law or agreed to in writing, software
> + * distributed under the License is distributed on an "AS IS" BASIS,
> + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
> + * See the License for the specific language governing permissions and
> + * limitations under the License.
> + ******************************************************************************/
> +package org.apache.drill.exec.physical.config;
> +
> +import java.util.Iterator;
> +import java.util.List;
> +
> +import org.apache.drill.common.logical.data.NamedExpression;
> +import org.apache.drill.exec.physical.OperatorCost;
> +import org.apache.drill.exec.physical.base.AbstractSingle;
> +import org.apache.drill.exec.physical.base.PhysicalOperator;
> +import org.apache.drill.exec.physical.base.PhysicalVisitor;
> +import org.apache.drill.exec.physical.base.Size;
> +
> +import com.fasterxml.jackson.annotation.JsonCreator;
> +import com.fasterxml.jackson.annotation.JsonProperty;
> +import com.fasterxml.jackson.annotation.JsonTypeName;
> +
> +@JsonTypeName("project")
> +public class Project extends AbstractSingle{
> +  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(Project.class);
> +
> +  private final List<NamedExpression> exprs;
> +  
> +  @JsonCreator
> +  public Project(@JsonProperty("exprs") List<NamedExpression> exprs, @JsonProperty("child") PhysicalOperator child) {
> +    super(child);
> +    this.exprs = exprs;
> +  }
> +
> +  public List<NamedExpression> getExprs() {
> +    return exprs;
> +  }
> +
> +  @Override
> +  public <T, X, E extends Throwable> T accept(PhysicalVisitor<T, X, E> physicalVisitor, X value) throws E{
> +    return physicalVisitor.visitProject(this, value);
> +  }
> +
> +  @Override
> +  public OperatorCost getCost() {
> +    return new OperatorCost(0, 0, 1000, child.getSize().getRecordCount());
> +  }
> +  
> +  @Override
> +  public Size getSize() {
> +    //TODO: This should really change the row width...
> +    return child.getSize();
> +  }
> +
> +  @Override
> +  protected PhysicalOperator getNewWithChild(PhysicalOperator child) {
> +    return new Project(exprs, child);
> +  }
> +  
> +  
> +}
> 
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/RandomReceiver.java
> ----------------------------------------------------------------------
> diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/RandomReceiver.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/RandomReceiver.java
> new file mode 100644
> index 0000000..ed41586
> --- /dev/null
> +++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/RandomReceiver.java
> @@ -0,0 +1,83 @@
> +/*******************************************************************************
> + * Licensed to the Apache Software Foundation (ASF) under one
> + * or more contributor license agreements.  See the NOTICE file
> + * distributed with this work for additional information
> + * regarding copyright ownership.  The ASF licenses this file
> + * to you under the Apache License, Version 2.0 (the
> + * "License"); you may not use this file except in compliance
> + * with the License.  You may obtain a copy of the License at
> + * 
> + * http://www.apache.org/licenses/LICENSE-2.0
> + * 
> + * Unless required by applicable law or agreed to in writing, software
> + * distributed under the License is distributed on an "AS IS" BASIS,
> + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
> + * See the License for the specific language governing permissions and
> + * limitations under the License.
> + ******************************************************************************/
> +package org.apache.drill.exec.physical.config;
> +
> +import java.util.List;
> +
> +import org.apache.drill.exec.physical.OperatorCost;
> +import org.apache.drill.exec.physical.base.AbstractReceiver;
> +import org.apache.drill.exec.physical.base.PhysicalOperator;
> +import org.apache.drill.exec.physical.base.PhysicalVisitor;
> +import org.apache.drill.exec.physical.base.Size;
> +import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
> +
> +import com.fasterxml.jackson.annotation.JsonCreator;
> +import com.fasterxml.jackson.annotation.JsonIgnore;
> +import com.fasterxml.jackson.annotation.JsonProperty;
> +import com.fasterxml.jackson.annotation.JsonTypeName;
> +import com.google.common.base.Preconditions;
> +
> +@JsonTypeName("random-receiver")
> +public class RandomReceiver extends AbstractReceiver{
> +  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(RandomReceiver.class);
> +
> +  private List<DrillbitEndpoint> senders;
> +  
> +  @JsonCreator
> +  public RandomReceiver(@JsonProperty("sender-major-fragment") int oppositeMajorFragmentId, @JsonProperty("senders") List<DrillbitEndpoint> senders){
> +    super(oppositeMajorFragmentId);
> +    this.senders = senders;
> +  }
> +  
> +  @Override
> +  public List<DrillbitEndpoint> getProvidingEndpoints() {
> +    return senders;
> +  }
> +
> +  @Override
> +  public boolean supportsOutOfOrderExchange() {
> +    return true;
> +  }
> +
> +  @Override
> +  public OperatorCost getCost() {
> +    //TODO: deal with receiver cost through exchange.
> +    return new OperatorCost(1,1,1,1);
> +  }
> +
> +  
> +  @Override
> +  public <T, X, E extends Throwable> T accept(PhysicalVisitor<T, X, E> physicalVisitor, X value) throws E {
> +    return physicalVisitor.visitRandomReceiver(this, value);
> +  }
> +
> +  @Override
> +  public Size getSize() {
> +    //TODO: deal with size info through exchange.
> +    return new Size(1,1);
> +  }
> +
> +  @Override
> +  public int getOppositeMajorFragmentId() {
> +    return 0;
> +  }
> +
> +  
> +
> +  
> +}
> 
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/RangeSender.java
> ----------------------------------------------------------------------
> diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/RangeSender.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/RangeSender.java
> new file mode 100644
> index 0000000..7d64dba
> --- /dev/null
> +++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/RangeSender.java
> @@ -0,0 +1,72 @@
> +/*******************************************************************************
> + * Licensed to the Apache Software Foundation (ASF) under one
> + * or more contributor license agreements.  See the NOTICE file
> + * distributed with this work for additional information
> + * regarding copyright ownership.  The ASF licenses this file
> + * to you under the Apache License, Version 2.0 (the
> + * "License"); you may not use this file except in compliance
> + * with the License.  You may obtain a copy of the License at
> + * 
> + * http://www.apache.org/licenses/LICENSE-2.0
> + * 
> + * Unless required by applicable law or agreed to in writing, software
> + * distributed under the License is distributed on an "AS IS" BASIS,
> + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
> + * See the License for the specific language governing permissions and
> + * limitations under the License.
> + ******************************************************************************/
> +package org.apache.drill.exec.physical.config;
> +
> +import java.util.List;
> +
> +import org.apache.drill.exec.physical.base.AbstractSender;
> +import org.apache.drill.exec.physical.base.PhysicalOperator;
> +import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
> +
> +import com.fasterxml.jackson.annotation.JsonCreator;
> +import com.fasterxml.jackson.annotation.JsonProperty;
> +import com.fasterxml.jackson.annotation.JsonTypeName;
> +
> +@JsonTypeName("range-sender")
> +public class RangeSender extends AbstractSender{
> +
> +  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(RangeSender.class);
> +
> +  List<EndpointPartition> partitions;
> +  
> +  @JsonCreator
> +  public RangeSender(@JsonProperty("receiver-major-fragment") int oppositeMajorFragmentId, @JsonProperty("child") PhysicalOperator child, @JsonProperty("partitions") List<EndpointPartition> partitions) {
> +    super(oppositeMajorFragmentId, child);
> +    this.partitions = partitions;
> +  }
> +
> +  @Override
> +  public List<DrillbitEndpoint> getDestinations() {
> +    return null;
> +  }
> +
> +
> +  @Override
> +  protected PhysicalOperator getNewWithChild(PhysicalOperator child) {
> +    return new RangeSender(oppositeMajorFragmentId, child, partitions);
> +  }
> +
> +
> +  public static class EndpointPartition{
> +    private final PartitionRange range;
> +    private final DrillbitEndpoint endpoint;
> +    
> +    @JsonCreator
> +    public EndpointPartition(@JsonProperty("range") PartitionRange range, @JsonProperty("endpoint") DrillbitEndpoint endpoint) {
> +      super();
> +      this.range = range;
> +      this.endpoint = endpoint;
> +    }
> +    public PartitionRange getRange() {
> +      return range;
> +    }
> +    public DrillbitEndpoint getEndpoint() {
> +      return endpoint;
> +    }
> +  }
> +}
> 
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/Screen.java
> ----------------------------------------------------------------------
> diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/Screen.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/Screen.java
> new file mode 100644
> index 0000000..86a201d
> --- /dev/null
> +++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/Screen.java
> @@ -0,0 +1,106 @@
> +/*******************************************************************************
> + * Licensed to the Apache Software Foundation (ASF) under one
> + * or more contributor license agreements.  See the NOTICE file
> + * distributed with this work for additional information
> + * regarding copyright ownership.  The ASF licenses this file
> + * to you under the Apache License, Version 2.0 (the
> + * "License"); you may not use this file except in compliance
> + * with the License.  You may obtain a copy of the License at
> + * 
> + * http://www.apache.org/licenses/LICENSE-2.0
> + * 
> + * Unless required by applicable law or agreed to in writing, software
> + * distributed under the License is distributed on an "AS IS" BASIS,
> + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
> + * See the License for the specific language governing permissions and
> + * limitations under the License.
> + ******************************************************************************/
> +package org.apache.drill.exec.physical.config;
> +
> +import java.util.Collections;
> +import java.util.List;
> +
> +import org.apache.drill.common.exceptions.PhysicalOperatorSetupException;
> +import org.apache.drill.exec.physical.EndpointAffinity;
> +import org.apache.drill.exec.physical.OperatorCost;
> +import org.apache.drill.exec.physical.base.AbstractStore;
> +import org.apache.drill.exec.physical.base.PhysicalOperator;
> +import org.apache.drill.exec.physical.base.PhysicalVisitor;
> +import org.apache.drill.exec.physical.base.Root;
> +import org.apache.drill.exec.physical.base.Size;
> +import org.apache.drill.exec.physical.base.Store;
> +import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
> +
> +import com.fasterxml.jackson.annotation.JacksonInject;
> +import com.fasterxml.jackson.annotation.JsonIgnore;
> +import com.fasterxml.jackson.annotation.JsonProperty;
> +import com.fasterxml.jackson.annotation.JsonTypeName;
> +import com.google.common.base.Preconditions;
> +
> +@JsonTypeName("screen")
> +public class Screen extends AbstractStore implements Root{
> +  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(Screen.class);
> +
> +  private final DrillbitEndpoint endpoint;
> +
> +  public Screen(@JsonProperty("child") PhysicalOperator child, @JacksonInject DrillbitEndpoint endpoint) {
> +    super(child);
> +    this.endpoint = endpoint;
> +  }
> +
> +  @Override
> +  public List<EndpointAffinity> getOperatorAffinity() {
> +    return Collections.singletonList(new EndpointAffinity(endpoint, 1000000000000l));
> +  }
> +
> +  @Override
> +  public int getMaxWidth() {
> +    return 1;
> +  }
> +
> +  @Override
> +  public void applyAssignments(List<DrillbitEndpoint> endpoints) throws PhysicalOperatorSetupException {
> +    // we actually don't have to do anything since nothing should have changed. we'll check just check that things
> +    // didn't get screwed up.
> +    if (endpoints.size() != 1) throw new PhysicalOperatorSetupException("A Screen operator can only be assigned to a single node.");
> +    DrillbitEndpoint endpoint = endpoints.iterator().next();
> +    logger.debug("Endpoint this: {}, assignment: {}", this.endpoint, endpoint);
> +    if (!endpoint.equals(this.endpoint)) {
> +      throw new PhysicalOperatorSetupException(String.format(
> +          "A Screen operator can only be assigned to its home node.  Expected endpoint %s, Actual endpoint: %s",
> +          this.endpoint, endpoint));
> +    }
> +  }
> +
> +  @Override
> +  public Store getSpecificStore(PhysicalOperator child, int minorFragmentId) {
> +    return new Screen(child, endpoint);
> +  }
> +
> +  @JsonIgnore
> +  public DrillbitEndpoint getEndpoint() {
> +    return endpoint;
> +  }
> +
> +  @Override
> +  public String toString() {
> +    return "Screen [endpoint=" + endpoint + ", getChild()=" + getChild() + "]";
> +  }
> +
> +  @Override
> +  public OperatorCost getCost() {
> +    return new OperatorCost(0, 0, 1000, child.getSize().getRecordCount());
> +  }
> +
> +  @Override
> +  protected PhysicalOperator getNewWithChild(PhysicalOperator child) {
> +    return new Screen(child, endpoint);
> +  }
> +
> +  @Override
> +  public <T, X, E extends Throwable> T accept(PhysicalVisitor<T, X, E> physicalVisitor, X value) throws E {
> +    return physicalVisitor.visitScreen(this, value);
> +  }
> +
> +  
> +}
> 
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/SingleSender.java
> ----------------------------------------------------------------------
> diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/SingleSender.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/SingleSender.java
> new file mode 100644
> index 0000000..79d937a
> --- /dev/null
> +++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/SingleSender.java
> @@ -0,0 +1,78 @@
> +/*******************************************************************************
> + * Licensed to the Apache Software Foundation (ASF) under one
> + * or more contributor license agreements.  See the NOTICE file
> + * distributed with this work for additional information
> + * regarding copyright ownership.  The ASF licenses this file
> + * to you under the Apache License, Version 2.0 (the
> + * "License"); you may not use this file except in compliance
> + * with the License.  You may obtain a copy of the License at
> + * 
> + * http://www.apache.org/licenses/LICENSE-2.0
> + * 
> + * Unless required by applicable law or agreed to in writing, software
> + * distributed under the License is distributed on an "AS IS" BASIS,
> + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
> + * See the License for the specific language governing permissions and
> + * limitations under the License.
> + ******************************************************************************/
> +package org.apache.drill.exec.physical.config;
> +
> +import java.util.Collections;
> +import java.util.List;
> +
> +import org.apache.drill.common.graph.GraphVisitor;
> +import org.apache.drill.exec.physical.OperatorCost;
> +import org.apache.drill.exec.physical.base.AbstractSender;
> +import org.apache.drill.exec.physical.base.PhysicalOperator;
> +import org.apache.drill.exec.physical.base.PhysicalVisitor;
> +import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
> +
> +import com.fasterxml.jackson.annotation.JsonCreator;
> +import com.fasterxml.jackson.annotation.JsonIgnore;
> +import com.fasterxml.jackson.annotation.JsonProperty;
> +import com.fasterxml.jackson.annotation.JsonTypeName;
> +
> +/**
> + * Sender that pushes all data to a single destination node.
> + */
> +@JsonTypeName("single-sender")
> +public class SingleSender extends AbstractSender {
> +  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(SingleSender.class);
> +
> +  private final DrillbitEndpoint destination;
> +  
> +  @JsonCreator
> +  public SingleSender(@JsonProperty("receiver-major-fragment") int oppositeMajorFragmentId, @JsonProperty("child") PhysicalOperator child, @JsonProperty("destination") DrillbitEndpoint destination) {
> +    super(oppositeMajorFragmentId, child);
> +    this.destination = destination;
> +  }
> +
> +  @Override
> +  @JsonIgnore
> +  public List<DrillbitEndpoint> getDestinations() {
> +    return Collections.singletonList(destination);
> +  }
> +
> +  @Override
> +  public OperatorCost getCost() {
> +    long recordSize = child.getSize().getRecordSize() * child.getSize().getRecordCount();
> +    return new OperatorCost((float) recordSize, recordSize, 0, child.getSize().getRecordCount()/(1<<16));
> +  }
> +
> +  @Override
> +  protected PhysicalOperator getNewWithChild(PhysicalOperator child) {
> +    return new SingleSender(oppositeMajorFragmentId, child, destination);
> +  }
> +
> +  @Override
> +  public <T, X, E extends Throwable> T accept(PhysicalVisitor<T, X, E> physicalVisitor, X value) throws E {
> +    return physicalVisitor.visitSingleSender(this, value);
> +  }
> + 
> +
> +  public DrillbitEndpoint getDestination() {
> +    return destination;
> +  }
> + 
> +  
> +}
> 
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/Sort.java
> ----------------------------------------------------------------------
> diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/Sort.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/Sort.java
> new file mode 100644
> index 0000000..e4ece6b
> --- /dev/null
> +++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/Sort.java
> @@ -0,0 +1,86 @@
> +/*******************************************************************************
> + * Licensed to the Apache Software Foundation (ASF) under one
> + * or more contributor license agreements.  See the NOTICE file
> + * distributed with this work for additional information
> + * regarding copyright ownership.  The ASF licenses this file
> + * to you under the Apache License, Version 2.0 (the
> + * "License"); you may not use this file except in compliance
> + * with the License.  You may obtain a copy of the License at
> + * 
> + * http://www.apache.org/licenses/LICENSE-2.0
> + * 
> + * Unless required by applicable law or agreed to in writing, software
> + * distributed under the License is distributed on an "AS IS" BASIS,
> + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
> + * See the License for the specific language governing permissions and
> + * limitations under the License.
> + ******************************************************************************/
> +package org.apache.drill.exec.physical.config;
> +
> +import java.util.List;
> +
> +import org.apache.drill.common.defs.OrderDef;
> +import org.apache.drill.exec.physical.OperatorCost;
> +import org.apache.drill.exec.physical.base.AbstractSingle;
> +import org.apache.drill.exec.physical.base.PhysicalOperator;
> +import org.apache.drill.exec.physical.base.PhysicalVisitor;
> +import org.apache.drill.exec.physical.base.Size;
> +
> +import com.fasterxml.jackson.annotation.JsonCreator;
> +import com.fasterxml.jackson.annotation.JsonIgnore;
> +import com.fasterxml.jackson.annotation.JsonProperty;
> +import com.fasterxml.jackson.annotation.JsonTypeName;
> +
> +@JsonTypeName("sort")
> +public class Sort extends AbstractSingle{
> +  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(Sort.class);
> +  
> +  private final List<OrderDef> orderings;
> +  private boolean reverse = false;
> +  
> +  @JsonCreator
> +  public Sort(@JsonProperty("child") PhysicalOperator child, @JsonProperty("orderings") List<OrderDef> orderings, @JsonProperty("reverse") boolean reverse) {
> +    super(child);
> +    this.orderings = orderings;
> +    this.reverse = reverse;
> +  }
> +
> +  public List<OrderDef> getOrderings() {
> +    return orderings;
> +  }
> +
> +  public boolean getReverse() {
> +    return reverse;
> +  }
> +
> +  @Override
> +  public <T, X, E extends Throwable> T accept(PhysicalVisitor<T, X, E> physicalVisitor, X value) throws E{
> +    return physicalVisitor.visitSort(this, value);
> +  }
> +
> +  @Override
> +  public OperatorCost getCost() {
> +    Size childSize = child.getSize();
> +    long n = childSize.getRecordCount();
> +    long width = childSize.getRecordSize();
> +
> +    //TODO: Magic Number, let's assume 1/10 of data can fit in memory. 
> +    int k = 10;
> +    long n2 = n/k;
> +    double cpuCost = 
> +        k * n2 * (Math.log(n2)/Math.log(2)) + // 
> +        n * (Math.log(k)/Math.log(2));
> +    double diskCost = n*width*2;
> +    
> +    return new OperatorCost(0, (float) diskCost, (float) n2*width, (float) cpuCost);
> +  }
> +
> +  @Override
> +  protected PhysicalOperator getNewWithChild(PhysicalOperator child) {
> +    return new Sort(child, orderings, reverse);
> +  }
> +
> +    
> +  
> +  
> +}
> 
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/UnionExchange.java
> ----------------------------------------------------------------------
> diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/UnionExchange.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/UnionExchange.java
> new file mode 100644
> index 0000000..56467ce
> --- /dev/null
> +++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/UnionExchange.java
> @@ -0,0 +1,79 @@
> +/*******************************************************************************
> + * Licensed to the Apache Software Foundation (ASF) under one
> + * or more contributor license agreements.  See the NOTICE file
> + * distributed with this work for additional information
> + * regarding copyright ownership.  The ASF licenses this file
> + * to you under the Apache License, Version 2.0 (the
> + * "License"); you may not use this file except in compliance
> + * with the License.  You may obtain a copy of the License at
> + * 
> + * http://www.apache.org/licenses/LICENSE-2.0
> + * 
> + * Unless required by applicable law or agreed to in writing, software
> + * distributed under the License is distributed on an "AS IS" BASIS,
> + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
> + * See the License for the specific language governing permissions and
> + * limitations under the License.
> + ******************************************************************************/
> +package org.apache.drill.exec.physical.config;
> +
> +import java.util.List;
> +
> +import org.apache.drill.common.exceptions.PhysicalOperatorSetupException;
> +import org.apache.drill.exec.physical.base.AbstractExchange;
> +import org.apache.drill.exec.physical.base.ExchangeCost;
> +import org.apache.drill.exec.physical.base.PhysicalOperator;
> +import org.apache.drill.exec.physical.base.PhysicalVisitor;
> +import org.apache.drill.exec.physical.base.Receiver;
> +import org.apache.drill.exec.physical.base.Sender;
> +import org.apache.drill.exec.physical.base.Size;
> +import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
> +
> +import com.fasterxml.jackson.annotation.JsonProperty;
> +import com.fasterxml.jackson.annotation.JsonTypeName;
> +
> +@JsonTypeName("union-exchange")
> +public class UnionExchange extends AbstractExchange{
> +
> +  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(UnionExchange.class);
> +
> +  private List<DrillbitEndpoint> senderLocations;
> +  private DrillbitEndpoint destinationLocation;
> +  
> +  public UnionExchange(@JsonProperty("child") PhysicalOperator child) {
> +    super(child);
> +  }
> +  
> +  @Override
> +  public void setupSenders(List<DrillbitEndpoint> senderLocations) {
> +    this.senderLocations = senderLocations;
> +  }
> +
> +  @Override
> +  protected void setupReceivers(List<DrillbitEndpoint> receiverLocations) throws PhysicalOperatorSetupException {
> +    if(receiverLocations.size() != 1) throw new PhysicalOperatorSetupException("A Union Exchange only supports a single receiver endpoint.");
> +    this.destinationLocation = receiverLocations.iterator().next();
> +  }
> +
> +  @Override
> +  public Sender getSender(int minorFragmentId, PhysicalOperator child) {
> +    return new SingleSender(this.receiverMajorFragmentId, child, destinationLocation);
> +  }
> +
> +  @Override
> +  public Receiver getReceiver(int minorFragmentId) {
> +    return new RandomReceiver(this.senderMajorFragmentId, senderLocations);
> +  }
> +
> +  @Override
> +  public int getMaxSendWidth() {
> +    return Integer.MAX_VALUE;
> +  }
> +
> +  @Override
> +  protected PhysicalOperator getNewWithChild(PhysicalOperator child) {
> +    return new UnionExchange(child);
> +  }
> +  
> +  
> +}
> 
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/BatchCreator.java
> ----------------------------------------------------------------------
> diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/BatchCreator.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/BatchCreator.java
> new file mode 100644
> index 0000000..9a7df56
> --- /dev/null
> +++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/BatchCreator.java
> @@ -0,0 +1,31 @@
> +/*******************************************************************************
> + * Licensed to the Apache Software Foundation (ASF) under one
> + * or more contributor license agreements.  See the NOTICE file
> + * distributed with this work for additional information
> + * regarding copyright ownership.  The ASF licenses this file
> + * to you under the Apache License, Version 2.0 (the
> + * "License"); you may not use this file except in compliance
> + * with the License.  You may obtain a copy of the License at
> + * 
> + * http://www.apache.org/licenses/LICENSE-2.0
> + * 
> + * Unless required by applicable law or agreed to in writing, software
> + * distributed under the License is distributed on an "AS IS" BASIS,
> + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
> + * See the License for the specific language governing permissions and
> + * limitations under the License.
> + ******************************************************************************/
> +package org.apache.drill.exec.physical.impl;
> +
> +import java.util.List;
> +
> +import org.apache.drill.common.exceptions.ExecutionSetupException;
> +import org.apache.drill.exec.ops.FragmentContext;
> +import org.apache.drill.exec.physical.base.PhysicalOperator;
> +import org.apache.drill.exec.record.RecordBatch;
> +
> +public interface BatchCreator<T extends PhysicalOperator> {
> +  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(BatchCreator.class);
> +  
> +  public RecordBatch getBatch(FragmentContext context, T config, List<RecordBatch> children) throws ExecutionSetupException;
> +}
> 
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/FilterRecordBatch.java
> ----------------------------------------------------------------------
> diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/FilterRecordBatch.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/FilterRecordBatch.java
> new file mode 100644
> index 0000000..6592ca1
> --- /dev/null
> +++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/FilterRecordBatch.java
> @@ -0,0 +1,108 @@
> +/*******************************************************************************
> + * Licensed to the Apache Software Foundation (ASF) under one
> + * or more contributor license agreements.  See the NOTICE file
> + * distributed with this work for additional information
> + * regarding copyright ownership.  The ASF licenses this file
> + * to you under the Apache License, Version 2.0 (the
> + * "License"); you may not use this file except in compliance
> + * with the License.  You may obtain a copy of the License at
> + * 
> + * http://www.apache.org/licenses/LICENSE-2.0
> + * 
> + * Unless required by applicable law or agreed to in writing, software
> + * distributed under the License is distributed on an "AS IS" BASIS,
> + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
> + * See the License for the specific language governing permissions and
> + * limitations under the License.
> + ******************************************************************************/
> +package org.apache.drill.exec.physical.impl;
> +
> +import org.apache.drill.exec.ops.FragmentContext;
> +import org.apache.drill.exec.record.BatchSchema;
> +import org.apache.drill.exec.record.InvalidValueAccessor;
> +import org.apache.drill.exec.record.RecordBatch;
> +import org.apache.drill.exec.record.vector.SelectionVector;
> +import org.apache.drill.exec.record.vector.ValueVector;
> +
> +public abstract class FilterRecordBatch implements RecordBatch {
> +  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(FilterRecordBatch.class);
> +
> +  private RecordBatch incoming;
> +  private SelectionVector selectionVector;
> +  private BatchSchema schema;
> +  private FilteringRecordBatchTransformer transformer;
> +  private int outstanding;
> +
> +  public FilterRecordBatch(RecordBatch batch) {
> +    this.incoming = batch;
> +  }
> +
> +  @Override
> +  public FragmentContext getContext() {
> +    return incoming.getContext();
> +  }
> +
> +  @Override
> +  public BatchSchema getSchema() {
> +    return schema;
> +  }
> +
> +  @Override
> +  public int getRecordCount() {
> +    return 0;
> +  }
> +
> +  @Override
> +  public void kill() {
> +    incoming.kill();
> +  }
> +
> +  @Override
> +  public <T extends ValueVector<T>> T getValueVector(int fieldId, Class<T> clazz) throws InvalidValueAccessor {
> +    return null;
> +  }
> +
> +  abstract int applyFilter(SelectionVector vector, int count);
> +
> +  /**
> +   * Release all assets.
> +   */
> +  private void close() {
> +
> +  }
> +
> +  @Override
> +  public IterOutcome next() {
> +    while (true) {
> +      IterOutcome o = incoming.next();
> +      switch (o) {
> +      case OK_NEW_SCHEMA:
> +        transformer = incoming.getContext().getFilteringExpression(null);
> +        schema = transformer.getSchema();
> +        // fall through to ok.
> +      case OK:
> +
> +      case NONE:
> +      case STOP:
> +        close();
> +        return IterOutcome.STOP;
> +      }
> +
> +      if (outstanding > 0) {
> +        // move data to output location.
> +
> +        for (int i = incoming.getRecordCount() - outstanding; i < incoming.getRecordCount(); i++) {
> +
> +        }
> +      }
> +
> +      // make sure the bit vector is as large as the current record batch.
> +      if (selectionVector.capacity() < incoming.getRecordCount()) {
> +        selectionVector.allocateNew(incoming.getRecordCount());
> +      }
> +
> +      return null;
> +    }
> +
> +  }
> +}
> 
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/FilteringRecordBatchTransformer.java
> ----------------------------------------------------------------------
> diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/FilteringRecordBatchTransformer.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/FilteringRecordBatchTransformer.java
> new file mode 100644
> index 0000000..191521a
> --- /dev/null
> +++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/FilteringRecordBatchTransformer.java
> @@ -0,0 +1,58 @@
> +/*******************************************************************************
> + * Licensed to the Apache Software Foundation (ASF) under one
> + * or more contributor license agreements.  See the NOTICE file
> + * distributed with this work for additional information
> + * regarding copyright ownership.  The ASF licenses this file
> + * to you under the Apache License, Version 2.0 (the
> + * "License"); you may not use this file except in compliance
> + * with the License.  You may obtain a copy of the License at
> + * 
> + * http://www.apache.org/licenses/LICENSE-2.0
> + * 
> + * Unless required by applicable law or agreed to in writing, software
> + * distributed under the License is distributed on an "AS IS" BASIS,
> + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
> + * See the License for the specific language governing permissions and
> + * limitations under the License.
> + ******************************************************************************/
> +package org.apache.drill.exec.physical.impl;
> +
> +import org.apache.drill.exec.record.BatchSchema;
> +import org.apache.drill.exec.record.RecordBatch;
> +import org.apache.drill.exec.record.vector.SelectionVector;
> +
> +public abstract class FilteringRecordBatchTransformer {
> +  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(FilteringRecordBatchTransformer.class);
> +  
> +  final RecordBatch incoming;
> +  final SelectionVector selectionVector;
> +  final BatchSchema schema;
> +  
> +  public FilteringRecordBatchTransformer(RecordBatch incoming, OutputMutator output, SelectionVector selectionVector) {
> +    super();
> +    this.incoming = incoming;
> +    this.selectionVector = selectionVector;
> +    this.schema = innerSetup();
> +  }
> +
> +  public abstract BatchSchema innerSetup();
> +  
> +  /**
> +   * Applies the filter to the selection index.  Ignores any values in the selection vector, instead creating a.
> +   * @return
> +   */
> +  public abstract int apply();
> +  
> +  /**
> +   * Applies the filter to the selection index.  Utilizes the existing selection index and only evaluates on those records.
> +   * @return
> +   */
> +  public abstract int applyWithSelection();
> +
> +  public BatchSchema getSchema() {
> +    return schema;
> +  }
> +  
> +  
> +  
> +}
> 
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ImplCreator.java
> ----------------------------------------------------------------------
> diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ImplCreator.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ImplCreator.java
> new file mode 100644
> index 0000000..d98c107
> --- /dev/null
> +++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ImplCreator.java
> @@ -0,0 +1,102 @@
> +/*******************************************************************************
> + * Licensed to the Apache Software Foundation (ASF) under one
> + * or more contributor license agreements.  See the NOTICE file
> + * distributed with this work for additional information
> + * regarding copyright ownership.  The ASF licenses this file
> + * to you under the Apache License, Version 2.0 (the
> + * "License"); you may not use this file except in compliance
> + * with the License.  You may obtain a copy of the License at
> + * 
> + * http://www.apache.org/licenses/LICENSE-2.0
> + * 
> + * Unless required by applicable law or agreed to in writing, software
> + * distributed under the License is distributed on an "AS IS" BASIS,
> + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
> + * See the License for the specific language governing permissions and
> + * limitations under the License.
> + ******************************************************************************/
> +package org.apache.drill.exec.physical.impl;
> +
> +import java.util.Collections;
> +import java.util.List;
> +
> +import org.apache.drill.common.exceptions.ExecutionSetupException;
> +import org.apache.drill.exec.ops.FragmentContext;
> +import org.apache.drill.exec.physical.base.AbstractPhysicalVisitor;
> +import org.apache.drill.exec.physical.base.FragmentRoot;
> +import org.apache.drill.exec.physical.base.PhysicalOperator;
> +import org.apache.drill.exec.physical.base.Scan;
> +import org.apache.drill.exec.physical.config.MockScanBatchCreator;
> +import org.apache.drill.exec.physical.config.MockScanPOP;
> +import org.apache.drill.exec.physical.config.RandomReceiver;
> +import org.apache.drill.exec.physical.config.Screen;
> +import org.apache.drill.exec.physical.config.SingleSender;
> +import org.apache.drill.exec.record.RecordBatch;
> +
> +import com.google.common.base.Preconditions;
> +import com.google.common.collect.Lists;
> +
> +public class ImplCreator extends AbstractPhysicalVisitor<RecordBatch, FragmentContext, ExecutionSetupException>{
> +  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ImplCreator.class);
> +
> +  private MockScanBatchCreator msc = new MockScanBatchCreator();
> +  private ScreenCreator sc = new ScreenCreator();
> +  private RandomReceiverCreator rrc = new RandomReceiverCreator();
> +  private SingleSenderCreator ssc = new SingleSenderCreator();
> +  private RootExec root = null;
> +  
> +  private ImplCreator(){}
> +  
> +  public RootExec getRoot(){
> +    return root;
> +  }
> +  
> +  
> +  @Override
> +  public RecordBatch visitScan(Scan<?> scan, FragmentContext context) throws ExecutionSetupException {
> +    Preconditions.checkNotNull(scan);
> +    Preconditions.checkNotNull(context);
> +    
> +    if(scan instanceof MockScanPOP){
> +      return msc.getBatch(context, (MockScanPOP) scan, Collections.<RecordBatch> emptyList());
> +    }else{
> +      return super.visitScan(scan, context);  
> +    }
> +    
> +  }
> +
> +  @Override
> +  public RecordBatch visitScreen(Screen op, FragmentContext context) throws ExecutionSetupException {
> +    Preconditions.checkArgument(root == null);
> +    root = sc.getRoot(context, op, getChildren(op, context));
> +    return null;
> +  }
> +
> +  
> +  
> +  @Override
> +  public RecordBatch visitSingleSender(SingleSender op, FragmentContext context) throws ExecutionSetupException {
> +    root = ssc.getRoot(context, op, getChildren(op, context));
> +    return null;
> +  }
> +
> +  @Override
> +  public RecordBatch visitRandomReceiver(RandomReceiver op, FragmentContext context) throws ExecutionSetupException {
> +    return rrc.getBatch(context, op, null);
> +  }
> +
> +  private List<RecordBatch> getChildren(PhysicalOperator op, FragmentContext context) throws ExecutionSetupException{
> +    List<RecordBatch> children = Lists.newArrayList();
> +    for(PhysicalOperator child : op){
> +      children.add(child.accept(this, context));
> +    }
> +    return children;
> +  }
> +  
> +  public static RootExec getExec(FragmentContext context, FragmentRoot root) throws ExecutionSetupException{
> +    ImplCreator i = new ImplCreator();
> +    root.accept(i, context);
> +    if(i.root == null) throw new ExecutionSetupException("The provided fragment did not have a root node that correctly created a RootExec value.");
> +    return i.getRoot();
> +  }
> +}
> 
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/OutputMutator.java
> ----------------------------------------------------------------------
> diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/OutputMutator.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/OutputMutator.java
> new file mode 100644
> index 0000000..ce0cf66
> --- /dev/null
> +++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/OutputMutator.java
> @@ -0,0 +1,28 @@
> +/*******************************************************************************
> + * Licensed to the Apache Software Foundation (ASF) under one
> + * or more contributor license agreements.  See the NOTICE file
> + * distributed with this work for additional information
> + * regarding copyright ownership.  The ASF licenses this file
> + * to you under the Apache License, Version 2.0 (the
> + * "License"); you may not use this file except in compliance
> + * with the License.  You may obtain a copy of the License at
> + * 
> + * http://www.apache.org/licenses/LICENSE-2.0
> + * 
> + * Unless required by applicable law or agreed to in writing, software
> + * distributed under the License is distributed on an "AS IS" BASIS,
> + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
> + * See the License for the specific language governing permissions and
> + * limitations under the License.
> + ******************************************************************************/
> +package org.apache.drill.exec.physical.impl;
> +
> +import org.apache.drill.exec.exception.SchemaChangeException;
> +import org.apache.drill.exec.record.BatchSchema;
> +import org.apache.drill.exec.record.vector.ValueVector;
> +
> +public interface OutputMutator {
> +  public void removeField(int fieldId) throws SchemaChangeException;
> +  public void addField(int fieldId, ValueVector<?> vector) throws SchemaChangeException ;
> +  public void setNewSchema() throws SchemaChangeException ;
> +}
> 
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/PhysicalConfig.java
> ----------------------------------------------------------------------
> diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/PhysicalConfig.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/PhysicalConfig.java
> new file mode 100644
> index 0000000..9995bc2
> --- /dev/null
> +++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/PhysicalConfig.java
> @@ -0,0 +1,29 @@
> +/*******************************************************************************
> + * Licensed to the Apache Software Foundation (ASF) under one
> + * or more contributor license agreements.  See the NOTICE file
> + * distributed with this work for additional information
> + * regarding copyright ownership.  The ASF licenses this file
> + * to you under the Apache License, Version 2.0 (the
> + * "License"); you may not use this file except in compliance
> + * with the License.  You may obtain a copy of the License at
> + * 
> + * http://www.apache.org/licenses/LICENSE-2.0
> + * 
> + * Unless required by applicable law or agreed to in writing, software
> + * distributed under the License is distributed on an "AS IS" BASIS,
> + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
> + * See the License for the specific language governing permissions and
> + * limitations under the License.
> + ******************************************************************************/
> +package org.apache.drill.exec.physical.impl;
> +
> +import java.lang.annotation.ElementType;
> +import java.lang.annotation.Retention;
> +import java.lang.annotation.RetentionPolicy;
> +import java.lang.annotation.Target;
> +
> +@Target({ElementType.TYPE})
> +@Retention(RetentionPolicy.RUNTIME)
> +public @interface PhysicalConfig {
> +  Class<?> value();
> +}
> 
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/RandomReceiverCreator.java
> ----------------------------------------------------------------------
> diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/RandomReceiverCreator.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/RandomReceiverCreator.java
> new file mode 100644
> index 0000000..4b991f8
> --- /dev/null
> +++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/RandomReceiverCreator.java
> @@ -0,0 +1,46 @@
> +/*******************************************************************************
> + * Licensed to the Apache Software Foundation (ASF) under one
> + * or more contributor license agreements.  See the NOTICE file
> + * distributed with this work for additional information
> + * regarding copyright ownership.  The ASF licenses this file
> + * to you under the Apache License, Version 2.0 (the
> + * "License"); you may not use this file except in compliance
> + * with the License.  You may obtain a copy of the License at
> + * 
> + * http://www.apache.org/licenses/LICENSE-2.0
> + * 
> + * Unless required by applicable law or agreed to in writing, software
> + * distributed under the License is distributed on an "AS IS" BASIS,
> + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
> + * See the License for the specific language governing permissions and
> + * limitations under the License.
> + ******************************************************************************/
> +package org.apache.drill.exec.physical.impl;
> +
> +import java.util.List;
> +
> +import org.apache.drill.common.exceptions.ExecutionSetupException;
> +import org.apache.drill.exec.ops.FragmentContext;
> +import org.apache.drill.exec.physical.config.RandomReceiver;
> +import org.apache.drill.exec.record.RecordBatch;
> +import org.apache.drill.exec.work.batch.IncomingBuffers;
> +import org.apache.drill.exec.work.batch.RawBatchBuffer;
> +
> +public class RandomReceiverCreator implements BatchCreator<RandomReceiver>{
> +  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(RandomReceiverCreator.class);
> +
> +  @Override
> +  public RecordBatch getBatch(FragmentContext context, RandomReceiver receiver, List<RecordBatch> children)
> +      throws ExecutionSetupException {
> +    assert children == null || children.isEmpty();
> +    IncomingBuffers bufHolder = context.getBuffers();
> +    assert bufHolder != null : "IncomingBuffers must be defined for any place a receiver is declared.";
> +    
> +    RawBatchBuffer[] buffers = bufHolder.getBuffers(receiver.getOppositeMajorFragmentId());
> +    assert buffers.length == 1;
> +    RawBatchBuffer buffer = buffers[0];
> +    return new WireRecordBatch(context, buffer);
> +  }
> +  
> +  
> +}
> 
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/RootCreator.java
> ----------------------------------------------------------------------
> diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/RootCreator.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/RootCreator.java
> new file mode 100644
> index 0000000..80def05
> --- /dev/null
> +++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/RootCreator.java
> @@ -0,0 +1,31 @@
> +/*******************************************************************************
> + * Licensed to the Apache Software Foundation (ASF) under one
> + * or more contributor license agreements.  See the NOTICE file
> + * distributed with this work for additional information
> + * regarding copyright ownership.  The ASF licenses this file
> + * to you under the Apache License, Version 2.0 (the
> + * "License"); you may not use this file except in compliance
> + * with the License.  You may obtain a copy of the License at
> + * 
> + * http://www.apache.org/licenses/LICENSE-2.0
> + * 
> + * Unless required by applicable law or agreed to in writing, software
> + * distributed under the License is distributed on an "AS IS" BASIS,
> + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
> + * See the License for the specific language governing permissions and
> + * limitations under the License.
> + ******************************************************************************/
> +package org.apache.drill.exec.physical.impl;
> +
> +import java.util.List;
> +
> +import org.apache.drill.common.exceptions.ExecutionSetupException;
> +import org.apache.drill.exec.ops.FragmentContext;
> +import org.apache.drill.exec.physical.base.PhysicalOperator;
> +import org.apache.drill.exec.record.RecordBatch;
> +
> +public interface RootCreator<T extends PhysicalOperator> {
> +  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(RootCreator.class);
> +  
> +  public RootExec getRoot(FragmentContext context, T config, List<RecordBatch> children) throws ExecutionSetupException;
> +}
> 
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/RootExec.java
> ----------------------------------------------------------------------
> diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/RootExec.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/RootExec.java
> new file mode 100644
> index 0000000..3f8aac7
> --- /dev/null
> +++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/RootExec.java
> @@ -0,0 +1,40 @@
> +/*******************************************************************************
> + * Licensed to the Apache Software Foundation (ASF) under one
> + * or more contributor license agreements.  See the NOTICE file
> + * distributed with this work for additional information
> + * regarding copyright ownership.  The ASF licenses this file
> + * to you under the Apache License, Version 2.0 (the
> + * "License"); you may not use this file except in compliance
> + * with the License.  You may obtain a copy of the License at
> + * 
> + * http://www.apache.org/licenses/LICENSE-2.0
> + * 
> + * Unless required by applicable law or agreed to in writing, software
> + * distributed under the License is distributed on an "AS IS" BASIS,
> + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
> + * See the License for the specific language governing permissions and
> + * limitations under the License.
> + ******************************************************************************/
> +package org.apache.drill.exec.physical.impl;
> +
> +import org.apache.drill.exec.exception.FragmentSetupException;
> +
> +/**
> + * A FragmentRoot is a node which is the last processing node in a query plan. FragmentTerminals include Exchange
> + * output nodes and storage nodes.  They are there driving force behind the completion of a query.
> + */
> +public interface RootExec {
> +  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(RootExec.class);
> +  
> +  /**
> +   * Do the next batch of work.  
> +   * @return Whether or not additional batches of work are necessary.  False means that this fragment is done.
> +   */
> +  public boolean next();
> +  
> +  /**
> +   * Inform all children to clean up and go away.
> +   */
> +  public void stop();
> +  
> +}
> 
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScanBatch.java
> ----------------------------------------------------------------------
> diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScanBatch.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScanBatch.java
> new file mode 100644
> index 0000000..33c1e29
> --- /dev/null
> +++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScanBatch.java
> @@ -0,0 +1,172 @@
> +/*******************************************************************************
> + * Licensed to the Apache Software Foundation (ASF) under one
> + * or more contributor license agreements.  See the NOTICE file
> + * distributed with this work for additional information
> + * regarding copyright ownership.  The ASF licenses this file
> + * to you under the Apache License, Version 2.0 (the
> + * "License"); you may not use this file except in compliance
> + * with the License.  You may obtain a copy of the License at
> + * 
> + * http://www.apache.org/licenses/LICENSE-2.0
> + * 
> + * Unless required by applicable law or agreed to in writing, software
> + * distributed under the License is distributed on an "AS IS" BASIS,
> + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
> + * See the License for the specific language governing permissions and
> + * limitations under the License.
> + ******************************************************************************/
> +package org.apache.drill.exec.physical.impl;
> +
> +import io.netty.buffer.ByteBuf;
> +
> +import java.util.Iterator;
> +import java.util.List;
> +
> +import org.apache.drill.common.exceptions.ExecutionSetupException;
> +import org.apache.drill.exec.exception.SchemaChangeException;
> +import org.apache.drill.exec.ops.FragmentContext;
> +import org.apache.drill.exec.proto.UserBitShared.FieldMetadata;
> +import org.apache.drill.exec.proto.UserBitShared.RecordBatchDef;
> +import org.apache.drill.exec.record.BatchSchema;
> +import org.apache.drill.exec.record.InvalidValueAccessor;
> +import org.apache.drill.exec.record.RecordBatch;
> +import org.apache.drill.exec.record.SchemaBuilder;
> +import org.apache.drill.exec.record.WritableBatch;
> +import org.apache.drill.exec.record.vector.ValueVector;
> +import org.apache.drill.exec.store.RecordReader;
> +
> +import com.carrotsearch.hppc.IntObjectOpenHashMap;
> +import com.carrotsearch.hppc.procedures.IntObjectProcedure;
> +import com.google.common.collect.Lists;
> +
> +/**
> + * Record batch used for a particular scan. Operators against one or more
> + */
> +public class ScanBatch implements RecordBatch {
> +  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ScanBatch.class);
> +
> +  private IntObjectOpenHashMap<ValueVector<?>> fields = new IntObjectOpenHashMap<ValueVector<?>>();
> +  private BatchSchema schema;
> +  private int recordCount;
> +  private boolean schemaChanged = true;
> +  private final FragmentContext context;
> +  private Iterator<RecordReader> readers;
> +  private RecordReader currentReader;
> +  private final Mutator mutator = new Mutator();
> +
> +  public ScanBatch(FragmentContext context, Iterator<RecordReader> readers)
> +      throws ExecutionSetupException {
> +    this.context = context;
> +    this.readers = readers;
> +    if (!readers.hasNext()) throw new ExecutionSetupException("A scan batch must contain at least one reader.");
> +    this.currentReader = readers.next();
> +    this.currentReader.setup(mutator);
> +  }
> +
> +  private void schemaChanged() {
> +    schema = null;
> +    schemaChanged = true;
> +  }
> +
> +  @Override
> +  public FragmentContext getContext() {
> +    return context;
> +  }
> +
> +  @Override
> +  public BatchSchema getSchema() {
> +    return schema;
> +  }
> +
> +  @Override
> +  public int getRecordCount() {
> +    return recordCount;
> +  }
> +
> +  @Override
> +  public void kill() {
> +    releaseAssets();
> +  }
> +
> +  private void releaseAssets() {
> +    fields.forEach(new IntObjectProcedure<ValueVector<?>>() {
> +      @Override
> +      public void apply(int key, ValueVector<?> value) {
> +        value.close();
> +      }
> +    });
> +  }
> +
> +  @SuppressWarnings("unchecked")
> +  @Override
> +  public <T extends ValueVector<T>> T getValueVector(int fieldId, Class<T> clazz) throws InvalidValueAccessor {
> +    if (fields.containsKey(fieldId)) throw new InvalidValueAccessor(String.format("Unknown value accesor for field id %d."));
> +    ValueVector<?> vector = this.fields.lget();
> +    if (vector.getClass().isAssignableFrom(clazz)) {
> +      return (T) vector;
> +    } else {
> +      throw new InvalidValueAccessor(String.format(
> +          "You requested a field accessor of type %s for field id %d but the actual type was %s.",
> +          clazz.getCanonicalName(), fieldId, vector.getClass().getCanonicalName()));
> +    }
> +  }
> +
> +  @Override
> +  public IterOutcome next() {
> +    while ((recordCount = currentReader.next()) == 0) {
> +      try {
> +        if (!readers.hasNext()) {
> +          currentReader.cleanup();
> +          releaseAssets();
> +          return IterOutcome.NONE;
> +        }
> +        currentReader.cleanup();
> +        currentReader = readers.next();
> +        currentReader.setup(mutator);
> +      } catch (ExecutionSetupException e) {
> +        this.context.fail(e);
> +        releaseAssets();
> +        return IterOutcome.STOP;
> +      }
> +    }
> +
> +    if (schemaChanged) {
> +      schemaChanged = false;
> +      return IterOutcome.OK_NEW_SCHEMA;
> +    } else {
> +      return IterOutcome.OK;
> +    }
> +  }
> +
> +  private class Mutator implements OutputMutator {
> +    private SchemaBuilder builder = BatchSchema.newBuilder();
> +    
> +    public void removeField(int fieldId) throws SchemaChangeException {
> +      schemaChanged();
> +      ValueVector<?> v = fields.remove(fieldId);
> +      if (v == null) throw new SchemaChangeException("Failure attempting to remove an unknown field.");
> +      v.close();
> +    }
> +
> +    public void addField(int fieldId, ValueVector<?> vector) {
> +      schemaChanged();
> +      ValueVector<?> v = fields.put(fieldId, vector);
> +      vector.getField();
> +      builder.addField(vector.getField());
> +      if (v != null) v.close();
> +    }
> +
> +    @Override
> +    public void setNewSchema() throws SchemaChangeException {
> +      ScanBatch.this.schema = this.builder.build();
> +      ScanBatch.this.schemaChanged = true;
> +    }
> +
> +  }
> +
> +  @Override
> +  public WritableBatch getWritableBatch() {
> +    return WritableBatch.get(this.getRecordCount(), fields);
> +  }
> +  
> +}
> 
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScreenCreator.java
> ----------------------------------------------------------------------
> diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScreenCreator.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScreenCreator.java
> new file mode 100644
> index 0000000..c0711db
> --- /dev/null
> +++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScreenCreator.java
> @@ -0,0 +1,90 @@
> +/*******************************************************************************
> + * Licensed to the Apache Software Foundation (ASF) under one
> + * or more contributor license agreements.  See the NOTICE file
> + * distributed with this work for additional information
> + * regarding copyright ownership.  The ASF licenses this file
> + * to you under the Apache License, Version 2.0 (the
> + * "License"); you may not use this file except in compliance
> + * with the License.  You may obtain a copy of the License at
> + * 
> + * http://www.apache.org/licenses/LICENSE-2.0
> + * 
> + * Unless required by applicable law or agreed to in writing, software
> + * distributed under the License is distributed on an "AS IS" BASIS,
> + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
> + * See the License for the specific language governing permissions and
> + * limitations under the License.
> + ******************************************************************************/
> +package org.apache.drill.exec.physical.impl;
> +
> +import java.util.List;
> +
> +import org.apache.drill.exec.ops.FragmentContext;
> +import org.apache.drill.exec.physical.config.Screen;
> +import org.apache.drill.exec.physical.impl.materialize.RecordMaterializer;
> +import org.apache.drill.exec.physical.impl.materialize.VectorRecordMaterializer;
> +import org.apache.drill.exec.record.RecordBatch;
> +import org.apache.drill.exec.record.RecordBatch.IterOutcome;
> +import org.apache.drill.exec.rpc.user.UserServer.UserClientConnection;
> +
> +import com.google.common.base.Preconditions;
> +
> +public class ScreenCreator implements RootCreator<Screen>{
> +  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ScreenCreator.class);
> +
> +  @Override
> +  public RootExec getRoot(FragmentContext context, Screen config, List<RecordBatch> children) {
> +    Preconditions.checkArgument(children.size() == 1);
> +    return new ScreenRoot(context, children.iterator().next());
> +  }
> +  
> +  
> +  private static class ScreenRoot implements RootExec{
> +
> +    final RecordBatch incoming;
> +    final FragmentContext context;
> +    final UserClientConnection connection;
> +    private RecordMaterializer materializer;
> +    
> +    public ScreenRoot(FragmentContext context, RecordBatch incoming){
> +      assert context.getConnection() != null : "A screen root should only be run on the driving node which is connected directly to the client.  As such, this should always be true.";
> +
> +      this.context = context;
> +      this.incoming = incoming;
> +      this.connection = context.getConnection();
> +    }
> +    
> +    @Override
> +    public boolean next() {
> +      IterOutcome outcome = incoming.next();
> +      boolean isLast = false;
> +      switch(outcome){
> +      case NONE:
> +      case STOP:
> +        connection.sendResult(materializer.convertNext(true));
> +        context.batchesCompleted.inc(1);
> +        context.recordsCompleted.inc(incoming.getRecordCount());
> +        return false;
> +        
> +      case OK_NEW_SCHEMA:
> +        materializer = new VectorRecordMaterializer(context, incoming);
> +        // fall through.
> +        // fall through
> +      case OK:
> +        connection.sendResult(materializer.convertNext(false));
> +        context.batchesCompleted.inc(1);
> +        context.recordsCompleted.inc(incoming.getRecordCount());
> +        return !isLast;
> +      default:
> +        throw new UnsupportedOperationException();
> +      }
> +    }
> +
> +    @Override
> +    public void stop() {
> +      incoming.kill();
> +    }
> +
> +    
> +  }
> +}
> 
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/SingleSenderCreator.java
> ----------------------------------------------------------------------
> diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/SingleSenderCreator.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/SingleSenderCreator.java
> new file mode 100644
> index 0000000..60c2d78
> --- /dev/null
> +++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/SingleSenderCreator.java
> @@ -0,0 +1,89 @@
> +/*******************************************************************************
> + * Licensed to the Apache Software Foundation (ASF) under one
> + * or more contributor license agreements.  See the NOTICE file
> + * distributed with this work for additional information
> + * regarding copyright ownership.  The ASF licenses this file
> + * to you under the Apache License, Version 2.0 (the
> + * "License"); you may not use this file except in compliance
> + * with the License.  You may obtain a copy of the License at
> + * 
> + * http://www.apache.org/licenses/LICENSE-2.0
> + * 
> + * Unless required by applicable law or agreed to in writing, software
> + * distributed under the License is distributed on an "AS IS" BASIS,
> + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
> + * See the License for the specific language governing permissions and
> + * limitations under the License.
> + ******************************************************************************/
> +package org.apache.drill.exec.physical.impl;
> +
> +import java.util.List;
> +
> +import org.apache.drill.common.exceptions.ExecutionSetupException;
> +import org.apache.drill.exec.ops.FragmentContext;
> +import org.apache.drill.exec.physical.config.SingleSender;
> +import org.apache.drill.exec.proto.ExecProtos.FragmentHandle;
> +import org.apache.drill.exec.record.FragmentWritableBatch;
> +import org.apache.drill.exec.record.RecordBatch;
> +import org.apache.drill.exec.record.RecordBatch.IterOutcome;
> +import org.apache.drill.exec.rpc.bit.BitTunnel;
> +
> +public class SingleSenderCreator implements RootCreator<SingleSender>{
> +
> +  @Override
> +  public RootExec getRoot(FragmentContext context, SingleSender config, List<RecordBatch> children)
> +      throws ExecutionSetupException {
> +    assert children != null && children.size() == 1;
> +    return new SingleSenderRootExec(context, children.iterator().next(), config);
> +  }
> +  
> +  
> +  private static class SingleSenderRootExec implements RootExec{
> +    static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(SingleSenderRootExec.class);
> +    private RecordBatch incoming;
> +    private BitTunnel tunnel;
> +    private FragmentHandle handle;
> +    private int recMajor;
> +    private FragmentContext context;
> +    
> +    public SingleSenderRootExec(FragmentContext context, RecordBatch batch, SingleSender config){
> +      logger.debug("Creating single sender root exec base on config: {}", config);
> +      this.incoming = batch;
> +      this.handle = context.getHandle();
> +      this.recMajor = config.getOppositeMajorFragmentId();
> +      this.tunnel = context.getCommunicator().getTunnel(config.getDestination());
> +      this.context = context;
> +    }
> +    
> +    @Override
> +    public boolean next() {
> +      IterOutcome out = incoming.next();
> +      logger.debug("Outcome of sender next {}", out);
> +      switch(out){
> +      case STOP:
> +      case NONE:
> +        FragmentWritableBatch b2 = new FragmentWritableBatch(false, handle.getQueryId(), handle.getMajorFragmentId(), handle.getMinorFragmentId(), recMajor, 0, incoming.getWritableBatch());
> +        tunnel.sendRecordBatch(context, b2);
> +        return false;
> +        
> +
> +      case OK:
> +      case OK_NEW_SCHEMA:
> +        FragmentWritableBatch batch = new FragmentWritableBatch(true, handle.getQueryId(), handle.getMajorFragmentId(), handle.getMinorFragmentId(), recMajor, 0, incoming.getWritableBatch());
> +        tunnel.sendRecordBatch(context, batch);
> +        return true;
> +
> +      case NOT_YET:
> +      default:
> +        throw new IllegalStateException();
> +      }
> +    }
> +
> +    @Override
> +    public void stop() {
> +    }
> +    
> +    
> +    
> +  }
> +}
> 
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/WireRecordBatch.java
> ----------------------------------------------------------------------
> diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/WireRecordBatch.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/WireRecordBatch.java
> new file mode 100644
> index 0000000..fc7f833
> --- /dev/null
> +++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/WireRecordBatch.java
> @@ -0,0 +1,99 @@
> +/*******************************************************************************
> + * Licensed to the Apache Software Foundation (ASF) under one
> + * or more contributor license agreements.  See the NOTICE file
> + * distributed with this work for additional information
> + * regarding copyright ownership.  The ASF licenses this file
> + * to you under the Apache License, Version 2.0 (the
> + * "License"); you may not use this file except in compliance
> + * with the License.  You may obtain a copy of the License at
> + * 
> + * http://www.apache.org/licenses/LICENSE-2.0
> + * 
> + * Unless required by applicable law or agreed to in writing, software
> + * distributed under the License is distributed on an "AS IS" BASIS,
> + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
> + * See the License for the specific language governing permissions and
> + * limitations under the License.
> + ******************************************************************************/
> +package org.apache.drill.exec.physical.impl;
> +
> +import java.util.Iterator;
> +
> +import org.apache.drill.exec.exception.SchemaChangeException;
> +import org.apache.drill.exec.ops.FragmentContext;
> +import org.apache.drill.exec.proto.UserBitShared.RecordBatchDef;
> +import org.apache.drill.exec.record.BatchSchema;
> +import org.apache.drill.exec.record.InvalidValueAccessor;
> +import org.apache.drill.exec.record.RawFragmentBatch;
> +import org.apache.drill.exec.record.RawFragmentBatchProvider;
> +import org.apache.drill.exec.record.RecordBatch;
> +import org.apache.drill.exec.record.RecordBatchLoader;
> +import org.apache.drill.exec.record.WritableBatch;
> +import org.apache.drill.exec.record.RecordBatch.IterOutcome;
> +import org.apache.drill.exec.record.vector.ValueVector;
> +
> +public class WireRecordBatch implements RecordBatch{
> +  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(WireRecordBatch.class);
> +
> +  private RecordBatchLoader batchLoader;
> +  private RawFragmentBatchProvider fragProvider;
> +  private FragmentContext context;
> +
> +  
> +  public WireRecordBatch(FragmentContext context, RawFragmentBatchProvider fragProvider) {
> +    this.fragProvider = fragProvider;
> +    this.context = context;
> +    this.batchLoader = new RecordBatchLoader(context.getAllocator());
> +  }
> +
> +  @Override
> +  public FragmentContext getContext() {
> +    return context;
> +  }
> +
> +  @Override
> +  public BatchSchema getSchema() {
> +    return null;
> +  }
> +
> +  @Override
> +  public int getRecordCount() {
> +    return batchLoader.getRecordCount();
> +  }
> +
> +  @Override
> +  public void kill() {
> +    fragProvider.kill(context);
> +  }
> +
> +  @Override
> +  public <T extends ValueVector<T>> T getValueVector(int fieldId, Class<T> clazz) throws InvalidValueAccessor {
> +    return batchLoader.getValueVector(fieldId, clazz);
> +  }
> +
> +  @Override
> +  public IterOutcome next() {
> +    RawFragmentBatch batch = this.fragProvider.getNext();
> +    try{
> +      if(batch == null) return IterOutcome.NONE;
> +
> +      RecordBatchDef rbd = batch.getHeader().getDef();
> +      boolean schemaChanged = batchLoader.load(rbd, batch.getBody());
> +      if(schemaChanged){
> +        return IterOutcome.OK_NEW_SCHEMA;
> +      }else{
> +        return IterOutcome.OK;
> +      }
> +    }catch(SchemaChangeException ex){
> +      context.fail(ex);
> +      return IterOutcome.STOP;
> +    }
> +  }
> +
> +  @Override
> +  public WritableBatch getWritableBatch() {
> +    return batchLoader.getWritableBatch();
> +  }
> +  
> +  
> +}
> 
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/materialize/QueryWritableBatch.java
> ----------------------------------------------------------------------
> diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/materialize/QueryWritableBatch.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/materialize/QueryWritableBatch.java
> new file mode 100644
> index 0000000..187e6e9
> --- /dev/null
> +++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/materialize/QueryWritableBatch.java
> @@ -0,0 +1,46 @@
> +/*******************************************************************************
> + * Licensed to the Apache Software Foundation (ASF) under one
> + * or more contributor license agreements.  See the NOTICE file
> + * distributed with this work for additional information
> + * regarding copyright ownership.  The ASF licenses this file
> + * to you under the Apache License, Version 2.0 (the
> + * "License"); you may not use this file except in compliance
> + * with the License.  You may obtain a copy of the License at
> + * 
> + * http://www.apache.org/licenses/LICENSE-2.0
> + * 
> + * Unless required by applicable law or agreed to in writing, software
> + * distributed under the License is distributed on an "AS IS" BASIS,
> + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
> + * See the License for the specific language governing permissions and
> + * limitations under the License.
> + ******************************************************************************/
> +package org.apache.drill.exec.physical.impl.materialize;
> +
> +import io.netty.buffer.ByteBuf;
> +
> +import org.apache.drill.exec.proto.UserProtos.QueryResult;
> +import org.apache.drill.exec.record.WritableBatch;
> +
> +public class QueryWritableBatch {
> +  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(QueryWritableBatch.class);
> +  
> +  private final QueryResult header;
> +  private final ByteBuf[] buffers;
> +  
> +  
> +  public QueryWritableBatch(QueryResult header, ByteBuf... buffers) {
> +    super();
> +    this.header = header;
> +    this.buffers = buffers;
> +  }
> +
> +  public ByteBuf[] getBuffers(){
> +    return buffers;
> +  }
> +
> +  public QueryResult getHeader() {
> +    return header;
> +  }
> +  
> +}
> 
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/materialize/RecordMaterializer.java
> ----------------------------------------------------------------------
> diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/materialize/RecordMaterializer.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/materialize/RecordMaterializer.java
> new file mode 100644
> index 0000000..17c65e9
> --- /dev/null
> +++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/materialize/RecordMaterializer.java
> @@ -0,0 +1,31 @@
> +/*******************************************************************************
> + * Licensed to the Apache Software Foundation (ASF) under one
> + * or more contributor license agreements.  See the NOTICE file
> + * distributed with this work for additional information
> + * regarding copyright ownership.  The ASF licenses this file
> + * to you under the Apache License, Version 2.0 (the
> + * "License"); you may not use this file except in compliance
> + * with the License.  You may obtain a copy of the License at
> + * 
> + * http://www.apache.org/licenses/LICENSE-2.0
> + * 
> + * Unless required by applicable law or agreed to in writing, software
> + * distributed under the License is distributed on an "AS IS" BASIS,
> + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
> + * See the License for the specific language governing permissions and
> + * limitations under the License.
> + ******************************************************************************/
> +package org.apache.drill.exec.physical.impl.materialize;
> +
> +import org.apache.drill.exec.ops.FragmentContext;
> +import org.apache.drill.exec.proto.UserBitShared.QueryId;
> +import org.apache.drill.exec.proto.UserProtos.QueryResult;
> +import org.apache.drill.exec.record.MaterializedField;
> +import org.apache.drill.exec.record.RecordBatch;
> +import org.apache.drill.exec.record.WritableBatch;
> +
> +public interface RecordMaterializer {
> +  
> +  public QueryWritableBatch convertNext(boolean isLast);
> +  
> +}
> 
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/materialize/VectorRecordMaterializer.java
> ----------------------------------------------------------------------
> diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/materialize/VectorRecordMaterializer.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/materialize/VectorRecordMaterializer.java
> new file mode 100644
> index 0000000..e2d2eb9
> --- /dev/null
> +++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/materialize/VectorRecordMaterializer.java
> @@ -0,0 +1,52 @@
> +/*******************************************************************************
> + * Licensed to the Apache Software Foundation (ASF) under one
> + * or more contributor license agreements.  See the NOTICE file
> + * distributed with this work for additional information
> + * regarding copyright ownership.  The ASF licenses this file
> + * to you under the Apache License, Version 2.0 (the
> + * "License"); you may not use this file except in compliance
> + * with the License.  You may obtain a copy of the License at
> + * 
> + * http://www.apache.org/licenses/LICENSE-2.0
> + * 
> + * Unless required by applicable law or agreed to in writing, software
> + * distributed under the License is distributed on an "AS IS" BASIS,
> + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
> + * See the License for the specific language governing permissions and
> + * limitations under the License.
> + ******************************************************************************/
> +package org.apache.drill.exec.physical.impl.materialize;
> +
> +import org.apache.drill.exec.ops.FragmentContext;
> +import org.apache.drill.exec.proto.UserBitShared.QueryId;
> +import org.apache.drill.exec.proto.UserProtos.QueryResult;
> +import org.apache.drill.exec.record.MaterializedField;
> +import org.apache.drill.exec.record.RecordBatch;
> +import org.apache.drill.exec.record.WritableBatch;
> +
> +public class VectorRecordMaterializer implements RecordMaterializer{
> +  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(VectorRecordMaterializer.class);
> +
> +  private QueryId queryId;
> +  private RecordBatch batch;
> +
> +  public VectorRecordMaterializer(FragmentContext context, RecordBatch batch) {
> +    this.queryId = context.getHandle().getQueryId();
> +    this.batch = batch;
> +
> +    for (MaterializedField f : batch.getSchema()) {
> +      logger.debug("New Field: {}", f);
> +    }
> +  }
> +
> +  public QueryWritableBatch convertNext(boolean isLast) {
> +    WritableBatch w = batch.getWritableBatch();
> +
> +    QueryResult header = QueryResult.newBuilder() //
> +        .setQueryId(queryId) //
> +        .setRowCount(batch.getRecordCount()) //
> +        .setDef(w.getDef()).setIsLastChunk(isLast).build();
> +    QueryWritableBatch batch = new QueryWritableBatch(header, w.getBuffers());
> +    return batch;
> +  }
> +}
> 
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/AbstractOpWrapperVisitor.java
> ----------------------------------------------------------------------
> diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/AbstractOpWrapperVisitor.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/AbstractOpWrapperVisitor.java
> new file mode 100644
> index 0000000..9b2cb85
> --- /dev/null
> +++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/AbstractOpWrapperVisitor.java
> @@ -0,0 +1,45 @@
> +/*******************************************************************************
> + * Licensed to the Apache Software Foundation (ASF) under one
> + * or more contributor license agreements.  See the NOTICE file
> + * distributed with this work for additional information
> + * regarding copyright ownership.  The ASF licenses this file
> + * to you under the Apache License, Version 2.0 (the
> + * "License"); you may not use this file except in compliance
> + * with the License.  You may obtain a copy of the License at
> + * 
> + * http://www.apache.org/licenses/LICENSE-2.0
> + * 
> + * Unless required by applicable law or agreed to in writing, software
> + * distributed under the License is distributed on an "AS IS" BASIS,
> + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
> + * See the License for the specific language governing permissions and
> + * limitations under the License.
> + ******************************************************************************/
> +package org.apache.drill.exec.planner;
> +
> +import org.apache.drill.exec.physical.base.AbstractPhysicalVisitor;
> +import org.apache.drill.exec.physical.base.Exchange;
> +import org.apache.drill.exec.planner.fragment.Wrapper;
> +
> +public abstract class AbstractOpWrapperVisitor<RET, EXCEP extends Throwable> extends
> +    AbstractPhysicalVisitor<RET, Wrapper, EXCEP> {
> +  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(AbstractOpWrapperVisitor.class);
> +
> +  @Override
> +  public final RET visitExchange(Exchange exchange, Wrapper wrapper) throws EXCEP {
> +    if (wrapper.getNode().getSendingExchange() == exchange) {
> +      return visitSendingExchange(exchange, wrapper);
> +    } else {
> +      return visitReceivingExchange(exchange, wrapper);
> +    }
> +  }
> +
> +  public RET visitSendingExchange(Exchange exchange, Wrapper wrapper) throws EXCEP {
> +    return visitOp(exchange, wrapper);
> +  }
> +
> +  public RET visitReceivingExchange(Exchange exchange, Wrapper wrapper) throws EXCEP {
> +    return visitOp(exchange, wrapper);
> +  }
> +
> +}
> 

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

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

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

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

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

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

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/RangeSender.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/RangeSender.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/RangeSender.java
new file mode 100644
index 0000000..7d64dba
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/RangeSender.java
@@ -0,0 +1,72 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.physical.config;
+
+import java.util.List;
+
+import org.apache.drill.exec.physical.base.AbstractSender;
+import org.apache.drill.exec.physical.base.PhysicalOperator;
+import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.annotation.JsonTypeName;
+
+@JsonTypeName("range-sender")
+public class RangeSender extends AbstractSender{
+
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(RangeSender.class);
+
+  List<EndpointPartition> partitions;
+  
+  @JsonCreator
+  public RangeSender(@JsonProperty("receiver-major-fragment") int oppositeMajorFragmentId, @JsonProperty("child") PhysicalOperator child, @JsonProperty("partitions") List<EndpointPartition> partitions) {
+    super(oppositeMajorFragmentId, child);
+    this.partitions = partitions;
+  }
+
+  @Override
+  public List<DrillbitEndpoint> getDestinations() {
+    return null;
+  }
+
+
+  @Override
+  protected PhysicalOperator getNewWithChild(PhysicalOperator child) {
+    return new RangeSender(oppositeMajorFragmentId, child, partitions);
+  }
+
+
+  public static class EndpointPartition{
+    private final PartitionRange range;
+    private final DrillbitEndpoint endpoint;
+    
+    @JsonCreator
+    public EndpointPartition(@JsonProperty("range") PartitionRange range, @JsonProperty("endpoint") DrillbitEndpoint endpoint) {
+      super();
+      this.range = range;
+      this.endpoint = endpoint;
+    }
+    public PartitionRange getRange() {
+      return range;
+    }
+    public DrillbitEndpoint getEndpoint() {
+      return endpoint;
+    }
+  }
+}

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

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/SingleSender.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/SingleSender.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/SingleSender.java
new file mode 100644
index 0000000..79d937a
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/SingleSender.java
@@ -0,0 +1,78 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.physical.config;
+
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.drill.common.graph.GraphVisitor;
+import org.apache.drill.exec.physical.OperatorCost;
+import org.apache.drill.exec.physical.base.AbstractSender;
+import org.apache.drill.exec.physical.base.PhysicalOperator;
+import org.apache.drill.exec.physical.base.PhysicalVisitor;
+import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonIgnore;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.annotation.JsonTypeName;
+
+/**
+ * Sender that pushes all data to a single destination node.
+ */
+@JsonTypeName("single-sender")
+public class SingleSender extends AbstractSender {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(SingleSender.class);
+
+  private final DrillbitEndpoint destination;
+  
+  @JsonCreator
+  public SingleSender(@JsonProperty("receiver-major-fragment") int oppositeMajorFragmentId, @JsonProperty("child") PhysicalOperator child, @JsonProperty("destination") DrillbitEndpoint destination) {
+    super(oppositeMajorFragmentId, child);
+    this.destination = destination;
+  }
+
+  @Override
+  @JsonIgnore
+  public List<DrillbitEndpoint> getDestinations() {
+    return Collections.singletonList(destination);
+  }
+
+  @Override
+  public OperatorCost getCost() {
+    long recordSize = child.getSize().getRecordSize() * child.getSize().getRecordCount();
+    return new OperatorCost((float) recordSize, recordSize, 0, child.getSize().getRecordCount()/(1<<16));
+  }
+
+  @Override
+  protected PhysicalOperator getNewWithChild(PhysicalOperator child) {
+    return new SingleSender(oppositeMajorFragmentId, child, destination);
+  }
+
+  @Override
+  public <T, X, E extends Throwable> T accept(PhysicalVisitor<T, X, E> physicalVisitor, X value) throws E {
+    return physicalVisitor.visitSingleSender(this, value);
+  }
+ 
+
+  public DrillbitEndpoint getDestination() {
+    return destination;
+  }
+ 
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/Sort.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/Sort.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/Sort.java
new file mode 100644
index 0000000..e4ece6b
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/Sort.java
@@ -0,0 +1,86 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.physical.config;
+
+import java.util.List;
+
+import org.apache.drill.common.defs.OrderDef;
+import org.apache.drill.exec.physical.OperatorCost;
+import org.apache.drill.exec.physical.base.AbstractSingle;
+import org.apache.drill.exec.physical.base.PhysicalOperator;
+import org.apache.drill.exec.physical.base.PhysicalVisitor;
+import org.apache.drill.exec.physical.base.Size;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonIgnore;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.annotation.JsonTypeName;
+
+@JsonTypeName("sort")
+public class Sort extends AbstractSingle{
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(Sort.class);
+  
+  private final List<OrderDef> orderings;
+  private boolean reverse = false;
+  
+  @JsonCreator
+  public Sort(@JsonProperty("child") PhysicalOperator child, @JsonProperty("orderings") List<OrderDef> orderings, @JsonProperty("reverse") boolean reverse) {
+    super(child);
+    this.orderings = orderings;
+    this.reverse = reverse;
+  }
+
+  public List<OrderDef> getOrderings() {
+    return orderings;
+  }
+
+  public boolean getReverse() {
+    return reverse;
+  }
+
+  @Override
+  public <T, X, E extends Throwable> T accept(PhysicalVisitor<T, X, E> physicalVisitor, X value) throws E{
+    return physicalVisitor.visitSort(this, value);
+  }
+
+  @Override
+  public OperatorCost getCost() {
+    Size childSize = child.getSize();
+    long n = childSize.getRecordCount();
+    long width = childSize.getRecordSize();
+
+    //TODO: Magic Number, let's assume 1/10 of data can fit in memory. 
+    int k = 10;
+    long n2 = n/k;
+    double cpuCost = 
+        k * n2 * (Math.log(n2)/Math.log(2)) + // 
+        n * (Math.log(k)/Math.log(2));
+    double diskCost = n*width*2;
+    
+    return new OperatorCost(0, (float) diskCost, (float) n2*width, (float) cpuCost);
+  }
+
+  @Override
+  protected PhysicalOperator getNewWithChild(PhysicalOperator child) {
+    return new Sort(child, orderings, reverse);
+  }
+
+    
+  
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/UnionExchange.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/UnionExchange.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/UnionExchange.java
new file mode 100644
index 0000000..56467ce
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/UnionExchange.java
@@ -0,0 +1,79 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.physical.config;
+
+import java.util.List;
+
+import org.apache.drill.common.exceptions.PhysicalOperatorSetupException;
+import org.apache.drill.exec.physical.base.AbstractExchange;
+import org.apache.drill.exec.physical.base.ExchangeCost;
+import org.apache.drill.exec.physical.base.PhysicalOperator;
+import org.apache.drill.exec.physical.base.PhysicalVisitor;
+import org.apache.drill.exec.physical.base.Receiver;
+import org.apache.drill.exec.physical.base.Sender;
+import org.apache.drill.exec.physical.base.Size;
+import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
+
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.annotation.JsonTypeName;
+
+@JsonTypeName("union-exchange")
+public class UnionExchange extends AbstractExchange{
+
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(UnionExchange.class);
+
+  private List<DrillbitEndpoint> senderLocations;
+  private DrillbitEndpoint destinationLocation;
+  
+  public UnionExchange(@JsonProperty("child") PhysicalOperator child) {
+    super(child);
+  }
+  
+  @Override
+  public void setupSenders(List<DrillbitEndpoint> senderLocations) {
+    this.senderLocations = senderLocations;
+  }
+
+  @Override
+  protected void setupReceivers(List<DrillbitEndpoint> receiverLocations) throws PhysicalOperatorSetupException {
+    if(receiverLocations.size() != 1) throw new PhysicalOperatorSetupException("A Union Exchange only supports a single receiver endpoint.");
+    this.destinationLocation = receiverLocations.iterator().next();
+  }
+
+  @Override
+  public Sender getSender(int minorFragmentId, PhysicalOperator child) {
+    return new SingleSender(this.receiverMajorFragmentId, child, destinationLocation);
+  }
+
+  @Override
+  public Receiver getReceiver(int minorFragmentId) {
+    return new RandomReceiver(this.senderMajorFragmentId, senderLocations);
+  }
+
+  @Override
+  public int getMaxSendWidth() {
+    return Integer.MAX_VALUE;
+  }
+
+  @Override
+  protected PhysicalOperator getNewWithChild(PhysicalOperator child) {
+    return new UnionExchange(child);
+  }
+  
+  
+}

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

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

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

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ImplCreator.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ImplCreator.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ImplCreator.java
new file mode 100644
index 0000000..d98c107
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ImplCreator.java
@@ -0,0 +1,102 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.physical.impl;
+
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.drill.common.exceptions.ExecutionSetupException;
+import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.physical.base.AbstractPhysicalVisitor;
+import org.apache.drill.exec.physical.base.FragmentRoot;
+import org.apache.drill.exec.physical.base.PhysicalOperator;
+import org.apache.drill.exec.physical.base.Scan;
+import org.apache.drill.exec.physical.config.MockScanBatchCreator;
+import org.apache.drill.exec.physical.config.MockScanPOP;
+import org.apache.drill.exec.physical.config.RandomReceiver;
+import org.apache.drill.exec.physical.config.Screen;
+import org.apache.drill.exec.physical.config.SingleSender;
+import org.apache.drill.exec.record.RecordBatch;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+
+public class ImplCreator extends AbstractPhysicalVisitor<RecordBatch, FragmentContext, ExecutionSetupException>{
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ImplCreator.class);
+
+  private MockScanBatchCreator msc = new MockScanBatchCreator();
+  private ScreenCreator sc = new ScreenCreator();
+  private RandomReceiverCreator rrc = new RandomReceiverCreator();
+  private SingleSenderCreator ssc = new SingleSenderCreator();
+  private RootExec root = null;
+  
+  private ImplCreator(){}
+  
+  public RootExec getRoot(){
+    return root;
+  }
+  
+  
+  @Override
+  public RecordBatch visitScan(Scan<?> scan, FragmentContext context) throws ExecutionSetupException {
+    Preconditions.checkNotNull(scan);
+    Preconditions.checkNotNull(context);
+    
+    if(scan instanceof MockScanPOP){
+      return msc.getBatch(context, (MockScanPOP) scan, Collections.<RecordBatch> emptyList());
+    }else{
+      return super.visitScan(scan, context);  
+    }
+    
+  }
+
+  @Override
+  public RecordBatch visitScreen(Screen op, FragmentContext context) throws ExecutionSetupException {
+    Preconditions.checkArgument(root == null);
+    root = sc.getRoot(context, op, getChildren(op, context));
+    return null;
+  }
+
+  
+  
+  @Override
+  public RecordBatch visitSingleSender(SingleSender op, FragmentContext context) throws ExecutionSetupException {
+    root = ssc.getRoot(context, op, getChildren(op, context));
+    return null;
+  }
+
+  @Override
+  public RecordBatch visitRandomReceiver(RandomReceiver op, FragmentContext context) throws ExecutionSetupException {
+    return rrc.getBatch(context, op, null);
+  }
+
+  private List<RecordBatch> getChildren(PhysicalOperator op, FragmentContext context) throws ExecutionSetupException{
+    List<RecordBatch> children = Lists.newArrayList();
+    for(PhysicalOperator child : op){
+      children.add(child.accept(this, context));
+    }
+    return children;
+  }
+  
+  public static RootExec getExec(FragmentContext context, FragmentRoot root) throws ExecutionSetupException{
+    ImplCreator i = new ImplCreator();
+    root.accept(i, context);
+    if(i.root == null) throw new ExecutionSetupException("The provided fragment did not have a root node that correctly created a RootExec value.");
+    return i.getRoot();
+  }
+}

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

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

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/RandomReceiverCreator.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/RandomReceiverCreator.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/RandomReceiverCreator.java
new file mode 100644
index 0000000..4b991f8
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/RandomReceiverCreator.java
@@ -0,0 +1,46 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.physical.impl;
+
+import java.util.List;
+
+import org.apache.drill.common.exceptions.ExecutionSetupException;
+import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.physical.config.RandomReceiver;
+import org.apache.drill.exec.record.RecordBatch;
+import org.apache.drill.exec.work.batch.IncomingBuffers;
+import org.apache.drill.exec.work.batch.RawBatchBuffer;
+
+public class RandomReceiverCreator implements BatchCreator<RandomReceiver>{
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(RandomReceiverCreator.class);
+
+  @Override
+  public RecordBatch getBatch(FragmentContext context, RandomReceiver receiver, List<RecordBatch> children)
+      throws ExecutionSetupException {
+    assert children == null || children.isEmpty();
+    IncomingBuffers bufHolder = context.getBuffers();
+    assert bufHolder != null : "IncomingBuffers must be defined for any place a receiver is declared.";
+    
+    RawBatchBuffer[] buffers = bufHolder.getBuffers(receiver.getOppositeMajorFragmentId());
+    assert buffers.length == 1;
+    RawBatchBuffer buffer = buffers[0];
+    return new WireRecordBatch(context, buffer);
+  }
+  
+  
+}

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

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

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScanBatch.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScanBatch.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScanBatch.java
new file mode 100644
index 0000000..33c1e29
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScanBatch.java
@@ -0,0 +1,172 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.physical.impl;
+
+import io.netty.buffer.ByteBuf;
+
+import java.util.Iterator;
+import java.util.List;
+
+import org.apache.drill.common.exceptions.ExecutionSetupException;
+import org.apache.drill.exec.exception.SchemaChangeException;
+import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.proto.UserBitShared.FieldMetadata;
+import org.apache.drill.exec.proto.UserBitShared.RecordBatchDef;
+import org.apache.drill.exec.record.BatchSchema;
+import org.apache.drill.exec.record.InvalidValueAccessor;
+import org.apache.drill.exec.record.RecordBatch;
+import org.apache.drill.exec.record.SchemaBuilder;
+import org.apache.drill.exec.record.WritableBatch;
+import org.apache.drill.exec.record.vector.ValueVector;
+import org.apache.drill.exec.store.RecordReader;
+
+import com.carrotsearch.hppc.IntObjectOpenHashMap;
+import com.carrotsearch.hppc.procedures.IntObjectProcedure;
+import com.google.common.collect.Lists;
+
+/**
+ * Record batch used for a particular scan. Operators against one or more
+ */
+public class ScanBatch implements RecordBatch {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ScanBatch.class);
+
+  private IntObjectOpenHashMap<ValueVector<?>> fields = new IntObjectOpenHashMap<ValueVector<?>>();
+  private BatchSchema schema;
+  private int recordCount;
+  private boolean schemaChanged = true;
+  private final FragmentContext context;
+  private Iterator<RecordReader> readers;
+  private RecordReader currentReader;
+  private final Mutator mutator = new Mutator();
+
+  public ScanBatch(FragmentContext context, Iterator<RecordReader> readers)
+      throws ExecutionSetupException {
+    this.context = context;
+    this.readers = readers;
+    if (!readers.hasNext()) throw new ExecutionSetupException("A scan batch must contain at least one reader.");
+    this.currentReader = readers.next();
+    this.currentReader.setup(mutator);
+  }
+
+  private void schemaChanged() {
+    schema = null;
+    schemaChanged = true;
+  }
+
+  @Override
+  public FragmentContext getContext() {
+    return context;
+  }
+
+  @Override
+  public BatchSchema getSchema() {
+    return schema;
+  }
+
+  @Override
+  public int getRecordCount() {
+    return recordCount;
+  }
+
+  @Override
+  public void kill() {
+    releaseAssets();
+  }
+
+  private void releaseAssets() {
+    fields.forEach(new IntObjectProcedure<ValueVector<?>>() {
+      @Override
+      public void apply(int key, ValueVector<?> value) {
+        value.close();
+      }
+    });
+  }
+
+  @SuppressWarnings("unchecked")
+  @Override
+  public <T extends ValueVector<T>> T getValueVector(int fieldId, Class<T> clazz) throws InvalidValueAccessor {
+    if (fields.containsKey(fieldId)) throw new InvalidValueAccessor(String.format("Unknown value accesor for field id %d."));
+    ValueVector<?> vector = this.fields.lget();
+    if (vector.getClass().isAssignableFrom(clazz)) {
+      return (T) vector;
+    } else {
+      throw new InvalidValueAccessor(String.format(
+          "You requested a field accessor of type %s for field id %d but the actual type was %s.",
+          clazz.getCanonicalName(), fieldId, vector.getClass().getCanonicalName()));
+    }
+  }
+
+  @Override
+  public IterOutcome next() {
+    while ((recordCount = currentReader.next()) == 0) {
+      try {
+        if (!readers.hasNext()) {
+          currentReader.cleanup();
+          releaseAssets();
+          return IterOutcome.NONE;
+        }
+        currentReader.cleanup();
+        currentReader = readers.next();
+        currentReader.setup(mutator);
+      } catch (ExecutionSetupException e) {
+        this.context.fail(e);
+        releaseAssets();
+        return IterOutcome.STOP;
+      }
+    }
+
+    if (schemaChanged) {
+      schemaChanged = false;
+      return IterOutcome.OK_NEW_SCHEMA;
+    } else {
+      return IterOutcome.OK;
+    }
+  }
+
+  private class Mutator implements OutputMutator {
+    private SchemaBuilder builder = BatchSchema.newBuilder();
+    
+    public void removeField(int fieldId) throws SchemaChangeException {
+      schemaChanged();
+      ValueVector<?> v = fields.remove(fieldId);
+      if (v == null) throw new SchemaChangeException("Failure attempting to remove an unknown field.");
+      v.close();
+    }
+
+    public void addField(int fieldId, ValueVector<?> vector) {
+      schemaChanged();
+      ValueVector<?> v = fields.put(fieldId, vector);
+      vector.getField();
+      builder.addField(vector.getField());
+      if (v != null) v.close();
+    }
+
+    @Override
+    public void setNewSchema() throws SchemaChangeException {
+      ScanBatch.this.schema = this.builder.build();
+      ScanBatch.this.schemaChanged = true;
+    }
+
+  }
+
+  @Override
+  public WritableBatch getWritableBatch() {
+    return WritableBatch.get(this.getRecordCount(), fields);
+  }
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScreenCreator.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScreenCreator.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScreenCreator.java
new file mode 100644
index 0000000..c0711db
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScreenCreator.java
@@ -0,0 +1,90 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.physical.impl;
+
+import java.util.List;
+
+import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.physical.config.Screen;
+import org.apache.drill.exec.physical.impl.materialize.RecordMaterializer;
+import org.apache.drill.exec.physical.impl.materialize.VectorRecordMaterializer;
+import org.apache.drill.exec.record.RecordBatch;
+import org.apache.drill.exec.record.RecordBatch.IterOutcome;
+import org.apache.drill.exec.rpc.user.UserServer.UserClientConnection;
+
+import com.google.common.base.Preconditions;
+
+public class ScreenCreator implements RootCreator<Screen>{
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ScreenCreator.class);
+
+  @Override
+  public RootExec getRoot(FragmentContext context, Screen config, List<RecordBatch> children) {
+    Preconditions.checkArgument(children.size() == 1);
+    return new ScreenRoot(context, children.iterator().next());
+  }
+  
+  
+  private static class ScreenRoot implements RootExec{
+
+    final RecordBatch incoming;
+    final FragmentContext context;
+    final UserClientConnection connection;
+    private RecordMaterializer materializer;
+    
+    public ScreenRoot(FragmentContext context, RecordBatch incoming){
+      assert context.getConnection() != null : "A screen root should only be run on the driving node which is connected directly to the client.  As such, this should always be true.";
+
+      this.context = context;
+      this.incoming = incoming;
+      this.connection = context.getConnection();
+    }
+    
+    @Override
+    public boolean next() {
+      IterOutcome outcome = incoming.next();
+      boolean isLast = false;
+      switch(outcome){
+      case NONE:
+      case STOP:
+        connection.sendResult(materializer.convertNext(true));
+        context.batchesCompleted.inc(1);
+        context.recordsCompleted.inc(incoming.getRecordCount());
+        return false;
+        
+      case OK_NEW_SCHEMA:
+        materializer = new VectorRecordMaterializer(context, incoming);
+        // fall through.
+        // fall through
+      case OK:
+        connection.sendResult(materializer.convertNext(false));
+        context.batchesCompleted.inc(1);
+        context.recordsCompleted.inc(incoming.getRecordCount());
+        return !isLast;
+      default:
+        throw new UnsupportedOperationException();
+      }
+    }
+
+    @Override
+    public void stop() {
+      incoming.kill();
+    }
+
+    
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/SingleSenderCreator.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/SingleSenderCreator.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/SingleSenderCreator.java
new file mode 100644
index 0000000..60c2d78
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/SingleSenderCreator.java
@@ -0,0 +1,89 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.physical.impl;
+
+import java.util.List;
+
+import org.apache.drill.common.exceptions.ExecutionSetupException;
+import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.physical.config.SingleSender;
+import org.apache.drill.exec.proto.ExecProtos.FragmentHandle;
+import org.apache.drill.exec.record.FragmentWritableBatch;
+import org.apache.drill.exec.record.RecordBatch;
+import org.apache.drill.exec.record.RecordBatch.IterOutcome;
+import org.apache.drill.exec.rpc.bit.BitTunnel;
+
+public class SingleSenderCreator implements RootCreator<SingleSender>{
+
+  @Override
+  public RootExec getRoot(FragmentContext context, SingleSender config, List<RecordBatch> children)
+      throws ExecutionSetupException {
+    assert children != null && children.size() == 1;
+    return new SingleSenderRootExec(context, children.iterator().next(), config);
+  }
+  
+  
+  private static class SingleSenderRootExec implements RootExec{
+    static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(SingleSenderRootExec.class);
+    private RecordBatch incoming;
+    private BitTunnel tunnel;
+    private FragmentHandle handle;
+    private int recMajor;
+    private FragmentContext context;
+    
+    public SingleSenderRootExec(FragmentContext context, RecordBatch batch, SingleSender config){
+      logger.debug("Creating single sender root exec base on config: {}", config);
+      this.incoming = batch;
+      this.handle = context.getHandle();
+      this.recMajor = config.getOppositeMajorFragmentId();
+      this.tunnel = context.getCommunicator().getTunnel(config.getDestination());
+      this.context = context;
+    }
+    
+    @Override
+    public boolean next() {
+      IterOutcome out = incoming.next();
+      logger.debug("Outcome of sender next {}", out);
+      switch(out){
+      case STOP:
+      case NONE:
+        FragmentWritableBatch b2 = new FragmentWritableBatch(false, handle.getQueryId(), handle.getMajorFragmentId(), handle.getMinorFragmentId(), recMajor, 0, incoming.getWritableBatch());
+        tunnel.sendRecordBatch(context, b2);
+        return false;
+        
+
+      case OK:
+      case OK_NEW_SCHEMA:
+        FragmentWritableBatch batch = new FragmentWritableBatch(true, handle.getQueryId(), handle.getMajorFragmentId(), handle.getMinorFragmentId(), recMajor, 0, incoming.getWritableBatch());
+        tunnel.sendRecordBatch(context, batch);
+        return true;
+
+      case NOT_YET:
+      default:
+        throw new IllegalStateException();
+      }
+    }
+
+    @Override
+    public void stop() {
+    }
+    
+    
+    
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/WireRecordBatch.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/WireRecordBatch.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/WireRecordBatch.java
new file mode 100644
index 0000000..fc7f833
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/WireRecordBatch.java
@@ -0,0 +1,99 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.physical.impl;
+
+import java.util.Iterator;
+
+import org.apache.drill.exec.exception.SchemaChangeException;
+import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.proto.UserBitShared.RecordBatchDef;
+import org.apache.drill.exec.record.BatchSchema;
+import org.apache.drill.exec.record.InvalidValueAccessor;
+import org.apache.drill.exec.record.RawFragmentBatch;
+import org.apache.drill.exec.record.RawFragmentBatchProvider;
+import org.apache.drill.exec.record.RecordBatch;
+import org.apache.drill.exec.record.RecordBatchLoader;
+import org.apache.drill.exec.record.WritableBatch;
+import org.apache.drill.exec.record.RecordBatch.IterOutcome;
+import org.apache.drill.exec.record.vector.ValueVector;
+
+public class WireRecordBatch implements RecordBatch{
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(WireRecordBatch.class);
+
+  private RecordBatchLoader batchLoader;
+  private RawFragmentBatchProvider fragProvider;
+  private FragmentContext context;
+
+  
+  public WireRecordBatch(FragmentContext context, RawFragmentBatchProvider fragProvider) {
+    this.fragProvider = fragProvider;
+    this.context = context;
+    this.batchLoader = new RecordBatchLoader(context.getAllocator());
+  }
+
+  @Override
+  public FragmentContext getContext() {
+    return context;
+  }
+
+  @Override
+  public BatchSchema getSchema() {
+    return null;
+  }
+
+  @Override
+  public int getRecordCount() {
+    return batchLoader.getRecordCount();
+  }
+
+  @Override
+  public void kill() {
+    fragProvider.kill(context);
+  }
+
+  @Override
+  public <T extends ValueVector<T>> T getValueVector(int fieldId, Class<T> clazz) throws InvalidValueAccessor {
+    return batchLoader.getValueVector(fieldId, clazz);
+  }
+
+  @Override
+  public IterOutcome next() {
+    RawFragmentBatch batch = this.fragProvider.getNext();
+    try{
+      if(batch == null) return IterOutcome.NONE;
+
+      RecordBatchDef rbd = batch.getHeader().getDef();
+      boolean schemaChanged = batchLoader.load(rbd, batch.getBody());
+      if(schemaChanged){
+        return IterOutcome.OK_NEW_SCHEMA;
+      }else{
+        return IterOutcome.OK;
+      }
+    }catch(SchemaChangeException ex){
+      context.fail(ex);
+      return IterOutcome.STOP;
+    }
+  }
+
+  @Override
+  public WritableBatch getWritableBatch() {
+    return batchLoader.getWritableBatch();
+  }
+  
+  
+}

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

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

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/materialize/VectorRecordMaterializer.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/materialize/VectorRecordMaterializer.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/materialize/VectorRecordMaterializer.java
new file mode 100644
index 0000000..e2d2eb9
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/materialize/VectorRecordMaterializer.java
@@ -0,0 +1,52 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.physical.impl.materialize;
+
+import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.proto.UserBitShared.QueryId;
+import org.apache.drill.exec.proto.UserProtos.QueryResult;
+import org.apache.drill.exec.record.MaterializedField;
+import org.apache.drill.exec.record.RecordBatch;
+import org.apache.drill.exec.record.WritableBatch;
+
+public class VectorRecordMaterializer implements RecordMaterializer{
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(VectorRecordMaterializer.class);
+
+  private QueryId queryId;
+  private RecordBatch batch;
+
+  public VectorRecordMaterializer(FragmentContext context, RecordBatch batch) {
+    this.queryId = context.getHandle().getQueryId();
+    this.batch = batch;
+
+    for (MaterializedField f : batch.getSchema()) {
+      logger.debug("New Field: {}", f);
+    }
+  }
+
+  public QueryWritableBatch convertNext(boolean isLast) {
+    WritableBatch w = batch.getWritableBatch();
+
+    QueryResult header = QueryResult.newBuilder() //
+        .setQueryId(queryId) //
+        .setRowCount(batch.getRecordCount()) //
+        .setDef(w.getDef()).setIsLastChunk(isLast).build();
+    QueryWritableBatch batch = new QueryWritableBatch(header, w.getBuffers());
+    return batch;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/AbstractOpWrapperVisitor.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/AbstractOpWrapperVisitor.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/AbstractOpWrapperVisitor.java
new file mode 100644
index 0000000..9b2cb85
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/AbstractOpWrapperVisitor.java
@@ -0,0 +1,45 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.planner;
+
+import org.apache.drill.exec.physical.base.AbstractPhysicalVisitor;
+import org.apache.drill.exec.physical.base.Exchange;
+import org.apache.drill.exec.planner.fragment.Wrapper;
+
+public abstract class AbstractOpWrapperVisitor<RET, EXCEP extends Throwable> extends
+    AbstractPhysicalVisitor<RET, Wrapper, EXCEP> {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(AbstractOpWrapperVisitor.class);
+
+  @Override
+  public final RET visitExchange(Exchange exchange, Wrapper wrapper) throws EXCEP {
+    if (wrapper.getNode().getSendingExchange() == exchange) {
+      return visitSendingExchange(exchange, wrapper);
+    } else {
+      return visitReceivingExchange(exchange, wrapper);
+    }
+  }
+
+  public RET visitSendingExchange(Exchange exchange, Wrapper wrapper) throws EXCEP {
+    return visitOp(exchange, wrapper);
+  }
+
+  public RET visitReceivingExchange(Exchange exchange, Wrapper wrapper) throws EXCEP {
+    return visitOp(exchange, wrapper);
+  }
+
+}


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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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


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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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


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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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


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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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


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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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


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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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