You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@drill.apache.org by ja...@apache.org on 2013/08/16 03:44:39 UTC

[01/27] git commit: Added JSONScanPOP and JSONScanBatch

Updated Branches:
  refs/heads/master bd41633f1 -> 47985bad0


Added JSONScanPOP and JSONScanBatch


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

Branch: refs/heads/master
Commit: 73fad99a752f3f37944e082353aaf790154953a1
Parents: bd41633
Author: Timothy Chen <tn...@gmail.com>
Authored: Wed Jun 19 23:21:01 2013 -0700
Committer: Timothy Chen <tn...@gmail.com>
Committed: Thu Aug 8 20:46:55 2013 -0700

----------------------------------------------------------------------
 .../templates/RepeatedValueVectors.java         |   7 +-
 .../physical/config/JSONScanBatchCreator.java   |  46 +++++
 .../drill/exec/physical/config/JSONScanPOP.java | 114 ++++++++++++
 .../drill/exec/physical/impl/ImplCreator.java   |  14 +-
 .../apache/drill/exec/schema/DiffSchema.java    |  20 +--
 .../drill/exec/store/JSONRecordReader.java      | 175 +++++++++++--------
 .../apache/drill/exec/store/VectorHolder.java   |  96 +++++-----
 .../drill/exec/vector/FixedWidthVector.java     |   1 -
 .../drill/exec/vector/RepeatedMutator.java      |  23 +++
 .../physical/impl/TestSimpleFragmentRun.java    |  13 +-
 .../resources/physical_json_scan_test1.json     |  23 +++
 11 files changed, 388 insertions(+), 144 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/73fad99a/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/RepeatedValueVectors.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/RepeatedValueVectors.java b/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/RepeatedValueVectors.java
index 1afe84b..c629a1d 100644
--- a/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/RepeatedValueVectors.java
+++ b/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/RepeatedValueVectors.java
@@ -249,7 +249,7 @@ import com.google.common.collect.Lists;
     }
   }
   
-  public final class Mutator implements ValueVector.Mutator{
+  public final class Mutator implements RepeatedMutator {
 
     
     private Mutator(){
@@ -262,10 +262,7 @@ import com.google.common.collect.Lists;
      * @param index   record of the element to add
      * @param value   value to add to the given row
      */
-    public void add(int index, <#if (type.width > 4)> ${minor.javaType!type.javaType}
-                               <#elseif type.major == "VarLen"> byte[]
-                               <#else> int
-                               </#if> value) {
+    public void add(int index, <#if type.major == "VarLen">byte[]<#elseif (type.width < 4)>int<#else>${minor.javaType!type.javaType}</#if> value) {
       int nextOffset = offsets.getAccessor().get(index+1);
       values.getMutator().set(nextOffset, value);
       offsets.getMutator().set(index+1, nextOffset+1);

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/73fad99a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/JSONScanBatchCreator.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/JSONScanBatchCreator.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/JSONScanBatchCreator.java
new file mode 100644
index 0000000..f93f03b
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/JSONScanBatchCreator.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.config;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+import org.apache.drill.common.exceptions.ExecutionSetupException;
+import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.physical.impl.BatchCreator;
+import org.apache.drill.exec.physical.impl.ScanBatch;
+import org.apache.drill.exec.record.RecordBatch;
+import org.apache.drill.exec.store.JSONRecordReader;
+import org.apache.drill.exec.store.RecordReader;
+
+import java.util.List;
+
+public class JSONScanBatchCreator implements BatchCreator<JSONScanPOP> {
+
+    @Override
+    public RecordBatch getBatch(FragmentContext context, JSONScanPOP config, List<RecordBatch> children) throws ExecutionSetupException {
+        Preconditions.checkArgument(children.isEmpty());
+        List<JSONScanPOP.ScanEntry> entries = config.getReadEntries();
+        List<RecordReader> readers = Lists.newArrayList();
+        for (JSONScanPOP.ScanEntry e : entries) {
+            readers.add(new JSONRecordReader(context, e.getUrl()));
+        }
+
+        return new ScanBatch(context, readers.iterator());
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/73fad99a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/JSONScanPOP.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/JSONScanPOP.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/JSONScanPOP.java
new file mode 100644
index 0000000..1dcf5e1
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/JSONScanPOP.java
@@ -0,0 +1,114 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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 com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.annotation.JsonTypeName;
+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;
+
+import java.io.File;
+import java.net.URI;
+import java.util.Collections;
+import java.util.LinkedList;
+import java.util.List;
+
+import static com.google.common.base.Preconditions.checkArgument;
+
+@JsonTypeName("json-scan")
+public class JSONScanPOP extends AbstractScan<JSONScanPOP.ScanEntry> {
+    private static int ESTIMATED_RECORD_SIZE = 1024; // 1kb
+
+    private LinkedList[] mappings;
+
+    @JsonCreator
+    public JSONScanPOP(@JsonProperty("entries") List<JSONScanPOP.ScanEntry> readEntries) {
+        super(readEntries);
+    }
+
+    @SuppressWarnings("unchecked")
+    @Override
+    public void applyAssignments(List<CoordinationProtos.DrillbitEndpoint> endpoints) {
+        checkArgument(endpoints.size() <= getReadEntries().size());
+
+        mappings = new LinkedList[endpoints.size()];
+
+        int i = 0;
+        for (ScanEntry e : this.getReadEntries()) {
+            if (i == endpoints.size()) i = 0;
+            LinkedList entries = mappings[i];
+            if (entries == null) {
+                entries = new LinkedList<>();
+                mappings[i] = entries;
+            }
+            entries.add(e);
+            i++;
+        }
+    }
+
+    @SuppressWarnings("unchecked")
+    @Override
+    public Scan<?> getSpecificScan(int minorFragmentId) {
+        checkArgument(minorFragmentId < mappings.length, "Mappings length [%s] should be longer than minor fragment id [%s] but it isn't.", mappings.length, minorFragmentId);
+        return new JSONScanPOP(mappings[minorFragmentId]);
+    }
+
+    @Override
+    public List<EndpointAffinity> getOperatorAffinity() {
+        return Collections.emptyList();
+    }
+
+    @Override
+    public PhysicalOperator getNewWithChildren(List<PhysicalOperator> children) {
+        return new JSONScanPOP(readEntries);
+    }
+
+    public static class ScanEntry implements ReadEntry {
+        private final String url;
+        private Size size;
+
+        @JsonCreator
+        public ScanEntry(@JsonProperty("url") String url) {
+            this.url = url;
+            long fileLength = new File(URI.create(url)).length();
+            size = new Size(fileLength / ESTIMATED_RECORD_SIZE, ESTIMATED_RECORD_SIZE);
+        }
+
+        @Override
+        public OperatorCost getCost() {
+            return new OperatorCost(1, 1, 2, 2);
+        }
+
+        @Override
+        public Size getSize() {
+            return size;
+        }
+
+        public String getUrl() {
+            return url;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/73fad99a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ImplCreator.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ImplCreator.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ImplCreator.java
index c31e9e4..1c15289 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ImplCreator.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ImplCreator.java
@@ -26,15 +26,7 @@ import org.apache.drill.exec.physical.base.AbstractPhysicalVisitor;
 import org.apache.drill.exec.physical.base.FragmentRoot;
 import org.apache.drill.exec.physical.base.PhysicalOperator;
 import org.apache.drill.exec.physical.base.Scan;
-import org.apache.drill.exec.physical.config.Filter;
-import org.apache.drill.exec.physical.config.MockScanBatchCreator;
-import org.apache.drill.exec.physical.config.MockScanPOP;
-import org.apache.drill.exec.physical.config.Project;
-import org.apache.drill.exec.physical.config.RandomReceiver;
-import org.apache.drill.exec.physical.config.Screen;
-import org.apache.drill.exec.physical.config.SelectionVectorRemover;
-import org.apache.drill.exec.physical.config.SingleSender;
-import org.apache.drill.exec.physical.config.Sort;
+import org.apache.drill.exec.physical.config.*;
 import org.apache.drill.exec.physical.impl.filter.FilterBatchCreator;
 import org.apache.drill.exec.physical.impl.project.ProjectBatchCreator;
 import org.apache.drill.exec.physical.impl.sort.SortBatchCreator;
@@ -74,7 +66,9 @@ public class ImplCreator extends AbstractPhysicalVisitor<RecordBatch, FragmentCo
     Preconditions.checkNotNull(context);
     
     if(scan instanceof MockScanPOP){
-      return msc.getBatch(context, (MockScanPOP) scan, Collections.<RecordBatch> emptyList());
+      return msc.getBatch(context, (MockScanPOP) scan, Collections.<RecordBatch>emptyList());
+    } else if(scan instanceof JSONScanPOP) {
+      return new JSONScanBatchCreator().getBatch(context, (JSONScanPOP)scan, Collections.<RecordBatch>emptyList());
     }else{
       return super.visitScan(scan, context);  
     }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/73fad99a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/schema/DiffSchema.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/schema/DiffSchema.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/schema/DiffSchema.java
index b654a92..68c3e12 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/schema/DiffSchema.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/schema/DiffSchema.java
@@ -26,6 +26,7 @@ import java.util.List;
 public class DiffSchema {
     List<Field> addedFields;
     List<Field> removedFields;
+    boolean hasChanged = false;
 
     public DiffSchema() {
         this.addedFields = Lists.newArrayList();
@@ -34,27 +35,22 @@ public class DiffSchema {
 
     public void recordNewField(Field field) {
         addedFields.add(field);
-    }
-
-    public boolean hasDiffFields() {
-        return !addedFields.isEmpty() || !removedFields.isEmpty();
-    }
-
-    public List<Field> getAddedFields() {
-        return addedFields;
-    }
-
-    public List<Field> getRemovedFields() {
-        return removedFields;
+        hasChanged = true;
     }
 
     public void reset() {
         addedFields.clear();
         removedFields.clear();
+        hasChanged = false;
     }
 
     public void addRemovedField(Field field) {
         removedFields.add(field);
+        hasChanged = true;
+    }
+
+    public boolean isHasChanged() {
+        return hasChanged;
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/73fad99a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/JSONRecordReader.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/JSONRecordReader.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/JSONRecordReader.java
index f72b519..ff7d315 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/JSONRecordReader.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/JSONRecordReader.java
@@ -1,16 +1,15 @@
 package org.apache.drill.exec.store;
 
-import static com.fasterxml.jackson.core.JsonToken.END_ARRAY;
-import static com.fasterxml.jackson.core.JsonToken.END_OBJECT;
-import static com.fasterxml.jackson.core.JsonToken.FIELD_NAME;
-
-import java.io.File;
-import java.io.IOException;
-import java.io.InputStreamReader;
-import java.nio.charset.Charset;
-import java.util.List;
-import java.util.Map;
-
+import com.fasterxml.jackson.core.JsonFactory;
+import com.fasterxml.jackson.core.JsonParser;
+import com.fasterxml.jackson.core.JsonToken;
+import com.google.common.base.Charsets;
+import com.google.common.collect.Iterables;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import com.google.common.io.Files;
+import com.google.common.io.InputSupplier;
+import com.google.common.io.Resources;
 import org.apache.drill.common.exceptions.DrillRuntimeException;
 import org.apache.drill.common.exceptions.ExecutionSetupException;
 import org.apache.drill.common.expression.ExpressionPosition;
@@ -22,27 +21,19 @@ import org.apache.drill.exec.memory.BufferAllocator;
 import org.apache.drill.exec.ops.FragmentContext;
 import org.apache.drill.exec.physical.impl.OutputMutator;
 import org.apache.drill.exec.record.MaterializedField;
-import org.apache.drill.exec.schema.DiffSchema;
-import org.apache.drill.exec.schema.Field;
-import org.apache.drill.exec.schema.ListSchema;
-import org.apache.drill.exec.schema.NamedField;
-import org.apache.drill.exec.schema.ObjectSchema;
-import org.apache.drill.exec.schema.OrderedField;
-import org.apache.drill.exec.schema.RecordSchema;
-import org.apache.drill.exec.schema.SchemaIdGenerator;
+import org.apache.drill.exec.schema.*;
 import org.apache.drill.exec.schema.json.jackson.JacksonHelper;
 import org.apache.drill.exec.vector.*;
 
-import com.fasterxml.jackson.core.JsonFactory;
-import com.fasterxml.jackson.core.JsonParser;
-import com.fasterxml.jackson.core.JsonToken;
-import com.google.common.base.Charsets;
-import com.google.common.collect.Iterables;
-import com.google.common.collect.Lists;
-import com.google.common.collect.Maps;
-import com.google.common.io.Files;
-import com.google.common.io.InputSupplier;
-import com.google.common.io.Resources;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.net.URI;
+import java.nio.charset.Charset;
+import java.util.List;
+import java.util.Map;
+
+import static com.fasterxml.jackson.core.JsonToken.*;
 
 public class JSONRecordReader implements RecordReader {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(JSONRecordReader.class);
@@ -89,7 +80,7 @@ public class JSONRecordReader implements RecordReader {
       if (inputPath.startsWith("resource:")) {
         input = Resources.newReaderSupplier(Resources.getResource(inputPath.substring(9)), Charsets.UTF_8);
       } else {
-        input = Files.newReaderSupplier(new File(inputPath), Charsets.UTF_8);
+        input = Files.newReaderSupplier(new File(URI.create(inputPath)), Charsets.UTF_8);
       }
 
       JsonFactory factory = new JsonFactory();
@@ -112,7 +103,7 @@ public class JSONRecordReader implements RecordReader {
     int nextRowIndex = 0;
 
     try {
-      while (ReadType.OBJECT.readRecord(null, this, null, nextRowIndex++)) {
+      while (ReadType.OBJECT.readRecord(null, this, null, nextRowIndex++, 0)) {
         parser.nextToken(); // Read to START_OBJECT token
 
         if (!parser.hasCurrentToken()) {
@@ -133,9 +124,19 @@ public class JSONRecordReader implements RecordReader {
         outputMutator.removeField(field.getAsMaterializedField());
       }
 
+      if (diffSchema.isHasChanged()) {
+        outputMutator.setNewSchema();
+      }
+
+
     } catch (IOException | SchemaChangeException e) {
       logger.error("Error reading next in Json reader", e);
     }
+
+    for (VectorHolder holder : valueVectorMap.values()) {
+      holder.populateVectorLength();
+    }
+
     return nextRowIndex;
   }
 
@@ -171,18 +172,10 @@ public class JSONRecordReader implements RecordReader {
     return removedFields;
   }
 
-  private DiffSchema getDiffSchema() {
-    return diffSchema;
-  }
-
   public BufferAllocator getAllocator() {
     return allocator;
   }
 
-  public OutputMutator getOutputMutator() {
-    return outputMutator;
-  }
-
   public static enum ReadType {
     ARRAY(END_ARRAY) {
       @Override
@@ -221,10 +214,12 @@ public class JSONRecordReader implements RecordReader {
       return endObject;
     }
 
+    @SuppressWarnings("ConstantConditions")
     public boolean readRecord(Field parentField,
                               JSONRecordReader reader,
                               String prefixFieldName,
-                              int rowIndex) throws IOException, SchemaChangeException {
+                              int rowIndex,
+                              int groupCount) throws IOException, SchemaChangeException {
       JsonParser parser = reader.getParser();
       JsonToken token = parser.nextToken();
       JsonToken endObject = getEndObject();
@@ -242,23 +237,26 @@ public class JSONRecordReader implements RecordReader {
         switch (token) {
           case START_ARRAY:
             readType = ReadType.ARRAY;
+            groupCount++;
             break;
           case START_OBJECT:
             readType = ReadType.OBJECT;
+            groupCount = 0;
             break;
         }
-        if (fieldType != null) { // Including nulls
-          boolean currentFieldNotFull = recordData(
-              parentField,
-              readType,
-              reader,
-              fieldType,
-              prefixFieldName,
-              fieldName,
-              rowIndex, colIndex);
-
-          isFull = isFull || !currentFieldNotFull;
 
+        if (fieldType != null) { // Including nulls
+          isFull = isFull ||
+              !recordData(
+                  parentField,
+                  readType,
+                  reader,
+                  fieldType,
+                  prefixFieldName,
+                  fieldName,
+                  rowIndex,
+                  colIndex,
+                  groupCount);
         }
         token = parser.nextToken();
         colIndex += 1;
@@ -286,7 +284,8 @@ public class JSONRecordReader implements RecordReader {
                                String prefixFieldName,
                                String fieldName,
                                int rowIndex,
-                               int colIndex) throws IOException, SchemaChangeException {
+                               int colIndex,
+                               int groupCount) throws IOException, SchemaChangeException {
       RecordSchema currentSchema = reader.getCurrentSchema();
       Field field = currentSchema.getField(fieldName, colIndex);
       boolean isFieldFound = field != null;
@@ -323,10 +322,14 @@ public class JSONRecordReader implements RecordReader {
         field.assignSchemaIfNull(newSchema);
 
         if (fieldSchema == null) reader.setCurrentSchema(newSchema);
-        readType.readRecord(field, reader, field.getFullFieldName(), rowIndex);
+        if(readType == ReadType.ARRAY) {
+          readType.readRecord(field, reader, field.getFullFieldName(), rowIndex, groupCount);
+        } else {
+          readType.readRecord(field, reader, field.getFullFieldName(), rowIndex, groupCount);
+        }
 
         reader.setCurrentSchema(currentSchema);
-      } else if (holder != null) {
+      } else {
         return addValueToVector(
             rowIndex,
             holder,
@@ -335,32 +338,54 @@ public class JSONRecordReader implements RecordReader {
                 reader.getParser(),
                 fieldType.getMinorType()
             ),
-            fieldType.getMinorType()
+            fieldType.getMinorType(),
+            groupCount
         );
       }
 
       return true;
     }
 
-    private static <T> boolean addValueToVector(int index, VectorHolder holder, BufferAllocator allocator, T val, MinorType minorType) {
+    private static <T> boolean addValueToVector(int index, VectorHolder holder, BufferAllocator allocator, T val, MinorType minorType, int groupCount) {
       switch (minorType) {
         case INT: {
-          holder.incAndCheckLength(32 + 1);
-          NullableIntVector int4 = (NullableIntVector) holder.getValueVector();
-          NullableIntVector.Mutator m = int4.getMutator();
-          if (val != null) {
-            m.set(index, (Integer) val);
+          holder.incAndCheckLength(32);
+          if (groupCount == 0) {
+            if (val != null) {
+              NullableIntVector int4 = (NullableIntVector) holder.getValueVector();
+              NullableIntVector.Mutator m = int4.getMutator();
+              m.set(index, (Integer) val);
+            }
+          } else {
+            if (val == null) {
+              throw new UnsupportedOperationException("Nullable repeated int is not supported.");
+            }
+
+            RepeatedIntVector repeatedInt4 = (RepeatedIntVector) holder.getValueVector();
+            RepeatedIntVector.Mutator m = repeatedInt4.getMutator();
+            m.add(index, (Integer) val);
           }
-          return holder.hasEnoughSpace(32 + 1);
+
+          return holder.hasEnoughSpace(32);
         }
         case FLOAT4: {
-          holder.incAndCheckLength(32 + 1);
-          NullableFloat4Vector float4 = (NullableFloat4Vector) holder.getValueVector();
-          NullableFloat4Vector.Mutator m = float4.getMutator();
-          if (val != null) {
-            m.set(index, (Float) val);
+          holder.incAndCheckLength(32);
+          if (groupCount == 0) {
+            if (val != null) {
+              NullableFloat4Vector float4 = (NullableFloat4Vector) holder.getValueVector();
+              NullableFloat4Vector.Mutator m = float4.getMutator();
+              m.set(index, (Float) val);
+            }
+          } else {
+            if (val == null) {
+              throw new UnsupportedOperationException("Nullable repeated float is not supported.");
+            }
+
+            RepeatedFloat4Vector repeatedFloat4 = (RepeatedFloat4Vector) holder.getValueVector();
+            RepeatedFloat4Vector.Mutator m = repeatedFloat4.getMutator();
+            m.add(groupCount, (Float) val);
           }
-          return holder.hasEnoughSpace(32 + 1);
+          return holder.hasEnoughSpace(32);
         }
         case VARCHAR: {
           if (val == null) {
@@ -369,10 +394,16 @@ public class JSONRecordReader implements RecordReader {
             byte[] bytes = ((String) val).getBytes(UTF_8);
             int length = bytes.length;
             holder.incAndCheckLength(length);
-            NullableVarCharVector varLen4 = (NullableVarCharVector) holder.getValueVector();
-            NullableVarCharVector.Mutator m = varLen4.getMutator();
-            m.set(index, bytes);
-            return holder.hasEnoughSpace(length + 4 + 1);
+            if (groupCount == 0) {
+              NullableVarCharVector varLen4 = (NullableVarCharVector) holder.getValueVector();
+              NullableVarCharVector.Mutator m = varLen4.getMutator();
+              m.set(index, bytes);
+            } else {
+              RepeatedVarCharVector repeatedVarLen4 = (RepeatedVarCharVector) holder.getValueVector();
+              RepeatedVarCharVector.Mutator m = repeatedVarLen4.getMutator();
+              m.add(index, bytes);
+            }
+            return holder.hasEnoughSpace(length);
           }
         }
         case BIT: {

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/73fad99a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/VectorHolder.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/VectorHolder.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/VectorHolder.java
index d594b9e..43d3cd9 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/VectorHolder.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/VectorHolder.java
@@ -18,58 +18,68 @@
 
 package org.apache.drill.exec.store;
 
-import org.apache.drill.exec.vector.FixedWidthVector;
-import org.apache.drill.exec.vector.ValueVector;
-import org.apache.drill.exec.vector.VariableWidthVector;
+import org.apache.drill.exec.vector.*;
 
 public class VectorHolder {
-    private int length;
-    private ValueVector vector;
-    private ValueVector.Mutator mutator;
-    private int currentLength;
+  private int count;
+  private int groupCount;
+  private int length;
+  private ValueVector vector;
+  private int currentLength;
 
-    VectorHolder(int length, ValueVector vector) {
-        this.length = length;
-        this.vector = vector;
-        this.mutator = vector.getMutator();
-    }
+  VectorHolder(int length, ValueVector vector) {
+    this.length = length;
+    this.vector = vector;
+  }
 
-    public ValueVector getValueVector() {
-        return vector;
-    }
+  public ValueVector getValueVector() {
+    return vector;
+  }
 
-    public void incAndCheckLength(int newLength) {
-        if (!hasEnoughSpace(newLength)) {
-            throw new BatchExceededException(length, currentLength + newLength);
-        }
-        currentLength += newLength;
+  public void incAndCheckLength(int newLength) {
+    if (!hasEnoughSpace(newLength)) {
+      throw new BatchExceededException(length, currentLength + newLength);
     }
+    count += 1;
+    currentLength += newLength;
+  }
 
-    public boolean hasEnoughSpace(int newLength) {
-        return length >= currentLength + newLength;
-    }
+  public void setGroupCount(int groupCount) {
+    this.groupCount = groupCount;
+  }
 
-    public int getLength() {
-        return length;
-    }
+  public boolean hasEnoughSpace(int newLength) {
+    return length >= currentLength + newLength;
+  }
 
-    public void reset() {
-        currentLength = 0;
-        allocateNew(length);
-        
-    }
-    
-    public void allocateNew(int valueLength){
-      if(vector instanceof FixedWidthVector){
-        ((FixedWidthVector)vector).allocateNew(valueLength);  
-      }else if(vector instanceof VariableWidthVector){
-        ((VariableWidthVector)vector).allocateNew(valueLength * 10, valueLength);  
-      }else{
-        throw new UnsupportedOperationException();
-      }
+  public int getLength() {
+    return length;
+  }
+
+  public void reset() {
+    currentLength = 0;
+    count = 0;
+    allocateNew(length);
+  }
+
+  public void populateVectorLength() {
+    ValueVector.Mutator mutator = vector.getMutator();
+    if(mutator instanceof NonRepeatedMutator) {
+      ((NonRepeatedMutator)mutator).setValueCount(count);
+    } else if(mutator instanceof RepeatedMutator) {
+      ((RepeatedMutator)mutator).setGroupAndValueCount(groupCount, count);
+    } else {
+      throw new UnsupportedOperationException("Mutator not supported: " + mutator.getClass().getName());
     }
-    
-    public ValueVector.Mutator getMutator(){
-      return mutator;
+  }
+
+  public void allocateNew(int valueLength) {
+    if (vector instanceof FixedWidthVector) {
+      ((FixedWidthVector) vector).allocateNew(valueLength);
+    } else if (vector instanceof VariableWidthVector) {
+      ((VariableWidthVector) vector).allocateNew(valueLength * 10, valueLength);
+    } else {
+      throw new UnsupportedOperationException();
     }
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/73fad99a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/FixedWidthVector.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/FixedWidthVector.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/FixedWidthVector.java
index 17e072b..e5d7a30 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/FixedWidthVector.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/FixedWidthVector.java
@@ -7,7 +7,6 @@ public interface FixedWidthVector extends ValueVector{
   /**
    * Allocate a new memory space for this vector.  Must be called prior to using the ValueVector.
    *
-   * @param totalBytes   Desired size of the underlying data buffer.
    * @param valueCount   Number of values in the vector.
    */
   public void allocateNew(int valueCount);

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/73fad99a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/RepeatedMutator.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/RepeatedMutator.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/RepeatedMutator.java
new file mode 100644
index 0000000..1227d02
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/RepeatedMutator.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.vector;
+
+public interface RepeatedMutator extends ValueVector.Mutator {
+  public void setGroupAndValueCount(int groupCount, int valueCount);
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/73fad99a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestSimpleFragmentRun.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestSimpleFragmentRun.java b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestSimpleFragmentRun.java
index e21289c..5d4e700 100644
--- a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestSimpleFragmentRun.java
+++ b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestSimpleFragmentRun.java
@@ -17,13 +17,19 @@
  ******************************************************************************/
 package org.apache.drill.exec.physical.impl;
 
-import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.*;
 
+import java.io.File;
+import java.io.IOException;
+import java.net.URI;
+import java.nio.charset.Charset;
 import java.util.List;
 
+import org.apache.drill.common.types.TypeProtos;
 import org.apache.drill.common.util.FileUtils;
 import org.apache.drill.exec.client.DrillClient;
 import org.apache.drill.exec.pop.PopUnitTestBase;
+import org.apache.drill.exec.proto.SchemaDefProtos;
 import org.apache.drill.exec.proto.UserProtos.QueryType;
 import org.apache.drill.exec.record.RecordBatchLoader;
 import org.apache.drill.exec.record.VectorWrapper;
@@ -31,14 +37,19 @@ import org.apache.drill.exec.rpc.user.QueryResultBatch;
 import org.apache.drill.exec.server.Drillbit;
 import org.apache.drill.exec.server.RemoteServiceSet;
 import org.apache.drill.exec.vector.ValueVector;
+import org.junit.Assert;
 import org.junit.Test;
 
+import com.carrotsearch.hppc.cursors.IntObjectCursor;
 import com.google.common.base.Charsets;
 import com.google.common.io.Files;
 
 public class TestSimpleFragmentRun extends PopUnitTestBase {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestSimpleFragmentRun.class);
 
+  private static final Charset UTF_8 = Charset.forName("UTF-8");
+
+
   @Test
   public void runNoExchangeFragment() throws Exception {
     try(RemoteServiceSet serviceSet = RemoteServiceSet.getLocalServiceSet(); 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/73fad99a/sandbox/prototype/exec/java-exec/src/test/resources/physical_json_scan_test1.json
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/resources/physical_json_scan_test1.json b/sandbox/prototype/exec/java-exec/src/test/resources/physical_json_scan_test1.json
new file mode 100644
index 0000000..91eb80c
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/test/resources/physical_json_scan_test1.json
@@ -0,0 +1,23 @@
+{
+    head:{
+        type:"APACHE_DRILL_PHYSICAL",
+        version:"1",
+        generator:{
+            type:"manual"
+        }
+    },
+	graph:[
+        {
+            @id:1,
+            pop:"json-scan",
+            entries:[
+            	{url: "file:////home/tnachen/src/incubator-drill/sandbox/prototype/exec/java-exec/src/test/resources/scan_json_test_1.json"}
+            ]
+        },
+        {
+            @id: 2,
+            child: 1,
+            pop: "screen"
+        }
+    ]
+}
\ No newline at end of file


[17/27] git commit: cleanup comparison function tests. use a single json file, and abstract out redundant code.

Posted by ja...@apache.org.
cleanup comparison function tests. use a single json file, and abstract out redundant code.


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

Branch: refs/heads/master
Commit: be77d5aaab5ac24b99ea6f28712537cc49b43949
Parents: 2884db7
Author: Steven Phillips <sp...@maprtech.com>
Authored: Thu Aug 15 18:23:07 2013 -0700
Committer: Jacques Nadeau <ja...@apache.org>
Committed: Thu Aug 15 18:23:07 2013 -0700

----------------------------------------------------------------------
 .../physical/impl/TestComparisonFunctions.java  | 749 +++----------------
 .../impl/TestComparisonFunctionsNullable.java   | 354 ---------
 .../resources/functions/comparisonTest.json     |  39 +
 .../test/resources/functions/float4Equal.json   |  35 -
 .../resources/functions/float4GreaterThan.json  |  35 -
 .../functions/float4GreaterThanEqual.json       |  35 -
 .../resources/functions/float4LessThan.json     |  35 -
 .../functions/float4LessThanEqual.json          |  35 -
 .../resources/functions/float4NotEqual.json     |  35 -
 .../test/resources/functions/float8Equal.json   |  35 -
 .../resources/functions/float8GreaterThan.json  |  35 -
 .../functions/float8GreaterThanEqual.json       |  35 -
 .../resources/functions/float8LessThan.json     |  35 -
 .../functions/float8LessThanEqual.json          |  35 -
 .../resources/functions/float8NotEqual.json     |  35 -
 .../src/test/resources/functions/intEqual.json  |  35 -
 .../resources/functions/intGreaterThan.json     |  35 -
 .../functions/intGreaterThanEqual.json          |  35 -
 .../test/resources/functions/intLessThan.json   |  35 -
 .../resources/functions/intLessThanEqual.json   |  35 -
 .../test/resources/functions/intNotEqual.json   |  35 -
 .../src/test/resources/functions/longEqual.json |  35 -
 .../resources/functions/longGreaterThan.json    |  35 -
 .../functions/longGreaterThanEqual.json         |  35 -
 .../test/resources/functions/longLessThan.json  |  35 -
 .../resources/functions/longLessThanEqual.json  |  35 -
 .../test/resources/functions/longNotEqual.json  |  35 -
 .../functions/nullableBigIntEqual.json          |  35 -
 .../functions/nullableBigIntGreaterThan.json    |  35 -
 .../nullableBigIntGreaterThanEqual.json         |  35 -
 .../functions/nullableBigIntLessThan.json       |  35 -
 .../functions/nullableBigIntLessThanEqual.json  |  35 -
 .../functions/nullableBigIntNotEqual.json       |  35 -
 .../resources/functions/nullableIntEqual.json   |  35 -
 .../functions/nullableIntGreaterThan.json       |  35 -
 .../functions/nullableIntGreaterThanEqual.json  |  35 -
 .../functions/nullableIntLessThan.json          |  35 -
 .../functions/nullableIntLessThanEqual.json     |  35 -
 .../functions/nullableIntNotEqual.json          |  35 -
 39 files changed, 136 insertions(+), 2266 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/be77d5aa/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestComparisonFunctions.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestComparisonFunctions.java b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestComparisonFunctions.java
index 4aa4dae..ff67cf2 100644
--- a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestComparisonFunctions.java
+++ b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestComparisonFunctions.java
@@ -1,14 +1,13 @@
 package org.apache.drill.exec.physical.impl;
 
 import com.google.common.base.Charsets;
-import com.google.common.io.Files;
+import com.google.common.io.Resources;
 import com.yammer.metrics.MetricRegistry;
 
 import mockit.Injectable;
 import mockit.NonStrictExpectations;
 
 import org.apache.drill.common.config.DrillConfig;
-import org.apache.drill.common.util.FileUtils;
 import org.apache.drill.exec.expr.fn.FunctionImplementationRegistry;
 import org.apache.drill.exec.memory.BufferAllocator;
 import org.apache.drill.exec.ops.FragmentContext;
@@ -19,7 +18,6 @@ import org.apache.drill.exec.proto.CoordinationProtos;
 import org.apache.drill.exec.proto.ExecProtos;
 import org.apache.drill.exec.rpc.user.UserServer;
 import org.apache.drill.exec.server.DrillbitContext;
-import org.junit.After;
 import org.junit.AfterClass;
 import org.junit.Test;
 
@@ -29,655 +27,102 @@ import static org.junit.Assert.assertTrue;
 public class TestComparisonFunctions {
     static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestComparisonFunctions.class);
 
-    DrillConfig c = DrillConfig.create();
-
-    @Test
-    public void testIntEqual(@Injectable final DrillbitContext bitContext,
-                        @Injectable UserServer.UserClientConnection connection) throws Throwable{
-
-        new NonStrictExpectations(){{
-            bitContext.getMetrics(); result = new MetricRegistry("test");
-            bitContext.getAllocator(); result = BufferAllocator.getAllocator(c);
-        }};
-
-        PhysicalPlanReader reader = new PhysicalPlanReader(c, c.getMapper(), CoordinationProtos.DrillbitEndpoint.getDefaultInstance());
-        PhysicalPlan plan = reader.readPhysicalPlan(Files.toString(FileUtils.getResourceAsFile("/functions/intEqual.json"), Charsets.UTF_8));
-        FunctionImplementationRegistry registry = new FunctionImplementationRegistry(c);
-        FragmentContext context = new FragmentContext(bitContext, ExecProtos.FragmentHandle.getDefaultInstance(), connection, null, registry);
-        SimpleRootExec exec = new SimpleRootExec(ImplCreator.getExec(context, (FragmentRoot) plan.getSortedOperators(false).iterator().next()));
-
-        while(exec.next()){
-            assertEquals(100, exec.getSelectionVector2().getCount());
-        }
-
-        if(context.getFailureCause() != null){
-            throw context.getFailureCause();
-        }
-
-        assertTrue(!context.isFailed());
-
-    }
-
-    @Test
-    public void testBigIntEqual(@Injectable final DrillbitContext bitContext,
-                              @Injectable UserServer.UserClientConnection connection) throws Throwable{
-
-        new NonStrictExpectations(){{
-            bitContext.getMetrics(); result = new MetricRegistry("test");
-            bitContext.getAllocator(); result = BufferAllocator.getAllocator(c);
-        }};
-
-        PhysicalPlanReader reader = new PhysicalPlanReader(c, c.getMapper(), CoordinationProtos.DrillbitEndpoint.getDefaultInstance());
-        PhysicalPlan plan = reader.readPhysicalPlan(Files.toString(FileUtils.getResourceAsFile("/functions/longEqual.json"), Charsets.UTF_8));
-        FunctionImplementationRegistry registry = new FunctionImplementationRegistry(c);
-        FragmentContext context = new FragmentContext(bitContext, ExecProtos.FragmentHandle.getDefaultInstance(), connection, null, registry);
-        SimpleRootExec exec = new SimpleRootExec(ImplCreator.getExec(context, (FragmentRoot) plan.getSortedOperators(false).iterator().next()));
-
-        while(exec.next()){
-            assertEquals(100, exec.getSelectionVector2().getCount());
-        }
-
-        if(context.getFailureCause() != null){
-            throw context.getFailureCause();
-        }
-
-        assertTrue(!context.isFailed());
-
-    }
-
-    @Test
-    public void testFloat4Equal(@Injectable final DrillbitContext bitContext,
-                                @Injectable UserServer.UserClientConnection connection) throws Throwable{
-
-        new NonStrictExpectations(){{
-            bitContext.getMetrics(); result = new MetricRegistry("test");
-            bitContext.getAllocator(); result = BufferAllocator.getAllocator(c);
-        }};
-
-        PhysicalPlanReader reader = new PhysicalPlanReader(c, c.getMapper(), CoordinationProtos.DrillbitEndpoint.getDefaultInstance());
-        PhysicalPlan plan = reader.readPhysicalPlan(Files.toString(FileUtils.getResourceAsFile("/functions/float4Equal.json"), Charsets.UTF_8));
-        FunctionImplementationRegistry registry = new FunctionImplementationRegistry(c);
-        FragmentContext context = new FragmentContext(bitContext, ExecProtos.FragmentHandle.getDefaultInstance(), connection, null, registry);
-        SimpleRootExec exec = new SimpleRootExec(ImplCreator.getExec(context, (FragmentRoot) plan.getSortedOperators(false).iterator().next()));
-
-        while(exec.next()){
-            assertEquals(100, exec.getSelectionVector2().getCount());
-        }
-
-        if(context.getFailureCause() != null){
-            throw context.getFailureCause();
-        }
-
-        assertTrue(!context.isFailed());
-
-    }
-
-    @Test
-    public void testFloat8Equal(@Injectable final DrillbitContext bitContext,
-                                @Injectable UserServer.UserClientConnection connection) throws Throwable{
-
-        new NonStrictExpectations(){{
-            bitContext.getMetrics(); result = new MetricRegistry("test");
-            bitContext.getAllocator(); result = BufferAllocator.getAllocator(c);
-        }};
-
-        PhysicalPlanReader reader = new PhysicalPlanReader(c, c.getMapper(), CoordinationProtos.DrillbitEndpoint.getDefaultInstance());
-        PhysicalPlan plan = reader.readPhysicalPlan(Files.toString(FileUtils.getResourceAsFile("/functions/float8Equal.json"), Charsets.UTF_8));
-        FunctionImplementationRegistry registry = new FunctionImplementationRegistry(c);
-        FragmentContext context = new FragmentContext(bitContext, ExecProtos.FragmentHandle.getDefaultInstance(), connection, null, registry);
-        SimpleRootExec exec = new SimpleRootExec(ImplCreator.getExec(context, (FragmentRoot) plan.getSortedOperators(false).iterator().next()));
-
-        while(exec.next()){
-            assertEquals(100, exec.getSelectionVector2().getCount());
-        }
-
-        if(context.getFailureCause() != null){
-            throw context.getFailureCause();
-        }
-
-        assertTrue(!context.isFailed());
-
-    }
-
-    @Test
-    public void testIntNotEqual(@Injectable final DrillbitContext bitContext,
-                              @Injectable UserServer.UserClientConnection connection) throws Throwable{
-
-        new NonStrictExpectations(){{
-            bitContext.getMetrics(); result = new MetricRegistry("test");
-            bitContext.getAllocator(); result = BufferAllocator.getAllocator(c);
-        }};
-
-        PhysicalPlanReader reader = new PhysicalPlanReader(c, c.getMapper(), CoordinationProtos.DrillbitEndpoint.getDefaultInstance());
-        PhysicalPlan plan = reader.readPhysicalPlan(Files.toString(FileUtils.getResourceAsFile("/functions/intNotEqual.json"), Charsets.UTF_8));
-        FunctionImplementationRegistry registry = new FunctionImplementationRegistry(c);
-        FragmentContext context = new FragmentContext(bitContext, ExecProtos.FragmentHandle.getDefaultInstance(), connection, null, registry);
-        SimpleRootExec exec = new SimpleRootExec(ImplCreator.getExec(context, (FragmentRoot) plan.getSortedOperators(false).iterator().next()));
-
-        while(exec.next()){
-            assertEquals(0, exec.getSelectionVector2().getCount());
-        }
-
-        if(context.getFailureCause() != null){
-            throw context.getFailureCause();
-        }
-
-        assertTrue(!context.isFailed());
-
-    }
-
-    @Test
-    public void testBigIntNotEqual(@Injectable final DrillbitContext bitContext,
-                                @Injectable UserServer.UserClientConnection connection) throws Throwable{
-
-        new NonStrictExpectations(){{
-            bitContext.getMetrics(); result = new MetricRegistry("test");
-            bitContext.getAllocator(); result = BufferAllocator.getAllocator(c);
-        }};
-
-        PhysicalPlanReader reader = new PhysicalPlanReader(c, c.getMapper(), CoordinationProtos.DrillbitEndpoint.getDefaultInstance());
-        PhysicalPlan plan = reader.readPhysicalPlan(Files.toString(FileUtils.getResourceAsFile("/functions/longNotEqual.json"), Charsets.UTF_8));
-        FunctionImplementationRegistry registry = new FunctionImplementationRegistry(c);
-        FragmentContext context = new FragmentContext(bitContext, ExecProtos.FragmentHandle.getDefaultInstance(), connection, null, registry);
-        SimpleRootExec exec = new SimpleRootExec(ImplCreator.getExec(context, (FragmentRoot) plan.getSortedOperators(false).iterator().next()));
-
-        while(exec.next()){
-            assertEquals(0, exec.getSelectionVector2().getCount());
-        }
-
-        if(context.getFailureCause() != null){
-            throw context.getFailureCause();
-        }
-
-        assertTrue(!context.isFailed());
-
-    }
-
-    @Test
-    public void testFloat4NotEqual(@Injectable final DrillbitContext bitContext,
-                                   @Injectable UserServer.UserClientConnection connection) throws Throwable{
-
-        new NonStrictExpectations(){{
-            bitContext.getMetrics(); result = new MetricRegistry("test");
-            bitContext.getAllocator(); result = BufferAllocator.getAllocator(c);
-        }};
-
-        PhysicalPlanReader reader = new PhysicalPlanReader(c, c.getMapper(), CoordinationProtos.DrillbitEndpoint.getDefaultInstance());
-        PhysicalPlan plan = reader.readPhysicalPlan(Files.toString(FileUtils.getResourceAsFile("/functions/float4NotEqual.json"), Charsets.UTF_8));
-        FunctionImplementationRegistry registry = new FunctionImplementationRegistry(c);
-        FragmentContext context = new FragmentContext(bitContext, ExecProtos.FragmentHandle.getDefaultInstance(), connection, null, registry);
-        SimpleRootExec exec = new SimpleRootExec(ImplCreator.getExec(context, (FragmentRoot) plan.getSortedOperators(false).iterator().next()));
-
-        while(exec.next()){
-            assertEquals(0, exec.getSelectionVector2().getCount());
-        }
-
-        if(context.getFailureCause() != null){
-            throw context.getFailureCause();
-        }
-
-        assertTrue(!context.isFailed());
-
-    }
-
-    @Test
-    public void testFloat8NotEqual(@Injectable final DrillbitContext bitContext,
-                                   @Injectable UserServer.UserClientConnection connection) throws Throwable{
-
-        new NonStrictExpectations(){{
-            bitContext.getMetrics(); result = new MetricRegistry("test");
-            bitContext.getAllocator(); result = BufferAllocator.getAllocator(c);
-        }};
-
-        PhysicalPlanReader reader = new PhysicalPlanReader(c, c.getMapper(), CoordinationProtos.DrillbitEndpoint.getDefaultInstance());
-        PhysicalPlan plan = reader.readPhysicalPlan(Files.toString(FileUtils.getResourceAsFile("/functions/float8NotEqual.json"), Charsets.UTF_8));
-        FunctionImplementationRegistry registry = new FunctionImplementationRegistry(c);
-        FragmentContext context = new FragmentContext(bitContext, ExecProtos.FragmentHandle.getDefaultInstance(), connection, null, registry);
-        SimpleRootExec exec = new SimpleRootExec(ImplCreator.getExec(context, (FragmentRoot) plan.getSortedOperators(false).iterator().next()));
-
-        while(exec.next()){
-            assertEquals(0, exec.getSelectionVector2().getCount());
-        }
-
-        if(context.getFailureCause() != null){
-            throw context.getFailureCause();
-        }
-
-        assertTrue(!context.isFailed());
-
-    }
-
-    @Test
-    public void testIntGreaterThan(@Injectable final DrillbitContext bitContext,
-                                   @Injectable UserServer.UserClientConnection connection) throws Throwable{
-
-        new NonStrictExpectations(){{
-            bitContext.getMetrics(); result = new MetricRegistry("test");
-            bitContext.getAllocator(); result = BufferAllocator.getAllocator(c);
-        }};
-
-        PhysicalPlanReader reader = new PhysicalPlanReader(c, c.getMapper(), CoordinationProtos.DrillbitEndpoint.getDefaultInstance());
-        PhysicalPlan plan = reader.readPhysicalPlan(Files.toString(FileUtils.getResourceAsFile("/functions/intGreaterThan.json"), Charsets.UTF_8));
-        FunctionImplementationRegistry registry = new FunctionImplementationRegistry(c);
-        FragmentContext context = new FragmentContext(bitContext, ExecProtos.FragmentHandle.getDefaultInstance(), connection, null, registry);
-        SimpleRootExec exec = new SimpleRootExec(ImplCreator.getExec(context, (FragmentRoot) plan.getSortedOperators(false).iterator().next()));
-
-        while(exec.next()){
-            assertEquals(0, exec.getSelectionVector2().getCount());
-        }
-
-        if(context.getFailureCause() != null){
-            throw context.getFailureCause();
-        }
-
-        assertTrue(!context.isFailed());
-
-    }
-
-    @Test
-    public void testBigIntGreaterThan(@Injectable final DrillbitContext bitContext,
-                                   @Injectable UserServer.UserClientConnection connection) throws Throwable{
-
-        new NonStrictExpectations(){{
-            bitContext.getMetrics(); result = new MetricRegistry("test");
-            bitContext.getAllocator(); result = BufferAllocator.getAllocator(c);
-        }};
-
-        PhysicalPlanReader reader = new PhysicalPlanReader(c, c.getMapper(), CoordinationProtos.DrillbitEndpoint.getDefaultInstance());
-        PhysicalPlan plan = reader.readPhysicalPlan(Files.toString(FileUtils.getResourceAsFile("/functions/longGreaterThan.json"), Charsets.UTF_8));
-        FunctionImplementationRegistry registry = new FunctionImplementationRegistry(c);
-        FragmentContext context = new FragmentContext(bitContext, ExecProtos.FragmentHandle.getDefaultInstance(), connection, null, registry);
-        SimpleRootExec exec = new SimpleRootExec(ImplCreator.getExec(context, (FragmentRoot) plan.getSortedOperators(false).iterator().next()));
-
-        while(exec.next()){
-            assertEquals(0, exec.getSelectionVector2().getCount());
-        }
-
-        if(context.getFailureCause() != null){
-            throw context.getFailureCause();
-        }
-
-        assertTrue(!context.isFailed());
-
-    }
-
-    @Test
-    public void testFloat4GreaterThan(@Injectable final DrillbitContext bitContext,
-                                      @Injectable UserServer.UserClientConnection connection) throws Throwable{
-
-        new NonStrictExpectations(){{
-            bitContext.getMetrics(); result = new MetricRegistry("test");
-            bitContext.getAllocator(); result = BufferAllocator.getAllocator(c);
-        }};
-
-        PhysicalPlanReader reader = new PhysicalPlanReader(c, c.getMapper(), CoordinationProtos.DrillbitEndpoint.getDefaultInstance());
-        PhysicalPlan plan = reader.readPhysicalPlan(Files.toString(FileUtils.getResourceAsFile("/functions/float4GreaterThan.json"), Charsets.UTF_8));
-        FunctionImplementationRegistry registry = new FunctionImplementationRegistry(c);
-        FragmentContext context = new FragmentContext(bitContext, ExecProtos.FragmentHandle.getDefaultInstance(), connection, null, registry);
-        SimpleRootExec exec = new SimpleRootExec(ImplCreator.getExec(context, (FragmentRoot) plan.getSortedOperators(false).iterator().next()));
-
-        while(exec.next()){
-            assertEquals(0, exec.getSelectionVector2().getCount());
-        }
-
-        if(context.getFailureCause() != null){
-            throw context.getFailureCause();
-        }
-
-        assertTrue(!context.isFailed());
-
-    }
-
-    @Test
-    public void testFloat8GreaterThan(@Injectable final DrillbitContext bitContext,
-                                      @Injectable UserServer.UserClientConnection connection) throws Throwable{
-
-        new NonStrictExpectations(){{
-            bitContext.getMetrics(); result = new MetricRegistry("test");
-            bitContext.getAllocator(); result = BufferAllocator.getAllocator(c);
-        }};
-
-        PhysicalPlanReader reader = new PhysicalPlanReader(c, c.getMapper(), CoordinationProtos.DrillbitEndpoint.getDefaultInstance());
-        PhysicalPlan plan = reader.readPhysicalPlan(Files.toString(FileUtils.getResourceAsFile("/functions/Float8GreaterThan.json"), Charsets.UTF_8));
-        FunctionImplementationRegistry registry = new FunctionImplementationRegistry(c);
-        FragmentContext context = new FragmentContext(bitContext, ExecProtos.FragmentHandle.getDefaultInstance(), connection, null, registry);
-        SimpleRootExec exec = new SimpleRootExec(ImplCreator.getExec(context, (FragmentRoot) plan.getSortedOperators(false).iterator().next()));
-
-        while(exec.next()){
-            assertEquals(0, exec.getSelectionVector2().getCount());
-        }
-
-        if(context.getFailureCause() != null){
-            throw context.getFailureCause();
-        }
-
-        assertTrue(!context.isFailed());
-
-    }
-
-    @Test
-    public void testIntGreaterThanEqual(@Injectable final DrillbitContext bitContext,
-                                        @Injectable UserServer.UserClientConnection connection) throws Throwable{
-
-        new NonStrictExpectations(){{
-            bitContext.getMetrics(); result = new MetricRegistry("test");
-            bitContext.getAllocator(); result = BufferAllocator.getAllocator(c);
-        }};
-
-        PhysicalPlanReader reader = new PhysicalPlanReader(c, c.getMapper(), CoordinationProtos.DrillbitEndpoint.getDefaultInstance());
-        PhysicalPlan plan = reader.readPhysicalPlan(Files.toString(FileUtils.getResourceAsFile("/functions/intGreaterThanEqual.json"), Charsets.UTF_8));
-        FunctionImplementationRegistry registry = new FunctionImplementationRegistry(c);
-        FragmentContext context = new FragmentContext(bitContext, ExecProtos.FragmentHandle.getDefaultInstance(), connection, null, registry);
-        SimpleRootExec exec = new SimpleRootExec(ImplCreator.getExec(context, (FragmentRoot) plan.getSortedOperators(false).iterator().next()));
-
-        while(exec.next()){
-            assertEquals(100, exec.getSelectionVector2().getCount());
-        }
-
-        if(context.getFailureCause() != null){
-            throw context.getFailureCause();
-        }
-
-        assertTrue(!context.isFailed());
-
-    }
-
-    @Test
-    public void testBigIntGreaterThanEqual(@Injectable final DrillbitContext bitContext,
-                                        @Injectable UserServer.UserClientConnection connection) throws Throwable{
-
-        new NonStrictExpectations(){{
-            bitContext.getMetrics(); result = new MetricRegistry("test");
-            bitContext.getAllocator(); result = BufferAllocator.getAllocator(c);
-        }};
-
-        PhysicalPlanReader reader = new PhysicalPlanReader(c, c.getMapper(), CoordinationProtos.DrillbitEndpoint.getDefaultInstance());
-        PhysicalPlan plan = reader.readPhysicalPlan(Files.toString(FileUtils.getResourceAsFile("/functions/longGreaterThanEqual.json"), Charsets.UTF_8));
-        FunctionImplementationRegistry registry = new FunctionImplementationRegistry(c);
-        FragmentContext context = new FragmentContext(bitContext, ExecProtos.FragmentHandle.getDefaultInstance(), connection, null, registry);
-        SimpleRootExec exec = new SimpleRootExec(ImplCreator.getExec(context, (FragmentRoot) plan.getSortedOperators(false).iterator().next()));
-
-        while(exec.next()){
-            assertEquals(100, exec.getSelectionVector2().getCount());
-        }
-
-        if(context.getFailureCause() != null){
-            throw context.getFailureCause();
-        }
-
-        assertTrue(!context.isFailed());
-
-    }
-
-    @Test
-    public void testFloat4GreaterThanEqual(@Injectable final DrillbitContext bitContext,
-                                           @Injectable UserServer.UserClientConnection connection) throws Throwable{
-
-        new NonStrictExpectations(){{
-            bitContext.getMetrics(); result = new MetricRegistry("test");
-            bitContext.getAllocator(); result = BufferAllocator.getAllocator(c);
-        }};
-
-        PhysicalPlanReader reader = new PhysicalPlanReader(c, c.getMapper(), CoordinationProtos.DrillbitEndpoint.getDefaultInstance());
-        PhysicalPlan plan = reader.readPhysicalPlan(Files.toString(FileUtils.getResourceAsFile("/functions/float4GreaterThanEqual.json"), Charsets.UTF_8));
-        FunctionImplementationRegistry registry = new FunctionImplementationRegistry(c);
-        FragmentContext context = new FragmentContext(bitContext, ExecProtos.FragmentHandle.getDefaultInstance(), connection, null, registry);
-        SimpleRootExec exec = new SimpleRootExec(ImplCreator.getExec(context, (FragmentRoot) plan.getSortedOperators(false).iterator().next()));
-
-        while(exec.next()){
-            assertEquals(100, exec.getSelectionVector2().getCount());
-        }
-
-        if(context.getFailureCause() != null){
-            throw context.getFailureCause();
-        }
-
-        assertTrue(!context.isFailed());
-
-    }
-
-    @Test
-    public void testFloat8GreaterThanEqual(@Injectable final DrillbitContext bitContext,
-                                           @Injectable UserServer.UserClientConnection connection) throws Throwable{
-
-        new NonStrictExpectations(){{
-            bitContext.getMetrics(); result = new MetricRegistry("test");
-            bitContext.getAllocator(); result = BufferAllocator.getAllocator(c);
-        }};
-
-        PhysicalPlanReader reader = new PhysicalPlanReader(c, c.getMapper(), CoordinationProtos.DrillbitEndpoint.getDefaultInstance());
-        PhysicalPlan plan = reader.readPhysicalPlan(Files.toString(FileUtils.getResourceAsFile("/functions/float8GreaterThanEqual.json"), Charsets.UTF_8));
-        FunctionImplementationRegistry registry = new FunctionImplementationRegistry(c);
-        FragmentContext context = new FragmentContext(bitContext, ExecProtos.FragmentHandle.getDefaultInstance(), connection, null, registry);
-        SimpleRootExec exec = new SimpleRootExec(ImplCreator.getExec(context, (FragmentRoot) plan.getSortedOperators(false).iterator().next()));
-
-        while(exec.next()){
-            assertEquals(100, exec.getSelectionVector2().getCount());
-        }
-
-        if(context.getFailureCause() != null){
-            throw context.getFailureCause();
-        }
-
-        assertTrue(!context.isFailed());
-
-    }
-
-    @Test
-    public void testIntLessThan(@Injectable final DrillbitContext bitContext,
-                                   @Injectable UserServer.UserClientConnection connection) throws Throwable{
-
-        new NonStrictExpectations(){{
-            bitContext.getMetrics(); result = new MetricRegistry("test");
-            bitContext.getAllocator(); result = BufferAllocator.getAllocator(c);
-        }};
-
-        PhysicalPlanReader reader = new PhysicalPlanReader(c, c.getMapper(), CoordinationProtos.DrillbitEndpoint.getDefaultInstance());
-        PhysicalPlan plan = reader.readPhysicalPlan(Files.toString(FileUtils.getResourceAsFile("/functions/intLessThan.json"), Charsets.UTF_8));
-        FunctionImplementationRegistry registry = new FunctionImplementationRegistry(c);
-        FragmentContext context = new FragmentContext(bitContext, ExecProtos.FragmentHandle.getDefaultInstance(), connection, null, registry);
-        SimpleRootExec exec = new SimpleRootExec(ImplCreator.getExec(context, (FragmentRoot) plan.getSortedOperators(false).iterator().next()));
-
-        while(exec.next()){
-            assertEquals(0, exec.getSelectionVector2().getCount());
-        }
-
-        if(context.getFailureCause() != null){
-            throw context.getFailureCause();
-        }
-
-        assertTrue(!context.isFailed());
-
-    }
-
-    @Test
-    public void testBigIntLessThan(@Injectable final DrillbitContext bitContext,
-                                @Injectable UserServer.UserClientConnection connection) throws Throwable{
-
-        new NonStrictExpectations(){{
-            bitContext.getMetrics(); result = new MetricRegistry("test");
-            bitContext.getAllocator(); result = BufferAllocator.getAllocator(c);
-        }};
-
-        PhysicalPlanReader reader = new PhysicalPlanReader(c, c.getMapper(), CoordinationProtos.DrillbitEndpoint.getDefaultInstance());
-        PhysicalPlan plan = reader.readPhysicalPlan(Files.toString(FileUtils.getResourceAsFile("/functions/longLessThan.json"), Charsets.UTF_8));
-        FunctionImplementationRegistry registry = new FunctionImplementationRegistry(c);
-        FragmentContext context = new FragmentContext(bitContext, ExecProtos.FragmentHandle.getDefaultInstance(), connection, null, registry);
-        SimpleRootExec exec = new SimpleRootExec(ImplCreator.getExec(context, (FragmentRoot) plan.getSortedOperators(false).iterator().next()));
-
-        while(exec.next()){
-            assertEquals(0, exec.getSelectionVector2().getCount());
-        }
-
-        if(context.getFailureCause() != null){
-            throw context.getFailureCause();
-        }
-
-        assertTrue(!context.isFailed());
-
-    }
-
-    @Test
-    public void testFloat4LessThan(@Injectable final DrillbitContext bitContext,
-                                   @Injectable UserServer.UserClientConnection connection) throws Throwable{
-
-        new NonStrictExpectations(){{
-            bitContext.getMetrics(); result = new MetricRegistry("test");
-            bitContext.getAllocator(); result = BufferAllocator.getAllocator(c);
-        }};
-
-        PhysicalPlanReader reader = new PhysicalPlanReader(c, c.getMapper(), CoordinationProtos.DrillbitEndpoint.getDefaultInstance());
-        PhysicalPlan plan = reader.readPhysicalPlan(Files.toString(FileUtils.getResourceAsFile("/functions/float4LessThan.json"), Charsets.UTF_8));
-        FunctionImplementationRegistry registry = new FunctionImplementationRegistry(c);
-        FragmentContext context = new FragmentContext(bitContext, ExecProtos.FragmentHandle.getDefaultInstance(), connection, null, registry);
-        SimpleRootExec exec = new SimpleRootExec(ImplCreator.getExec(context, (FragmentRoot) plan.getSortedOperators(false).iterator().next()));
-
-        while(exec.next()){
-            assertEquals(0, exec.getSelectionVector2().getCount());
-        }
-
-        if(context.getFailureCause() != null){
-            throw context.getFailureCause();
-        }
-
-        assertTrue(!context.isFailed());
-
-    }
-
-    @Test
-    public void testFloat8LessThan(@Injectable final DrillbitContext bitContext,
-                                   @Injectable UserServer.UserClientConnection connection) throws Throwable{
-
-        new NonStrictExpectations(){{
-            bitContext.getMetrics(); result = new MetricRegistry("test");
-            bitContext.getAllocator(); result = BufferAllocator.getAllocator(c);
-        }};
-
-        PhysicalPlanReader reader = new PhysicalPlanReader(c, c.getMapper(), CoordinationProtos.DrillbitEndpoint.getDefaultInstance());
-        PhysicalPlan plan = reader.readPhysicalPlan(Files.toString(FileUtils.getResourceAsFile("/functions/float8LessThan.json"), Charsets.UTF_8));
-        FunctionImplementationRegistry registry = new FunctionImplementationRegistry(c);
-        FragmentContext context = new FragmentContext(bitContext, ExecProtos.FragmentHandle.getDefaultInstance(), connection, null, registry);
-        SimpleRootExec exec = new SimpleRootExec(ImplCreator.getExec(context, (FragmentRoot) plan.getSortedOperators(false).iterator().next()));
-
-        while(exec.next()){
-            assertEquals(0, exec.getSelectionVector2().getCount());
-        }
-
-        if(context.getFailureCause() != null){
-            throw context.getFailureCause();
-        }
-
-        assertTrue(!context.isFailed());
-
-    }
-
-    @Test
-    public void testIntLessThanEqual(@Injectable final DrillbitContext bitContext,
-                                        @Injectable UserServer.UserClientConnection connection) throws Throwable{
-
-        new NonStrictExpectations(){{
-            bitContext.getMetrics(); result = new MetricRegistry("test");
-            bitContext.getAllocator(); result = BufferAllocator.getAllocator(c);
-        }};
-
-        PhysicalPlanReader reader = new PhysicalPlanReader(c, c.getMapper(), CoordinationProtos.DrillbitEndpoint.getDefaultInstance());
-        PhysicalPlan plan = reader.readPhysicalPlan(Files.toString(FileUtils.getResourceAsFile("/functions/intLessThanEqual.json"), Charsets.UTF_8));
-        FunctionImplementationRegistry registry = new FunctionImplementationRegistry(c);
-        FragmentContext context = new FragmentContext(bitContext, ExecProtos.FragmentHandle.getDefaultInstance(), connection, null, registry);
-        SimpleRootExec exec = new SimpleRootExec(ImplCreator.getExec(context, (FragmentRoot) plan.getSortedOperators(false).iterator().next()));
-
-        while(exec.next()){
-            assertEquals(100, exec.getSelectionVector2().getCount());
-        }
-
-        if(context.getFailureCause() != null){
-            throw context.getFailureCause();
-        }
-
-        assertTrue(!context.isFailed());
-
-    }
-
-    @Test
-    public void testBigIntLessThanEqual(@Injectable final DrillbitContext bitContext,
-                                     @Injectable UserServer.UserClientConnection connection) throws Throwable{
-
-        new NonStrictExpectations(){{
-            bitContext.getMetrics(); result = new MetricRegistry("test");
-            bitContext.getAllocator(); result = BufferAllocator.getAllocator(c);
-        }};
-
-        PhysicalPlanReader reader = new PhysicalPlanReader(c, c.getMapper(), CoordinationProtos.DrillbitEndpoint.getDefaultInstance());
-        PhysicalPlan plan = reader.readPhysicalPlan(Files.toString(FileUtils.getResourceAsFile("/functions/longLessThanEqual.json"), Charsets.UTF_8));
-        FunctionImplementationRegistry registry = new FunctionImplementationRegistry(c);
-        FragmentContext context = new FragmentContext(bitContext, ExecProtos.FragmentHandle.getDefaultInstance(), connection, null, registry);
-        SimpleRootExec exec = new SimpleRootExec(ImplCreator.getExec(context, (FragmentRoot) plan.getSortedOperators(false).iterator().next()));
-
-        while(exec.next()){
-            assertEquals(100, exec.getSelectionVector2().getCount());
-        }
-
-        if(context.getFailureCause() != null){
-            throw context.getFailureCause();
-        }
-
-        assertTrue(!context.isFailed());
-
-    }
-
-    @Test
-    public void testFloat4LessThanEqual(@Injectable final DrillbitContext bitContext,
-                                        @Injectable UserServer.UserClientConnection connection) throws Throwable{
-
-        new NonStrictExpectations(){{
-            bitContext.getMetrics(); result = new MetricRegistry("test");
-            bitContext.getAllocator(); result = BufferAllocator.getAllocator(c);
-        }};
-
-        PhysicalPlanReader reader = new PhysicalPlanReader(c, c.getMapper(), CoordinationProtos.DrillbitEndpoint.getDefaultInstance());
-        PhysicalPlan plan = reader.readPhysicalPlan(Files.toString(FileUtils.getResourceAsFile("/functions/float4LessThanEqual.json"), Charsets.UTF_8));
-        FunctionImplementationRegistry registry = new FunctionImplementationRegistry(c);
-        FragmentContext context = new FragmentContext(bitContext, ExecProtos.FragmentHandle.getDefaultInstance(), connection, null, registry);
-        SimpleRootExec exec = new SimpleRootExec(ImplCreator.getExec(context, (FragmentRoot) plan.getSortedOperators(false).iterator().next()));
-
-        while(exec.next()){
-            assertEquals(100, exec.getSelectionVector2().getCount());
-        }
-
-        if(context.getFailureCause() != null){
-            throw context.getFailureCause();
-        }
-
-        assertTrue(!context.isFailed());
-
-    }
-    
-    @Test
-    public void testFloat8LessThanEqual(@Injectable final DrillbitContext bitContext,
-                                        @Injectable UserServer.UserClientConnection connection) throws Throwable{
-
-        new NonStrictExpectations(){{
-            bitContext.getMetrics(); result = new MetricRegistry("test");
-            bitContext.getAllocator(); result = BufferAllocator.getAllocator(c);
-        }};
-
-        PhysicalPlanReader reader = new PhysicalPlanReader(c, c.getMapper(), CoordinationProtos.DrillbitEndpoint.getDefaultInstance());
-        PhysicalPlan plan = reader.readPhysicalPlan(Files.toString(FileUtils.getResourceAsFile("/functions/float8LessThanEqual.json"), Charsets.UTF_8));
-        FunctionImplementationRegistry registry = new FunctionImplementationRegistry(c);
-        FragmentContext context = new FragmentContext(bitContext, ExecProtos.FragmentHandle.getDefaultInstance(), connection, null, registry);
-        SimpleRootExec exec = new SimpleRootExec(ImplCreator.getExec(context, (FragmentRoot) plan.getSortedOperators(false).iterator().next()));
-
-        while(exec.next()){
-            assertEquals(100, exec.getSelectionVector2().getCount());
-        }
-
-        if(context.getFailureCause() != null){
-            throw context.getFailureCause();
-        }
-
-        assertTrue(!context.isFailed());
-
-    }
+  DrillConfig c = DrillConfig.create();
+    String COMPARISON_TEST_PHYSICAL_PLAN = "functions/comparisonTest.json";
+  PhysicalPlanReader reader;
+  FunctionImplementationRegistry registry;
+  FragmentContext context;
+
+  public void runTest(@Injectable final DrillbitContext bitContext,
+                      @Injectable UserServer.UserClientConnection connection, String expression, int expectedResults) throws Throwable {
+
+    new NonStrictExpectations(){{
+      bitContext.getMetrics(); result = new MetricRegistry("test");
+      bitContext.getAllocator(); result = BufferAllocator.getAllocator(c);
+    }};
+
+    String planString = Resources.toString(Resources.getResource(COMPARISON_TEST_PHYSICAL_PLAN), Charsets.UTF_8).replaceAll("EXPRESSION", expression);
+    if(reader == null) reader = new PhysicalPlanReader(c, c.getMapper(), CoordinationProtos.DrillbitEndpoint.getDefaultInstance());
+    if(registry == null) registry = new FunctionImplementationRegistry(c);
+    if(context == null) context = new FragmentContext(bitContext, ExecProtos.FragmentHandle.getDefaultInstance(), connection, null, registry);
+    PhysicalPlan plan = reader.readPhysicalPlan(planString);
+    SimpleRootExec exec = new SimpleRootExec(ImplCreator.getExec(context, (FragmentRoot) plan.getSortedOperators(false).iterator().next()));
+
+    while(exec.next()){
+      assertEquals(String.format("Expression: %s;", expression), expectedResults, exec.getSelectionVector2().getCount());
+    }
+
+    if(context.getFailureCause() != null){
+      throw context.getFailureCause();
+    }
+
+    assertTrue(!context.isFailed());
+  }
+
+  @Test
+  public void testInt(@Injectable final DrillbitContext bitContext,
+                           @Injectable UserServer.UserClientConnection connection) throws Throwable{
+    runTest(bitContext, connection, "intColumn == intColumn", 100);
+    runTest(bitContext, connection, "intColumn != intColumn", 0);
+    runTest(bitContext, connection, "intColumn > intColumn", 0);
+    runTest(bitContext, connection, "intColumn < intColumn", 0);
+    runTest(bitContext, connection, "intColumn >= intColumn", 100);
+    runTest(bitContext, connection, "intColumn <= intColumn", 100);
+  }
+
+  @Test
+  public void testBigInt(@Injectable final DrillbitContext bitContext,
+                      @Injectable UserServer.UserClientConnection connection) throws Throwable{
+    runTest(bitContext, connection, "bigIntColumn == bigIntColumn", 100);
+    runTest(bitContext, connection, "bigIntColumn != bigIntColumn", 0);
+    runTest(bitContext, connection, "bigIntColumn > bigIntColumn", 0);
+    runTest(bitContext, connection, "bigIntColumn < bigIntColumn", 0);
+    runTest(bitContext, connection, "bigIntColumn >= bigIntColumn", 100);
+    runTest(bitContext, connection, "bigIntColumn <= bigIntColumn", 100);
+  }
+
+  @Test
+  public void testFloat4(@Injectable final DrillbitContext bitContext,
+                         @Injectable UserServer.UserClientConnection connection) throws Throwable{
+    runTest(bitContext, connection, "float4Column == float4Column", 100);
+    runTest(bitContext, connection, "float4Column != float4Column", 0);
+    runTest(bitContext, connection, "float4Column > float4Column", 0);
+    runTest(bitContext, connection, "float4Column < float4Column", 0);
+    runTest(bitContext, connection, "float4Column >= float4Column", 100);
+    runTest(bitContext, connection, "float4Column <= float4Column", 100);
+  }
+
+  @Test
+  public void testFloat8(@Injectable final DrillbitContext bitContext,
+                         @Injectable UserServer.UserClientConnection connection) throws Throwable{
+    runTest(bitContext, connection, "float8Column == float8Column", 100);
+    runTest(bitContext, connection, "float8Column != float8Column", 0);
+    runTest(bitContext, connection, "float8Column > float8Column", 0);
+    runTest(bitContext, connection, "float8Column < float8Column", 0);
+    runTest(bitContext, connection, "float8Column >= float8Column", 100);
+    runTest(bitContext, connection, "float8Column <= float8Column", 100);
+  }
+
+  @Test
+  public void testIntNullable(@Injectable final DrillbitContext bitContext,
+                      @Injectable UserServer.UserClientConnection connection) throws Throwable{
+    runTest(bitContext, connection, "intNullableColumn == intNullableColumn", 50);
+    runTest(bitContext, connection, "intNullableColumn != intNullableColumn", 0);
+    runTest(bitContext, connection, "intNullableColumn > intNullableColumn", 0);
+    runTest(bitContext, connection, "intNullableColumn < intNullableColumn", 0);
+    runTest(bitContext, connection, "intNullableColumn >= intNullableColumn", 50);
+    runTest(bitContext, connection, "intNullableColumn <= intNullableColumn", 50);
+  }
+  @Test
+  public void testBigIntNullable(@Injectable final DrillbitContext bitContext,
+                         @Injectable UserServer.UserClientConnection connection) throws Throwable{
+    runTest(bitContext, connection, "bigIntNullableColumn == bigIntNullableColumn", 50);
+    runTest(bitContext, connection, "bigIntNullableColumn != bigIntNullableColumn", 0);
+    runTest(bitContext, connection, "bigIntNullableColumn > bigIntNullableColumn", 0);
+    runTest(bitContext, connection, "bigIntNullableColumn < bigIntNullableColumn", 0);
+    runTest(bitContext, connection, "bigIntNullableColumn >= bigIntNullableColumn", 50);
+    runTest(bitContext, connection, "bigIntNullableColumn <= bigIntNullableColumn", 50);
+  }
 
     @AfterClass
     public static void tearDown() throws Exception{

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/be77d5aa/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestComparisonFunctionsNullable.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestComparisonFunctionsNullable.java b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestComparisonFunctionsNullable.java
deleted file mode 100644
index 362fc2b..0000000
--- a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestComparisonFunctionsNullable.java
+++ /dev/null
@@ -1,354 +0,0 @@
-package org.apache.drill.exec.physical.impl;
-
-import com.google.common.base.Charsets;
-import com.google.common.io.Files;
-import com.yammer.metrics.MetricRegistry;
-
-import mockit.Injectable;
-import mockit.NonStrictExpectations;
-
-import org.apache.drill.common.config.DrillConfig;
-import org.apache.drill.common.util.FileUtils;
-import org.apache.drill.exec.expr.fn.FunctionImplementationRegistry;
-import org.apache.drill.exec.memory.BufferAllocator;
-import org.apache.drill.exec.ops.FragmentContext;
-import org.apache.drill.exec.physical.PhysicalPlan;
-import org.apache.drill.exec.physical.base.FragmentRoot;
-import org.apache.drill.exec.planner.PhysicalPlanReader;
-import org.apache.drill.exec.proto.CoordinationProtos;
-import org.apache.drill.exec.proto.ExecProtos;
-import org.apache.drill.exec.rpc.user.UserServer;
-import org.apache.drill.exec.server.DrillbitContext;
-import org.junit.After;
-import org.junit.AfterClass;
-import org.junit.Test;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-
-public class TestComparisonFunctionsNullable {
-    static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestComparisonFunctionsNullable.class);
-
-    DrillConfig c = DrillConfig.create();
-    @Test
-    public void testNullableIntEqual(@Injectable final DrillbitContext bitContext,
-                        @Injectable UserServer.UserClientConnection connection) throws Throwable{
-
-        new NonStrictExpectations(){{
-            bitContext.getMetrics(); result = new MetricRegistry("test");
-            bitContext.getAllocator(); result = BufferAllocator.getAllocator(c);
-        }};
-
-        PhysicalPlanReader reader = new PhysicalPlanReader(c, c.getMapper(), CoordinationProtos.DrillbitEndpoint.getDefaultInstance());
-        PhysicalPlan plan = reader.readPhysicalPlan(Files.toString(FileUtils.getResourceAsFile("/functions/nullableIntEqual.json"), Charsets.UTF_8));
-        FunctionImplementationRegistry registry = new FunctionImplementationRegistry(c);
-        FragmentContext context = new FragmentContext(bitContext, ExecProtos.FragmentHandle.getDefaultInstance(), connection, null, registry);
-        SimpleRootExec exec = new SimpleRootExec(ImplCreator.getExec(context, (FragmentRoot) plan.getSortedOperators(false).iterator().next()));
-
-        while(exec.next()){
-            assertEquals(50, exec.getSelectionVector2().getCount());
-        }
-
-        if(context.getFailureCause() != null){
-            throw context.getFailureCause();
-        }
-
-        assertTrue(!context.isFailed());
-
-    }
-
-    @Test
-    public void testNullableBigIntEqual(@Injectable final DrillbitContext bitContext,
-                                     @Injectable UserServer.UserClientConnection connection) throws Throwable{
-
-        new NonStrictExpectations(){{
-            bitContext.getMetrics(); result = new MetricRegistry("test");
-            bitContext.getAllocator(); result = BufferAllocator.getAllocator(c);
-        }};
-
-        PhysicalPlanReader reader = new PhysicalPlanReader(c, c.getMapper(), CoordinationProtos.DrillbitEndpoint.getDefaultInstance());
-        PhysicalPlan plan = reader.readPhysicalPlan(Files.toString(FileUtils.getResourceAsFile("/functions/nullableBigIntEqual.json"), Charsets.UTF_8));
-        FunctionImplementationRegistry registry = new FunctionImplementationRegistry(c);
-        FragmentContext context = new FragmentContext(bitContext, ExecProtos.FragmentHandle.getDefaultInstance(), connection, null, registry);
-        SimpleRootExec exec = new SimpleRootExec(ImplCreator.getExec(context, (FragmentRoot) plan.getSortedOperators(false).iterator().next()));
-
-        while(exec.next()){
-            assertEquals(50, exec.getSelectionVector2().getCount());
-        }
-
-        if(context.getFailureCause() != null){
-            throw context.getFailureCause();
-        }
-
-        assertTrue(!context.isFailed());
-
-    }
-
-    @Test
-    public void testNullableIntNotEqual(@Injectable final DrillbitContext bitContext,
-                                           @Injectable UserServer.UserClientConnection connection) throws Throwable{
-
-        new NonStrictExpectations(){{
-            bitContext.getMetrics(); result = new MetricRegistry("test");
-            bitContext.getAllocator(); result = BufferAllocator.getAllocator(c);
-        }};
-
-        PhysicalPlanReader reader = new PhysicalPlanReader(c, c.getMapper(), CoordinationProtos.DrillbitEndpoint.getDefaultInstance());
-        PhysicalPlan plan = reader.readPhysicalPlan(Files.toString(FileUtils.getResourceAsFile("/functions/nullableIntNotEqual.json"), Charsets.UTF_8));
-        FunctionImplementationRegistry registry = new FunctionImplementationRegistry(c);
-        FragmentContext context = new FragmentContext(bitContext, ExecProtos.FragmentHandle.getDefaultInstance(), connection, null, registry);
-        SimpleRootExec exec = new SimpleRootExec(ImplCreator.getExec(context, (FragmentRoot) plan.getSortedOperators(false).iterator().next()));
-
-        while(exec.next()){
-            assertEquals(50, exec.getSelectionVector2().getCount());
-        }
-
-        if(context.getFailureCause() != null){
-            throw context.getFailureCause();
-        }
-
-        assertTrue(!context.isFailed());
-
-    }
-
-    @Test
-    public void testNullableBigIntNotEqual(@Injectable final DrillbitContext bitContext,
-                                        @Injectable UserServer.UserClientConnection connection) throws Throwable{
-
-        new NonStrictExpectations(){{
-            bitContext.getMetrics(); result = new MetricRegistry("test");
-            bitContext.getAllocator(); result = BufferAllocator.getAllocator(c);
-        }};
-
-        PhysicalPlanReader reader = new PhysicalPlanReader(c, c.getMapper(), CoordinationProtos.DrillbitEndpoint.getDefaultInstance());
-        PhysicalPlan plan = reader.readPhysicalPlan(Files.toString(FileUtils.getResourceAsFile("/functions/nullableBigIntNotEqual.json"), Charsets.UTF_8));
-        FunctionImplementationRegistry registry = new FunctionImplementationRegistry(c);
-        FragmentContext context = new FragmentContext(bitContext, ExecProtos.FragmentHandle.getDefaultInstance(), connection, null, registry);
-        SimpleRootExec exec = new SimpleRootExec(ImplCreator.getExec(context, (FragmentRoot) plan.getSortedOperators(false).iterator().next()));
-
-        while(exec.next()){
-            assertEquals(50, exec.getSelectionVector2().getCount());
-        }
-
-        if(context.getFailureCause() != null){
-            throw context.getFailureCause();
-        }
-
-        assertTrue(!context.isFailed());
-
-    }
-
-    @Test
-    public void testNullableIntLessThan(@Injectable final DrillbitContext bitContext,
-                                           @Injectable UserServer.UserClientConnection connection) throws Throwable{
-
-        new NonStrictExpectations(){{
-            bitContext.getMetrics(); result = new MetricRegistry("test");
-            bitContext.getAllocator(); result = BufferAllocator.getAllocator(c);
-        }};
-
-        PhysicalPlanReader reader = new PhysicalPlanReader(c, c.getMapper(), CoordinationProtos.DrillbitEndpoint.getDefaultInstance());
-        PhysicalPlan plan = reader.readPhysicalPlan(Files.toString(FileUtils.getResourceAsFile("/functions/nullableIntLessThan.json"), Charsets.UTF_8));
-        FunctionImplementationRegistry registry = new FunctionImplementationRegistry(c);
-        FragmentContext context = new FragmentContext(bitContext, ExecProtos.FragmentHandle.getDefaultInstance(), connection, null, registry);
-        SimpleRootExec exec = new SimpleRootExec(ImplCreator.getExec(context, (FragmentRoot) plan.getSortedOperators(false).iterator().next()));
-
-        while(exec.next()){
-            assertEquals(0, exec.getSelectionVector2().getCount());
-        }
-
-        if(context.getFailureCause() != null){
-            throw context.getFailureCause();
-        }
-
-        assertTrue(!context.isFailed());
-    }
-
-    @Test
-    public void testNullableBigIntLessThan(@Injectable final DrillbitContext bitContext,
-                                        @Injectable UserServer.UserClientConnection connection) throws Throwable{
-
-        new NonStrictExpectations(){{
-            bitContext.getMetrics(); result = new MetricRegistry("test");
-            bitContext.getAllocator(); result = BufferAllocator.getAllocator(c);
-        }};
-
-        PhysicalPlanReader reader = new PhysicalPlanReader(c, c.getMapper(), CoordinationProtos.DrillbitEndpoint.getDefaultInstance());
-        PhysicalPlan plan = reader.readPhysicalPlan(Files.toString(FileUtils.getResourceAsFile("/functions/nullableBigIntLessThan.json"), Charsets.UTF_8));
-        FunctionImplementationRegistry registry = new FunctionImplementationRegistry(c);
-        FragmentContext context = new FragmentContext(bitContext, ExecProtos.FragmentHandle.getDefaultInstance(), connection, null, registry);
-        SimpleRootExec exec = new SimpleRootExec(ImplCreator.getExec(context, (FragmentRoot) plan.getSortedOperators(false).iterator().next()));
-
-        while(exec.next()){
-            assertEquals(50, exec.getSelectionVector2().getCount());
-        }
-
-        if(context.getFailureCause() != null){
-            throw context.getFailureCause();
-        }
-
-        assertTrue(!context.isFailed());
-    }
-
-    @Test
-    public void testNullableIntLessThanEqual(@Injectable final DrillbitContext bitContext,
-                                           @Injectable UserServer.UserClientConnection connection) throws Throwable{
-
-        new NonStrictExpectations(){{
-            bitContext.getMetrics(); result = new MetricRegistry("test");
-            bitContext.getAllocator(); result = BufferAllocator.getAllocator(c);
-        }};
-
-        PhysicalPlanReader reader = new PhysicalPlanReader(c, c.getMapper(), CoordinationProtos.DrillbitEndpoint.getDefaultInstance());
-        PhysicalPlan plan = reader.readPhysicalPlan(Files.toString(FileUtils.getResourceAsFile("/functions/nullableIntLessThanEqual.json"), Charsets.UTF_8));
-        FunctionImplementationRegistry registry = new FunctionImplementationRegistry(c);
-        FragmentContext context = new FragmentContext(bitContext, ExecProtos.FragmentHandle.getDefaultInstance(), connection, null, registry);
-        SimpleRootExec exec = new SimpleRootExec(ImplCreator.getExec(context, (FragmentRoot) plan.getSortedOperators(false).iterator().next()));
-
-        while(exec.next()){
-            assertEquals(50, exec.getSelectionVector2().getCount());
-        }
-
-        if(context.getFailureCause() != null){
-            throw context.getFailureCause();
-        }
-
-        assertTrue(!context.isFailed());
-    }
-
-    @Test
-    public void testNullableBigIntLessThanEqual(@Injectable final DrillbitContext bitContext,
-                                             @Injectable UserServer.UserClientConnection connection) throws Throwable{
-
-        new NonStrictExpectations(){{
-            bitContext.getMetrics(); result = new MetricRegistry("test");
-            bitContext.getAllocator(); result = BufferAllocator.getAllocator(c);
-        }};
-
-        PhysicalPlanReader reader = new PhysicalPlanReader(c, c.getMapper(), CoordinationProtos.DrillbitEndpoint.getDefaultInstance());
-        PhysicalPlan plan = reader.readPhysicalPlan(Files.toString(FileUtils.getResourceAsFile("/functions/nullableBigIntLessThanEqual.json"), Charsets.UTF_8));
-        FunctionImplementationRegistry registry = new FunctionImplementationRegistry(c);
-        FragmentContext context = new FragmentContext(bitContext, ExecProtos.FragmentHandle.getDefaultInstance(), connection, null, registry);
-        SimpleRootExec exec = new SimpleRootExec(ImplCreator.getExec(context, (FragmentRoot) plan.getSortedOperators(false).iterator().next()));
-
-        while(exec.next()){
-            assertEquals(50, exec.getSelectionVector2().getCount());
-        }
-
-        if(context.getFailureCause() != null){
-            throw context.getFailureCause();
-        }
-
-        assertTrue(!context.isFailed());
-    }
-
-    @Test
-    public void testNullableIntGreaterThan(@Injectable final DrillbitContext bitContext,
-                                                @Injectable UserServer.UserClientConnection connection) throws Throwable{
-
-        new NonStrictExpectations(){{
-            bitContext.getMetrics(); result = new MetricRegistry("test");
-            bitContext.getAllocator(); result = BufferAllocator.getAllocator(c);
-        }};
-
-        PhysicalPlanReader reader = new PhysicalPlanReader(c, c.getMapper(), CoordinationProtos.DrillbitEndpoint.getDefaultInstance());
-        PhysicalPlan plan = reader.readPhysicalPlan(Files.toString(FileUtils.getResourceAsFile("/functions/nullableIntGreaterThan.json"), Charsets.UTF_8));
-        FunctionImplementationRegistry registry = new FunctionImplementationRegistry(c);
-        FragmentContext context = new FragmentContext(bitContext, ExecProtos.FragmentHandle.getDefaultInstance(), connection, null, registry);
-        SimpleRootExec exec = new SimpleRootExec(ImplCreator.getExec(context, (FragmentRoot) plan.getSortedOperators(false).iterator().next()));
-
-        while(exec.next()){
-            assertEquals(0, exec.getSelectionVector2().getCount());
-        }
-
-        if(context.getFailureCause() != null){
-            throw context.getFailureCause();
-        }
-
-        assertTrue(!context.isFailed());
-    }
-
-    @Test
-    public void testNullableBigIntGreaterThan(@Injectable final DrillbitContext bitContext,
-                                           @Injectable UserServer.UserClientConnection connection) throws Throwable{
-
-        new NonStrictExpectations(){{
-            bitContext.getMetrics(); result = new MetricRegistry("test");
-            bitContext.getAllocator(); result = BufferAllocator.getAllocator(c);
-        }};
-
-        PhysicalPlanReader reader = new PhysicalPlanReader(c, c.getMapper(), CoordinationProtos.DrillbitEndpoint.getDefaultInstance());
-        PhysicalPlan plan = reader.readPhysicalPlan(Files.toString(FileUtils.getResourceAsFile("/functions/nullableBigIntGreaterThan.json"), Charsets.UTF_8));
-        FunctionImplementationRegistry registry = new FunctionImplementationRegistry(c);
-        FragmentContext context = new FragmentContext(bitContext, ExecProtos.FragmentHandle.getDefaultInstance(), connection, null, registry);
-        SimpleRootExec exec = new SimpleRootExec(ImplCreator.getExec(context, (FragmentRoot) plan.getSortedOperators(false).iterator().next()));
-
-        while(exec.next()){
-            assertEquals(0, exec.getSelectionVector2().getCount());
-        }
-
-        if(context.getFailureCause() != null){
-            throw context.getFailureCause();
-        }
-
-        assertTrue(!context.isFailed());
-    }
-
-    @Test
-    public void testNullableIntGreaterThanEqual(@Injectable final DrillbitContext bitContext,
-                                              @Injectable UserServer.UserClientConnection connection) throws Throwable{
-
-        new NonStrictExpectations(){{
-            bitContext.getMetrics(); result = new MetricRegistry("test");
-            bitContext.getAllocator(); result = BufferAllocator.getAllocator(c);
-        }};
-
-        PhysicalPlanReader reader = new PhysicalPlanReader(c, c.getMapper(), CoordinationProtos.DrillbitEndpoint.getDefaultInstance());
-        PhysicalPlan plan = reader.readPhysicalPlan(Files.toString(FileUtils.getResourceAsFile("/functions/nullableIntGreaterThanEqual.json"), Charsets.UTF_8));
-        FunctionImplementationRegistry registry = new FunctionImplementationRegistry(c);
-        FragmentContext context = new FragmentContext(bitContext, ExecProtos.FragmentHandle.getDefaultInstance(), connection, null, registry);
-        SimpleRootExec exec = new SimpleRootExec(ImplCreator.getExec(context, (FragmentRoot) plan.getSortedOperators(false).iterator().next()));
-
-        while(exec.next()){
-            assertEquals(50, exec.getSelectionVector2().getCount());
-        }
-
-        if(context.getFailureCause() != null){
-            throw context.getFailureCause();
-        }
-
-        assertTrue(!context.isFailed());
-    }
-
-    @Test
-    public void testNullableBigIntGreaterThanEqual(@Injectable final DrillbitContext bitContext,
-                                                @Injectable UserServer.UserClientConnection connection) throws Throwable{
-
-        new NonStrictExpectations(){{
-            bitContext.getMetrics(); result = new MetricRegistry("test");
-            bitContext.getAllocator(); result = BufferAllocator.getAllocator(c);
-        }};
-
-        PhysicalPlanReader reader = new PhysicalPlanReader(c, c.getMapper(), CoordinationProtos.DrillbitEndpoint.getDefaultInstance());
-        PhysicalPlan plan = reader.readPhysicalPlan(Files.toString(FileUtils.getResourceAsFile("/functions/nullableBigIntGreaterThanEqual.json"), Charsets.UTF_8));
-        FunctionImplementationRegistry registry = new FunctionImplementationRegistry(c);
-        FragmentContext context = new FragmentContext(bitContext, ExecProtos.FragmentHandle.getDefaultInstance(), connection, null, registry);
-        SimpleRootExec exec = new SimpleRootExec(ImplCreator.getExec(context, (FragmentRoot) plan.getSortedOperators(false).iterator().next()));
-
-        while(exec.next()){
-            assertEquals(50, exec.getSelectionVector2().getCount());
-        }
-
-        if(context.getFailureCause() != null){
-            throw context.getFailureCause();
-        }
-
-        assertTrue(!context.isFailed());
-    }
-
-    @AfterClass
-    public static void tearDown() throws Exception{
-      // pause to get logger to catch up.
-      Thread.sleep(1000);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/be77d5aa/sandbox/prototype/exec/java-exec/src/test/resources/functions/comparisonTest.json
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/resources/functions/comparisonTest.json b/sandbox/prototype/exec/java-exec/src/test/resources/functions/comparisonTest.json
new file mode 100644
index 0000000..eac6e68
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/test/resources/functions/comparisonTest.json
@@ -0,0 +1,39 @@
+{
+    head:{
+        type:"APACHE_DRILL_PHYSICAL",
+        version:"1",
+        generator:{
+            type:"manual"
+        }
+    },
+	graph:[
+        {
+            @id:1,
+            pop:"mock-sub-scan",
+            url: "http://apache.org",
+            entries:[
+            	{records: 100, types: [
+            	  {name: "intColumn", type: "INT", mode: "REQUIRED"},
+            	  {name: "bigIntColumn", type: "BIGINT", mode: "REQUIRED"},
+            	  {name: "float4Column", type: "FLOAT4", mode: "REQUIRED"},
+            	  {name: "float8Column", type: "FLOAT8", mode: "REQUIRED"},
+            	  {name: "intNullableColumn", type: "INT", mode: "OPTIONAL"},
+            	  {name: "bigIntNullableColumn", type: "BIGINT", mode: "OPTIONAL"},
+            	  {name: "float4NullableColumn", type: "FLOAT4", mode: "OPTIONAL"},
+            	  {name: "float8NullableColumn", type: "FLOAT8", mode: "OPTIONAL"}
+            	]}
+            ]
+        },
+        {
+            @id:2,
+            child: 1,
+            pop:"filter",
+            expr: "EXPRESSION"
+        },
+        {
+            @id: 3,
+            child: 2,
+            pop: "screen"
+        }
+    ]
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/be77d5aa/sandbox/prototype/exec/java-exec/src/test/resources/functions/float4Equal.json
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/resources/functions/float4Equal.json b/sandbox/prototype/exec/java-exec/src/test/resources/functions/float4Equal.json
deleted file mode 100644
index 7ebe3dd..0000000
--- a/sandbox/prototype/exec/java-exec/src/test/resources/functions/float4Equal.json
+++ /dev/null
@@ -1,35 +0,0 @@
-{
-    head:{
-        type:"APACHE_DRILL_PHYSICAL",
-        version:"1",
-        generator:{
-            type:"manual"
-        }
-    },
-	graph:[
-        {
-            @id:1,
-            pop:"mock-sub-scan",
-            url: "http://apache.org",
-            entries:[
-            	{records: 100, types: [
-            	  {name: "blue", type: "FLOAT4", mode: "REQUIRED"},
-            	  {name: "red", type: "BIGINT", mode: "REQUIRED"},
-            	  {name: "yellow", type: "FLOAT4", mode: "REQUIRED"},
-            	  {name: "green", type: "INT", mode: "REQUIRED"}
-            	]}
-            ]
-        },
-        {
-            @id:2,
-            child: 1,
-            pop:"filter",
-            expr: "blue == blue"
-        },
-        {
-            @id: 3,
-            child: 2,
-            pop: "screen"
-        }
-    ]
-}
\ No newline at end of file

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

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/be77d5aa/sandbox/prototype/exec/java-exec/src/test/resources/functions/float4GreaterThanEqual.json
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/resources/functions/float4GreaterThanEqual.json b/sandbox/prototype/exec/java-exec/src/test/resources/functions/float4GreaterThanEqual.json
deleted file mode 100644
index 51f0a13..0000000
--- a/sandbox/prototype/exec/java-exec/src/test/resources/functions/float4GreaterThanEqual.json
+++ /dev/null
@@ -1,35 +0,0 @@
-{
-    head:{
-        type:"APACHE_DRILL_PHYSICAL",
-        version:"1",
-        generator:{
-            type:"manual"
-        }
-    },
-	graph:[
-        {
-            @id:1,
-            pop:"mock-sub-scan",
-            url: "http://apache.org",
-            entries:[
-            	{records: 100, types: [
-            	  {name: "blue", type: "FLOAT4", mode: "REQUIRED"},
-            	  {name: "red", type: "BIGINT", mode: "REQUIRED"},
-            	  {name: "yellow", type: "FLOAT4", mode: "REQUIRED"},
-            	  {name: "green", type: "INT", mode: "REQUIRED"}
-            	]}
-            ]
-        },
-        {
-            @id:2,
-            child: 1,
-            pop:"filter",
-            expr: "blue >= blue"
-        },
-        {
-            @id: 3,
-            child: 2,
-            pop: "screen"
-        }
-    ]
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/be77d5aa/sandbox/prototype/exec/java-exec/src/test/resources/functions/float4LessThan.json
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/resources/functions/float4LessThan.json b/sandbox/prototype/exec/java-exec/src/test/resources/functions/float4LessThan.json
deleted file mode 100644
index 95ef169..0000000
--- a/sandbox/prototype/exec/java-exec/src/test/resources/functions/float4LessThan.json
+++ /dev/null
@@ -1,35 +0,0 @@
-{
-    head:{
-        type:"APACHE_DRILL_PHYSICAL",
-        version:"1",
-        generator:{
-            type:"manual"
-        }
-    },
-	graph:[
-        {
-            @id:1,
-            pop:"mock-sub-scan",
-            url: "http://apache.org",
-            entries:[
-            	{records: 100, types: [
-            	  {name: "blue", type: "FLOAT4", mode: "REQUIRED"},
-            	  {name: "red", type: "BIGINT", mode: "REQUIRED"},
-            	  {name: "yellow", type: "FLOAT4", mode: "REQUIRED"},
-            	  {name: "green", type: "INT", mode: "REQUIRED"}
-            	]}
-            ]
-        },
-        {
-            @id:2,
-            child: 1,
-            pop:"filter",
-            expr: "blue < blue"
-        },
-        {
-            @id: 3,
-            child: 2,
-            pop: "screen"
-        }
-    ]
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/be77d5aa/sandbox/prototype/exec/java-exec/src/test/resources/functions/float4LessThanEqual.json
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/resources/functions/float4LessThanEqual.json b/sandbox/prototype/exec/java-exec/src/test/resources/functions/float4LessThanEqual.json
deleted file mode 100644
index b58d3b6..0000000
--- a/sandbox/prototype/exec/java-exec/src/test/resources/functions/float4LessThanEqual.json
+++ /dev/null
@@ -1,35 +0,0 @@
-{
-    head:{
-        type:"APACHE_DRILL_PHYSICAL",
-        version:"1",
-        generator:{
-            type:"manual"
-        }
-    },
-	graph:[
-        {
-            @id:1,
-            pop:"mock-sub-scan",
-            url: "http://apache.org",
-            entries:[
-            	{records: 100, types: [
-            	  {name: "blue", type: "FLOAT4", mode: "REQUIRED"},
-            	  {name: "red", type: "BIGINT", mode: "REQUIRED"},
-            	  {name: "yellow", type: "FLOAT4", mode: "REQUIRED"},
-            	  {name: "green", type: "INT", mode: "REQUIRED"}
-            	]}
-            ]
-        },
-        {
-            @id:2,
-            child: 1,
-            pop:"filter",
-            expr: "blue <= blue"
-        },
-        {
-            @id: 3,
-            child: 2,
-            pop: "screen"
-        }
-    ]
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/be77d5aa/sandbox/prototype/exec/java-exec/src/test/resources/functions/float4NotEqual.json
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/resources/functions/float4NotEqual.json b/sandbox/prototype/exec/java-exec/src/test/resources/functions/float4NotEqual.json
deleted file mode 100644
index ba001d4..0000000
--- a/sandbox/prototype/exec/java-exec/src/test/resources/functions/float4NotEqual.json
+++ /dev/null
@@ -1,35 +0,0 @@
-{
-    head:{
-        type:"APACHE_DRILL_PHYSICAL",
-        version:"1",
-        generator:{
-            type:"manual"
-        }
-    },
-	graph:[
-        {
-            @id:1,
-            pop:"mock-sub-scan",
-            url: "http://apache.org",
-            entries:[
-            	{records: 100, types: [
-            	  {name: "blue", type: "FLOAT4", mode: "REQUIRED"},
-            	  {name: "red", type: "BIGINT", mode: "REQUIRED"},
-            	  {name: "yellow", type: "FLOAT4", mode: "REQUIRED"},
-            	  {name: "green", type: "INT", mode: "REQUIRED"}
-            	]}
-            ]
-        },
-        {
-            @id:2,
-            child: 1,
-            pop:"filter",
-            expr: "blue <> blue"
-        },
-        {
-            @id: 3,
-            child: 2,
-            pop: "screen"
-        }
-    ]
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/be77d5aa/sandbox/prototype/exec/java-exec/src/test/resources/functions/float8Equal.json
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/resources/functions/float8Equal.json b/sandbox/prototype/exec/java-exec/src/test/resources/functions/float8Equal.json
deleted file mode 100644
index 9edfd51..0000000
--- a/sandbox/prototype/exec/java-exec/src/test/resources/functions/float8Equal.json
+++ /dev/null
@@ -1,35 +0,0 @@
-{
-    head:{
-        type:"APACHE_DRILL_PHYSICAL",
-        version:"1",
-        generator:{
-            type:"manual"
-        }
-    },
-	graph:[
-        {
-            @id:1,
-            pop:"mock-sub-scan",
-            url: "http://apache.org",
-            entries:[
-            	{records: 100, types: [
-            	  {name: "blue", type: "FLOAT8", mode: "REQUIRED"},
-            	  {name: "red", type: "BIGINT", mode: "REQUIRED"},
-            	  {name: "yellow", type: "FLOAT4", mode: "REQUIRED"},
-            	  {name: "green", type: "INT", mode: "REQUIRED"}
-            	]}
-            ]
-        },
-        {
-            @id:2,
-            child: 1,
-            pop:"filter",
-            expr: "blue == blue"
-        },
-        {
-            @id: 3,
-            child: 2,
-            pop: "screen"
-        }
-    ]
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/be77d5aa/sandbox/prototype/exec/java-exec/src/test/resources/functions/float8GreaterThan.json
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/resources/functions/float8GreaterThan.json b/sandbox/prototype/exec/java-exec/src/test/resources/functions/float8GreaterThan.json
deleted file mode 100644
index 08bb328..0000000
--- a/sandbox/prototype/exec/java-exec/src/test/resources/functions/float8GreaterThan.json
+++ /dev/null
@@ -1,35 +0,0 @@
-{
-    head:{
-        type:"APACHE_DRILL_PHYSICAL",
-        version:"1",
-        generator:{
-            type:"manual"
-        }
-    },
-	graph:[
-        {
-            @id:1,
-            pop:"mock-sub-scan",
-            url: "http://apache.org",
-            entries:[
-            	{records: 100, types: [
-            	  {name: "blue", type: "FLOAT8", mode: "REQUIRED"},
-            	  {name: "red", type: "BIGINT", mode: "REQUIRED"},
-            	  {name: "yellow", type: "FLOAT4", mode: "REQUIRED"},
-            	  {name: "green", type: "INT", mode: "REQUIRED"}
-            	]}
-            ]
-        },
-        {
-            @id:2,
-            child: 1,
-            pop:"filter",
-            expr: "blue > blue"
-        },
-        {
-            @id: 3,
-            child: 2,
-            pop: "screen"
-        }
-    ]
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/be77d5aa/sandbox/prototype/exec/java-exec/src/test/resources/functions/float8GreaterThanEqual.json
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/resources/functions/float8GreaterThanEqual.json b/sandbox/prototype/exec/java-exec/src/test/resources/functions/float8GreaterThanEqual.json
deleted file mode 100644
index 67afb47..0000000
--- a/sandbox/prototype/exec/java-exec/src/test/resources/functions/float8GreaterThanEqual.json
+++ /dev/null
@@ -1,35 +0,0 @@
-{
-    head:{
-        type:"APACHE_DRILL_PHYSICAL",
-        version:"1",
-        generator:{
-            type:"manual"
-        }
-    },
-	graph:[
-        {
-            @id:1,
-            pop:"mock-sub-scan",
-            url: "http://apache.org",
-            entries:[
-            	{records: 100, types: [
-            	  {name: "blue", type: "FLOAT8", mode: "REQUIRED"},
-            	  {name: "red", type: "BIGINT", mode: "REQUIRED"},
-            	  {name: "yellow", type: "FLOAT4", mode: "REQUIRED"},
-            	  {name: "green", type: "INT", mode: "REQUIRED"}
-            	]}
-            ]
-        },
-        {
-            @id:2,
-            child: 1,
-            pop:"filter",
-            expr: "blue >= blue"
-        },
-        {
-            @id: 3,
-            child: 2,
-            pop: "screen"
-        }
-    ]
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/be77d5aa/sandbox/prototype/exec/java-exec/src/test/resources/functions/float8LessThan.json
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/resources/functions/float8LessThan.json b/sandbox/prototype/exec/java-exec/src/test/resources/functions/float8LessThan.json
deleted file mode 100644
index af485b2..0000000
--- a/sandbox/prototype/exec/java-exec/src/test/resources/functions/float8LessThan.json
+++ /dev/null
@@ -1,35 +0,0 @@
-{
-    head:{
-        type:"APACHE_DRILL_PHYSICAL",
-        version:"1",
-        generator:{
-            type:"manual"
-        }
-    },
-	graph:[
-        {
-            @id:1,
-            pop:"mock-sub-scan",
-            url: "http://apache.org",
-            entries:[
-            	{records: 100, types: [
-            	  {name: "blue", type: "FLOAT8", mode: "REQUIRED"},
-            	  {name: "red", type: "BIGINT", mode: "REQUIRED"},
-            	  {name: "yellow", type: "FLOAT4", mode: "REQUIRED"},
-            	  {name: "green", type: "INT", mode: "REQUIRED"}
-            	]}
-            ]
-        },
-        {
-            @id:2,
-            child: 1,
-            pop:"filter",
-            expr: "blue < blue"
-        },
-        {
-            @id: 3,
-            child: 2,
-            pop: "screen"
-        }
-    ]
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/be77d5aa/sandbox/prototype/exec/java-exec/src/test/resources/functions/float8LessThanEqual.json
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/resources/functions/float8LessThanEqual.json b/sandbox/prototype/exec/java-exec/src/test/resources/functions/float8LessThanEqual.json
deleted file mode 100644
index 6f5a21e..0000000
--- a/sandbox/prototype/exec/java-exec/src/test/resources/functions/float8LessThanEqual.json
+++ /dev/null
@@ -1,35 +0,0 @@
-{
-    head:{
-        type:"APACHE_DRILL_PHYSICAL",
-        version:"1",
-        generator:{
-            type:"manual"
-        }
-    },
-	graph:[
-        {
-            @id:1,
-            pop:"mock-sub-scan",
-            url: "http://apache.org",
-            entries:[
-            	{records: 100, types: [
-            	  {name: "blue", type: "FLOAT8", mode: "REQUIRED"},
-            	  {name: "red", type: "BIGINT", mode: "REQUIRED"},
-            	  {name: "yellow", type: "FLOAT4", mode: "REQUIRED"},
-            	  {name: "green", type: "INT", mode: "REQUIRED"}
-            	]}
-            ]
-        },
-        {
-            @id:2,
-            child: 1,
-            pop:"filter",
-            expr: "blue <= blue"
-        },
-        {
-            @id: 3,
-            child: 2,
-            pop: "screen"
-        }
-    ]
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/be77d5aa/sandbox/prototype/exec/java-exec/src/test/resources/functions/float8NotEqual.json
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/resources/functions/float8NotEqual.json b/sandbox/prototype/exec/java-exec/src/test/resources/functions/float8NotEqual.json
deleted file mode 100644
index cf3d053..0000000
--- a/sandbox/prototype/exec/java-exec/src/test/resources/functions/float8NotEqual.json
+++ /dev/null
@@ -1,35 +0,0 @@
-{
-    head:{
-        type:"APACHE_DRILL_PHYSICAL",
-        version:"1",
-        generator:{
-            type:"manual"
-        }
-    },
-	graph:[
-        {
-            @id:1,
-            pop:"mock-sub-scan",
-            url: "http://apache.org",
-            entries:[
-            	{records: 100, types: [
-            	  {name: "blue", type: "FLOAT8", mode: "REQUIRED"},
-            	  {name: "red", type: "BIGINT", mode: "REQUIRED"},
-            	  {name: "yellow", type: "FLOAT4", mode: "REQUIRED"},
-            	  {name: "green", type: "INT", mode: "REQUIRED"}
-            	]}
-            ]
-        },
-        {
-            @id:2,
-            child: 1,
-            pop:"filter",
-            expr: "blue <> blue"
-        },
-        {
-            @id: 3,
-            child: 2,
-            pop: "screen"
-        }
-    ]
-}
\ No newline at end of file

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

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/be77d5aa/sandbox/prototype/exec/java-exec/src/test/resources/functions/intGreaterThan.json
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/resources/functions/intGreaterThan.json b/sandbox/prototype/exec/java-exec/src/test/resources/functions/intGreaterThan.json
deleted file mode 100644
index 9e21f4b..0000000
--- a/sandbox/prototype/exec/java-exec/src/test/resources/functions/intGreaterThan.json
+++ /dev/null
@@ -1,35 +0,0 @@
-{
-    head:{
-        type:"APACHE_DRILL_PHYSICAL",
-        version:"1",
-        generator:{
-            type:"manual"
-        }
-    },
-	graph:[
-        {
-            @id:1,
-            pop:"mock-sub-scan",
-            url: "http://apache.org",
-            entries:[
-            	{records: 100, types: [
-            	  {name: "blue", type: "INT", mode: "REQUIRED"},
-            	  {name: "red", type: "BIGINT", mode: "REQUIRED"},
-            	  {name: "yellow", type: "FLOAT8", mode: "REQUIRED"},
-            	  {name: "green", type: "INT", mode: "REQUIRED"}
-            	]}
-            ]
-        },
-        {
-            @id:2,
-            child: 1,
-            pop:"filter",
-            expr: "blue > blue"
-        },
-        {
-            @id: 3,
-            child: 2,
-            pop: "screen"
-        }
-    ]
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/be77d5aa/sandbox/prototype/exec/java-exec/src/test/resources/functions/intGreaterThanEqual.json
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/resources/functions/intGreaterThanEqual.json b/sandbox/prototype/exec/java-exec/src/test/resources/functions/intGreaterThanEqual.json
deleted file mode 100644
index 9692f82..0000000
--- a/sandbox/prototype/exec/java-exec/src/test/resources/functions/intGreaterThanEqual.json
+++ /dev/null
@@ -1,35 +0,0 @@
-{
-    head:{
-        type:"APACHE_DRILL_PHYSICAL",
-        version:"1",
-        generator:{
-            type:"manual"
-        }
-    },
-	graph:[
-        {
-            @id:1,
-            pop:"mock-sub-scan",
-            url: "http://apache.org",
-            entries:[
-            	{records: 100, types: [
-            	  {name: "blue", type: "INT", mode: "REQUIRED"},
-            	  {name: "red", type: "BIGINT", mode: "REQUIRED"},
-            	  {name: "yellow", type: "FLOAT8", mode: "REQUIRED"},
-            	  {name: "green", type: "INT", mode: "REQUIRED"}
-            	]}
-            ]
-        },
-        {
-            @id:2,
-            child: 1,
-            pop:"filter",
-            expr: "blue >= blue"
-        },
-        {
-            @id: 3,
-            child: 2,
-            pop: "screen"
-        }
-    ]
-}
\ No newline at end of file

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

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/be77d5aa/sandbox/prototype/exec/java-exec/src/test/resources/functions/intLessThanEqual.json
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/resources/functions/intLessThanEqual.json b/sandbox/prototype/exec/java-exec/src/test/resources/functions/intLessThanEqual.json
deleted file mode 100644
index 834cf7e..0000000
--- a/sandbox/prototype/exec/java-exec/src/test/resources/functions/intLessThanEqual.json
+++ /dev/null
@@ -1,35 +0,0 @@
-{
-    head:{
-        type:"APACHE_DRILL_PHYSICAL",
-        version:"1",
-        generator:{
-            type:"manual"
-        }
-    },
-	graph:[
-        {
-            @id:1,
-            pop:"mock-sub-scan",
-            url: "http://apache.org",
-            entries:[
-            	{records: 100, types: [
-            	  {name: "blue", type: "INT", mode: "REQUIRED"},
-            	  {name: "red", type: "BIGINT", mode: "REQUIRED"},
-            	  {name: "yellow", type: "FLOAT8", mode: "REQUIRED"},
-            	  {name: "green", type: "INT", mode: "REQUIRED"}
-            	]}
-            ]
-        },
-        {
-            @id:2,
-            child: 1,
-            pop:"filter",
-            expr: "blue <= blue"
-        },
-        {
-            @id: 3,
-            child: 2,
-            pop: "screen"
-        }
-    ]
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/be77d5aa/sandbox/prototype/exec/java-exec/src/test/resources/functions/intNotEqual.json
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/resources/functions/intNotEqual.json b/sandbox/prototype/exec/java-exec/src/test/resources/functions/intNotEqual.json
deleted file mode 100644
index ceb2913..0000000
--- a/sandbox/prototype/exec/java-exec/src/test/resources/functions/intNotEqual.json
+++ /dev/null
@@ -1,35 +0,0 @@
-{
-    head:{
-        type:"APACHE_DRILL_PHYSICAL",
-        version:"1",
-        generator:{
-            type:"manual"
-        }
-    },
-	graph:[
-        {
-            @id:1,
-            pop:"mock-sub-scan",
-            url: "http://apache.org",
-            entries:[
-            	{records: 100, types: [
-            	  {name: "blue", type: "INT", mode: "REQUIRED"},
-            	  {name: "red", type: "BIGINT", mode: "REQUIRED"},
-            	  {name: "yellow", type: "FLOAT8", mode: "REQUIRED"},
-            	  {name: "green", type: "INT", mode: "REQUIRED"}
-            	]}
-            ]
-        },
-        {
-            @id:2,
-            child: 1,
-            pop:"filter",
-            expr: "blue <> blue"
-        },
-        {
-            @id: 3,
-            child: 2,
-            pop: "screen"
-        }
-    ]
-}
\ No newline at end of file


[25/27] git commit: optimized generated code for partition sender

Posted by ja...@apache.org.
optimized generated code for partition sender


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

Branch: refs/heads/master
Commit: 8ffc674b7ce699aa6905166bf2060bc4e48e45c3
Parents: 4a10ea1
Author: Ben Becker <be...@gmail.com>
Authored: Sat Aug 10 01:20:53 2013 -0700
Committer: Jacques Nadeau <ja...@apache.org>
Committed: Thu Aug 15 18:31:31 2013 -0700

----------------------------------------------------------------------
 .../partitionsender/OutgoingRecordBatch.java    |  52 ++++-----
 .../PartitionSenderRootExec.java                | 115 ++++++++++++-------
 .../exec/work/RemoteFragmentRunnerListener.java |  11 +-
 3 files changed, 107 insertions(+), 71 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/8ffc674b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/partitionsender/OutgoingRecordBatch.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/partitionsender/OutgoingRecordBatch.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/partitionsender/OutgoingRecordBatch.java
index 4ab598c..b40ce4c 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/partitionsender/OutgoingRecordBatch.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/partitionsender/OutgoingRecordBatch.java
@@ -19,11 +19,10 @@
 package org.apache.drill.exec.physical.impl.partitionsender;
 
 import java.util.Iterator;
-import java.util.List;
 
 import com.google.common.base.Preconditions;
-import com.google.common.collect.Lists;
 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.HashPartitionSender;
 import org.apache.drill.exec.proto.ExecProtos;
@@ -52,7 +51,6 @@ public class OutgoingRecordBatch implements RecordBatch {
   private RecordBatch incoming;
   private FragmentContext context;
   private BatchSchema outSchema;
-  private List<ValueVector> valueVectors;
   private VectorContainer vectorContainer;
   private int recordCount;
   private int recordCapacity;
@@ -65,29 +63,25 @@ public class OutgoingRecordBatch implements RecordBatch {
     initializeBatch();
   }
 
-  public OutgoingRecordBatch() {  }
-
-  public void init(HashPartitionSender operator, BitTunnel tunnel, RecordBatch incoming, FragmentContext context) {
-    this.incoming = incoming;
-    this.context = context;
-    this.operator = operator;
-    this.tunnel = tunnel;
-    resetBatch();
-  }
-
   public void flushIfNecessary() {
-    if (recordCount == recordCapacity - 1) flush();
+    if (recordCount == recordCapacity) logger.debug("Flush is necesary:  Count is " + recordCount + ", capacity is " + recordCapacity);
+    try {
+      if (recordCount == recordCapacity) flush();
+    } catch (SchemaChangeException e) {
+      // TODO:
+      logger.error("Unable to flush outgoing record batch: " + e);
+    }
   }
 
   public void incRecordCount() {
     ++recordCount;
   }
   
-  public void flush() {
+  public void flush() throws SchemaChangeException {
     if (recordCount == 0) {
       logger.warn("Attempted to flush an empty record batch");
-      return;
     }
+    logger.debug("Flushing record batch.  count is: " + recordCount + ", capacity is " + recordCapacity);
     final ExecProtos.FragmentHandle handle = context.getHandle();
     FragmentWritableBatch writableBatch = new FragmentWritableBatch(isLast,
                                                                     handle.getQueryId(),
@@ -96,23 +90,26 @@ public class OutgoingRecordBatch implements RecordBatch {
                                                                     operator.getOppositeMajorFragmentId(),
                                                                     0,
                                                                     getWritableBatch());
-    tunnel.sendRecordBatch(statusHandler, context, writableBatch);
+     tunnel.sendRecordBatch(statusHandler, context, writableBatch);
 
     // reset values and reallocate the buffer for each value vector.  NOTE: the value vector is directly
     // referenced by generated code and must not be replaced.
     recordCount = 0;
     for (VectorWrapper v : vectorContainer) {
-      getAllocator(TypeHelper.getNewVector(v.getField(), context.getAllocator()),
-                   v.getValueVector()).alloc(recordCapacity);
+      logger.debug("Reallocating vv to capacity " + recordCapacity + " after flush. " + v.getValueVector());
+      getAllocator(v.getValueVector(),
+                   TypeHelper.getNewVector(v.getField(), context.getAllocator())).alloc(recordCapacity);
     }
+    if (!ok) { throw new SchemaChangeException("Flush ended NOT OK!"); }
   }
 
+
   /**
    * Create a new output schema and allocate space for value vectors based on the incoming record batch.
    */
   public void initializeBatch() {
+    isLast = false;
     recordCapacity = incoming.getRecordCount();
-    valueVectors = Lists.newArrayList();
     vectorContainer = new VectorContainer();
 
     SchemaBuilder bldr = BatchSchema.newBuilder().setSelectionVectorMode(BatchSchema.SelectionVectorMode.NONE);
@@ -122,12 +119,13 @@ public class OutgoingRecordBatch implements RecordBatch {
       bldr.addField(v.getField());
 
       // allocate a new value vector
-      vectorContainer.add(v.getValueVector());
       ValueVector outgoingVector = TypeHelper.getNewVector(v.getField(), context.getAllocator());
-      getAllocator(outgoingVector, v.getValueVector()).alloc(recordCapacity);
-      valueVectors.add(outgoingVector);
+      getAllocator(v.getValueVector(), outgoingVector).alloc(recordCapacity);
+      vectorContainer.add(outgoingVector);
+      logger.debug("Reallocating to cap " + recordCapacity + " because of newly init'd vector : " + v.getValueVector());
     }
     outSchema = bldr.build();
+    logger.debug("Initialized OutgoingRecordBatch.  RecordCount: " + recordCount + ", cap: " + recordCapacity + " Schema: " + outSchema);
   }
 
   /**
@@ -135,13 +133,11 @@ public class OutgoingRecordBatch implements RecordBatch {
    * on the incoming record batch.
    */
   public void resetBatch() {
+    isLast = false;
     recordCount = 0;
     recordCapacity = 0;
-    if (valueVectors != null) {
-      for(ValueVector v : valueVectors){
-        v.close();
-      }
-    }
+    for (VectorWrapper v : vectorContainer)
+      v.getValueVector().clear();
     initializeBatch();
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/8ffc674b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/partitionsender/PartitionSenderRootExec.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/partitionsender/PartitionSenderRootExec.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/partitionsender/PartitionSenderRootExec.java
index 313ddf3..aa25c96 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/partitionsender/PartitionSenderRootExec.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/partitionsender/PartitionSenderRootExec.java
@@ -18,7 +18,6 @@
 
 package org.apache.drill.exec.physical.impl.partitionsender;
 
-import com.beust.jcommander.internal.Lists;
 import com.sun.codemodel.*;
 import org.apache.drill.common.expression.*;
 import org.apache.drill.exec.exception.ClassTransformationException;
@@ -32,9 +31,10 @@ import org.apache.drill.exec.proto.CoordinationProtos;
 import org.apache.drill.exec.record.RecordBatch;
 import org.apache.drill.exec.record.TypedFieldId;
 import org.apache.drill.exec.record.VectorWrapper;
+import org.apache.drill.exec.vector.TypeHelper;
+import org.apache.drill.exec.vector.ValueVector;
 
 import java.io.IOException;
-import java.util.List;
 
 class PartitionSenderRootExec implements RootExec {
 
@@ -56,10 +56,10 @@ class PartitionSenderRootExec implements RootExec {
     this.outgoing = new OutgoingRecordBatch[operator.getDestinations().size()];
     int fieldId = 0;
     for (CoordinationProtos.DrillbitEndpoint endpoint : operator.getDestinations())
-      outgoing[fieldId] = new OutgoingRecordBatch(operator,
-                             context.getCommunicator().getTunnel(endpoint),
-                             incoming,
-                             context);
+      outgoing[fieldId++] = new OutgoingRecordBatch(operator,
+                                                    context.getCommunicator().getTunnel(endpoint),
+                                                    incoming,
+                                                    context);
     try {
       createPartitioner();
     } catch (SchemaChangeException e) {
@@ -87,19 +87,25 @@ class PartitionSenderRootExec implements RootExec {
           partitioner.partitionBatch(incoming);
 
         // send all pending batches
-        flushOutgoingBatches(true, false);
+        try {
+          flushOutgoingBatches(true, false);
+        } catch (SchemaChangeException e) {
+          incoming.kill();
+          logger.error("Error while creating partitioning sender or flushing outgoing batches", e);
+          context.fail(e);
+          return false;
+        }
         return false;
 
       case OK_NEW_SCHEMA:
-        // send all existing batches
-        flushOutgoingBatches(false, true);
-        // update OutgoingRecordBatch's schema and value vectors
         try {
+          // send all existing batches
+          flushOutgoingBatches(false, true);
+          // update OutgoingRecordBatch's schema and generate partitioning code
           createPartitioner();
-          partitioner.setup(context, incoming, outgoing);
         } catch (SchemaChangeException e) {
           incoming.kill();
-          logger.error("Failed to create partitioning sender during query ", e);
+          logger.error("Error while creating partitioning sender or flushing outgoing batches", e);
           context.fail(e);
           return false;
         }
@@ -133,54 +139,85 @@ class PartitionSenderRootExec implements RootExec {
     }
 
     // generate code to copy from an incoming value vector to the destination partition's outgoing value vector
-    int fieldId = 0;
     JExpression inIndex = JExpr.direct("inIndex");
     JExpression outIndex = JExpr.direct("outIndex");
+    JType outgoingVectorArrayType = cg.getModel().ref(ValueVector.class).array().array();
+    JType outgoingBatchArrayType = cg.getModel().ref(OutgoingRecordBatch.class).array();
     cg.rotateBlock();
 
-    // declare array of record batches for each partition
+    // declare and assign the array of outgoing record batches
     JVar outgoingBatches = cg.clazz.field(JMod.NONE,
-                                          cg.getModel().ref(OutgoingRecordBatch.class).array(),
+                                          outgoingBatchArrayType,
                                           "outgoingBatches");
-
     cg.getSetupBlock().assign(outgoingBatches, JExpr.direct("outgoing"));
 
-    // declare incoming value vectors
-    List<JVar> incomingVVs = Lists.newArrayList();
-    for (VectorWrapper<?> vvIn : incoming)
-      incomingVVs.add(cg.declareVectorValueSetupAndMember("incoming", new TypedFieldId(vvIn.getField().getType(),
-                                                                                       fieldId++,
-                                                                                       vvIn.isHyper())));
+    // declare a two-dimensional array of value vectors; batch is first dimension, ValueVector is the second
+    JVar outgoingVectors = cg.clazz.field(JMod.NONE,
+                                          outgoingVectorArrayType,
+                                          "outgoingVectors");
 
+    // create 2d array and build initialization list.  For example:
+    //     outgoingVectors = new ValueVector[][] { 
+    //                              new ValueVector[] {vv1, vv2},
+    //                              new ValueVector[] {vv3, vv4}
+    //                       });
+    JArray outgoingVectorInit = JExpr.newArray(cg.getModel().ref(ValueVector.class).array());
+
+    int fieldId = 0;
     int batchId = 0;
-    fieldId = 0;
-    // generate switch statement for each destination batch
-    JSwitch switchStatement = cg.getBlock()._switch(outIndex);
     for (OutgoingRecordBatch batch : outgoing) {
 
-      // generate case statement for this batch
-      JBlock caseBlock = switchStatement._case(JExpr.lit(batchId)).body();
-
+      JArray outgoingVectorInitBatch = JExpr.newArray(cg.getModel().ref(ValueVector.class));
       for (VectorWrapper<?> vv : batch) {
-        // declare outgoing value vector and a corresponding counter
+        // declare outgoing value vector and assign it to the array
         JVar outVV = cg.declareVectorValueSetupAndMember("outgoing[" + batchId + "]",
                                                          new TypedFieldId(vv.getField().getType(),
                                                                           fieldId,
                                                                           false));
-
-        caseBlock.add(outVV.invoke("copyFrom")
-                              .arg(inIndex)
-                              .arg(JExpr.direct("outgoingBatches[" + batchId + "]").invoke("getRecordCount"))
-                              .arg(incomingVVs.get(fieldId)));
+        // add vv to initialization list (e.g. { vv1, vv2, vv3 } )
+        outgoingVectorInitBatch.add(outVV);
         ++fieldId;
       }
-      caseBlock.add(JExpr.direct("outgoingBatches[" + batchId + "]").invoke("incRecordCount"));
-      caseBlock.add(JExpr.direct("outgoingBatches[" + batchId + "]").invoke("flushIfNecessary"));
-      fieldId = 0;
-      caseBlock._break();
+
+      // add VV array to initialization list (e.g. new ValueVector[] { ... })
+      outgoingVectorInit.add(outgoingVectorInitBatch);
       ++batchId;
+      fieldId = 0;
     }
 
+    // generate outgoing value vector 2d array initialization list.
+    cg.getSetupBlock().assign(outgoingVectors, outgoingVectorInit);
+
+    for (VectorWrapper<?> vvIn : incoming) {
+      // declare incoming value vectors
+      JVar incomingVV = cg.declareVectorValueSetupAndMember("incoming", new TypedFieldId(vvIn.getField().getType(),
+                                                                                         fieldId,
+                                                                                         vvIn.isHyper()));
+
+      // generate the copyFrom() invocation with explicit cast to the appropriate type
+      Class<?> vvType = TypeHelper.getValueVectorClass(vvIn.getField().getType().getMinorType(),
+                                                       vvIn.getField().getType().getMode());
+      JClass vvClass = cg.getModel().ref(vvType);
+      // the following block generates calls to copyFrom(); e.g.:
+      // ((IntVector) outgoingVectors[outIndex][0]).copyFrom(inIndex,
+      //                                                     outgoingBatches[outIndex].getRecordCount(),
+      //                                                     vv1);
+      cg.getBlock().add(
+        ((JExpression) JExpr.cast(vvClass,
+              ((JExpression)
+                     outgoingVectors
+                       .component(outIndex))
+                       .component(JExpr.lit(fieldId))))
+                       .invoke("copyFrom")
+                       .arg(inIndex)
+                       .arg(((JExpression) outgoingBatches.component(outIndex)).invoke("getRecordCount"))
+                       .arg(incomingVV));
+
+      // generate the OutgoingRecordBatch helper invocations
+      cg.getBlock().add(((JExpression) outgoingBatches.component(outIndex)).invoke("incRecordCount"));
+      cg.getBlock().add(((JExpression) outgoingBatches.component(outIndex)).invoke("flushIfNecessary"));
+      ++fieldId;
+    }
     try {
       // compile and setup generated code
       partitioner = context.getImplementationClassMultipleOutput(cg);
@@ -199,7 +236,7 @@ class PartitionSenderRootExec implements RootExec {
    * @param isLastBatch    true if this is the last incoming batch
    * @param schemaChanged  true if the schema has changed
    */
-  public void flushOutgoingBatches(boolean isLastBatch, boolean schemaChanged) {
+  public void flushOutgoingBatches(boolean isLastBatch, boolean schemaChanged) throws SchemaChangeException {
     for (OutgoingRecordBatch batch : outgoing) {
       logger.debug("Attempting to flush all outgoing batches");
       if (isLastBatch)

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/8ffc674b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/RemoteFragmentRunnerListener.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/RemoteFragmentRunnerListener.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/RemoteFragmentRunnerListener.java
index 4ecbd0e..48d7f5d 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/RemoteFragmentRunnerListener.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/RemoteFragmentRunnerListener.java
@@ -20,17 +20,20 @@ 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 RemoteFragmentRunnerListener extends AbstractFragmentRunnerListener{
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(RemoteFragmentRunnerListener.class);
+public class RemotingFragmentRunnerListener extends AbstractFragmentRunnerListener{
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(RemotingFragmentRunnerListener.class);
   
   private final BitTunnel tunnel;
 
-  public RemoteFragmentRunnerListener(FragmentContext context, BitTunnel tunnel) {
+  public RemotingFragmentRunnerListener(FragmentContext context, BitTunnel tunnel) {
     super(context);
     this.tunnel = tunnel;
   }
@@ -38,7 +41,7 @@ public class RemoteFragmentRunnerListener extends AbstractFragmentRunnerListener
   
   @Override
   protected void statusChange(FragmentHandle handle, FragmentStatus status) {
-    logger.debug("Sending remote status message. {}", status);
+    logger.debug("Sending status change message message to remote node: " + status);
     tunnel.sendFragmentStatus(status);
   }
   


[04/27] git commit: Fix late type binding for json record reader

Posted by ja...@apache.org.
Fix late type binding for json record reader


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

Branch: refs/heads/master
Commit: b1e48b32e3bc5e240a01d75f83ac5d2be4b2e7ae
Parents: a15f5b1
Author: Timothy Chen <tn...@gmail.com>
Authored: Sun Aug 11 11:55:24 2013 -0700
Committer: Timothy Chen <tn...@gmail.com>
Committed: Sun Aug 11 11:55:24 2013 -0700

----------------------------------------------------------------------
 .../org/apache/drill/common/types/Types.java    |  27 ++++-
 .../org/apache/drill/exec/schema/Field.java     | 111 +++++++++++--------
 .../exec/schema/json/jackson/JacksonHelper.java |   1 +
 .../drill/exec/store/JSONRecordReader.java      |  49 +++++---
 .../drill/exec/store/JSONRecordReaderTest.java  |  41 +++++--
 .../src/test/resources/scan_json_test_5.json    |  33 +++---
 6 files changed, 170 insertions(+), 92 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/b1e48b32/sandbox/prototype/common/src/main/java/org/apache/drill/common/types/Types.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/types/Types.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/types/Types.java
index e81bc89..f07f726 100644
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/types/Types.java
+++ b/sandbox/prototype/common/src/main/java/org/apache/drill/common/types/Types.java
@@ -4,6 +4,8 @@ import org.apache.drill.common.types.TypeProtos.DataMode;
 import org.apache.drill.common.types.TypeProtos.MajorType;
 import org.apache.drill.common.types.TypeProtos.MinorType;
 
+import static org.apache.drill.common.types.TypeProtos.DataMode.REPEATED;
+
 public class Types {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(Types.class);
   
@@ -16,7 +18,7 @@ public class Types {
   }
   
   public static boolean isNumericType(MajorType type){
-    if(type.getMode() == DataMode.REPEATED) return false;
+    if(type.getMode() == REPEATED) return false;
     
     switch(type.getMinorType()){
     case BIGINT:
@@ -40,7 +42,7 @@ public class Types {
   }
   
   public static boolean usesHolderForGet(MajorType type){
-    if(type.getMode() == DataMode.REPEATED) return true;
+    if(type.getMode() == REPEATED) return true;
     switch(type.getMinorType()){
     case BIGINT:
     case DECIMAL4:
@@ -76,7 +78,7 @@ public class Types {
   
   
   public static boolean isStringScalarType(MajorType type){
-    if(type.getMode() == DataMode.REPEATED) return false;
+    if(type.getMode() == REPEATED) return false;
     switch(type.getMinorType()){
     case FIXEDCHAR:
     case FIXED16CHAR:
@@ -89,7 +91,7 @@ public class Types {
   }
   
   public static boolean isBytesScalarType(MajorType type){
-    if(type.getMode() == DataMode.REPEATED) return false;
+    if(type.getMode() == REPEATED) return false;
     switch(type.getMinorType()){
     case FIXEDBINARY:
     case VARBINARY:
@@ -100,7 +102,7 @@ public class Types {
   }
   
   public static Comparability getComparability(MajorType type){
-    if(type.getMode() == DataMode.REPEATED) return Comparability.NONE;
+    if(type.getMode() == REPEATED) return Comparability.NONE;
     if(type.getMinorType() == MinorType.LATE) return Comparability.UNKNOWN;
     
     switch(type.getMinorType()){
@@ -144,12 +146,25 @@ public class Types {
   }
   
   public static MajorType repeated(MinorType type){
-    return MajorType.newBuilder().setMode(DataMode.REPEATED).setMinorType(type).build();
+    return MajorType.newBuilder().setMode(REPEATED).setMinorType(type).build();
   }
   
   public static MajorType optional(MinorType type){
     return MajorType.newBuilder().setMode(DataMode.OPTIONAL).setMinorType(type).build();
   }
+
+  public static MajorType overrideMinorType(MajorType originalMajorType, MinorType overrideMinorType) {
+    switch(originalMajorType.getMode()) {
+      case REPEATED:
+        return repeated(overrideMinorType);
+      case OPTIONAL:
+        return optional(overrideMinorType);
+      case REQUIRED:
+        return required(overrideMinorType);
+      default:
+        throw new UnsupportedOperationException();
+    }
+  }
   
   
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/b1e48b32/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/schema/Field.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/schema/Field.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/schema/Field.java
index 85bbdf3..080be92 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/schema/Field.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/schema/Field.java
@@ -20,6 +20,7 @@ package org.apache.drill.exec.schema;
 
 import org.apache.drill.common.expression.ExpressionPosition;
 import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.common.types.TypeProtos;
 import org.apache.drill.common.types.TypeProtos.MajorType;
 import org.apache.drill.exec.record.MaterializedField;
 
@@ -27,64 +28,80 @@ import com.google.common.base.Objects;
 import com.google.common.base.Strings;
 
 public abstract class Field {
-    final MajorType fieldType;
-    final String prefixFieldName;
-    RecordSchema schema;
-    RecordSchema parentSchema;
-    boolean read;
-
-    public Field(RecordSchema parentSchema, MajorType type, String prefixFieldName) {
-        fieldType = type;
-        this.prefixFieldName = prefixFieldName;
-        this.parentSchema = parentSchema;
+  final String prefixFieldName;
+  MajorType fieldType;
+  RecordSchema schema;
+  RecordSchema parentSchema;
+  boolean read;
+
+  public Field(RecordSchema parentSchema, MajorType type, String prefixFieldName) {
+    fieldType = type;
+    this.prefixFieldName = prefixFieldName;
+    this.parentSchema = parentSchema;
+  }
+
+  public MaterializedField getAsMaterializedField() {
+    return MaterializedField.create(new SchemaPath(getFieldName(), ExpressionPosition.UNKNOWN), fieldType);
+  }
+
+  public abstract String getFieldName();
+
+  public String getFullFieldName() {
+    String fieldName = getFieldName();
+    if(Strings.isNullOrEmpty(prefixFieldName)) {
+      return fieldName;
+    } else if(Strings.isNullOrEmpty(fieldName)) {
+      return prefixFieldName;
+    } else {
+      return prefixFieldName + "." + getFieldName();
     }
+  }
 
-    public MaterializedField getAsMaterializedField(){
-      return MaterializedField.create(new SchemaPath(getFieldName(), ExpressionPosition.UNKNOWN), fieldType);
-    }
-    
-    public abstract String getFieldName();
+  public void setRead(boolean read) {
+    this.read = read;
+  }
 
-    public String getFullFieldName() {
-        return Strings.isNullOrEmpty(prefixFieldName) ? getFieldName() : prefixFieldName + "." + getFieldName();
-    }
+  protected abstract Objects.ToStringHelper addAttributesToHelper(Objects.ToStringHelper helper);
 
-    public void setRead(boolean read) {
-        this.read = read;
-    }
+  Objects.ToStringHelper getAttributesStringHelper() {
+    return Objects.toStringHelper(this).add("type", fieldType)
+        .add("fullFieldName", getFullFieldName())
+        .add("schema", schema == null ? null : schema.toSchemaString()).omitNullValues();
+  }
 
-    protected abstract Objects.ToStringHelper addAttributesToHelper(Objects.ToStringHelper helper);
+  @Override
+  public String toString() {
+    return addAttributesToHelper(getAttributesStringHelper()).toString();
+  }
 
-    Objects.ToStringHelper getAttributesStringHelper() {
-        return Objects.toStringHelper(this).add("type", fieldType)
-                .add("fullFieldName", getFullFieldName())
-                .add("schema", schema == null ? null : schema.toSchemaString()).omitNullValues();
-    }
+  public RecordSchema getAssignedSchema() {
+    return schema;
+  }
 
-    @Override
-    public String toString() {
-        return addAttributesToHelper(getAttributesStringHelper()).toString();
+  public void assignSchemaIfNull(RecordSchema newSchema) {
+    if (!hasSchema()) {
+      schema = newSchema;
     }
+  }
 
-    public RecordSchema getAssignedSchema() {
-        return schema;
-    }
+  public boolean isRead() {
+    return read;
+  }
 
-    public void assignSchemaIfNull(RecordSchema newSchema) {
-        if (!hasSchema()) {
-            schema = newSchema;
-        }
-    }
+  public boolean hasSchema() {
+    return schema != null;
+  }
 
-    public boolean isRead() {
-        return read;
-    }
+  public MajorType getFieldType() {
+    return fieldType;
+  }
 
-    public boolean hasSchema() {
-        return schema != null;
-    }
+  public void setFieldType(MajorType fieldType) {
+    this.fieldType = fieldType;
+  }
 
-    public MajorType getFieldType() {
-        return fieldType;
-    }
+  @Override
+  public int hashCode() {
+    return getFullFieldName().hashCode();
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/b1e48b32/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/schema/json/jackson/JacksonHelper.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/schema/json/jackson/JacksonHelper.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/schema/json/jackson/JacksonHelper.java
index d8f0646..22167b1 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/schema/json/jackson/JacksonHelper.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/schema/json/jackson/JacksonHelper.java
@@ -92,6 +92,7 @@ public class JacksonHelper {
       case BIT:
         return parser.getBooleanValue();
       case LATE:
+      case NULL:
         return null;
       default:
         throw new RuntimeException("Unexpected Field type to return value: " + fieldType.toString());

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/b1e48b32/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/JSONRecordReader.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/JSONRecordReader.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/JSONRecordReader.java
index a4887c0..21b8c1b 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/JSONRecordReader.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/JSONRecordReader.java
@@ -16,6 +16,7 @@ import org.apache.drill.common.expression.ExpressionPosition;
 import org.apache.drill.common.expression.SchemaPath;
 import org.apache.drill.common.types.TypeProtos.MajorType;
 import org.apache.drill.common.types.TypeProtos.MinorType;
+import org.apache.drill.common.types.Types;
 import org.apache.drill.exec.exception.SchemaChangeException;
 import org.apache.drill.exec.memory.BufferAllocator;
 import org.apache.drill.exec.ops.FragmentContext;
@@ -42,7 +43,7 @@ public class JSONRecordReader implements RecordReader {
 
   private final String inputPath;
 
-  private final Map<Field, VectorHolder> valueVectorMap;
+  private final Map<String, VectorHolder> valueVectorMap;
 
   private JsonParser parser;
   private SchemaIdGenerator generator;
@@ -181,13 +182,11 @@ public class JSONRecordReader implements RecordReader {
       @Override
       public Field createField(RecordSchema parentSchema, String prefixFieldName, String fieldName, MajorType fieldType, int index) {
         return new NamedField(parentSchema, prefixFieldName, fieldName, fieldType);
-        //return new OrderedField(parentSchema, fieldType, prefixFieldName, index);
       }
 
       @Override
       public RecordSchema createSchema() throws IOException {
         return new ObjectSchema();
-        //return new ListSchema();
       }
     },
     OBJECT(END_OBJECT) {
@@ -287,18 +286,30 @@ public class JSONRecordReader implements RecordReader {
                                int colIndex,
                                int groupCount) throws IOException, SchemaChangeException {
       RecordSchema currentSchema = reader.getCurrentSchema();
-      Field field = currentSchema.getField(fieldName, colIndex);
+      Field field = currentSchema.getField(fieldName == null ? prefixFieldName : fieldName, colIndex);
       boolean isFieldFound = field != null;
       List<Field> removedFields = reader.getRemovedFields();
-      if (!isFieldFound || !field.getFieldType().equals(fieldType)) {
-        if (isFieldFound) {
+      boolean newFieldLateBound = fieldType.getMinorType().equals(MinorType.LATE);
+
+      if (isFieldFound && !field.getFieldType().equals(fieldType)) {
+        boolean existingFieldLateBound = field.getFieldType().getMinorType().equals(MinorType.LATE);
+
+        if (newFieldLateBound && !existingFieldLateBound) {
+          fieldType = Types.overrideMinorType(fieldType, field.getFieldType().getMinorType());
+        } else if (!newFieldLateBound && existingFieldLateBound) {
+          field.setFieldType(Types.overrideMinorType(field.getFieldType(), fieldType.getMinorType()));
+        } else if (!newFieldLateBound && !existingFieldLateBound) {
           if (field.hasSchema()) {
             removeChildFields(removedFields, field);
           }
           removedFields.add(field);
           currentSchema.removeField(field, colIndex);
+
+          isFieldFound = false;
         }
+      }
 
+      if (!isFieldFound) {
         field = createField(
             currentSchema,
             prefixFieldName,
@@ -316,16 +327,19 @@ public class JSONRecordReader implements RecordReader {
       VectorHolder holder = getOrCreateVectorHolder(reader, field);
       if (readType != null) {
         RecordSchema fieldSchema = field.getAssignedSchema();
-        reader.setCurrentSchema(fieldSchema);
-
         RecordSchema newSchema = readType.createSchema();
-        field.assignSchemaIfNull(newSchema);
 
-        if (fieldSchema == null) reader.setCurrentSchema(newSchema);
-        readType.readRecord(reader, field.getFullFieldName(), rowIndex, groupCount);
+        if (readType != ReadType.ARRAY) {
+          reader.setCurrentSchema(fieldSchema);
+          if (fieldSchema == null) reader.setCurrentSchema(newSchema);
+          readType.readRecord(reader, field.getFullFieldName(), rowIndex, groupCount);
+        } else {
+          readType.readRecord(reader, field.getFullFieldName(), rowIndex, groupCount);
+        }
 
         reader.setCurrentSchema(currentSchema);
-      } else {
+
+      } else if (holder != null && !newFieldLateBound && fieldType.getMinorType() != MinorType.LATE) {
         return addValueToVector(
             rowIndex,
             holder,
@@ -447,22 +461,23 @@ public class JSONRecordReader implements RecordReader {
   }
 
   private VectorHolder getOrCreateVectorHolder(Field field) throws SchemaChangeException {
-    VectorHolder holder = valueVectorMap.get(field);
+    String fullFieldName = field.getFullFieldName();
+    VectorHolder holder = valueVectorMap.get(fullFieldName);
 
     if (holder == null) {
       MajorType type = field.getFieldType();
-      MaterializedField f = MaterializedField.create(new SchemaPath(field.getFullFieldName(), ExpressionPosition.UNKNOWN), type);
-
-      MinorType minorType = f.getType().getMinorType();
+      MinorType minorType = type.getMinorType();
 
       if (minorType.equals(MinorType.MAP) || minorType.equals(MinorType.LATE)) {
         return null;
       }
 
+      MaterializedField f = MaterializedField.create(new SchemaPath(fullFieldName, ExpressionPosition.UNKNOWN), type);
+
       ValueVector v = TypeHelper.getNewVector(f, allocator);
       AllocationHelper.allocate(v, batchSize, 50);
       holder = new VectorHolder(batchSize, v);
-      valueVectorMap.put(field, holder);
+      valueVectorMap.put(fullFieldName, holder);
       outputMutator.addField(v);
       return holder;
     }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/b1e48b32/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/JSONRecordReaderTest.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/JSONRecordReaderTest.java b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/JSONRecordReaderTest.java
index b39ac8a..6b353ae 100644
--- a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/JSONRecordReaderTest.java
+++ b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/JSONRecordReaderTest.java
@@ -70,10 +70,6 @@ public class JSONRecordReaderTest {
     assertEquals(expectedMinorType, def.getMajorType().getMinorType());
     String[] parts = name.split("\\.");
     int expected = parts.length;
-    boolean expectingArray = List.class.isAssignableFrom(value.getClass());
-    if (expectingArray) {
-      expected += 1;
-    }
     assertEquals(expected, def.getNameList().size());
     for(int i = 0; i < parts.length; ++i) {
       assertEquals(parts[i], def.getName(i).getName());
@@ -203,12 +199,12 @@ public class JSONRecordReaderTest {
     assertEquals("c", removedFields.get(0).getName());
     removedFields.clear();
     assertEquals(1, jr.next());
-    assertEquals(8, addFields.size()); // The reappearing of field 'c' is also included
+    assertEquals(7, addFields.size()); // The reappearing of field 'c' is also included
     assertField(addFields.get(0), 0, MinorType.INT, 12345, "test");
     assertField(addFields.get(3), 0, MinorType.BIT, true, "bool");
     assertField(addFields.get(5), 0, MinorType.INT, 6, "d");
-    assertField(addFields.get(6), 0, MinorType.FLOAT4, (float) 5.16, "c");
-    assertField(addFields.get(7), 0, MinorType.VARCHAR, "test3".getBytes(UTF_8), "str2");
+    assertField(addFields.get(2), 0, MinorType.FLOAT4, (float) 5.16, "c");
+    assertField(addFields.get(6), 0, MinorType.VARCHAR, "test3".getBytes(UTF_8), "str2");
     assertEquals(2, removedFields.size());
     Iterables.find(removedFields, new Predicate<MaterializedField>() {
       @Override
@@ -282,4 +278,35 @@ public class JSONRecordReaderTest {
     assertEquals(0, jr.next());
     assertTrue(mutator.getRemovedFields().isEmpty());
   }
+
+  @Test
+  public void testRepeatedMissingFields(@Injectable final FragmentContext context) throws ExecutionSetupException {
+    new Expectations() {
+      {
+        context.getAllocator();
+        returns(new DirectBufferAllocator());
+      }
+    };
+
+    JSONRecordReader jr = new JSONRecordReader(context, getResource("scan_json_test_5.json"));
+
+    MockOutputMutator mutator = new MockOutputMutator();
+    List<ValueVector> addFields = mutator.getAddFields();
+    jr.setup(mutator);
+    assertEquals(9, jr.next());
+    assertEquals(1, addFields.size());
+    assertField(addFields.get(0), 0, MinorType.INT, Arrays.<Integer>asList(), "test");
+    assertField(addFields.get(0), 1, MinorType.INT, Arrays.asList(1, 2, 3), "test");
+    assertField(addFields.get(0), 2, MinorType.INT, Arrays.<Integer>asList(), "test");
+    assertField(addFields.get(0), 3, MinorType.INT, Arrays.<Integer>asList(), "test");
+    assertField(addFields.get(0), 4, MinorType.INT, Arrays.asList(4, 5, 6), "test");
+    assertField(addFields.get(0), 5, MinorType.INT, Arrays.<Integer>asList(), "test");
+    assertField(addFields.get(0), 6, MinorType.INT, Arrays.<Integer>asList(), "test");
+    assertField(addFields.get(0), 7, MinorType.INT, Arrays.asList(7, 8, 9), "test");
+    assertField(addFields.get(0), 8, MinorType.INT, Arrays.<Integer>asList(), "test");
+
+
+    assertEquals(0, jr.next());
+    assertTrue(mutator.getRemovedFields().isEmpty());
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/b1e48b32/sandbox/prototype/exec/java-exec/src/test/resources/scan_json_test_5.json
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/resources/scan_json_test_5.json b/sandbox/prototype/exec/java-exec/src/test/resources/scan_json_test_5.json
index ae1aaf2..4977c60 100644
--- a/sandbox/prototype/exec/java-exec/src/test/resources/scan_json_test_5.json
+++ b/sandbox/prototype/exec/java-exec/src/test/resources/scan_json_test_5.json
@@ -1,21 +1,24 @@
 {
-    "test": 123,
-    "test2": [1,2,3],
-    "a": {
-    	 "b": 1
-    }
+    "test": []
 }
 {
-    "test": 1234,
-    "test3": false,
-    "a": {
-    	 "b": 2
-    }
+    "test": [1,2,3]
+}
+{
+    "test": []
+}
+{
+    "test": null
+}
+{
+    "test": [4,5,6]
+}
+{
+}
+{
+}
+{
+    "test": [7,8,9]
 }
 {
-    "test": 1234,
-    "test2": 1.5,
-    "a": {
-    	 "b": 2
-    }
 }
\ No newline at end of file


[02/27] git commit: Repeated values in JsonRecordReader

Posted by ja...@apache.org.
Repeated values in JsonRecordReader


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

Branch: refs/heads/master
Commit: 38ab96f335537c6bbdb6a4a64b1c6e13755172f6
Parents: 73fad99
Author: Timothy Chen <tn...@gmail.com>
Authored: Sat Aug 3 15:29:35 2013 -0700
Committer: Timothy Chen <tn...@gmail.com>
Committed: Thu Aug 8 23:36:41 2013 -0700

----------------------------------------------------------------------
 .../drill/common/expression/SchemaPath.java     |   2 +-
 .../templates/RepeatedValueVectors.java         |  10 +-
 .../exec/schema/json/jackson/JacksonHelper.java | 116 +++++++++++--------
 .../drill/exec/store/JSONRecordReader.java      |  86 ++++++++------
 .../apache/drill/exec/store/VectorHolder.java   |  20 ++--
 .../drill/exec/vector/AllocationHelper.java     |   2 +-
 .../drill/exec/vector/RepeatedMutator.java      |  23 ----
 .../physical/impl/TestSimpleFragmentRun.java    |  81 ++++++++++++-
 .../drill/exec/store/JSONRecordReaderTest.java  |  56 ++++++++-
 .../src/test/resources/scan_json_test_4.json    |  23 ++--
 10 files changed, 279 insertions(+), 140 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/38ab96f3/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 19d1069..6f1a733 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
@@ -42,7 +42,7 @@ public class SchemaPath extends LogicalExpressionBase {
       ")*$";
 
   // reads well in RegexBuddy
-  private static final String SEGMENT_REGEX = "(?:\n" + "(\\[\\d+\\])\n" + "|\n" + "'?\n"
+  private static final String SEGMENT_REGEX = "(?:\n" + "\\[(\\d+)\\]\n" + "|\n" + "'?\n"
       + "([^\\.\\[\\+\\-\\!\\]\\}]+)  # identifier\n" + "'?\n" + ")\n"
       + "([\\+\\-\\!\\]\\}]?)         # collision type";
   private static final int GROUP_INDEX = 1;

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/38ab96f3/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/RepeatedValueVectors.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/RepeatedValueVectors.java b/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/RepeatedValueVectors.java
index c629a1d..f4a7049 100644
--- a/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/RepeatedValueVectors.java
+++ b/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/RepeatedValueVectors.java
@@ -1,4 +1,4 @@
-
+import org.apache.drill.exec.vector.ValueVector;
 
 <@pp.dropOutputFile />
 <#list types as type>
@@ -249,7 +249,7 @@ import com.google.common.collect.Lists;
     }
   }
   
-  public final class Mutator implements RepeatedMutator {
+  public final class Mutator implements ValueVector.Mutator {
 
     
     private Mutator(){
@@ -264,12 +264,18 @@ import com.google.common.collect.Lists;
      */
     public void add(int index, <#if type.major == "VarLen">byte[]<#elseif (type.width < 4)>int<#else>${minor.javaType!type.javaType}</#if> value) {
       int nextOffset = offsets.getAccessor().get(index+1);
+      if (index > 0 && nextOffset == 0) {
+        nextOffset = offsets.getAccessor().get(index);
+      }
       values.getMutator().set(nextOffset, value);
       offsets.getMutator().set(index+1, nextOffset+1);
     }
 
     public void add(int index, ${minor.class}Holder holder){
       int nextOffset = offsets.getAccessor().get(index+1);
+      if (index > 0 && nextOffset == 0) {
+        nextOffset = offsets.getAccessor().get(index);
+      }
       values.getMutator().set(nextOffset, holder);
       offsets.getMutator().set(index+1, nextOffset+1);
     }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/38ab96f3/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/schema/json/jackson/JacksonHelper.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/schema/json/jackson/JacksonHelper.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/schema/json/jackson/JacksonHelper.java
index 0e2c052..d8f0646 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/schema/json/jackson/JacksonHelper.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/schema/json/jackson/JacksonHelper.java
@@ -18,63 +18,83 @@
 
 package org.apache.drill.exec.schema.json.jackson;
 
-import java.io.IOException;
-
-import org.apache.drill.common.types.Types;
-import org.apache.drill.common.types.TypeProtos.DataMode;
+import com.fasterxml.jackson.core.JsonParser;
+import com.fasterxml.jackson.core.JsonToken;
 import org.apache.drill.common.types.TypeProtos.MajorType;
 import org.apache.drill.common.types.TypeProtos.MinorType;
+import org.apache.drill.common.types.Types;
 
-import com.fasterxml.jackson.core.JsonParser;
-import com.fasterxml.jackson.core.JsonToken;
+import java.io.IOException;
 
 public class JacksonHelper {
 
-    public static final MajorType STRING_TYPE = Types.optional(MinorType.VARCHAR);
-    public static final MajorType BOOLEAN_TYPE = Types.optional(MinorType.BIT);
-    public static final MajorType ARRAY_TYPE = Types.repeated(MinorType.LATE);
-    public static final MajorType MAP_TYPE = Types.repeated(MinorType.MAP);
-    public static final MajorType INT_TYPE = Types.optional(MinorType.INT);
-    public static final MajorType FLOAT_TYPE = Types.optional(MinorType.FLOAT4);
-    public static final MajorType NULL_TYPE = Types.optional(MinorType.LATE);
+  public static final MajorType STRING_TYPE = Types.optional(MinorType.VARCHAR);
+  public static final MajorType REPEATED_STRING_TYPE = Types.repeated(MinorType.VARCHAR);
+  public static final MajorType BOOLEAN_TYPE = Types.optional(MinorType.BIT);
+  public static final MajorType REPEATED_BOOLEAN_TYPE = Types.repeated(MinorType.BIT);
+  public static final MajorType ARRAY_TYPE = Types.repeated(MinorType.LATE);
+  public static final MajorType MAP_TYPE = Types.repeated(MinorType.MAP);
+  public static final MajorType INT_TYPE = Types.optional(MinorType.INT);
+  public static final MajorType REPEATED_INT_TYPE = Types.repeated(MinorType.INT);
+  public static final MajorType FLOAT_TYPE = Types.optional(MinorType.FLOAT4);
+  public static final MajorType REPEATED_FLOAT_TYPE = Types.repeated(MinorType.FLOAT4);
+  public static final MajorType NULL_TYPE = Types.optional(MinorType.LATE);
 
-    public static MajorType getFieldType(JsonToken token) {
-        switch(token) {
-            case VALUE_STRING:
-                return STRING_TYPE;
-            case VALUE_FALSE:
-                return BOOLEAN_TYPE;
-            case VALUE_TRUE:
-                return BOOLEAN_TYPE;
-            case START_ARRAY:
-                return ARRAY_TYPE;
-            case START_OBJECT:
-                return MAP_TYPE;
-            case VALUE_NUMBER_INT:
-                return INT_TYPE;
-            case VALUE_NUMBER_FLOAT:
-                return FLOAT_TYPE;
-            case VALUE_NULL:
-                return NULL_TYPE;
-        }
+  public static MajorType getFieldType(JsonToken token, boolean repeated) {
+    if (repeated) {
+      switch (token) {
+        case VALUE_STRING:
+          return REPEATED_STRING_TYPE;
+        case VALUE_FALSE:
+        case VALUE_TRUE:
+          return REPEATED_BOOLEAN_TYPE;
+        case START_ARRAY:
+          return ARRAY_TYPE;
+        case START_OBJECT:
+          return MAP_TYPE;
+        case VALUE_NUMBER_INT:
+          return REPEATED_INT_TYPE;
+        case VALUE_NUMBER_FLOAT:
+          return REPEATED_FLOAT_TYPE;
+      }
+    } else {
 
-        throw new UnsupportedOperationException("Unsupported Jackson type: " + token);
+      switch (token) {
+        case VALUE_STRING:
+          return STRING_TYPE;
+        case VALUE_FALSE:
+        case VALUE_TRUE:
+          return BOOLEAN_TYPE;
+        case START_ARRAY:
+          return ARRAY_TYPE;
+        case START_OBJECT:
+          return MAP_TYPE;
+        case VALUE_NUMBER_INT:
+          return INT_TYPE;
+        case VALUE_NUMBER_FLOAT:
+          return FLOAT_TYPE;
+        case VALUE_NULL:
+          return NULL_TYPE;
+      }
     }
 
-    public static Object getValueFromFieldType(JsonParser parser, MinorType fieldType) throws IOException {
-        switch (fieldType) {
-            case INT:
-                return parser.getIntValue();
-            case VARCHAR:
-                return parser.getValueAsString();
-            case FLOAT4:
-                return parser.getFloatValue();
-            case BIT:
-                return parser.getBooleanValue();
-            case LATE:
-                return null;
-            default:
-                throw new RuntimeException("Unexpected Field type to return value: " + fieldType.toString());
-        }
+    throw new UnsupportedOperationException("Unsupported Jackson type: " + token + ", Repeated: " + repeated);
+  }
+
+  public static Object getValueFromFieldType(JsonParser parser, MinorType fieldType) throws IOException {
+    switch (fieldType) {
+      case INT:
+        return parser.getIntValue();
+      case VARCHAR:
+        return parser.getValueAsString();
+      case FLOAT4:
+        return parser.getFloatValue();
+      case BIT:
+        return parser.getBooleanValue();
+      case LATE:
+        return null;
+      default:
+        throw new RuntimeException("Unexpected Field type to return value: " + fieldType.toString());
     }
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/38ab96f3/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/JSONRecordReader.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/JSONRecordReader.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/JSONRecordReader.java
index ff7d315..a4887c0 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/JSONRecordReader.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/JSONRecordReader.java
@@ -103,7 +103,7 @@ public class JSONRecordReader implements RecordReader {
     int nextRowIndex = 0;
 
     try {
-      while (ReadType.OBJECT.readRecord(null, this, null, nextRowIndex++, 0)) {
+      while (ReadType.OBJECT.readRecord(this, null, nextRowIndex++, 0)) {
         parser.nextToken(); // Read to START_OBJECT token
 
         if (!parser.hasCurrentToken()) {
@@ -180,12 +180,14 @@ public class JSONRecordReader implements RecordReader {
     ARRAY(END_ARRAY) {
       @Override
       public Field createField(RecordSchema parentSchema, String prefixFieldName, String fieldName, MajorType fieldType, int index) {
-        return new OrderedField(parentSchema, fieldType, prefixFieldName, index);
+        return new NamedField(parentSchema, prefixFieldName, fieldName, fieldType);
+        //return new OrderedField(parentSchema, fieldType, prefixFieldName, index);
       }
 
       @Override
       public RecordSchema createSchema() throws IOException {
-        return new ListSchema();
+        return new ObjectSchema();
+        //return new ListSchema();
       }
     },
     OBJECT(END_OBJECT) {
@@ -215,8 +217,7 @@ public class JSONRecordReader implements RecordReader {
     }
 
     @SuppressWarnings("ConstantConditions")
-    public boolean readRecord(Field parentField,
-                              JSONRecordReader reader,
+    public boolean readRecord(JSONRecordReader reader,
                               String prefixFieldName,
                               int rowIndex,
                               int groupCount) throws IOException, SchemaChangeException {
@@ -232,7 +233,7 @@ public class JSONRecordReader implements RecordReader {
         }
 
         String fieldName = parser.getCurrentName();
-        MajorType fieldType = JacksonHelper.getFieldType(token);
+        MajorType fieldType = JacksonHelper.getFieldType(token, this == ReadType.ARRAY);
         ReadType readType = null;
         switch (token) {
           case START_ARRAY:
@@ -246,17 +247,17 @@ public class JSONRecordReader implements RecordReader {
         }
 
         if (fieldType != null) { // Including nulls
-          isFull = isFull ||
-              !recordData(
-                  parentField,
-                  readType,
-                  reader,
-                  fieldType,
-                  prefixFieldName,
-                  fieldName,
-                  rowIndex,
-                  colIndex,
-                  groupCount);
+          boolean currentFieldFull = !recordData(
+              readType,
+              reader,
+              fieldType,
+              prefixFieldName,
+              fieldName,
+              rowIndex,
+              colIndex,
+              groupCount);
+
+          isFull = isFull || currentFieldFull;
         }
         token = parser.nextToken();
         colIndex += 1;
@@ -277,8 +278,7 @@ public class JSONRecordReader implements RecordReader {
       }
     }
 
-    private boolean recordData(Field parentField,
-                               JSONRecordReader.ReadType readType,
+    private boolean recordData(JSONRecordReader.ReadType readType,
                                JSONRecordReader reader,
                                MajorType fieldType,
                                String prefixFieldName,
@@ -322,18 +322,13 @@ public class JSONRecordReader implements RecordReader {
         field.assignSchemaIfNull(newSchema);
 
         if (fieldSchema == null) reader.setCurrentSchema(newSchema);
-        if(readType == ReadType.ARRAY) {
-          readType.readRecord(field, reader, field.getFullFieldName(), rowIndex, groupCount);
-        } else {
-          readType.readRecord(field, reader, field.getFullFieldName(), rowIndex, groupCount);
-        }
+        readType.readRecord(reader, field.getFullFieldName(), rowIndex, groupCount);
 
         reader.setCurrentSchema(currentSchema);
       } else {
         return addValueToVector(
             rowIndex,
             holder,
-            reader.getAllocator(),
             JacksonHelper.getValueFromFieldType(
                 reader.getParser(),
                 fieldType.getMinorType()
@@ -346,10 +341,10 @@ public class JSONRecordReader implements RecordReader {
       return true;
     }
 
-    private static <T> boolean addValueToVector(int index, VectorHolder holder, BufferAllocator allocator, T val, MinorType minorType, int groupCount) {
+    private static <T> boolean addValueToVector(int index, VectorHolder holder, T val, MinorType minorType, int groupCount) {
       switch (minorType) {
         case INT: {
-          holder.incAndCheckLength(32);
+          holder.incAndCheckLength(32 + 1);
           if (groupCount == 0) {
             if (val != null) {
               NullableIntVector int4 = (NullableIntVector) holder.getValueVector();
@@ -363,13 +358,14 @@ public class JSONRecordReader implements RecordReader {
 
             RepeatedIntVector repeatedInt4 = (RepeatedIntVector) holder.getValueVector();
             RepeatedIntVector.Mutator m = repeatedInt4.getMutator();
+            holder.setGroupCount(index);
             m.add(index, (Integer) val);
           }
 
-          return holder.hasEnoughSpace(32);
+          return holder.hasEnoughSpace(32 + 1);
         }
         case FLOAT4: {
-          holder.incAndCheckLength(32);
+          holder.incAndCheckLength(32 + 1);
           if (groupCount == 0) {
             if (val != null) {
               NullableFloat4Vector float4 = (NullableFloat4Vector) holder.getValueVector();
@@ -383,9 +379,10 @@ public class JSONRecordReader implements RecordReader {
 
             RepeatedFloat4Vector repeatedFloat4 = (RepeatedFloat4Vector) holder.getValueVector();
             RepeatedFloat4Vector.Mutator m = repeatedFloat4.getMutator();
-            m.add(groupCount, (Float) val);
+            holder.setGroupCount(index);
+            m.add(index, (Float) val);
           }
-          return holder.hasEnoughSpace(32);
+          return holder.hasEnoughSpace(32 + 1);
         }
         case VARCHAR: {
           if (val == null) {
@@ -401,16 +398,29 @@ public class JSONRecordReader implements RecordReader {
             } else {
               RepeatedVarCharVector repeatedVarLen4 = (RepeatedVarCharVector) holder.getValueVector();
               RepeatedVarCharVector.Mutator m = repeatedVarLen4.getMutator();
+              holder.setGroupCount(index);
               m.add(index, bytes);
             }
-            return holder.hasEnoughSpace(length);
+            return holder.hasEnoughSpace(length + 4 + 1);
           }
         }
         case BIT: {
-          holder.incAndCheckLength(1);
-          NullableBitVector bit = (NullableBitVector) holder.getValueVector();
-          if (val != null) {
-            bit.getMutator().set(index, (Boolean) val ? 1 : 0);
+          holder.incAndCheckLength(1 + 1);
+          if (groupCount == 0) {
+            if (val != null) {
+              NullableBitVector bit = (NullableBitVector) holder.getValueVector();
+              NullableBitVector.Mutator m = bit.getMutator();
+              m.set(index, (Boolean) val ? 1 : 0);
+            }
+          } else {
+            if (val == null) {
+              throw new UnsupportedOperationException("Nullable repeated boolean is not supported.");
+            }
+
+            RepeatedBitVector repeatedBit = (RepeatedBitVector) holder.getValueVector();
+            RepeatedBitVector.Mutator m = repeatedBit.getMutator();
+            holder.setGroupCount(index);
+            m.add(index, (Boolean) val ? 1 : 0);
           }
           return holder.hasEnoughSpace(1 + 1);
         }
@@ -443,7 +453,9 @@ public class JSONRecordReader implements RecordReader {
       MajorType type = field.getFieldType();
       MaterializedField f = MaterializedField.create(new SchemaPath(field.getFullFieldName(), ExpressionPosition.UNKNOWN), type);
 
-      if (f.getType().getMinorType().equals(MinorType.MAP)) {
+      MinorType minorType = f.getType().getMinorType();
+
+      if (minorType.equals(MinorType.MAP) || minorType.equals(MinorType.LATE)) {
         return null;
       }
 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/38ab96f3/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/VectorHolder.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/VectorHolder.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/VectorHolder.java
index 43d3cd9..2c28082 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/VectorHolder.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/VectorHolder.java
@@ -64,22 +64,18 @@ public class VectorHolder {
 
   public void populateVectorLength() {
     ValueVector.Mutator mutator = vector.getMutator();
-    if(mutator instanceof NonRepeatedMutator) {
-      ((NonRepeatedMutator)mutator).setValueCount(count);
-    } else if(mutator instanceof RepeatedMutator) {
-      ((RepeatedMutator)mutator).setGroupAndValueCount(groupCount, count);
+    if(vector instanceof RepeatedFixedWidthVector || vector instanceof RepeatedVariableWidthVector) {
+      mutator.setValueCount(groupCount);
     } else {
-      throw new UnsupportedOperationException("Mutator not supported: " + mutator.getClass().getName());
+      mutator.setValueCount(count);
     }
   }
 
   public void allocateNew(int valueLength) {
-    if (vector instanceof FixedWidthVector) {
-      ((FixedWidthVector) vector).allocateNew(valueLength);
-    } else if (vector instanceof VariableWidthVector) {
-      ((VariableWidthVector) vector).allocateNew(valueLength * 10, valueLength);
-    } else {
-      throw new UnsupportedOperationException();
-    }
+    AllocationHelper.allocate(vector, valueLength, 10, 5);
+  }
+
+  public void allocateNew(int valueLength, int repeatedPerTop) {
+    AllocationHelper.allocate(vector, valueLength, 10, repeatedPerTop);
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/38ab96f3/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/AllocationHelper.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/AllocationHelper.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/AllocationHelper.java
index 69c17f4..5007dbd 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/AllocationHelper.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/AllocationHelper.java
@@ -10,7 +10,7 @@ public class AllocationHelper {
   public static void allocate(ValueVector v, int valueCount, int bytesPerValue, int repeatedPerTop){
     if(v instanceof FixedWidthVector){
       ((FixedWidthVector) v).allocateNew(valueCount);
-    }else if(v instanceof VariableWidthVector){
+    } else if (v instanceof VariableWidthVector) {
       ((VariableWidthVector) v).allocateNew(valueCount * bytesPerValue, valueCount);
     }else if(v instanceof RepeatedFixedWidthVector){
       ((RepeatedFixedWidthVector) v).allocateNew(valueCount, valueCount * repeatedPerTop);

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

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/38ab96f3/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestSimpleFragmentRun.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestSimpleFragmentRun.java b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestSimpleFragmentRun.java
index 5d4e700..cabe9b3 100644
--- a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestSimpleFragmentRun.java
+++ b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestSimpleFragmentRun.java
@@ -87,7 +87,6 @@ public class TestSimpleFragmentRun extends PopUnitTestBase {
         System.out.println();
       }
 
-
       for (int i = 0; i < batchLoader.getRecordCount(); i++) {
         boolean first = true;
         recordCount++;
@@ -101,13 +100,87 @@ public class TestSimpleFragmentRun extends PopUnitTestBase {
         }
         if(!first) System.out.println();
       }
-    
-  
-
     }
     logger.debug("Received results {}", results);
     assertEquals(recordCount, 200);
     }
   }
 
+
+  @Test
+  public void runJSONScanPopFragment() 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_json_scan_test1.json"), Charsets.UTF_8));
+
+      // look at records
+      RecordBatchLoader batchLoader = new RecordBatchLoader(bit.getContext().getAllocator());
+      int recordCount = 0;
+
+      int expectedBatchCount = 2;
+
+      assertEquals(expectedBatchCount, results.size());
+
+      for (int i = 0; i < results.size(); ++i) {
+        QueryResultBatch batch = results.get(i);
+        if (i == 0) {
+          assertTrue(batch.hasData());
+        } else {
+          assertFalse(batch.hasData());
+          return;
+        }
+
+        assertTrue(batchLoader.load(batch.getHeader().getDef(), batch.getData()));
+        boolean firstColumn = true;
+
+        // print headers.
+        System.out.println("\n\n========NEW SCHEMA=========\n\n");
+        for (VectorWrapper<?> v : batchLoader) {
+
+          if (firstColumn) {
+            firstColumn = false;
+          } else {
+            System.out.print("\t");
+          }
+          System.out.print(v.getField().getName());
+          System.out.print("[");
+          System.out.print(v.getField().getType().getMinorType());
+          System.out.print("]");
+        }
+
+        System.out.println();
+
+
+        for (int r = 0; i < batchLoader.getRecordCount(); r++) {
+          boolean first = true;
+          recordCount++;
+          for (VectorWrapper<?> v : batchLoader) {
+            if (first) {
+              first = false;
+            } else {
+              System.out.print("\t");
+            }
+
+            ValueVector.Accessor accessor = v.getValueVector().getAccessor();
+
+            if (v.getField().getType().getMinorType() == TypeProtos.MinorType.VARCHAR) {
+              System.out.println(new String((byte[]) accessor.getObject(r), UTF_8));
+            } else {
+              System.out.print(accessor.getObject(r));
+            }
+          }
+          if (!first) System.out.println();
+        }
+
+      }
+
+      assertEquals(2, recordCount);
+    }
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/38ab96f3/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/JSONRecordReaderTest.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/JSONRecordReaderTest.java b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/JSONRecordReaderTest.java
index 0ebb529..b39ac8a 100644
--- a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/JSONRecordReaderTest.java
+++ b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/JSONRecordReaderTest.java
@@ -5,6 +5,7 @@ import static org.junit.Assert.assertTrue;
 
 import java.io.IOException;
 import java.nio.charset.Charset;
+import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.List;
 
@@ -68,7 +69,12 @@ public class JSONRecordReaderTest {
     SchemaDefProtos.FieldDef def = metadata.getDef();
     assertEquals(expectedMinorType, def.getMajorType().getMinorType());
     String[] parts = name.split("\\.");
-    assertEquals(parts.length, def.getNameList().size());
+    int expected = parts.length;
+    boolean expectingArray = List.class.isAssignableFrom(value.getClass());
+    if (expectingArray) {
+      expected += 1;
+    }
+    assertEquals(expected, def.getNameList().size());
     for(int i = 0; i < parts.length; ++i) {
       assertEquals(parts[i], def.getName(i).getName());
     }
@@ -78,10 +84,21 @@ public class JSONRecordReaderTest {
     }
 
     T val = (T) valueVector.getAccessor().getObject(index);
-    if (val instanceof byte[]) {
-      assertTrue(Arrays.equals((byte[]) value, (byte[]) val));
+    assertValue(value, val);
+  }
+
+  private void assertValue(Object expected, Object found) {
+    if (found instanceof byte[]) {
+      assertTrue(Arrays.equals((byte[]) expected, (byte[]) found));
+    } else if(found instanceof ArrayList) {
+      List expectedArray = (List) expected;
+      List foundArray = (List) found;
+      assertEquals(expectedArray.size(), foundArray.size());
+      for(int i = 0; i < expectedArray.size(); ++i) {
+        assertValue(expectedArray.get(i), foundArray.get(i));
+      }
     } else {
-      assertEquals(value, val);
+      assertEquals(expected, found);
     }
   }
 
@@ -234,4 +251,35 @@ public class JSONRecordReaderTest {
     assertEquals(0, jr.next());
     assertTrue(mutator.getRemovedFields().isEmpty());
   }
+
+  @Test
+  public void testRepeatedFields(@Injectable final FragmentContext context) throws ExecutionSetupException {
+    new Expectations() {
+      {
+        context.getAllocator();
+        returns(new DirectBufferAllocator());
+      }
+    };
+
+    JSONRecordReader jr = new JSONRecordReader(context, getResource("scan_json_test_4.json"));
+
+    MockOutputMutator mutator = new MockOutputMutator();
+    List<ValueVector> addFields = mutator.getAddFields();
+    jr.setup(mutator);
+    assertEquals(2, jr.next());
+    assertEquals(7, addFields.size());
+    assertField(addFields.get(0), 0, MinorType.INT, 123, "test");
+    assertField(addFields.get(1), 0, MinorType.INT, Arrays.asList(1, 2, 3), "test2");
+    assertField(addFields.get(2), 0, MinorType.INT, Arrays.asList(4, 5, 6), "test3.a");
+    assertField(addFields.get(3), 0, MinorType.INT, Arrays.asList(7, 8, 9), "test3.b");
+    assertField(addFields.get(4), 0, MinorType.INT, Arrays.asList(10, 11, 12), "test3.c.d");
+    assertField(addFields.get(5), 0, MinorType.FLOAT4, Arrays.<Float>asList((float) 1.1, (float) 1.2, (float) 1.3), "testFloat");
+    assertField(addFields.get(6), 0, MinorType.VARCHAR, Arrays.asList("hello".getBytes(UTF_8), "drill".getBytes(UTF_8)), "testStr");
+    assertField(addFields.get(1), 1, MinorType.INT, Arrays.asList(1, 2), "test2");
+    assertField(addFields.get(2), 1, MinorType.INT, Arrays.asList(7, 7, 7, 8), "test3.a");
+    assertField(addFields.get(5), 1, MinorType.FLOAT4, Arrays.<Float>asList((float) 2.2, (float) 2.3,(float) 2.4), "testFloat");
+
+    assertEquals(0, jr.next());
+    assertTrue(mutator.getRemovedFields().isEmpty());
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/38ab96f3/sandbox/prototype/exec/java-exec/src/test/resources/scan_json_test_4.json
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/resources/scan_json_test_4.json b/sandbox/prototype/exec/java-exec/src/test/resources/scan_json_test_4.json
index 0fb3202..fd003ac 100644
--- a/sandbox/prototype/exec/java-exec/src/test/resources/scan_json_test_4.json
+++ b/sandbox/prototype/exec/java-exec/src/test/resources/scan_json_test_4.json
@@ -1,14 +1,21 @@
 {
     "test": 123,
     "test2": [1,2,3],
-    "a": {
-    	 "b": 1
-    }
+    "test3": {
+        "a": [4,5,6],
+        "b": [7,8,9],
+        "c": {
+            "d": [10, 11, 12]
+        }
+    },
+    "testFloat": [1.1, 1.2, 1.3],
+    "testStr": ["hello", "drill"]
 }
 {
-    "test": 1234,
-    "test3": false,
-    "a": {
-    	 "b": 2
-    }
+    "test2": [1,2],
+    "test3": {
+        "a": [7,7,7,8],
+        "b": []
+    },
+    "testFloat": [2.2, 2.3, 2.4]
 }
\ No newline at end of file


[20/27] git commit: fix issue with end of batch

Posted by ja...@apache.org.
fix issue with end of batch


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

Branch: refs/heads/master
Commit: add8c724b9490feba442ae831d9b9c0d6babe413
Parents: 93121cb
Author: Ben Becker <be...@gmail.com>
Authored: Sun Aug 11 15:12:32 2013 -0700
Committer: Jacques Nadeau <ja...@apache.org>
Committed: Thu Aug 15 18:31:31 2013 -0700

----------------------------------------------------------------------
 .../partitionsender/OutgoingRecordBatch.java    | 71 +++++++++++++-------
 .../PartitionSenderRootExec.java                | 13 ++--
 2 files changed, 52 insertions(+), 32 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/add8c724/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/partitionsender/OutgoingRecordBatch.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/partitionsender/OutgoingRecordBatch.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/partitionsender/OutgoingRecordBatch.java
index 6eff778..927cc75 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/partitionsender/OutgoingRecordBatch.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/partitionsender/OutgoingRecordBatch.java
@@ -19,7 +19,9 @@
 package org.apache.drill.exec.physical.impl.partitionsender;
 
 import java.util.Iterator;
+import java.util.List;
 
+import com.beust.jcommander.internal.Lists;
 import com.google.common.base.Preconditions;
 import org.apache.drill.common.expression.SchemaPath;
 import org.apache.drill.exec.exception.SchemaChangeException;
@@ -76,34 +78,58 @@ public class OutgoingRecordBatch implements RecordBatch {
   public void incRecordCount() {
     ++recordCount;
   }
-  
-  public void flush() throws SchemaChangeException {
-    if (recordCount == 0) {
-      // TODO:  recordCount of 0 with isLast causes recordLoader to throw an NPE.  Probably
-      //        need to send notification rather than an actual batch.
-      logger.warn("Attempted to flush an empty record batch" + (isLast ? " (last batch)" : ""));
-      return;
-    }
 
+  /**
+   * Send the record batch to the target node, then reset the value vectors
+   * 
+   * @return true if a flush was needed; otherwise false
+   * @throws SchemaChangeException
+   */
+  public boolean flush() throws SchemaChangeException {
+    logger.error("Creating FragmentWritableBatch.  IsLast? " + (isLast ? " (last batch)" : ""));
     final ExecProtos.FragmentHandle handle = context.getHandle();
-    FragmentWritableBatch writableBatch = new FragmentWritableBatch(isLast,
-                                                                    handle.getQueryId(),
-                                                                    handle.getMajorFragmentId(),
-                                                                    handle.getMinorFragmentId(),
-                                                                    operator.getOppositeMajorFragmentId(),
-                                                                    0,
-                                                                    getWritableBatch());
-    tunnel.sendRecordBatch(statusHandler, context, writableBatch);
-
-    // reset values and reallocate the buffer for each value vector.  NOTE: the value vector is directly
-    // referenced by generated code and must not be replaced.
+
+    if (recordCount != 0) {
+      FragmentWritableBatch writableBatch = new FragmentWritableBatch(isLast,
+                                                                      handle.getQueryId(),
+                                                                      handle.getMajorFragmentId(),
+                                                                      handle.getMinorFragmentId(),
+                                                                      operator.getOppositeMajorFragmentId(),
+                                                                      0,
+                                                                      getWritableBatch());
+      tunnel.sendRecordBatch(statusHandler, context, writableBatch);
+    } else {
+      logger.debug("Flush requested on an empty outgoing record batch" + (isLast ? " (last batch)" : ""));
+
+      if (isLast) {
+
+        // if the last batch is empty, it must not contain any value vectors.
+        vectorContainer = new VectorContainer();
+
+        // send final batch
+        FragmentWritableBatch writableBatch = new FragmentWritableBatch(isLast,
+                                                                        handle.getQueryId(),
+                                                                        handle.getMajorFragmentId(),
+                                                                        handle.getMinorFragmentId(),
+                                                                        operator.getOppositeMajorFragmentId(),
+                                                                        0,
+                                                                        getWritableBatch());
+        tunnel.sendRecordBatch(statusHandler, context, writableBatch);
+        return true;
+
+      }
+    }
+
+    // reset values and reallocate the buffer for each value vector based on the incoming batch.
+    // NOTE: the value vector is directly referenced by generated code; therefore references
+    // must remain valid.
     recordCount = 0;
     for (VectorWrapper v : vectorContainer) {
-      logger.debug("Reallocating vv to capacity " + recordCapacity + " after flush. " + v.getValueVector());
-      getAllocator(v.getValueVector(),
-                   v.getValueVector()).alloc(recordCapacity);
+      logger.debug("Reallocating vv to capacity " + incoming.getRecordCount() + " after flush.");
+      getAllocator(v.getValueVector(), v.getValueVector()).alloc(incoming.getRecordCount());
     }
     if (!ok) { throw new SchemaChangeException("Flush ended NOT OK!"); }
+    return true;
   }
 
 
@@ -141,7 +167,6 @@ public class OutgoingRecordBatch implements RecordBatch {
     recordCapacity = 0;
     for (VectorWrapper v : vectorContainer)
       v.getValueVector().clear();
-    initializeBatch();
   }
 
   public void setIsLast() {

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/add8c724/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/partitionsender/PartitionSenderRootExec.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/partitionsender/PartitionSenderRootExec.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/partitionsender/PartitionSenderRootExec.java
index 293a711..b2ca64e 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/partitionsender/PartitionSenderRootExec.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/partitionsender/PartitionSenderRootExec.java
@@ -80,12 +80,8 @@ class PartitionSenderRootExec implements RootExec {
     RecordBatch.IterOutcome out = incoming.next();
     logger.debug("Partitioner.next(): got next record batch with status {}", out);
     switch(out){
-      case STOP:
       case NONE:
-        // populate outgoing batches
-        if (incoming.getRecordCount() > 0)
-          partitioner.partitionBatch(incoming);
-
+      case STOP:
         try {
           // send any pending batches
           for (OutgoingRecordBatch batch : outgoing) {
@@ -96,10 +92,7 @@ class PartitionSenderRootExec implements RootExec {
           incoming.kill();
           logger.error("Error while creating partitioning sender or flushing outgoing batches", e);
           context.fail(e);
-          return false;
         }
-        context.batchesCompleted.inc(1);
-        context.recordsCompleted.inc(incoming.getRecordCount());
         return false;
 
       case OK_NEW_SCHEMA:
@@ -249,8 +242,10 @@ class PartitionSenderRootExec implements RootExec {
       if (isLastBatch)
         batch.setIsLast();
       batch.flush();
-      if (schemaChanged)
+      if (schemaChanged) {
         batch.resetBatch();
+        batch.initializeBatch();
+      }
     }
   }
 }


[07/27] Initial Parquet commit. Suports INT, LONG, FLOAT, DOUBLE, distributed scheduling.

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/0a2f997f/sandbox/prototype/exec/ref/src/main/java/org/apache/drill/exec/ref/rse/QueueRSE.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/ref/src/main/java/org/apache/drill/exec/ref/rse/QueueRSE.java b/sandbox/prototype/exec/ref/src/main/java/org/apache/drill/exec/ref/rse/QueueRSE.java
index 3bb1648..c870cca 100644
--- a/sandbox/prototype/exec/ref/src/main/java/org/apache/drill/exec/ref/rse/QueueRSE.java
+++ b/sandbox/prototype/exec/ref/src/main/java/org/apache/drill/exec/ref/rse/QueueRSE.java
@@ -54,7 +54,24 @@ public class QueueRSE extends RSEBase {
   
   @JsonTypeName("queue")
   public static class QueueRSEConfig extends StorageEngineConfigBase {
-    
+
+    @Override
+    public boolean equals(Object o) {
+      if (this == o) return true;
+      if (o == null || getClass() != o.getClass()) return false;
+
+      QueueRSEConfig that = (QueueRSEConfig) o;
+
+      if (encoding != that.encoding) return false;
+
+      return true;
+    }
+
+    @Override
+    public int hashCode() {
+      return encoding != null ? encoding.hashCode() : 0;
+    }
+
     public static enum Encoding {JSON, RECORD};
     
     private final Encoding encoding;

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/0a2f997f/sandbox/prototype/sqlparser/src/main/java/org/apache/drill/optiq/DrillScan.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/sqlparser/src/main/java/org/apache/drill/optiq/DrillScan.java b/sandbox/prototype/sqlparser/src/main/java/org/apache/drill/optiq/DrillScan.java
index ab622b1..9f11f8c 100644
--- a/sandbox/prototype/sqlparser/src/main/java/org/apache/drill/optiq/DrillScan.java
+++ b/sandbox/prototype/sqlparser/src/main/java/org/apache/drill/optiq/DrillScan.java
@@ -11,7 +11,7 @@ import com.fasterxml.jackson.databind.JsonNode;
 import com.fasterxml.jackson.databind.node.ObjectNode;
 
 /**
- * Scan of a Drill table.
+ * GroupScan of a Drill table.
  */
 public class DrillScan extends TableAccessRelBase implements DrillRel {
   private final DrillTable drillTable;


[24/27] git commit: fix schema path serialization

Posted by ja...@apache.org.
fix schema path serialization


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

Branch: refs/heads/master
Commit: b32c9edd96553a8947a853d2735277f043d56cee
Parents: 2c8094b
Author: Ben Becker <be...@gmail.com>
Authored: Mon Aug 12 13:30:53 2013 -0700
Committer: Jacques Nadeau <ja...@apache.org>
Committed: Thu Aug 15 18:31:31 2013 -0700

----------------------------------------------------------------------
 .../apache/drill/common/expression/ExpressionStringBuilder.java   | 2 --
 .../src/main/java/org/apache/drill/exec/expr/fn/impl/Hash.java    | 3 +--
 .../exec/java-exec/src/test/resources/sender/hash_exchange.json   | 2 +-
 3 files changed, 2 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/b32c9edd/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/ExpressionStringBuilder.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/ExpressionStringBuilder.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/ExpressionStringBuilder.java
index 09a7d57..bce660d 100644
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/ExpressionStringBuilder.java
+++ b/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/ExpressionStringBuilder.java
@@ -66,9 +66,7 @@ public class ExpressionStringBuilder extends AbstractExprVisitor<Void, StringBui
 
   @Override
   public Void visitSchemaPath(SchemaPath path, StringBuilder sb) throws RuntimeException {
-    sb.append("'");
     sb.append(path.getPath());
-    sb.append("'");
     return null;
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/b32c9edd/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/Hash.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/Hash.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/Hash.java
index 2ffd389..18f7509 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/Hash.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/Hash.java
@@ -18,7 +18,6 @@
 
 package org.apache.drill.exec.expr.fn.impl;
 
-import com.google.common.hash.Hashing;
 import org.apache.drill.common.expression.*;
 import org.apache.drill.exec.expr.DrillFunc;
 import org.apache.drill.exec.expr.annotations.FunctionTemplate;
@@ -40,7 +39,7 @@ public class Hash implements DrillFunc {
 
   public void eval() {
     // TODO: implement hash function for other types
-    out.value = Hashing.murmur3_128().hashLong(in.value).asInt();
+    out.value = com.google.common.hash.Hashing.murmur3_128().hashLong(in.value).asInt();
   }
 
   public static class Provider implements CallProvider{

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/b32c9edd/sandbox/prototype/exec/java-exec/src/test/resources/sender/hash_exchange.json
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/resources/sender/hash_exchange.json b/sandbox/prototype/exec/java-exec/src/test/resources/sender/hash_exchange.json
index 38116e7..78f3394 100644
--- a/sandbox/prototype/exec/java-exec/src/test/resources/sender/hash_exchange.json
+++ b/sandbox/prototype/exec/java-exec/src/test/resources/sender/hash_exchange.json
@@ -28,7 +28,7 @@
             @id: 2,
             child: 1,
             pop: "hash-to-random-exchange",
-            expr: "hash(1)"
+            expr: "hash(red)"
         },
         {
              @id: 3,


[27/27] git commit: Fix SQL parser logback file not found exception

Posted by ja...@apache.org.
Fix SQL parser logback file not found exception


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

Branch: refs/heads/master
Commit: 47985bad03a6210823258cf934bcbbd57cacdde0
Parents: 8cd9cac
Author: Jacques Nadeau <ja...@apache.org>
Authored: Thu Aug 15 18:41:27 2013 -0700
Committer: Jacques Nadeau <ja...@apache.org>
Committed: Thu Aug 15 18:41:27 2013 -0700

----------------------------------------------------------------------
 .../sqlparser/src/test/resources/logback.xml         | 15 ---------------
 1 file changed, 15 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/47985bad/sandbox/prototype/sqlparser/src/test/resources/logback.xml
----------------------------------------------------------------------
diff --git a/sandbox/prototype/sqlparser/src/test/resources/logback.xml b/sandbox/prototype/sqlparser/src/test/resources/logback.xml
index c8de412..3b3289a 100644
--- a/sandbox/prototype/sqlparser/src/test/resources/logback.xml
+++ b/sandbox/prototype/sqlparser/src/test/resources/logback.xml
@@ -15,32 +15,17 @@
       <pattern>%d{HH:mm:ss.SSS} [%thread] %-5level %logger{36} - %msg%n</pattern>
     </encoder>
   </appender>
-
-  <appender name="FILE" class="ch.qos.logback.core.rolling.RollingFileAppender">
-    <file>/logs/test-sqlparser.log</file>
-    <encoder>
-      <pattern>%date %level [%thread] %logger{10} [%file:%line] %msg%n</pattern>
-    </encoder>
-    <rollingPolicy class="ch.qos.logback.core.rolling.TimeBasedRollingPolicy">
-	    <!-- daily rollover -->
-	    <fileNamePattern>/logs/test-sqlparser.%d{yyyy-MM-dd}.log</fileNamePattern>
-	    <!-- keep 30 days' worth of history -->
-	    <maxHistory>30</maxHistory>
-    </rollingPolicy>
-  </appender>
   
   <logger name="org.apache.drill" additivity="false">
     <level value="debug" />
     <appender-ref ref="SOCKET" />
     <appender-ref ref="STDOUT" />
-    <appender-ref ref="FILE" />
   </logger>
 
   <root>
     <level value="error" />
     <appender-ref ref="SOCKET" />
     <appender-ref ref="STDOUT" />
-    <appender-ref ref="FILE" />
   </root>
 
 </configuration>
\ No newline at end of file


[11/27] git commit: Initial Parquet commit. Suports INT, LONG, FLOAT, DOUBLE, distributed scheduling.

Posted by ja...@apache.org.
Initial Parquet commit.  Suports INT, LONG, FLOAT, DOUBLE, distributed scheduling.


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

Branch: refs/heads/master
Commit: 0a2f997ff8d95b816238edc78f8ccf4c5cbbb924
Parents: 0a327ed
Author: Jason Altekruse <al...@gmial.com>
Authored: Thu Aug 8 11:50:47 2013 -0700
Committer: Jacques Nadeau <ja...@apache.org>
Committed: Thu Aug 15 16:57:08 2013 -0700

----------------------------------------------------------------------
 .../org/apache/drill/common/JSONOptions.java    |   5 +-
 .../drill/common/logical/LogicalPlan.java       |   2 +-
 .../common/logical/StorageEngineConfig.java     |   2 +
 .../common/logical/StorageEngineConfigBase.java |   2 +
 .../drill/storage/MockStorageEngineConfig.java  |  18 +-
 sandbox/prototype/exec/java-exec/pom.xml        |  67 ++-
 .../templates/VariableLengthVectors.java        |   6 +-
 .../org/apache/drill/exec/ExecConstants.java    |   1 +
 .../exec/exception/OptimizerException.java      |  17 +-
 .../org/apache/drill/exec/ops/QueryContext.java |   7 +
 .../apache/drill/exec/opt/BasicOptimizer.java   | 205 ++++---
 .../drill/exec/physical/ReadEntryFromHDFS.java  |  54 ++
 .../drill/exec/physical/ReadEntryWithPath.java  |  41 ++
 .../exec/physical/base/AbstractGroupScan.java   |  45 ++
 .../physical/base/AbstractPhysicalVisitor.java  |   9 +-
 .../drill/exec/physical/base/AbstractScan.java  |  84 ---
 .../drill/exec/physical/base/GroupScan.java     |  36 ++
 .../apache/drill/exec/physical/base/Leaf.java   |   2 +-
 .../exec/physical/base/PhysicalOperator.java    |   2 +-
 .../exec/physical/base/PhysicalVisitor.java     |   3 +-
 .../apache/drill/exec/physical/base/Scan.java   |  21 +-
 .../drill/exec/physical/base/SubScan.java       |  23 +
 .../exec/physical/config/MockGroupScanPOP.java  | 221 +++++++
 .../exec/physical/config/MockRecordReader.java  |   4 +-
 .../physical/config/MockScanBatchCreator.java   |   6 +-
 .../drill/exec/physical/config/MockScanPOP.java | 193 ------
 .../exec/physical/config/MockStorageEngine.java |   8 +-
 .../exec/physical/config/MockSubScanPOP.java    | 115 ++++
 .../drill/exec/physical/impl/ImplCreator.java   |  45 +-
 .../drill/exec/physical/impl/OutputMutator.java |   1 +
 .../drill/exec/physical/impl/ScanBatch.java     |   6 +
 .../exec/physical/impl/SingleSenderCreator.java |   3 +
 .../drill/exec/planner/PhysicalPlanReader.java  |  14 +-
 .../planner/fragment/MakeFragmentsVisitor.java  |   9 +-
 .../exec/planner/fragment/Materializer.java     |  17 +-
 .../planner/fragment/SimpleParallelizer.java    |   2 +
 .../exec/planner/fragment/StatsCollector.java   |  19 +-
 .../drill/exec/planner/fragment/Wrapper.java    |  20 +-
 .../exec/record/FragmentWritableBatch.java      |   6 +
 .../drill/exec/record/RecordBatchLoader.java    |  38 +-
 .../apache/drill/exec/record/WritableBatch.java |   1 -
 .../org/apache/drill/exec/rpc/RpcEncoder.java   |  12 +-
 .../drill/exec/server/DrillbitContext.java      |  10 +-
 .../drill/exec/service/ServiceEngine.java       |   5 +-
 .../drill/exec/store/AbstractStorageEngine.java |   4 +-
 .../drill/exec/store/AffinityCalculator.java    | 112 ++++
 .../apache/drill/exec/store/StorageEngine.java  |  12 +-
 .../drill/exec/store/StorageEngineRegistry.java |   6 +-
 .../apache/drill/exec/store/VectorHolder.java   |   9 +-
 .../drill/exec/store/parquet/BitReader.java     |  87 +++
 .../drill/exec/store/parquet/ColumnReader.java  | 115 ++++
 .../store/parquet/FixedByteAlignedReader.java   |  48 ++
 .../exec/store/parquet/PageReadStatus.java      | 116 ++++
 .../exec/store/parquet/ParquetGroupScan.java    | 357 +++++++++++
 .../exec/store/parquet/ParquetRecordReader.java | 403 +++++++++++++
 .../exec/store/parquet/ParquetRowGroupScan.java | 137 +++++
 .../store/parquet/ParquetScanBatchCreator.java  |  73 +++
 .../store/parquet/ParquetStorageEngine.java     | 116 ++++
 .../parquet/ParquetStorageEngineConfig.java     |  66 +++
 .../exec/store/parquet/VarLenBinaryReader.java  | 130 ++++
 .../drill/exec/vector/BaseDataValueVector.java  |   6 +-
 .../work/AbstractFragmentRunnerListener.java    |   4 +-
 .../exec/work/RemoteFragmentRunnerListener.java |  45 ++
 .../work/RemotingFragmentRunnerListener.java    |  48 --
 .../exec/work/batch/BitComHandlerImpl.java      |   6 +-
 .../apache/drill/exec/work/foreman/Foreman.java |   2 +-
 .../work/fragment/RemoteFragmentHandler.java    |   4 +-
 .../parquet/hadoop/CodecFactoryExposer.java     |  42 ++
 .../src/main/resources/drill-module.conf        |   7 +-
 .../impl/TestDistributedFragmentRun.java        |   5 +-
 .../apache/drill/exec/store/ByteArrayUtil.java  | 181 ++++++
 .../drill/exec/store/JSONRecordReaderTest.java  |   5 +
 .../apache/drill/exec/store/MockScantTest.java  | 115 ++++
 .../exec/store/ParquetRecordReaderTest.java     | 594 +++++++++++++++++++
 .../exec/store/TestAffinityCalculator.java      | 229 +++++++
 .../exec/store/TestParquetPhysicalPlan.java     |  56 ++
 .../src/test/resources/drill-module.conf        |   4 +-
 .../src/test/resources/filter/test1.json        |   2 +-
 .../test/resources/functions/float4Equal.json   |   2 +-
 .../resources/functions/float4GreaterThan.json  |   2 +-
 .../functions/float4GreaterThanEqual.json       |   2 +-
 .../resources/functions/float4LessThan.json     |   2 +-
 .../functions/float4LessThanEqual.json          |   2 +-
 .../resources/functions/float4NotEqual.json     |   2 +-
 .../test/resources/functions/float8Equal.json   |   2 +-
 .../resources/functions/float8GreaterThan.json  |   2 +-
 .../functions/float8GreaterThanEqual.json       |   2 +-
 .../resources/functions/float8LessThan.json     |   2 +-
 .../functions/float8LessThanEqual.json          |   2 +-
 .../resources/functions/float8NotEqual.json     |   2 +-
 .../src/test/resources/functions/intEqual.json  |   2 +-
 .../resources/functions/intGreaterThan.json     |   2 +-
 .../functions/intGreaterThanEqual.json          |   2 +-
 .../test/resources/functions/intLessThan.json   |   2 +-
 .../resources/functions/intLessThanEqual.json   |   2 +-
 .../test/resources/functions/intNotEqual.json   |   2 +-
 .../src/test/resources/functions/longEqual.json |   2 +-
 .../resources/functions/longGreaterThan.json    |   2 +-
 .../functions/longGreaterThanEqual.json         |   2 +-
 .../test/resources/functions/longLessThan.json  |   2 +-
 .../resources/functions/longLessThanEqual.json  |   2 +-
 .../test/resources/functions/longNotEqual.json  |   2 +-
 .../functions/nullableBigIntEqual.json          |   2 +-
 .../functions/nullableBigIntGreaterThan.json    |   2 +-
 .../nullableBigIntGreaterThanEqual.json         |   2 +-
 .../functions/nullableBigIntLessThan.json       |   2 +-
 .../functions/nullableBigIntLessThanEqual.json  |   2 +-
 .../functions/nullableBigIntNotEqual.json       |   2 +-
 .../resources/functions/nullableIntEqual.json   |   2 +-
 .../functions/nullableIntGreaterThan.json       |   2 +-
 .../functions/nullableIntGreaterThanEqual.json  |   2 +-
 .../functions/nullableIntLessThan.json          |   2 +-
 .../functions/nullableIntLessThanEqual.json     |   2 +-
 .../functions/nullableIntNotEqual.json          |   2 +-
 .../resources/functions/testByteSubstring.json  |   2 +-
 .../test/resources/functions/testIsNotNull.json |   2 +-
 .../test/resources/functions/testIsNull.json    |   2 +-
 .../test/resources/functions/testSubstring.json |   2 +-
 .../functions/testSubstringNegative.json        |   2 +-
 .../java-exec/src/test/resources/mock-scan.json |  31 +
 .../src/test/resources/parquet_scan_screen.json |  44 ++
 .../parquet_scan_union_screen_physical.json     |  35 ++
 .../src/test/resources/physical_repeated_1.json |   2 +-
 .../src/test/resources/project/test1.json       |   2 +-
 .../src/test/resources/remover/test1.json       |   2 +-
 .../src/test/resources/sort/one_key_sort.json   |   2 +-
 .../src/test/resources/sort/two_key_sort.json   |   2 +-
 .../prototype/exec/java-exec/src/test/sh/runbit |   2 +-
 .../org/apache/drill/exec/ref/ROPConverter.java |   5 +-
 .../apache/drill/exec/ref/rse/ClasspathRSE.java |   5 +
 .../apache/drill/exec/ref/rse/ConsoleRSE.java   |   8 +-
 .../drill/exec/ref/rse/FileSystemRSE.java       |  17 +
 .../org/apache/drill/exec/ref/rse/QueueRSE.java |  19 +-
 .../java/org/apache/drill/optiq/DrillScan.java  |   2 +-
 134 files changed, 4196 insertions(+), 617 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/0a2f997f/sandbox/prototype/common/src/main/java/org/apache/drill/common/JSONOptions.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/JSONOptions.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/JSONOptions.java
index 8a185a4..d091e17 100644
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/JSONOptions.java
+++ b/sandbox/prototype/common/src/main/java/org/apache/drill/common/JSONOptions.java
@@ -65,7 +65,10 @@ public class JSONOptions {
   public <T> T getListWith(DrillConfig config, TypeReference<T> t) throws IOException {
       ObjectMapper mapper = config.getMapper();
       return mapper.treeAsTokens(root).readValueAs(t);
-     // return mapper.treeToValue(root,  mapper.getTypeFactory().constructCollectionType(List.class, c));
+  }
+
+  public <T> T getListWith(ObjectMapper mapper, TypeReference<T> t) throws IOException {
+    return mapper.treeAsTokens(root).readValueAs(t);
   }
   
   public JsonNode path(String name){

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/0a2f997f/sandbox/prototype/common/src/main/java/org/apache/drill/common/logical/LogicalPlan.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/logical/LogicalPlan.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/logical/LogicalPlan.java
index 05fbd1f..742001a 100644
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/logical/LogicalPlan.java
+++ b/sandbox/prototype/common/src/main/java/org/apache/drill/common/logical/LogicalPlan.java
@@ -63,7 +63,7 @@ public class LogicalPlan {
     return GraphAlgos.TopoSorter.sortLogical(graph);
   }
 
-  public StorageEngineConfig getStorageEngine(String name) {
+  public StorageEngineConfig getStorageEngineConfig(String name) {
     return storageEngineMap.get(name);
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/0a2f997f/sandbox/prototype/common/src/main/java/org/apache/drill/common/logical/StorageEngineConfig.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/logical/StorageEngineConfig.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/logical/StorageEngineConfig.java
index 3a893d6..b73a2c1 100644
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/logical/StorageEngineConfig.java
+++ b/sandbox/prototype/common/src/main/java/org/apache/drill/common/logical/StorageEngineConfig.java
@@ -23,4 +23,6 @@ import com.fasterxml.jackson.annotation.JsonTypeInfo;
 
 @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, include = JsonTypeInfo.As.PROPERTY, property="type")
 public interface StorageEngineConfig{
+
+  public boolean equals(Object o);
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/0a2f997f/sandbox/prototype/common/src/main/java/org/apache/drill/common/logical/StorageEngineConfigBase.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/logical/StorageEngineConfigBase.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/logical/StorageEngineConfigBase.java
index 853196c..51dbef3 100644
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/logical/StorageEngineConfigBase.java
+++ b/sandbox/prototype/common/src/main/java/org/apache/drill/common/logical/StorageEngineConfigBase.java
@@ -35,5 +35,7 @@ public abstract class StorageEngineConfigBase implements StorageEngineConfig{
     logger.debug("Adding Storage Engine Configs including {}", (Object) sec );
     return sec;
   }
+
+  public abstract boolean equals(Object o);
   
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/0a2f997f/sandbox/prototype/common/src/test/java/org/apache/drill/storage/MockStorageEngineConfig.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/test/java/org/apache/drill/storage/MockStorageEngineConfig.java b/sandbox/prototype/common/src/test/java/org/apache/drill/storage/MockStorageEngineConfig.java
index bc24b2e..5843c38 100644
--- a/sandbox/prototype/common/src/test/java/org/apache/drill/storage/MockStorageEngineConfig.java
+++ b/sandbox/prototype/common/src/test/java/org/apache/drill/storage/MockStorageEngineConfig.java
@@ -39,5 +39,21 @@ public class MockStorageEngineConfig extends StorageEngineConfigBase{
     return url;
   }
 
-  
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) return true;
+    if (o == null || getClass() != o.getClass()) return false;
+
+    MockStorageEngineConfig that = (MockStorageEngineConfig) o;
+
+    if (url != null ? !url.equals(that.url) : that.url != null) return false;
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    return url != null ? url.hashCode() : 0;
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/0a2f997f/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 4b13952..cd9bc9a 100644
--- a/sandbox/prototype/exec/java-exec/pom.xml
+++ b/sandbox/prototype/exec/java-exec/pom.xml
@@ -1,7 +1,7 @@
 <?xml version="1.0"?>
 <project
-  xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"
-  xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance">
+    xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"
+    xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance">
   <modelVersion>4.0.0</modelVersion>
   <parent>
     <artifactId>exec-parent</artifactId>
@@ -46,7 +46,12 @@
     <dependency>
       <groupId>com.twitter</groupId>
       <artifactId>parquet-column</artifactId>
-      <version>1.0.0</version>
+      <version>1.0.1-SNAPSHOT</version>
+    </dependency>
+    <dependency>
+      <groupId>com.twitter</groupId>
+      <artifactId>parquet-hadoop</artifactId>
+      <version>1.0.1-SNAPSHOT</version>
     </dependency>
     <dependency>
       <groupId>com.yammer.metrics</groupId>
@@ -101,16 +106,57 @@
     <dependency>
       <groupId>org.apache.hadoop</groupId>
       <artifactId>hadoop-core</artifactId>
-      <version>1.1.0</version>
+      <version>1.0.3-mapr-2.1.2.1</version>
       <exclusions>
         <exclusion>
           <artifactId>jets3t</artifactId>
           <groupId>net.java.dev.jets3t</groupId>
         </exclusion>
         <exclusion>
+          <artifactId>log4j</artifactId>
+          <groupId>log4j</groupId>
+        </exclusion>
+
+        <exclusion>
+          <artifactId>mockito-all</artifactId>
+          <groupId>org.mockito</groupId>
+        </exclusion>
+        <exclusion>
+          <artifactId>commons-logging-api</artifactId>
+          <groupId>commons-logging</groupId>
+        </exclusion>
+        <exclusion>
           <artifactId>commons-logging</artifactId>
           <groupId>commons-logging</groupId>
         </exclusion>
+        <exclusion>
+          <artifactId>slf4j-log4j12</artifactId>
+          <groupId>org.slf4j</groupId>
+        </exclusion>
+        <exclusion>
+          <artifactId>servlet-api-2.5</artifactId>
+          <groupId>org.mortbay.jetty</groupId>
+        </exclusion>
+        <exclusion>
+          <artifactId>jasper-runtime</artifactId>
+          <groupId>tomcat</groupId>
+        </exclusion>
+        <exclusion>
+          <artifactId>jasper-compiler</artifactId>
+          <groupId>tomcat</groupId>
+        </exclusion>
+        <exclusion>
+          <artifactId>jetty</artifactId>
+          <groupId>org.mortbay.jetty</groupId>
+        </exclusion>
+        <exclusion>
+          <artifactId>jersey-server</artifactId>
+          <groupId>com.sun.jersey</groupId>
+        </exclusion>
+        <exclusion>
+          <artifactId>core</artifactId>
+          <groupId>org.eclipse.jdt</groupId>
+        </exclusion>
       </exclusions>
     </dependency>
     <dependency>
@@ -191,13 +237,13 @@
                   </fileset>
                 </path>
                 <pathconvert pathsep=" " property="proto.files"
-                  refid="proto.path.files" />
+                             refid="proto.path.files" />
 
                 <exec executable="protoc">
                   <arg value="--java_out=${target.gen.source.path}" />
                   <arg value="--proto_path=${proto.cas.path}" />
                   <arg
-                    value="--proto_path=${project.basedir}/../../common/src/main/protobuf/" />
+                      value="--proto_path=${project.basedir}/../../common/src/main/protobuf/" />
                   <arg line="${proto.files}" />
                 </exec>
               </tasks>
@@ -244,4 +290,13 @@
     </plugins>
   </build>
 
+
+  <repositories>
+    <repository>
+      <id>mapr-releases</id>
+      <url>http://repository.mapr.com/maven/</url>
+      <snapshots><enabled>false</enabled></snapshots>
+      <releases><enabled>true</enabled></releases>
+    </repository>
+  </repositories>
 </project>

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/0a2f997f/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/VariableLengthVectors.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/VariableLengthVectors.java b/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/VariableLengthVectors.java
index 061234c..7ceafe4 100644
--- a/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/VariableLengthVectors.java
+++ b/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/VariableLengthVectors.java
@@ -21,11 +21,9 @@ import org.apache.drill.exec.proto.UserBitShared.FieldMetadata;
 import org.apache.drill.exec.record.DeadBuf;
 import org.apache.drill.exec.record.MaterializedField;
 import org.apache.drill.exec.record.TransferPair;
-import org.mortbay.jetty.servlet.Holder;
 
 import com.google.common.base.Charsets;
 
-import antlr.collections.impl.Vector;
 
 /**
  * ${minor.class}Vector implements a vector of variable width values.  Elements in the vector
@@ -197,6 +195,10 @@ public final class ${minor.class}Vector extends BaseDataValueVector implements V
     public int getValueCount() {
       return valueCount;
     }
+
+    public UInt${type.width}Vector getOffsetVector(){
+      return offsetVector;
+    }
   }
   
   /**

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/0a2f997f/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ExecConstants.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ExecConstants.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ExecConstants.java
index 4eb0f4c..5e7ddf0 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ExecConstants.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ExecConstants.java
@@ -30,4 +30,5 @@ public interface ExecConstants {
   public static final String INITIAL_USER_PORT = "drill.exec.rpc.user.port";
   public static final String METRICS_CONTEXT_NAME = "drill.exec.metrics.context";
   public static final String FUNCTION_PACKAGES = "drill.exec.functions";
+  public static final String USE_IP_ADDRESS = "drill.exec.rpc.use.ip";
 }

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

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/0a2f997f/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 1c251b8..44117ff 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,12 +19,15 @@ package org.apache.drill.exec.ops;
 
 import java.util.Collection;
 
+import org.apache.drill.common.logical.StorageEngineConfig;
 import org.apache.drill.exec.cache.DistributedCache;
+import org.apache.drill.exec.exception.SetupException;
 import org.apache.drill.exec.planner.PhysicalPlanReader;
 import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
 import org.apache.drill.exec.proto.UserBitShared.QueryId;
 import org.apache.drill.exec.rpc.bit.BitCom;
 import org.apache.drill.exec.server.DrillbitContext;
+import org.apache.drill.exec.store.StorageEngine;
 
 public class QueryContext {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(QueryContext.class);
@@ -46,6 +49,10 @@ public class QueryContext {
     return queryId;
   }
 
+  public StorageEngine getStorageEngine(StorageEngineConfig config) throws SetupException {
+    return drillbitContext.getStorageEngine(config);
+  }
+
   public DistributedCache getCache(){
     return drillbitContext.getCache();
   }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/0a2f997f/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/opt/BasicOptimizer.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/opt/BasicOptimizer.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/opt/BasicOptimizer.java
index 289ec4b..c4a7e43 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/opt/BasicOptimizer.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/opt/BasicOptimizer.java
@@ -2,22 +2,28 @@ package org.apache.drill.exec.opt;
 
 import java.io.IOException;
 import java.util.ArrayList;
-import java.util.Arrays;
 import java.util.Collection;
 import java.util.List;
 
 import org.apache.drill.common.PlanProperties;
 import org.apache.drill.common.config.DrillConfig;
-import org.apache.drill.common.expression.*;
+import org.apache.drill.common.expression.ExpressionPosition;
+import org.apache.drill.common.expression.FunctionCall;
+import org.apache.drill.common.expression.FunctionDefinition;
+import org.apache.drill.common.expression.NoArgValidator;
+import org.apache.drill.common.expression.OutputTypeDeterminer;
 import org.apache.drill.common.logical.LogicalPlan;
-import org.apache.drill.common.logical.data.*;
 import org.apache.drill.common.logical.data.Filter;
 import org.apache.drill.common.logical.data.Project;
+import org.apache.drill.common.logical.data.Scan;
+import org.apache.drill.common.logical.data.SinkOperator;
+import org.apache.drill.common.logical.data.Store;
 import org.apache.drill.common.logical.data.visitors.AbstractLogicalVisitor;
 import org.apache.drill.common.types.TypeProtos;
 import org.apache.drill.common.types.TypeProtos.DataMode;
 import org.apache.drill.common.types.TypeProtos.MinorType;
 import org.apache.drill.exec.exception.OptimizerException;
+import org.apache.drill.exec.exception.SetupException;
 import org.apache.drill.exec.ops.QueryContext;
 import org.apache.drill.exec.physical.PhysicalPlan;
 import org.apache.drill.exec.physical.base.PhysicalOperator;
@@ -27,112 +33,127 @@ import com.fasterxml.jackson.core.type.TypeReference;
 
 public class BasicOptimizer extends Optimizer{
 
-    private DrillConfig config;
-    private QueryContext context;
-
-    public BasicOptimizer(DrillConfig config, QueryContext context){
-        this.config = config;
-        this.context = context;
+  private DrillConfig config;
+  private QueryContext context;
+
+  public BasicOptimizer(DrillConfig config, QueryContext context){
+    this.config = config;
+    this.context = context;
+  }
+
+  @Override
+  public void init(DrillConfig config) {
+
+  }
+
+  @Override
+  public PhysicalPlan optimize(OptimizationContext context, LogicalPlan plan) {
+    Object obj = new Object();
+    Collection<SinkOperator> roots = plan.getGraph().getRoots();
+    List<PhysicalOperator> physOps = new ArrayList<PhysicalOperator>(roots.size());
+    LogicalConverter converter = new LogicalConverter(plan);
+    for ( SinkOperator op : roots){
+      try {
+        PhysicalOperator pop  = op.accept(converter, obj);
+        System.out.println(pop);
+        physOps.add(pop);
+      } catch (OptimizerException e) {
+        e.printStackTrace();
+      } catch (Throwable throwable) {
+        throwable.printStackTrace();
+      }
     }
 
-    @Override
-    public void init(DrillConfig config) {
+    PlanProperties props = new PlanProperties();
+    props.type = PlanProperties.PlanType.APACHE_DRILL_PHYSICAL;
+    props.version = plan.getProperties().version;
+    props.generator = plan.getProperties().generator;
+    PhysicalPlan p = new PhysicalPlan(props, physOps);
+    return p;
+    //return new PhysicalPlan(props, physOps);
+  }
 
-    }
+  @Override
+  public void close() {
 
-    @Override
-    public PhysicalPlan optimize(OptimizationContext context, LogicalPlan plan) {
-        Object obj = new Object();
-        Collection<SinkOperator> roots = plan.getGraph().getRoots();
-        List<PhysicalOperator> physOps = new ArrayList<PhysicalOperator>(roots.size());
-        LogicalConverter converter = new LogicalConverter();
-        for ( SinkOperator op : roots){
-            try {
-                PhysicalOperator pop  = op.accept(converter, obj);
-                System.out.println(pop);
-                physOps.add(pop);
-            } catch (OptimizerException e) {
-                e.printStackTrace();
-            } catch (Throwable throwable) {
-                throwable.printStackTrace();
-            }
-        }
+  }
 
-        PlanProperties props = new PlanProperties();
-        props.type = PlanProperties.PlanType.APACHE_DRILL_PHYSICAL;
-        props.version = plan.getProperties().version;
-        props.generator = plan.getProperties().generator;
-        return new PhysicalPlan(props, physOps);
-    }
+  public static class BasicOptimizationContext implements OptimizationContext {
 
     @Override
-    public void close() {
-
+    public int getPriority() {
+      return 1;
     }
+  }
 
-    public static class BasicOptimizationContext implements OptimizationContext {
+  private class LogicalConverter extends AbstractLogicalVisitor<PhysicalOperator, Object, OptimizerException> {
 
-        @Override
-        public int getPriority() {
-            return 1;
-        }
+    // storing a reference to the plan for access to other elements outside of the query graph
+    // such as the storage engine configs
+    LogicalPlan logicalPlan;
+
+    public LogicalConverter(LogicalPlan logicalPlan){
+      this.logicalPlan = logicalPlan;
     }
 
 
-    private class LogicalConverter extends AbstractLogicalVisitor<PhysicalOperator, Object, OptimizerException> {
-
-        @Override
-        public MockScanPOP visitScan(Scan scan, Object obj) throws OptimizerException {
-            List<MockScanPOP.MockScanEntry> myObjects;
-
-            try {
-                if ( scan.getStorageEngine().equals("local-logs")){
-                    myObjects = scan.getSelection().getListWith(config,
-                            new TypeReference<ArrayList<MockScanPOP.MockScanEntry>>() {
-                    });
-                }
-                else{
-                    myObjects = new ArrayList<>();
-                    MockScanPOP.MockColumn[] cols = {
-                        new MockScanPOP.MockColumn("RED", MinorType.BIGINT, DataMode.REQUIRED, null, null, null),
-                        new MockScanPOP.MockColumn("GREEN", MinorType.BIGINT, DataMode.REQUIRED,null, null, null)
-                    };
-                    myObjects.add(new MockScanPOP.MockScanEntry(100, cols));
-                }
-            } catch (IOException e) {
-                e.printStackTrace();
-                throw new OptimizerException("Error reading selection attribute of Scan node in Logical to Physical plan conversion.");
-            }
-
-            return new MockScanPOP("http://apache.org", myObjects);
-        }
+    @Override
+    public PhysicalOperator visitScan(Scan scan, Object obj) throws OptimizerException {
+      List<MockGroupScanPOP.MockScanEntry> myObjects;
 
-        @Override
-        public Screen visitStore(Store store, Object obj) throws OptimizerException {
-            if ( ! store.iterator().hasNext()){
-                throw new OptimizerException("Store node in logical plan does not have a child.");
-            }
-            return new Screen(store.iterator().next().accept(this, obj), context.getCurrentEndpoint());
+      try {
+        if (scan.getStorageEngine().equals("parquet")) {
+          return context.getStorageEngine(logicalPlan.getStorageEngineConfig(scan.getStorageEngine())).getPhysicalScan(scan);
         }
-
-        @Override
-        public PhysicalOperator visitProject(Project project, Object obj) throws OptimizerException {
-          return project.getInput().accept(this, obj);
-//            return new org.apache.drill.exec.physical.config.Project(
-//                Arrays.asList(project.getSelections()), project.iterator().next().accept(this, obj));
+        if (scan.getStorageEngine().equals("local-logs")) {
+          myObjects = scan.getSelection().getListWith(config,
+              new TypeReference<ArrayList<MockGroupScanPOP.MockScanEntry>>() {
+              });
+        } else {
+          myObjects = new ArrayList<>();
+          MockGroupScanPOP.MockColumn[] cols = {
+              new MockGroupScanPOP.MockColumn("blah", MinorType.INT, DataMode.REQUIRED, 4, 4, 4),
+              new MockGroupScanPOP.MockColumn("blah_2", MinorType.INT, DataMode.REQUIRED, 4, 4, 4) };
+          myObjects.add(new MockGroupScanPOP.MockScanEntry(50, cols));
         }
+      } catch (IOException e) {
+        throw new OptimizerException(
+            "Error reading selection attribute of GroupScan node in Logical to Physical plan conversion.", e);
+      } catch (SetupException e) {
+        throw new OptimizerException(
+            "Storage engine not found: " + scan.getStorageEngine(), e);
+      }
+
+      return new MockGroupScanPOP("http://apache.org", myObjects);
+    }
 
-      @Override
-      public PhysicalOperator visitFilter(Filter filter, Object obj) throws OptimizerException {
-        TypeProtos.MajorType.Builder b = TypeProtos.MajorType.getDefaultInstance().newBuilderForType();
-        b.setMode(DataMode.REQUIRED);
-        b.setMinorType(MinorType.BIGINT);
-
-        return new SelectionVectorRemover(new org.apache.drill.exec.physical.config.Filter(
-            filter.iterator().next().accept(this, obj), /*filter.getExpr() */
-            new FunctionCall(FunctionDefinition.simple("alternate", new NoArgValidator(),
-                new OutputTypeDeterminer.FixedType(b.build())), null, new ExpressionPosition("asdf", 1)),
-            1.0f));
+    @Override
+    public PhysicalOperator visitStore(Store store, Object obj) throws OptimizerException {
+      if (!store.iterator().hasNext()) {
+        throw new OptimizerException("Store node in logical plan does not have a child.");
       }
+      return new Screen(store.iterator().next().accept(this, obj), context.getCurrentEndpoint());
+    }
+
+    @Override
+    public PhysicalOperator visitProject(Project project, Object obj) throws OptimizerException {
+      return project.getInput().accept(this, obj);
+      // return new org.apache.drill.exec.physical.config.Project(
+      // Arrays.asList(project.getSelections()), project.iterator().next().accept(this, obj));
+    }
+
+    @Override
+    public PhysicalOperator visitFilter(Filter filter, Object obj) throws OptimizerException {
+      TypeProtos.MajorType.Builder b = TypeProtos.MajorType.getDefaultInstance().newBuilderForType();
+      b.setMode(DataMode.REQUIRED);
+      b.setMinorType(MinorType.BIGINT);
+
+      return new SelectionVectorRemover(new org.apache.drill.exec.physical.config.Filter(filter.iterator().next()
+          .accept(this, obj), /* filter.getExpr() */
+      new FunctionCall(FunctionDefinition.simple("alternate", new NoArgValidator(), new OutputTypeDeterminer.FixedType(
+          b.build())), null, new ExpressionPosition("asdf", 1)), 1.0f));
     }
+
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/0a2f997f/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/ReadEntryFromHDFS.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/ReadEntryFromHDFS.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/ReadEntryFromHDFS.java
new file mode 100644
index 0000000..b90f1bc
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/ReadEntryFromHDFS.java
@@ -0,0 +1,54 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.physical;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import org.apache.drill.exec.physical.base.Size;
+
+public class ReadEntryFromHDFS extends ReadEntryWithPath {
+
+  private long start;
+  private long length;
+
+  @JsonCreator
+  public ReadEntryFromHDFS(@JsonProperty("path") String path,@JsonProperty("start") long start, @JsonProperty("length") long length) {
+    this.path = path;
+    this.start = start;
+    this.length = length;
+  }
+
+  @Override
+  public OperatorCost getCost() {
+    return new OperatorCost(1, 2, 1, 1);
+  }
+
+  @Override
+  public Size getSize() {
+    // TODO - these values are wrong, I cannot know these until after I read a file
+    return new Size(10, 10);
+  }
+
+  public long getStart() {
+    return start;
+  }
+
+  public long getLength() {
+    return length;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/0a2f997f/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/ReadEntryWithPath.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/ReadEntryWithPath.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/ReadEntryWithPath.java
new file mode 100644
index 0000000..57d1d0b
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/ReadEntryWithPath.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.physical;
+
+import org.apache.drill.exec.physical.base.Size;
+
+public class ReadEntryWithPath implements ReadEntry {
+
+  protected String path;
+
+  public String getPath(){
+   return path;
+  }
+
+  @Override
+  public OperatorCost getCost() {
+    throw new UnsupportedOperationException(this.getClass().getCanonicalName() + " is only for extracting path data from " +
+        "selections inside a scan node from a logical plan, it cannot be used in an executing plan and has no cost.");
+  }
+
+  @Override
+  public Size getSize() {
+    throw new UnsupportedOperationException(this.getClass().getCanonicalName() + " is only for extracting path data from " +
+        "selections on a scan node from a logical plan, it cannot be used in an executing plan and has no size.");
+  }
+}

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

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/0a2f997f/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractPhysicalVisitor.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractPhysicalVisitor.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractPhysicalVisitor.java
index f782325..3b58803 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractPhysicalVisitor.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractPhysicalVisitor.java
@@ -62,8 +62,13 @@ public abstract class AbstractPhysicalVisitor<T, X, E extends Throwable> impleme
   }
 
   @Override
-  public T visitScan(Scan<?> scan, X value) throws E{
-    return visitOp(scan, value);
+  public T visitGroupScan(GroupScan groupScan, X value) throws E{
+    return visitOp(groupScan, value);
+  }
+
+  @Override
+  public T visitSubScan(SubScan subScan, X value) throws E{
+    return visitOp(subScan, value);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/0a2f997f/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
deleted file mode 100644
index dbde9c5..0000000
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractScan.java
+++ /dev/null
@@ -1,84 +0,0 @@
-/*******************************************************************************
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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/0a2f997f/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/GroupScan.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/GroupScan.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/GroupScan.java
new file mode 100644
index 0000000..acafd6c
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/GroupScan.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 com.fasterxml.jackson.annotation.JsonIgnore;
+import org.apache.drill.exec.physical.ReadEntry;
+import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
+
+import com.fasterxml.jackson.annotation.JsonProperty;
+
+public interface GroupScan extends Scan, HasAffinity{
+
+  public abstract void applyAssignments(List<DrillbitEndpoint> endpoints);
+
+  public abstract SubScan getSpecificScan(int minorFragmentId);
+
+  @JsonIgnore
+  public int getMaxParallelizationWidth();
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/0a2f997f/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
index d4ed456..7764739 100644
--- 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
@@ -19,7 +19,7 @@ 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.
+ * node are GroupScan 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/0a2f997f/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
index d412c2d..c24836b 100644
--- 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
@@ -34,7 +34,7 @@ import com.fasterxml.jackson.annotation.ObjectIdGenerators;
 @JsonPropertyOrder({ "@id" })
 @JsonIdentityInfo(generator = ObjectIdGenerators.IntSequenceGenerator.class, property = "@id")
 @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, include = JsonTypeInfo.As.PROPERTY, property = "pop")
-public interface PhysicalOperator extends GraphValue<PhysicalOperator> {
+public interface  PhysicalOperator extends GraphValue<PhysicalOperator> {
 
   /**
    * Get the cost of execution of this particular operator.

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/0a2f997f/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/PhysicalVisitor.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/PhysicalVisitor.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/PhysicalVisitor.java
index f36633f..8e09e3a 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/PhysicalVisitor.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/PhysicalVisitor.java
@@ -39,7 +39,8 @@ public interface PhysicalVisitor<RETURN, EXTRA, EXCEP extends Throwable> {
   
   
   public RETURN visitExchange(Exchange exchange, EXTRA value) throws EXCEP;
-  public RETURN visitScan(Scan<?> scan, EXTRA value) throws EXCEP;
+  public RETURN visitGroupScan(GroupScan groupScan, EXTRA value) throws EXCEP;
+  public RETURN visitSubScan(SubScan subScan, EXTRA value) throws EXCEP;
   public RETURN visitStore(Store store, EXTRA value) throws EXCEP;
 
   public RETURN visitFilter(Filter filter, EXTRA value) throws EXCEP;

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/0a2f997f/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
index 2207f79..f56e9f9 100644
--- 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
@@ -6,9 +6,9 @@
  * to you under the Apache License, Version 2.0 (the
  * "License"); you may not use this file except in compliance
  * with the License.  You may obtain a copy of the License at
- * 
+ *
  * http://www.apache.org/licenses/LICENSE-2.0
- * 
+ *
  * Unless required by applicable law or agreed to in writing, software
  * distributed under the License is distributed on an "AS IS" BASIS,
  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
@@ -17,20 +17,7 @@
  ******************************************************************************/
 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
+public interface Scan extends Leaf {
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/0a2f997f/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/SubScan.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/SubScan.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/SubScan.java
new file mode 100644
index 0000000..f75ba19
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/SubScan.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.physical.base;
+
+import org.apache.drill.exec.physical.ReadEntry;
+
+public interface SubScan extends Scan {
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/0a2f997f/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockGroupScanPOP.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockGroupScanPOP.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockGroupScanPOP.java
new file mode 100644
index 0000000..a28c7d8
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockGroupScanPOP.java
@@ -0,0 +1,221 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.common.types.TypeProtos.DataMode;
+import org.apache.drill.common.types.TypeProtos.MajorType;
+import org.apache.drill.common.types.TypeProtos.MinorType;
+import org.apache.drill.exec.physical.EndpointAffinity;
+import org.apache.drill.exec.physical.OperatorCost;
+import org.apache.drill.exec.physical.ReadEntry;
+import org.apache.drill.exec.physical.base.*;
+import org.apache.drill.exec.physical.base.AbstractGroupScan;
+import org.apache.drill.exec.physical.base.GroupScan;
+import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
+import org.apache.drill.exec.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 MockGroupScanPOP extends AbstractGroupScan {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(MockGroupScanPOP.class);
+
+  private final String url;
+  protected final List<MockScanEntry> readEntries;
+  private final OperatorCost cost;
+  private final Size size;
+  private  LinkedList<MockScanEntry>[] mappings;
+
+  @JsonCreator
+  public MockGroupScanPOP(@JsonProperty("url") String url, @JsonProperty("entries") List<MockScanEntry> readEntries) {
+    this.readEntries = readEntries;
+    OperatorCost cost = new OperatorCost(0,0,0,0);
+    Size size = new Size(0,0);
+    for(MockScanEntry r : readEntries){
+      cost = cost.add(r.getCost());
+      size = size.add(r.getSize());
+    }
+    this.cost = cost;
+    this.size = size;
+    this.url = url;
+  }
+
+  public String getUrl() {
+    return url;
+  }
+
+  @JsonProperty("entries")
+  public List<MockScanEntry> getReadEntries() {
+    return readEntries;
+  }
+  
+  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 SubScan 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 MockSubScanPOP(url, mappings[minorFragmentId]);
+  }
+
+  @Override
+  public int getMaxParallelizationWidth() {
+    return readEntries.size();
+  }
+
+  @Override
+  public OperatorCost getCost() {
+    return cost;
+  }
+
+  @Override
+  public Size getSize() {
+    return size;
+  }
+
+  @Override
+  @JsonIgnore
+  public PhysicalOperator getNewWithChildren(List<PhysicalOperator> children) {
+    Preconditions.checkArgument(children.isEmpty());
+    return new MockGroupScanPOP(url, readEntries);
+
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/0a2f997f/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockRecordReader.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockRecordReader.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockRecordReader.java
index 11b9243..bd57823 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockRecordReader.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockRecordReader.java
@@ -23,8 +23,8 @@ import org.apache.drill.common.expression.SchemaPath;
 import org.apache.drill.common.types.TypeProtos.MajorType;
 import org.apache.drill.exec.exception.SchemaChangeException;
 import org.apache.drill.exec.ops.FragmentContext;
-import org.apache.drill.exec.physical.config.MockScanPOP.MockColumn;
-import org.apache.drill.exec.physical.config.MockScanPOP.MockScanEntry;
+import org.apache.drill.exec.physical.config.MockGroupScanPOP.MockColumn;
+import org.apache.drill.exec.physical.config.MockGroupScanPOP.MockScanEntry;
 import org.apache.drill.exec.physical.impl.OutputMutator;
 import org.apache.drill.exec.record.MaterializedField;
 import org.apache.drill.exec.store.RecordReader;

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/0a2f997f/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockScanBatchCreator.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockScanBatchCreator.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockScanBatchCreator.java
index bfc19af..a06aaee 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockScanBatchCreator.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockScanBatchCreator.java
@@ -21,7 +21,7 @@ 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.config.MockGroupScanPOP.MockScanEntry;
 import org.apache.drill.exec.physical.impl.BatchCreator;
 import org.apache.drill.exec.physical.impl.ScanBatch;
 import org.apache.drill.exec.record.RecordBatch;
@@ -30,11 +30,11 @@ 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>{
+public class MockScanBatchCreator implements BatchCreator<MockSubScanPOP>{
   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 {
+  public RecordBatch getBatch(FragmentContext context, MockSubScanPOP config, List<RecordBatch> children) throws ExecutionSetupException {
     Preconditions.checkArgument(children.isEmpty());
     List<MockScanEntry> entries = config.getReadEntries();
     List<RecordReader> readers = Lists.newArrayList();

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/0a2f997f/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
deleted file mode 100644
index 151d541..0000000
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockScanPOP.java
+++ /dev/null
@@ -1,193 +0,0 @@
-/*******************************************************************************
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.common.types.TypeProtos.DataMode;
-import org.apache.drill.common.types.TypeProtos.MajorType;
-import org.apache.drill.common.types.TypeProtos.MinorType;
-import org.apache.drill.exec.physical.EndpointAffinity;
-import org.apache.drill.exec.physical.OperatorCost;
-import org.apache.drill.exec.physical.ReadEntry;
-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.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);
-
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/0a2f997f/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
index 0044628..6348686 100644
--- 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
@@ -22,11 +22,10 @@ import java.util.Collection;
 
 import org.apache.drill.common.logical.data.Scan;
 import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.physical.ReadEntry;
 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;
 
@@ -39,11 +38,6 @@ public class MockStorageEngine extends AbstractStorageEngine{
   }
 
   @Override
-  public Collection<ReadEntry> getReadEntries(Scan scan) throws IOException {
-    return null;
-  }
-
-  @Override
   public ListMultimap<ReadEntry, DrillbitEndpoint> getReadLocations(Collection<ReadEntry> entries) {
     return null;
   }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/0a2f997f/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockSubScanPOP.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockSubScanPOP.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockSubScanPOP.java
new file mode 100644
index 0000000..7380617
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockSubScanPOP.java
@@ -0,0 +1,115 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.physical.config;
+
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+
+import com.google.common.collect.Iterators;
+import org.apache.drill.common.graph.GraphVisitor;
+import org.apache.drill.common.types.TypeProtos.DataMode;
+import org.apache.drill.common.types.TypeProtos.MajorType;
+import org.apache.drill.common.types.TypeProtos.MinorType;
+import org.apache.drill.exec.physical.EndpointAffinity;
+import org.apache.drill.exec.physical.OperatorCost;
+import org.apache.drill.exec.physical.ReadEntry;
+import org.apache.drill.exec.physical.base.*;
+import org.apache.drill.exec.physical.base.AbstractGroupScan;
+import org.apache.drill.exec.physical.base.GroupScan;
+import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
+import org.apache.drill.exec.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-sub-scan")
+public class MockSubScanPOP extends AbstractBase implements SubScan {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(MockGroupScanPOP.class);
+
+  private final String url;
+  protected final List<MockGroupScanPOP.MockScanEntry> readEntries;
+  private final OperatorCost cost;
+  private final Size size;
+  private  LinkedList<MockGroupScanPOP.MockScanEntry>[] mappings;
+
+  @JsonCreator
+  public MockSubScanPOP(@JsonProperty("url") String url, @JsonProperty("entries") List<MockGroupScanPOP.MockScanEntry> readEntries) {
+    this.readEntries = readEntries;
+    OperatorCost cost = new OperatorCost(0,0,0,0);
+    Size size = new Size(0,0);
+    for(MockGroupScanPOP.MockScanEntry r : readEntries){
+      cost = cost.add(r.getCost());
+      size = size.add(r.getSize());
+    }
+    this.cost = cost;
+    this.size = size;
+    this.url = url;
+  }
+
+  public String getUrl() {
+    return url;
+  }
+
+  @JsonProperty("entries")
+  public List<MockGroupScanPOP.MockScanEntry> getReadEntries() {
+    return readEntries;
+  }
+
+  @Override
+  public Iterator<PhysicalOperator> iterator() {
+    return Iterators.emptyIterator();
+  }
+
+  @Override
+  public OperatorCost getCost() {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public Size getSize() {
+    throw new UnsupportedOperationException();
+  }
+
+  // will want to replace these two methods with an interface above for AbstractSubScan
+  @Override
+  public boolean isExecutable() {
+    return true;  //To change body of implemented methods use File | Settings | File Templates.
+  }
+
+  @Override
+  public <T, X, E extends Throwable> T accept(PhysicalVisitor<T, X, E> physicalVisitor, X value) throws E{
+    return physicalVisitor.visitSubScan(this, value);
+  }
+  // see comment above about replacing this
+
+  @Override
+  @JsonIgnore
+  public PhysicalOperator getNewWithChildren(List<PhysicalOperator> children) {
+    Preconditions.checkArgument(children.isEmpty());
+    return new MockSubScanPOP(url, readEntries);
+
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/0a2f997f/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ImplCreator.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ImplCreator.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ImplCreator.java
index 1c15289..61c9383 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ImplCreator.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ImplCreator.java
@@ -6,9 +6,9 @@
  * to you under the Apache License, Version 2.0 (the
  * "License"); you may not use this file except in compliance
  * with the License.  You may obtain a copy of the License at
- * 
+ *
  * http://www.apache.org/licenses/LICENSE-2.0
- * 
+ *
  * Unless required by applicable law or agreed to in writing, software
  * distributed under the License is distributed on an "AS IS" BASIS,
  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
@@ -26,6 +26,15 @@ import org.apache.drill.exec.physical.base.AbstractPhysicalVisitor;
 import org.apache.drill.exec.physical.base.FragmentRoot;
 import org.apache.drill.exec.physical.base.PhysicalOperator;
 import org.apache.drill.exec.physical.base.Scan;
+import org.apache.drill.exec.physical.config.Filter;
+import org.apache.drill.exec.physical.config.MockScanBatchCreator;
+import org.apache.drill.exec.physical.config.Project;
+import org.apache.drill.exec.physical.config.RandomReceiver;
+import org.apache.drill.exec.physical.config.Screen;
+import org.apache.drill.exec.physical.config.SelectionVectorRemover;
+import org.apache.drill.exec.physical.config.SingleSender;
+import org.apache.drill.exec.physical.config.Sort;
+import org.apache.drill.exec.physical.base.*;
 import org.apache.drill.exec.physical.config.*;
 import org.apache.drill.exec.physical.impl.filter.FilterBatchCreator;
 import org.apache.drill.exec.physical.impl.project.ProjectBatchCreator;
@@ -35,11 +44,15 @@ import org.apache.drill.exec.record.RecordBatch;
 
 import com.google.common.base.Preconditions;
 import com.google.common.collect.Lists;
+import org.apache.drill.exec.store.parquet.ParquetGroupScan;
+import org.apache.drill.exec.store.parquet.ParquetRowGroupScan;
+import org.apache.drill.exec.store.parquet.ParquetScanBatchCreator;
 
 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 ParquetScanBatchCreator parquetScan = new ParquetScanBatchCreator();
   private ScreenCreator sc = new ScreenCreator();
   private RandomReceiverCreator rrc = new RandomReceiverCreator();
   private SingleSenderCreator ssc = new SingleSenderCreator();
@@ -48,22 +61,25 @@ public class ImplCreator extends AbstractPhysicalVisitor<RecordBatch, FragmentCo
   private SVRemoverCreator svc = new SVRemoverCreator();
   private SortBatchCreator sbc = new SortBatchCreator();
   private RootExec root = null;
-  
+
   private ImplCreator(){}
-  
+
   public RootExec getRoot(){
     return root;
   }
-  
+
   @Override
   public RecordBatch visitProject(Project op, FragmentContext context) throws ExecutionSetupException {
     return pbc.getBatch(context, op, getChildren(op, context));
   }
 
   @Override
-  public RecordBatch visitScan(Scan<?> scan, FragmentContext context) throws ExecutionSetupException {
-    Preconditions.checkNotNull(scan);
+  public RecordBatch visitSubScan(SubScan subScan, FragmentContext context) throws ExecutionSetupException {
+    Preconditions.checkNotNull(subScan);
     Preconditions.checkNotNull(context);
+
+    if(subScan instanceof MockSubScanPOP){
+      return msc.getBatch(context, (MockSubScanPOP) subScan, Collections.<RecordBatch> emptyList());
     
     if(scan instanceof MockScanPOP){
       return msc.getBatch(context, (MockScanPOP) scan, Collections.<RecordBatch>emptyList());
@@ -72,16 +88,21 @@ public class ImplCreator extends AbstractPhysicalVisitor<RecordBatch, FragmentCo
     }else{
       return super.visitScan(scan, context);  
     }
-    
+    else if (subScan instanceof ParquetRowGroupScan){
+      return parquetScan.getBatch(context, (ParquetRowGroupScan) subScan,  Collections.<RecordBatch> emptyList());
+    }
+    else{
+      return super.visitSubScan(subScan, context);
+    }
+
   }
 
-  
   @Override
   public RecordBatch visitOp(PhysicalOperator op, FragmentContext context) throws ExecutionSetupException {
     if(op instanceof SelectionVectorRemover){
       return svc.getBatch(context, (SelectionVectorRemover) op, getChildren(op, context));
     }else{
-      return super.visitOp(op, context);  
+      return super.visitOp(op, context);
     }
   }
 
@@ -97,7 +118,7 @@ public class ImplCreator extends AbstractPhysicalVisitor<RecordBatch, FragmentCo
     root = sc.getRoot(context, op, getChildren(op, context));
     return null;
   }
-  
+
   @Override
   public RecordBatch visitFilter(Filter filter, FragmentContext context) throws ExecutionSetupException {
     return fbc.getBatch(context, filter, getChildren(filter, context));
@@ -121,7 +142,7 @@ public class ImplCreator extends AbstractPhysicalVisitor<RecordBatch, FragmentCo
     }
     return children;
   }
-  
+
   public static RootExec getExec(FragmentContext context, FragmentRoot root) throws ExecutionSetupException{
     ImplCreator i = new ImplCreator();
     root.accept(i, context);

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/0a2f997f/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/OutputMutator.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/OutputMutator.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/OutputMutator.java
index 7e72683..3e9f1e2 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/OutputMutator.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/OutputMutator.java
@@ -24,5 +24,6 @@ import org.apache.drill.exec.vector.ValueVector;
 public interface OutputMutator {
   public void removeField(MaterializedField field) throws SchemaChangeException;
   public void addField(ValueVector vector) throws SchemaChangeException ;
+  public void removeAllFields();
   public void setNewSchema() throws SchemaChangeException;
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/0a2f997f/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScanBatch.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScanBatch.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScanBatch.java
index 5a543b0..4227450 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScanBatch.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScanBatch.java
@@ -161,6 +161,12 @@ public class ScanBatch implements RecordBatch {
     }
 
     @Override
+    public void removeAllFields() {
+      holder.clear();
+      fieldVectorMap.clear();
+    }
+
+    @Override
     public void setNewSchema() throws SchemaChangeException {
       ScanBatch.this.schema = this.builder.build();
       ScanBatch.this.schemaChanged = true;


[13/27] git commit: fixes for memory management and rpc throttling

Posted by ja...@apache.org.
fixes for memory management and rpc throttling


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

Branch: refs/heads/master
Commit: 402be7e04e744004beb16e9222cf649f2da6fc93
Parents: 0a2f997
Author: Jacques Nadeau <ja...@apache.org>
Authored: Thu Aug 15 08:45:20 2013 -0700
Committer: Jacques Nadeau <ja...@apache.org>
Committed: Thu Aug 15 16:59:09 2013 -0700

----------------------------------------------------------------------
 .../drill/common/logical/LogicalPlan.java       |   6 +-
 sandbox/prototype/exec/bufferl/pom.xml          |  37 +-
 .../main/java/io/netty/buffer/PoolArenaL.java   |   3 +-
 .../java/io/netty/buffer/PooledByteBufL.java    |   3 +-
 .../java/io/netty/buffer/PooledHeapBufferL.java | 282 +++++++++
 .../buffer/PooledUnsafeDirectByteBufL.java      |   3 +-
 sandbox/prototype/exec/java-exec/pom.xml        |   6 +-
 .../templates/FixedValueVectors.java            |   1 -
 .../templates/NullableValueVectors.java         |   6 +-
 .../templates/RepeatedValueVectors.java         |   8 +-
 .../templates/VariableLengthVectors.java        |   9 +-
 .../apache/drill/exec/client/DrillClient.java   |  11 +-
 .../apache/drill/exec/opt/BasicOptimizer.java   |  37 +-
 .../exec/physical/config/MockStorageEngine.java |  52 --
 .../drill/exec/physical/impl/ScanBatch.java     |   3 +
 .../drill/exec/physical/impl/ScreenCreator.java |  23 +-
 .../exec/physical/impl/SingleSenderCreator.java |   2 -
 .../exec/physical/impl/WireRecordBatch.java     |   1 +
 .../exec/record/FragmentWritableBatch.java      |   6 +-
 .../drill/exec/record/RawFragmentBatch.java     |   4 +
 .../drill/exec/record/VectorContainer.java      |   1 +
 .../apache/drill/exec/record/WritableBatch.java |   1 -
 .../exec/rpc/AbstractHandshakeHandler.java      |   2 +
 .../drill/exec/rpc/CoordinationQueue.java       |  10 +-
 .../drill/exec/rpc/DrillRpcFutureImpl.java      |   1 -
 .../apache/drill/exec/rpc/RemoteConnection.java |  22 +-
 .../java/org/apache/drill/exec/rpc/RpcBus.java  |  12 +-
 .../drill/exec/rpc/user/QueryResultBatch.java   |   6 +-
 .../drill/exec/store/AbstractStorageEngine.java |   6 +-
 .../drill/exec/store/StorageEngineRegistry.java |   2 +-
 .../exec/store/mock/MockStorageEngine.java      |  51 ++
 .../exec/store/parquet/ParquetRecordReader.java |   7 +-
 .../store/parquet/ParquetStorageEngine.java     |   5 +-
 .../exec/work/RemoteFragmentRunnerListener.java |   2 +-
 .../org/apache/drill/exec/work/WorkManager.java |   2 +-
 .../work/batch/AbstractFragmentCollector.java   |   2 +-
 .../work/batch/UnlimitedRawBatchBuffer.java     |  73 +++
 .../exec/work/batch/UnlmitedRawBatchBuffer.java |  73 ---
 .../physical/impl/TestSimpleFragmentRun.java    |   2 +-
 .../apache/drill/exec/store/MockScantTest.java  | 115 ----
 .../exec/store/ParquetRecordReaderTest.java     | 594 -------------------
 .../store/parquet/ParquetRecordReaderTest.java  | 347 +++++++++++
 .../exec/store/parquet/TestFileGenerator.java   | 210 +++++++
 .../src/test/resources/scan_screen_logical.json |   8 +-
 .../exec/java-exec/src/test/sh/logback.xml      |  35 ++
 .../prototype/exec/java-exec/src/test/sh/runbit |   4 +-
 sandbox/prototype/pom.xml                       |  10 +
 47 files changed, 1168 insertions(+), 938 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/402be7e0/sandbox/prototype/common/src/main/java/org/apache/drill/common/logical/LogicalPlan.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/logical/LogicalPlan.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/logical/LogicalPlan.java
index 742001a..6692661 100644
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/logical/LogicalPlan.java
+++ b/sandbox/prototype/common/src/main/java/org/apache/drill/common/logical/LogicalPlan.java
@@ -17,8 +17,8 @@
  ******************************************************************************/
 package org.apache.drill.common.logical;
 
-import java.io.File;
 import java.io.IOException;
+import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 
@@ -38,8 +38,6 @@ 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.google.common.base.Charsets;
-import com.google.common.io.Files;
 
 @JsonPropertyOrder({ "head", "storage", "query" })
 public class LogicalPlan {
@@ -53,7 +51,7 @@ public class LogicalPlan {
   public LogicalPlan(@JsonProperty("head") PlanProperties head,
       @JsonProperty("storage") Map<String, StorageEngineConfig> storageEngineMap,
       @JsonProperty("query") List<LogicalOperator> operators) {
-    this.storageEngineMap = storageEngineMap;
+    this.storageEngineMap = storageEngineMap != null ? storageEngineMap : new HashMap<String, StorageEngineConfig>();
     this.properties = head;
     this.graph = Graph.newGraph(operators, SinkOperator.class, SourceOperator.class);
   }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/402be7e0/sandbox/prototype/exec/bufferl/pom.xml
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/bufferl/pom.xml b/sandbox/prototype/exec/bufferl/pom.xml
index baa2c3d..11eef91 100644
--- a/sandbox/prototype/exec/bufferl/pom.xml
+++ b/sandbox/prototype/exec/bufferl/pom.xml
@@ -1,25 +1,26 @@
 <?xml version="1.0" encoding="UTF-8"?>
-<!--
-  ~ Copyright 2012 The Netty Project
-  ~
-  ~ The Netty Project licenses this file to you under the Apache License,
-  ~ version 2.0 (the "License"); you may not use this file except in compliance
-  ~ with the License. You may obtain a copy of the License at:
-  ~
-  ~   http://www.apache.org/licenses/LICENSE-2.0
-  ~
-  ~ Unless required by applicable law or agreed to in writing, software
-  ~ distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
-  ~ WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
-  ~ License for the specific language governing permissions and limitations
-  ~ under the License.
-  -->
-<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
+<!-- ~ Copyright 2012 The Netty Project ~ ~ The Netty Project licenses this 
+  file to you under the Apache License, ~ version 2.0 (the "License"); you 
+  may not use this file except in compliance ~ with the License. You may obtain 
+  a copy of the License at: ~ ~ http://www.apache.org/licenses/LICENSE-2.0 
+  ~ ~ Unless required by applicable law or agreed to in writing, software ~ 
+  distributed under the License is distributed on an "AS IS" BASIS, WITHOUT 
+  ~ WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the 
+  ~ License for the specific language governing permissions and limitations 
+  ~ under the License. -->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+  xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
 
   <modelVersion>4.0.0</modelVersion>
 
+  <parent>
+    <groupId>org.apache.drill.exec</groupId>
+    <artifactId>exec-parent</artifactId>
+    <version>1.0-SNAPSHOT</version>
+  </parent>
+
   <groupId>org.apache.drill.exec</groupId>
-  <version>4.0.3.Final</version>
+  <version>4.0.7.Final</version>
   <artifactId>netty-bufferl</artifactId>
 
   <name>Netty/Drill/Buffer</name>
@@ -31,5 +32,5 @@
       <version>${project.version}</version>
     </dependency>
   </dependencies>
-  
+
 </project>

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/402be7e0/sandbox/prototype/exec/bufferl/src/main/java/io/netty/buffer/PoolArenaL.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/bufferl/src/main/java/io/netty/buffer/PoolArenaL.java b/sandbox/prototype/exec/bufferl/src/main/java/io/netty/buffer/PoolArenaL.java
index db9818d..12fd1ae 100644
--- a/sandbox/prototype/exec/bufferl/src/main/java/io/netty/buffer/PoolArenaL.java
+++ b/sandbox/prototype/exec/bufferl/src/main/java/io/netty/buffer/PoolArenaL.java
@@ -354,8 +354,7 @@ abstract class PoolArenaL<T> {
 
         @Override
         protected PooledByteBufL<byte[]> newByteBuf(int maxCapacity) {
-          throw new UnsupportedOperationException();
-//            return PooledHeapByteBufL.newInstance(maxCapacity);
+            return PooledHeapByteBufL.newInstance(maxCapacity);
         }
 
         @Override

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/402be7e0/sandbox/prototype/exec/bufferl/src/main/java/io/netty/buffer/PooledByteBufL.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/bufferl/src/main/java/io/netty/buffer/PooledByteBufL.java b/sandbox/prototype/exec/bufferl/src/main/java/io/netty/buffer/PooledByteBufL.java
index c25c2e9..ded7c62 100644
--- a/sandbox/prototype/exec/bufferl/src/main/java/io/netty/buffer/PooledByteBufL.java
+++ b/sandbox/prototype/exec/bufferl/src/main/java/io/netty/buffer/PooledByteBufL.java
@@ -148,7 +148,7 @@ abstract class PooledByteBufL<T> extends AbstractReferenceCountedByteBuf {
             this.handle = -1;
             memory = null;
             chunk.arena.free(chunk, handle);
-            if (ResourceLeakDetector.ENABLED) {
+            if (leak != null) {
                 leak.close();
             } else {
                 recycle();
@@ -160,7 +160,6 @@ abstract class PooledByteBufL<T> extends AbstractReferenceCountedByteBuf {
     private void recycle() {
         Recycler.Handle recyclerHandle = this.recyclerHandle;
         if (recyclerHandle != null) {
-            setRefCnt(1);
             ((Recycler<Object>) recycler()).recycle(this, recyclerHandle);
         }
     }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/402be7e0/sandbox/prototype/exec/bufferl/src/main/java/io/netty/buffer/PooledHeapBufferL.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/bufferl/src/main/java/io/netty/buffer/PooledHeapBufferL.java b/sandbox/prototype/exec/bufferl/src/main/java/io/netty/buffer/PooledHeapBufferL.java
new file mode 100644
index 0000000..70b517c
--- /dev/null
+++ b/sandbox/prototype/exec/bufferl/src/main/java/io/netty/buffer/PooledHeapBufferL.java
@@ -0,0 +1,282 @@
+package io.netty.buffer;
+
+/*
+ * Copyright 2012 The Netty Project
+ *
+ * The Netty Project licenses this file tothe License at:
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations
+ * under the License.
+ */
+
+
+import io.netty.util.Recycler;
+import io.netty.util.internal.PlatformDependent;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.nio.ByteBuffer;
+import java.nio.channels.ClosedChannelException;
+import java.nio.channels.GatheringByteChannel;
+import java.nio.channels.ScatteringByteChannel;
+
+final class PooledHeapByteBufL extends PooledByteBufL<byte[]> {
+
+    private static final Recycler<PooledHeapByteBufL> RECYCLER = new Recycler<PooledHeapByteBufL>() {
+        @Override
+        protected PooledHeapByteBufL newObject(Handle handle) {
+            return new PooledHeapByteBufL(handle, 0);
+        }
+    };
+
+    static PooledHeapByteBufL newInstance(int maxCapacity) {
+        PooledHeapByteBufL buf = RECYCLER.get();
+        buf.setRefCnt(1);
+        buf.maxCapacity(maxCapacity);
+        return buf;
+    }
+
+    private PooledHeapByteBufL(Recycler.Handle recyclerHandle, int maxCapacity) {
+        super(recyclerHandle, maxCapacity);
+    }
+
+    @Override
+    public boolean isDirect() {
+        return false;
+    }
+
+    @Override
+    protected byte _getByte(int index) {
+        return memory[idx(index)];
+    }
+
+    @Override
+    protected short _getShort(int index) {
+        index = idx(index);
+        return (short) (memory[index] << 8 | memory[index + 1] & 0xFF);
+    }
+
+    @Override
+    protected int _getUnsignedMedium(int index) {
+        index = idx(index);
+        return (memory[index]     & 0xff) << 16 |
+               (memory[index + 1] & 0xff) <<  8 |
+                memory[index + 2] & 0xff;
+    }
+
+    @Override
+    protected int _getInt(int index) {
+        index = idx(index);
+        return (memory[index]     & 0xff) << 24 |
+               (memory[index + 1] & 0xff) << 16 |
+               (memory[index + 2] & 0xff) <<  8 |
+                memory[index + 3] & 0xff;
+    }
+
+    @Override
+    protected long _getLong(int index) {
+        index = idx(index);
+        return ((long) memory[index]     & 0xff) << 56 |
+               ((long) memory[index + 1] & 0xff) << 48 |
+               ((long) memory[index + 2] & 0xff) << 40 |
+               ((long) memory[index + 3] & 0xff) << 32 |
+               ((long) memory[index + 4] & 0xff) << 24 |
+               ((long) memory[index + 5] & 0xff) << 16 |
+               ((long) memory[index + 6] & 0xff) <<  8 |
+                (long) memory[index + 7] & 0xff;
+    }
+
+    @Override
+    public ByteBuf getBytes(int index, ByteBuf dst, int dstIndex, int length) {
+        checkDstIndex(index, length, dstIndex, dst.capacity());
+        if (dst.hasMemoryAddress()) {
+            PlatformDependent.copyMemory(memory, idx(index), dst.memoryAddress() + dstIndex, length);
+        } else if (dst.hasArray()) {
+            getBytes(index, dst.array(), dst.arrayOffset() + dstIndex, length);
+        } else {
+            dst.setBytes(dstIndex, memory, idx(index), length);
+        }
+        return this;
+    }
+
+    @Override
+    public ByteBuf getBytes(int index, byte[] dst, int dstIndex, int length) {
+        checkDstIndex(index, length, dstIndex, dst.length);
+        System.arraycopy(memory, idx(index), dst, dstIndex, length);
+        return this;
+    }
+
+    @Override
+    public ByteBuf getBytes(int index, ByteBuffer dst) {
+        checkIndex(index);
+        dst.put(memory, idx(index), Math.min(capacity() - index, dst.remaining()));
+        return this;
+    }
+
+    @Override
+    public ByteBuf getBytes(int index, OutputStream out, int length) throws IOException {
+        checkIndex(index, length);
+        out.write(memory, idx(index), length);
+        return this;
+    }
+
+    @Override
+    public int getBytes(int index, GatheringByteChannel out, int length) throws IOException {
+        checkIndex(index, length);
+        index = idx(index);
+        return out.write((ByteBuffer) internalNioBuffer().clear().position(index).limit(index + length));
+    }
+
+    @Override
+    protected void _setByte(int index, int value) {
+        memory[idx(index)] = (byte) value;
+    }
+
+    @Override
+    protected void _setShort(int index, int value) {
+        index = idx(index);
+        memory[index]     = (byte) (value >>> 8);
+        memory[index + 1] = (byte) value;
+    }
+
+    @Override
+    protected void _setMedium(int index, int   value) {
+        index = idx(index);
+        memory[index]     = (byte) (value >>> 16);
+        memory[index + 1] = (byte) (value >>> 8);
+        memory[index + 2] = (byte) value;
+    }
+
+    @Override
+    protected void _setInt(int index, int   value) {
+        index = idx(index);
+        memory[index]     = (byte) (value >>> 24);
+        memory[index + 1] = (byte) (value >>> 16);
+        memory[index + 2] = (byte) (value >>> 8);
+        memory[index + 3] = (byte) value;
+    }
+
+    @Override
+    protected void _setLong(int index, long  value) {
+        index = idx(index);
+        memory[index]     = (byte) (value >>> 56);
+        memory[index + 1] = (byte) (value >>> 48);
+        memory[index + 2] = (byte) (value >>> 40);
+        memory[index + 3] = (byte) (value >>> 32);
+        memory[index + 4] = (byte) (value >>> 24);
+        memory[index + 5] = (byte) (value >>> 16);
+        memory[index + 6] = (byte) (value >>> 8);
+        memory[index + 7] = (byte) value;
+    }
+
+    @Override
+    public ByteBuf setBytes(int index, ByteBuf src, int srcIndex, int length) {
+        checkSrcIndex(index, length, srcIndex, src.capacity());
+        if (src.hasMemoryAddress()) {
+            PlatformDependent.copyMemory(src.memoryAddress() + srcIndex, memory, idx(index), length);
+        } else if (src.hasArray()) {
+            setBytes(index, src.array(), src.arrayOffset() + srcIndex, length);
+        } else {
+            src.getBytes(srcIndex, memory, idx(index), length);
+        }
+        return this;
+    }
+
+    @Override
+    public ByteBuf setBytes(int index, byte[] src, int srcIndex, int length) {
+        checkSrcIndex(index, length, srcIndex, src.length);
+        System.arraycopy(src, srcIndex, memory, idx(index), length);
+        return this;
+    }
+
+    @Override
+    public ByteBuf setBytes(int index, ByteBuffer src) {
+        int length = src.remaining();
+        checkIndex(index, length);
+        src.get(memory, idx(index), length);
+        return this;
+    }
+
+    @Override
+    public int setBytes(int index, InputStream in, int length) throws IOException {
+        checkIndex(index, length);
+        return in.read(memory, idx(index), length);
+    }
+
+    @Override
+    public int setBytes(int index, ScatteringByteChannel in, int length) throws IOException {
+        checkIndex(index, length);
+        index = idx(index);
+        try {
+            return in.read((ByteBuffer) internalNioBuffer().clear().position(index).limit(index + length));
+        } catch (ClosedChannelException e) {
+            return -1;
+        }
+    }
+
+    @Override
+    public ByteBuf copy(int index, int length) {
+        checkIndex(index, length);
+        ByteBuf copy = alloc().heapBuffer(length, maxCapacity());
+        copy.writeBytes(memory, idx(index), length);
+        return copy;
+    }
+
+    @Override
+    public int nioBufferCount() {
+        return 1;
+    }
+
+    @Override
+    public ByteBuffer[] nioBuffers(int index, int length) {
+        return new ByteBuffer[] { nioBuffer(index, length) };
+    }
+
+    @Override
+    public ByteBuffer internalNioBuffer(int index, int length) {
+        checkIndex(index, length);
+        index = idx(index);
+        return (ByteBuffer) internalNioBuffer().clear().position(index).limit(index + length);
+    }
+
+    @Override
+    public boolean hasArray() {
+        return true;
+    }
+
+    @Override
+    public byte[] array() {
+        return memory;
+    }
+
+    @Override
+    public int arrayOffset() {
+        return offset;
+    }
+
+    @Override
+    public boolean hasMemoryAddress() {
+        return false;
+    }
+
+    @Override
+    public long memoryAddress() {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    protected ByteBuffer newInternalNioBuffer(byte[] memory) {
+        return ByteBuffer.wrap(memory);
+    }
+
+    @Override
+    protected Recycler<?> recycler() {
+        return RECYCLER;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/402be7e0/sandbox/prototype/exec/bufferl/src/main/java/io/netty/buffer/PooledUnsafeDirectByteBufL.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/bufferl/src/main/java/io/netty/buffer/PooledUnsafeDirectByteBufL.java b/sandbox/prototype/exec/bufferl/src/main/java/io/netty/buffer/PooledUnsafeDirectByteBufL.java
index 949f9fb..99daf62 100644
--- a/sandbox/prototype/exec/bufferl/src/main/java/io/netty/buffer/PooledUnsafeDirectByteBufL.java
+++ b/sandbox/prototype/exec/bufferl/src/main/java/io/netty/buffer/PooledUnsafeDirectByteBufL.java
@@ -30,7 +30,7 @@ import java.nio.channels.GatheringByteChannel;
 import java.nio.channels.ScatteringByteChannel;
 
 final class PooledUnsafeDirectByteBufL extends PooledByteBufL<ByteBuffer> {
-
+  
     private static final boolean NATIVE_ORDER = ByteOrder.nativeOrder() == ByteOrder.LITTLE_ENDIAN;
 
     private static final Recycler<PooledUnsafeDirectByteBufL> RECYCLER = new Recycler<PooledUnsafeDirectByteBufL>() {
@@ -42,6 +42,7 @@ final class PooledUnsafeDirectByteBufL extends PooledByteBufL<ByteBuffer> {
 
     static PooledUnsafeDirectByteBufL newInstance(int maxCapacity) {
         PooledUnsafeDirectByteBufL buf = RECYCLER.get();
+        buf.setRefCnt(1);
         buf.maxCapacity(maxCapacity);
         return buf;
     }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/402be7e0/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 cd9bc9a..a2e8501 100644
--- a/sandbox/prototype/exec/java-exec/pom.xml
+++ b/sandbox/prototype/exec/java-exec/pom.xml
@@ -65,7 +65,7 @@
     </dependency>
     <dependency>
       <groupId>org.apache.drill.exec</groupId>
-      <version>4.0.3.Final</version>
+      <version>4.0.7.Final</version>
       <artifactId>netty-bufferl</artifactId>
     </dependency>
     <dependency>
@@ -166,8 +166,8 @@
     </dependency>
     <dependency>
       <groupId>io.netty</groupId>
-      <artifactId>netty-all</artifactId>
-      <version>4.0.3.Final</version>
+      <artifactId>netty-handler</artifactId>
+      <version>4.0.7.Final</version>
     </dependency>
     <dependency>
       <groupId>com.google.protobuf</groupId>

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/402be7e0/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/FixedValueVectors.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/FixedValueVectors.java b/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/FixedValueVectors.java
index 147762e..311e715 100644
--- a/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/FixedValueVectors.java
+++ b/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/FixedValueVectors.java
@@ -61,7 +61,6 @@ public final class ${minor.class}Vector extends BaseDataValueVector implements F
   public void allocateNew(int valueCount) {
     clear();
     this.data = allocator.buffer(valueCount * ${type.width});
-    this.data.retain();
     this.data.readerIndex(0);
   }
   

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/402be7e0/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/NullableValueVectors.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/NullableValueVectors.java b/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/NullableValueVectors.java
index 483166b..ec7af46 100644
--- a/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/NullableValueVectors.java
+++ b/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/NullableValueVectors.java
@@ -10,6 +10,7 @@ import java.lang.UnsupportedOperationException;
 
 package org.apache.drill.exec.vector;
 
+
 import static com.google.common.base.Preconditions.checkArgument;
 import static com.google.common.base.Preconditions.checkState;
 import io.netty.buffer.ByteBuf;
@@ -30,6 +31,7 @@ import org.apache.drill.exec.vector.UInt2Vector;
 import org.apache.drill.exec.vector.UInt4Vector;
 
 import com.google.common.collect.Lists;
+import com.google.common.collect.ObjectArrays;
 
 /**
  * Nullable${minor.class} implements a vector of values which could be null.  Elements in the vector
@@ -59,7 +61,9 @@ public final class ${className} extends BaseValueVector implements <#if type.maj
   
   @Override
   public ByteBuf[] getBuffers() {
-    return ArrayUtils.addAll(bits.getBuffers(), values.getBuffers());
+    ByteBuf[] buffers = ObjectArrays.concat(bits.getBuffers(), values.getBuffers(), ByteBuf.class);
+    clear();
+    return buffers;
   }
   
   @Override

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/402be7e0/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/RepeatedValueVectors.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/RepeatedValueVectors.java b/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/RepeatedValueVectors.java
index c1660e8..44d036b 100644
--- a/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/RepeatedValueVectors.java
+++ b/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/RepeatedValueVectors.java
@@ -24,8 +24,11 @@ import org.apache.drill.exec.proto.UserBitShared.FieldMetadata;
 import org.apache.drill.exec.record.DeadBuf;
 import org.apache.drill.exec.record.MaterializedField;
 import org.apache.drill.exec.record.TransferPair;
+
 import java.util.List;
+
 import com.google.common.collect.Lists;
+import com.google.common.collect.ObjectArrays;
 
 @SuppressWarnings("unused")
 /**
@@ -176,8 +179,11 @@ import com.google.common.collect.Lists;
   }
   </#if>
 
+  @Override
   public ByteBuf[] getBuffers() {
-    return ArrayUtils.addAll(offsets.getBuffers(), values.getBuffers());
+    ByteBuf[] buffers = ObjectArrays.concat(offsets.getBuffers(), values.getBuffers(), ByteBuf.class);
+    clear();
+    return buffers;
   }
 
   public void clear(){

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/402be7e0/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/VariableLengthVectors.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/VariableLengthVectors.java b/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/VariableLengthVectors.java
index 7ceafe4..3be6dc2 100644
--- a/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/VariableLengthVectors.java
+++ b/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/VariableLengthVectors.java
@@ -23,6 +23,7 @@ import org.apache.drill.exec.record.MaterializedField;
 import org.apache.drill.exec.record.TransferPair;
 
 import com.google.common.base.Charsets;
+import com.google.common.collect.ObjectArrays;
 
 
 /**
@@ -74,7 +75,7 @@ public final class ${minor.class}Vector extends BaseDataValueVector implements V
   
   @Override
   public FieldMetadata getMetadata() {
-    int len = valueCount * ${type.width} + getVarByteLength();
+    int len = (valueCount+1) * ${type.width} + getVarByteLength();
     return FieldMetadata.newBuilder()
              .setDef(getField().getDef())
              .setValueCount(valueCount)
@@ -104,9 +105,12 @@ public final class ${minor.class}Vector extends BaseDataValueVector implements V
     offsetVector.clear();
   }
 
+  
   @Override
   public ByteBuf[] getBuffers() {
-    return ArrayUtils.addAll(offsetVector.getBuffers(), super.getBuffers());
+    ByteBuf[] buffers = ObjectArrays.concat(offsetVector.getBuffers(), super.getBuffers(), ByteBuf.class);
+    clear();
+    return buffers;
   }
   
   public TransferPair getTransferPair(){
@@ -151,7 +155,6 @@ public final class ${minor.class}Vector extends BaseDataValueVector implements V
     clear();
     assert totalBytes >= 0;
     data = allocator.buffer(totalBytes);
-    data.retain();
     data.readerIndex(0);
     offsetVector.allocateNew(valueCount+1);
   }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/402be7e0/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 1b49d54..cf99abd 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
@@ -34,6 +34,7 @@ import java.util.Vector;
 import org.apache.drill.common.config.DrillConfig;
 import org.apache.drill.exec.coord.ClusterCoordinator;
 import org.apache.drill.exec.coord.ZKClusterCoordinator;
+import org.apache.drill.exec.memory.DirectBufferAllocator;
 import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
 import org.apache.drill.exec.proto.UserProtos.QueryType;
 import org.apache.drill.exec.rpc.BasicClientWithConnection.ServerConnection;
@@ -58,6 +59,7 @@ public class DrillClient implements Closeable{
   private UserClient client;
   private volatile ClusterCoordinator clusterCoordinator;
   private volatile boolean connected = false;
+  private final DirectBufferAllocator allocator = new DirectBufferAllocator();
   
   public DrillClient() {
     this(DrillConfig.create());
@@ -99,8 +101,7 @@ public class DrillClient implements Closeable{
     checkState(!endpoints.isEmpty(), "No DrillbitEndpoint can be found");
     // just use the first endpoint for now
     DrillbitEndpoint endpoint = endpoints.iterator().next();
-    ByteBufAllocator bb = new PooledByteBufAllocatorL(true);
-    this.client = new UserClient(bb, new NioEventLoopGroup(1, new NamedThreadFactory("Client-")));
+    this.client = new UserClient(allocator.getUnderlyingAllocator(), new NioEventLoopGroup(1, new NamedThreadFactory("Client-")));
     try {
       logger.debug("Connecting to server {}:{}", endpoint.getAddress(), endpoint.getUserPort());
       FutureHandler f = new FutureHandler();
@@ -112,6 +113,12 @@ public class DrillClient implements Closeable{
     }
   }
 
+  
+  
+  public DirectBufferAllocator getAllocator() {
+    return allocator;
+  }
+
   /**
    * Closes this client's connection to the server
    *

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/402be7e0/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/opt/BasicOptimizer.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/opt/BasicOptimizer.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/opt/BasicOptimizer.java
index c4a7e43..b5eea03 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/opt/BasicOptimizer.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/opt/BasicOptimizer.java
@@ -13,6 +13,7 @@ import org.apache.drill.common.expression.FunctionDefinition;
 import org.apache.drill.common.expression.NoArgValidator;
 import org.apache.drill.common.expression.OutputTypeDeterminer;
 import org.apache.drill.common.logical.LogicalPlan;
+import org.apache.drill.common.logical.StorageEngineConfig;
 import org.apache.drill.common.logical.data.Filter;
 import org.apache.drill.common.logical.data.Project;
 import org.apache.drill.common.logical.data.Scan;
@@ -27,7 +28,10 @@ import org.apache.drill.exec.exception.SetupException;
 import org.apache.drill.exec.ops.QueryContext;
 import org.apache.drill.exec.physical.PhysicalPlan;
 import org.apache.drill.exec.physical.base.PhysicalOperator;
-import org.apache.drill.exec.physical.config.*;
+import org.apache.drill.exec.physical.config.MockGroupScanPOP;
+import org.apache.drill.exec.physical.config.Screen;
+import org.apache.drill.exec.physical.config.SelectionVectorRemover;
+import org.apache.drill.exec.store.StorageEngine;
 
 import com.fasterxml.jackson.core.type.TypeReference;
 
@@ -99,32 +103,15 @@ public class BasicOptimizer extends Optimizer{
 
     @Override
     public PhysicalOperator visitScan(Scan scan, Object obj) throws OptimizerException {
-      List<MockGroupScanPOP.MockScanEntry> myObjects;
-
+      StorageEngineConfig config = logicalPlan.getStorageEngineConfig(scan.getStorageEngine());
+      if(config == null) throw new OptimizerException(String.format("Logical plan referenced the storage engine config %s but the logical plan didn't have that available as a config.", scan.getStorageEngine()));
+      StorageEngine engine;
       try {
-        if (scan.getStorageEngine().equals("parquet")) {
-          return context.getStorageEngine(logicalPlan.getStorageEngineConfig(scan.getStorageEngine())).getPhysicalScan(scan);
-        }
-        if (scan.getStorageEngine().equals("local-logs")) {
-          myObjects = scan.getSelection().getListWith(config,
-              new TypeReference<ArrayList<MockGroupScanPOP.MockScanEntry>>() {
-              });
-        } else {
-          myObjects = new ArrayList<>();
-          MockGroupScanPOP.MockColumn[] cols = {
-              new MockGroupScanPOP.MockColumn("blah", MinorType.INT, DataMode.REQUIRED, 4, 4, 4),
-              new MockGroupScanPOP.MockColumn("blah_2", MinorType.INT, DataMode.REQUIRED, 4, 4, 4) };
-          myObjects.add(new MockGroupScanPOP.MockScanEntry(50, cols));
-        }
-      } catch (IOException e) {
-        throw new OptimizerException(
-            "Error reading selection attribute of GroupScan node in Logical to Physical plan conversion.", e);
-      } catch (SetupException e) {
-        throw new OptimizerException(
-            "Storage engine not found: " + scan.getStorageEngine(), e);
+        engine = context.getStorageEngine(config);
+        return engine.getPhysicalScan(scan);
+      } catch (SetupException | IOException e) {
+        throw new OptimizerException("Failure while attempting to retrieve storage engine.", e);
       }
-
-      return new MockGroupScanPOP("http://apache.org", myObjects);
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/402be7e0/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
deleted file mode 100644
index 6348686..0000000
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockStorageEngine.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.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.physical.ReadEntry;
-import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
-import org.apache.drill.exec.store.AbstractStorageEngine;
-import org.apache.drill.exec.store.RecordReader;
-
-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 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/402be7e0/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScanBatch.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScanBatch.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScanBatch.java
index 4227450..ea98c29 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScanBatch.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScanBatch.java
@@ -162,6 +162,9 @@ public class ScanBatch implements RecordBatch {
 
     @Override
     public void removeAllFields() {
+      for(VectorWrapper<?> vw : holder){
+        vw.release();
+      }
       holder.clear();
       fieldVectorMap.clear();
     }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/402be7e0/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScreenCreator.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScreenCreator.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScreenCreator.java
index 5c5e2e5..9b31407 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScreenCreator.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScreenCreator.java
@@ -17,6 +17,8 @@
  ******************************************************************************/
 package org.apache.drill.exec.physical.impl;
 
+import io.netty.buffer.ByteBuf;
+
 import java.util.List;
 
 import org.apache.drill.exec.ops.FragmentContext;
@@ -76,16 +78,16 @@ public class ScreenCreator implements RootCreator<Screen>{
       logger.debug("Screen Outcome {}", outcome);
       switch(outcome){
       case STOP: {
-          QueryResult header1 = QueryResult.newBuilder() //
+          QueryResult header = QueryResult.newBuilder() //
               .setQueryId(context.getHandle().getQueryId()) //
               .setRowCount(0) //
               .addError(ErrorHelper.logAndConvertError(context.getIdentity(), "Screen received stop request sent.", context.getFailureCause(), logger))
               .setDef(RecordBatchDef.getDefaultInstance()) //
               .setIsLastChunk(true) //
               .build();
-          QueryWritableBatch batch1 = new QueryWritableBatch(header1);
+          QueryWritableBatch batch = new QueryWritableBatch(header);
+          connection.sendResult(listener, batch);
 
-          connection.sendResult(listener, batch1);
           return false;
       }
       case NONE: {
@@ -93,16 +95,18 @@ public class ScreenCreator implements RootCreator<Screen>{
           // receive no results.
           context.batchesCompleted.inc(1);
           context.recordsCompleted.inc(incoming.getRecordCount());
-          QueryResult header2 = QueryResult.newBuilder() //
+          QueryResult header = QueryResult.newBuilder() //
               .setQueryId(context.getHandle().getQueryId()) //
               .setRowCount(0) //
               .setDef(RecordBatchDef.getDefaultInstance()) //
               .setIsLastChunk(true) //
               .build();
-          QueryWritableBatch batch2 = new QueryWritableBatch(header2);
-          connection.sendResult(listener, batch2);
+          QueryWritableBatch batch = new QueryWritableBatch(header);
+          connection.sendResult(listener, batch);
+
         }else{
-          connection.sendResult(listener, materializer.convertNext(true));
+          QueryWritableBatch batch = materializer.convertNext(true);
+          connection.sendResult(listener, batch);
         }
         return false;
       }
@@ -112,7 +116,8 @@ public class ScreenCreator implements RootCreator<Screen>{
       case OK:
         context.batchesCompleted.inc(1);
         context.recordsCompleted.inc(incoming.getRecordCount());
-        connection.sendResult(listener, materializer.convertNext(false));
+        QueryWritableBatch batch = materializer.convertNext(false);
+        connection.sendResult(listener, batch);
         return true;
       default:
         throw new UnsupportedOperationException();
@@ -128,6 +133,8 @@ public class ScreenCreator implements RootCreator<Screen>{
     
     private class SendListener extends BaseRpcOutcomeListener<Ack>{
 
+
+
       @Override
       public void failed(RpcException ex) {
         logger.error("Failure while sending data to user.", ex);

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/402be7e0/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/SingleSenderCreator.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/SingleSenderCreator.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/SingleSenderCreator.java
index a40031e..69455a1 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/SingleSenderCreator.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/SingleSenderCreator.java
@@ -75,14 +75,12 @@ public class SingleSenderCreator implements RootCreator<SingleSender>{
       case NONE:
         FragmentWritableBatch b2 = new FragmentWritableBatch(true, handle.getQueryId(), handle.getMajorFragmentId(), handle.getMinorFragmentId(), recMajor, 0, incoming.getWritableBatch());
         tunnel.sendRecordBatch(new RecordSendFailure(), context, b2);
-        b2.release();
         return false;
 
       case OK_NEW_SCHEMA:
       case OK:
         FragmentWritableBatch batch = new FragmentWritableBatch(false, handle.getQueryId(), handle.getMajorFragmentId(), handle.getMinorFragmentId(), recMajor, 0, incoming.getWritableBatch());
         tunnel.sendRecordBatch(new RecordSendFailure(), context, batch);
-        batch.release();
         return true;
 
       case NOT_YET:

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/402be7e0/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/WireRecordBatch.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/WireRecordBatch.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/WireRecordBatch.java
index 93f643d..c128504 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/WireRecordBatch.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/WireRecordBatch.java
@@ -105,6 +105,7 @@ public class WireRecordBatch implements RecordBatch{
 
       RecordBatchDef rbd = batch.getHeader().getDef();
       boolean schemaChanged = batchLoader.load(rbd, batch.getBody());
+      batch.release();
       if(schemaChanged){
         this.schema = batchLoader.getSchema();
         return IterOutcome.OK_NEW_SCHEMA;

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/402be7e0/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
index 964ef5c..3c25204 100644
--- 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
@@ -55,11 +55,7 @@ public class FragmentWritableBatch{
     return header;
   }
   
-  public void release(){
-    for(ByteBuf b : buffers){
-      b.release();
-    }
-  }
+
   
   
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/402be7e0/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/RawFragmentBatch.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/RawFragmentBatch.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/RawFragmentBatch.java
index 164bf59..217d34d 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/RawFragmentBatch.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/RawFragmentBatch.java
@@ -46,5 +46,9 @@ public class RawFragmentBatch {
   public String toString() {
     return "RawFragmentBatch [header=" + header + ", body=" + body + "]";
   }
+  
+  public void release(){
+    if(body != null) body.release();
+  }
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/402be7e0/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/VectorContainer.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/VectorContainer.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/VectorContainer.java
index 923fbd5..57aad79 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/VectorContainer.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/VectorContainer.java
@@ -70,6 +70,7 @@ public class VectorContainer implements Iterable<VectorWrapper<?>> {
     for (Iterator<VectorWrapper<?>> iter = wrappers.iterator(); iter.hasNext();) {
       VectorWrapper<?> w = iter.next();
       if (!w.isHyper() && v == w.getValueVector()) {
+        w.release();
         iter.remove();
         return;
       }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/402be7e0/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/WritableBatch.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/WritableBatch.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/WritableBatch.java
index cac042b..eb9c2c7 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/WritableBatch.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/WritableBatch.java
@@ -79,7 +79,6 @@ public class WritableBatch {
       
       for (ByteBuf b : vv.getBuffers()) {
         buffers.add(b);
-        b.retain();
       }
       // remove vv access to buffers.
       vv.clear();

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/402be7e0/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/AbstractHandshakeHandler.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/AbstractHandshakeHandler.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/AbstractHandshakeHandler.java
index 112b537..e5f2cc3 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/AbstractHandshakeHandler.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/AbstractHandshakeHandler.java
@@ -51,6 +51,8 @@ public abstract class AbstractHandshakeHandler<T extends MessageLite> extends Me
   
     T msg = parser.parseFrom(inbound.getProtobufBodyAsIS());
     consumeHandshake(ctx, msg);
+    inbound.pBody.release();
+    if(inbound.dBody != null) inbound.dBody.release();
     
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/402be7e0/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 f36530f..4b7d611 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
@@ -52,9 +52,9 @@ public class CoordinationQueue {
     }
   }
 
-  public <V> ChannelListenerWithCoordinationId get(RpcOutcomeListener<V> handler, Class<V> clazz){
+  public <V> ChannelListenerWithCoordinationId get(RpcOutcomeListener<V> handler, Class<V> clazz, RemoteConnection connection){
     int i = circularInt.getNext();
-    RpcListener<V> future = new RpcListener<V>(handler, clazz, i);
+    RpcListener<V> future = new RpcListener<V>(handler, clazz, i, connection);
     Object old = map.put(i, future);
     if (old != null)
       throw new IllegalStateException(
@@ -66,17 +66,19 @@ public class CoordinationQueue {
     final RpcOutcomeListener<T> handler;
     final Class<T> clazz;
     final int coordinationId;
+    final RemoteConnection connection;
     
-    public RpcListener(RpcOutcomeListener<T> handler, Class<T> clazz, int coordinationId) {
+    public RpcListener(RpcOutcomeListener<T> handler, Class<T> clazz, int coordinationId, RemoteConnection connection) {
       super();
       this.handler = handler;
       this.clazz = clazz;
       this.coordinationId = coordinationId;
+      this.connection = connection;
     }
 
     @Override
     public void operationComplete(ChannelFuture future) throws Exception {
-      
+      connection.releasePermit();
       if(!future.isSuccess()){
         removeFromMap(coordinationId);
         future.get();

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/402be7e0/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/DrillRpcFutureImpl.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/DrillRpcFutureImpl.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/DrillRpcFutureImpl.java
index 3b2452c..6a9d8c7 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/DrillRpcFutureImpl.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/DrillRpcFutureImpl.java
@@ -67,7 +67,6 @@ class DrillRpcFutureImpl<V> extends AbstractCheckedFuture<V, RpcException> imple
   @Override
   public void success(V value, ByteBuf buffer) {
     this.buffer = buffer;
-    if(buffer != null) buffer.retain();
     ( (InnerFuture<V>)delegate()).setValue(value);
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/402be7e0/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
index cedba10..35b5938 100644
--- 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
@@ -19,12 +19,32 @@ package org.apache.drill.exec.rpc;
 
 import io.netty.channel.Channel;
 
+import java.util.concurrent.Semaphore;
+
 public class RemoteConnection{
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(RemoteConnection.class);
   private final Channel channel;
   
-  public RemoteConnection(Channel channel) {
+  final Semaphore throttle;
+  
+  public void acquirePermit() throws InterruptedException{
+    if(RpcConstants.EXTRA_DEBUGGING) logger.debug("Acquiring send permit.");
+    this.throttle.acquire();
+    if(RpcConstants.EXTRA_DEBUGGING) logger.debug("Send permit acquired.");
+  }
+  
+  public void releasePermit() {
+    throttle.release();
+  }
+  
+  public RemoteConnection(Channel channel, int maxOutstanding) {
     super();
     this.channel = channel;
+    this.throttle  = new Semaphore(maxOutstanding);
+  }
+  
+  public RemoteConnection(Channel channel) {
+    this(channel, 100);
   }
 
 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/402be7e0/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 b08aa96..4e672b7 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
@@ -82,11 +82,15 @@ public abstract class RpcBus<T extends EnumLite, C extends RemoteConnection> imp
 
     try {
       Preconditions.checkNotNull(protobufBody);
-      ChannelListenerWithCoordinationId futureListener = queue.get(listener, clazz);
+      ChannelListenerWithCoordinationId futureListener = queue.get(listener, clazz, connection);
       OutboundRpcMessage m = new OutboundRpcMessage(RpcMode.REQUEST, rpcType, futureListener.getCoordinationId(), protobufBody, dataBodies);
+      connection.acquirePermit();
       ChannelFuture channelFuture = connection.getChannel().writeAndFlush(m);
       channelFuture.addListener(futureListener);
       completed = true;
+    } catch (InterruptedException e) {
+      completed = true;
+      listener.failed(new RpcException("Interrupted while attempting to acquire outbound queue.", e));
     } finally {
       if (!completed) {
         if (pBuffer != null) pBuffer.release();
@@ -136,8 +140,7 @@ public abstract class RpcBus<T extends EnumLite, C extends RemoteConnection> imp
       case REQUEST:
         // handle message and ack.
         Response r = handle(connection, msg.rpcType, msg.pBody, msg.dBody);
-        msg.pBody.release();
-        if(msg.dBody != null) msg.dBody.release(); // we release our ownership.  Handle could have taken over ownership.
+        msg.release();  // we release our ownership.  Handle could have taken over ownership.
         assert rpcConfig.checkResponseSend(r.rpcType, r.pBody.getClass());
         OutboundRpcMessage outMessage = new OutboundRpcMessage(RpcMode.RESPONSE, r.rpcType, msg.coordinationId,
             r.pBody, r.dBodies);
@@ -152,9 +155,8 @@ public abstract class RpcBus<T extends EnumLite, C extends RemoteConnection> imp
         RpcOutcome<?> rpcFuture = queue.getFuture(msg.rpcType, msg.coordinationId, m.getClass());
         Parser<?> parser = m.getParserForType();
         Object value = parser.parseFrom(new ByteBufInputStream(msg.pBody, msg.pBody.readableBytes()));
-        msg.pBody.release();
         rpcFuture.set(value, msg.dBody);
-        if(msg.dBody != null) msg.dBody.release();
+        msg.release();  // we release our ownership.  Handle could have taken over ownership.
         if (RpcConstants.EXTRA_DEBUGGING) logger.debug("Updated rpc future {} with value {}", rpcFuture, value);
         }catch(Exception ex){
           logger.error("Failure while handling response.", ex);

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/402be7e0/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/QueryResultBatch.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/QueryResultBatch.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/QueryResultBatch.java
index 48b0dae..c3aad22 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/QueryResultBatch.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/QueryResultBatch.java
@@ -31,7 +31,7 @@ public class QueryResultBatch {
 //    logger.debug("New Result Batch with header {} and data {}", header, data);
     this.header = header;
     this.data = data;
-    if(data != null) data.retain();
+    if(this.data != null) data.retain();
   }
 
   public QueryResult getHeader() {
@@ -47,6 +47,10 @@ public class QueryResultBatch {
     return data != null;
   }
 
+  public void release(){
+    if(data != null) data.release();
+  }
+  
   @Override
   public String toString() {
     return "QueryResultBatch [header=" + header + ", data=" + data + "]";

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/402be7e0/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 9c48052..d854789 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
@@ -31,9 +31,13 @@ import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
 import com.google.common.collect.ListMultimap;
 import com.google.common.collect.Multimap;
 
-public class AbstractStorageEngine implements StorageEngine{
+public abstract class AbstractStorageEngine implements StorageEngine{
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(AbstractStorageEngine.class);
 
+  protected AbstractStorageEngine(){
+  }
+  
+  
   @Override
   public boolean supportsRead() {
     return false;

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/402be7e0/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/StorageEngineRegistry.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/StorageEngineRegistry.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/StorageEngineRegistry.java
index 26504a2..4551c1f 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/StorageEngineRegistry.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/StorageEngineRegistry.java
@@ -51,7 +51,7 @@ public class StorageEngineRegistry {
       for(Constructor<?> c : engine.getConstructors()){
         Class<?>[] params = c.getParameterTypes();
         if(params.length != 2 || params[1] != DrillbitContext.class || !StorageEngineConfig.class.isAssignableFrom(params[0])){
-          logger.debug("Skipping StorageEngine constructor {} for engine class {} since it doesn't implement a [constructor(StorageEngineConfig, DrillbitContext)]", c, engine);
+          logger.info("Skipping StorageEngine constructor {} for engine class {} since it doesn't implement a [constructor(StorageEngineConfig, DrillbitContext)]", c, engine);
           continue;
         }
         availableEngines.put(params[0], (Constructor<? extends StorageEngine>) c);

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/402be7e0/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MockStorageEngine.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MockStorageEngine.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MockStorageEngine.java
new file mode 100644
index 0000000..23ac2b8
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MockStorageEngine.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.store.mock;
+
+import java.io.IOException;
+import java.util.ArrayList;
+
+import org.apache.drill.common.logical.data.Scan;
+import org.apache.drill.exec.physical.base.AbstractGroupScan;
+import org.apache.drill.exec.physical.config.MockGroupScanPOP;
+import org.apache.drill.exec.physical.config.MockGroupScanPOP.MockScanEntry;
+import org.apache.drill.exec.server.DrillbitContext;
+import org.apache.drill.exec.store.AbstractStorageEngine;
+import org.apache.drill.storage.MockStorageEngineConfig;
+
+import com.fasterxml.jackson.core.type.TypeReference;
+import com.fasterxml.jackson.databind.ObjectMapper;
+
+public class MockStorageEngine extends AbstractStorageEngine {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(MockStorageEngine.class);
+
+  public MockStorageEngine(MockStorageEngineConfig configuration, DrillbitContext context) {
+
+  }
+
+  @Override
+  public AbstractGroupScan getPhysicalScan(Scan scan) throws IOException {
+
+    ArrayList<MockScanEntry> readEntries = scan.getSelection().getListWith(new ObjectMapper(),
+        new TypeReference<ArrayList<MockScanEntry>>() {
+        });
+    
+    return new MockGroupScanPOP(null, readEntries);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/402be7e0/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetRecordReader.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetRecordReader.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetRecordReader.java
index 2ad7b44..2d36a08 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetRecordReader.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetRecordReader.java
@@ -60,8 +60,8 @@ public class ParquetRecordReader implements RecordReader {
   private static final long DEFAULT_BATCH_LENGTH_IN_BITS = DEFAULT_BATCH_LENGTH * 8; // 256kb
 
   // TODO - should probably find a smarter way to set this, currently 2 megabytes
-  private static final int VAR_LEN_FIELD_LENGTH = 1024 * 1024 * 2;
-  public static final int PARQUET_PAGE_MAX_SIZE = 1024 * 1024 * 5;
+  private static final int VAR_LEN_FIELD_LENGTH = 1024 * 1024 * 1;
+  public static final int PARQUET_PAGE_MAX_SIZE = 1024 * 1024 * 1;
   private static final String SEPERATOR = System.getProperty("file.separator");
 
 
@@ -398,6 +398,7 @@ public class ParquetRecordReader implements RecordReader {
   @Override
   public void cleanup() {
     columnStatuses.clear();
-    bufferWithAllData.clear();
+    this.varLengthReader.columns.clear();
+    bufferWithAllData.release();
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/402be7e0/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetStorageEngine.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetStorageEngine.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetStorageEngine.java
index f070f0f..cf5b5d8 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetStorageEngine.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetStorageEngine.java
@@ -24,21 +24,24 @@ import java.util.Collection;
 
 import com.fasterxml.jackson.core.type.TypeReference;
 import com.fasterxml.jackson.databind.ObjectMapper;
+
 import org.apache.drill.common.exceptions.ExecutionSetupException;
 import org.apache.drill.common.logical.data.Scan;
 import org.apache.drill.exec.ops.FragmentContext;
 import org.apache.drill.exec.physical.ReadEntry;
 import org.apache.drill.exec.physical.ReadEntryWithPath;
-import org.apache.drill.exec.physical.config.MockStorageEngine;
 import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
 import org.apache.drill.exec.server.DrillbitContext;
 import org.apache.drill.exec.store.AbstractStorageEngine;
 import org.apache.drill.exec.store.RecordReader;
+import org.apache.drill.exec.store.mock.MockStorageEngine;
 
 import com.google.common.collect.ListMultimap;
+
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.FileSystem;
+
 import parquet.format.converter.ParquetMetadataConverter;
 import parquet.hadoop.CodecFactoryExposer;
 import parquet.hadoop.ParquetFileReader;

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/402be7e0/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/RemoteFragmentRunnerListener.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/RemoteFragmentRunnerListener.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/RemoteFragmentRunnerListener.java
index 93f1af7..4ecbd0e 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/RemoteFragmentRunnerListener.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/RemoteFragmentRunnerListener.java
@@ -38,7 +38,7 @@ public class RemoteFragmentRunnerListener extends AbstractFragmentRunnerListener
   
   @Override
   protected void statusChange(FragmentHandle handle, FragmentStatus status) {
-    logger.debug("Sending remote failure.");
+    logger.debug("Sending remote status message. {}", status);
     tunnel.sendFragmentStatus(status);
   }
   

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/402be7e0/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/WorkManager.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/WorkManager.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/WorkManager.java
index 1170a1e..fa51b09 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/WorkManager.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/WorkManager.java
@@ -63,7 +63,7 @@ public class WorkManager implements Closeable{
   private final BitComHandler bitComWorker;
   private final UserWorker userWorker;
   private final WorkerBee bee;
-  private Executor executor = Executors.newFixedThreadPool(4, new NamedThreadFactory("Working Thread - "));
+  private Executor executor = Executors.newFixedThreadPool(4, new NamedThreadFactory("WorkManager-"));
   private final EventThread eventThread;
   
   public WorkManager(BootStrapContext context){

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/402be7e0/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/AbstractFragmentCollector.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/AbstractFragmentCollector.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/AbstractFragmentCollector.java
index dd55377..9e420c1 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/AbstractFragmentCollector.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/AbstractFragmentCollector.java
@@ -49,7 +49,7 @@ public abstract class AbstractFragmentCollector implements BatchCollector{
     this.oppositeMajorFragmentId = receiver.getOppositeMajorFragmentId();
     this.buffers = new RawBatchBuffer[minInputsRequired];
     for(int i = 0; i < buffers.length; i++){
-      buffers[i] = new UnlmitedRawBatchBuffer();
+      buffers[i] = new UnlimitedRawBatchBuffer();
     }
     if (receiver.supportsOutOfOrderExchange()) {
       this.remainingRequired = new AtomicInteger(1);

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/402be7e0/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/UnlimitedRawBatchBuffer.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/UnlimitedRawBatchBuffer.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/UnlimitedRawBatchBuffer.java
new file mode 100644
index 0000000..faff6c7
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/UnlimitedRawBatchBuffer.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 UnlimitedRawBatchBuffer implements RawBatchBuffer{
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(UnlimitedRawBatchBuffer.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 b;
+    
+  }
+
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/402be7e0/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
deleted file mode 100644
index 71ae576..0000000
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/UnlmitedRawBatchBuffer.java
+++ /dev/null
@@ -1,73 +0,0 @@
-/*******************************************************************************
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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 b;
-    
-  }
-
-  
-}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/402be7e0/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestSimpleFragmentRun.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestSimpleFragmentRun.java b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestSimpleFragmentRun.java
index db6c437..6587237 100644
--- a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestSimpleFragmentRun.java
+++ b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestSimpleFragmentRun.java
@@ -62,7 +62,7 @@ public class TestSimpleFragmentRun extends PopUnitTestBase {
     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());
+    RecordBatchLoader batchLoader = new RecordBatchLoader(client.getAllocator());
     int recordCount = 0;
     for (QueryResultBatch batch : results) {
 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/402be7e0/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/MockScantTest.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/MockScantTest.java b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/MockScantTest.java
deleted file mode 100644
index f9a1ecb..0000000
--- a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/MockScantTest.java
+++ /dev/null
@@ -1,115 +0,0 @@
-/*******************************************************************************
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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 com.beust.jcommander.internal.Lists;
-import com.google.common.base.Charsets;
-import com.google.common.io.Files;
-import org.apache.drill.common.config.DrillConfig;
-import org.apache.drill.common.types.TypeProtos;
-import org.apache.drill.common.util.FileUtils;
-import org.apache.drill.exec.client.DrillClient;
-import org.apache.drill.exec.exception.SchemaChangeException;
-import org.apache.drill.exec.physical.impl.OutputMutator;
-import org.apache.drill.exec.proto.UserProtos;
-import org.apache.drill.exec.record.MaterializedField;
-import org.apache.drill.exec.record.RecordBatchLoader;
-import org.apache.drill.exec.record.VectorWrapper;
-import org.apache.drill.exec.rpc.RpcException;
-import org.apache.drill.exec.rpc.user.QueryResultBatch;
-import org.apache.drill.exec.rpc.user.UserResultsListener;
-import org.apache.drill.exec.server.BootStrapContext;
-import org.apache.drill.exec.server.Drillbit;
-import org.apache.drill.exec.server.RemoteServiceSet;
-import org.apache.drill.exec.store.parquet.ParquetStorageEngine;
-import org.apache.drill.exec.vector.BaseDataValueVector;
-import org.apache.drill.exec.vector.ValueVector;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.junit.Test;
-import parquet.bytes.BytesInput;
-import parquet.column.ColumnDescriptor;
-import parquet.column.page.Page;
-import parquet.column.page.PageReadStore;
-import parquet.column.page.PageReader;
-import parquet.hadoop.Footer;
-import parquet.hadoop.ParquetFileWriter;
-import parquet.hadoop.metadata.CompressionCodecName;
-import parquet.hadoop.metadata.ParquetMetadata;
-import parquet.schema.MessageType;
-import parquet.schema.MessageTypeParser;
-
-import java.io.File;
-import java.io.IOException;
-import java.util.Arrays;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.CountDownLatch;
-
-import static org.junit.Assert.*;
-import static parquet.column.Encoding.PLAIN;
-
-
-public class MockScantTest {
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(StorageEngineRegistry.class);
-
-  private boolean VERBOSE_DEBUG = false;
-
-  private class ParquetResultListener implements UserResultsListener {
-
-    CountDownLatch latch = new CountDownLatch(1);
-    @Override
-    public void submissionFailed(RpcException ex) {
-      latch.countDown();
-    }
-
-    @Override
-    public void resultArrived(QueryResultBatch result) {
-      if(result.getHeader().getIsLastChunk()) latch.countDown();
-      result.getData().release(1);
-    }
-
-    public void await() throws Exception {
-      latch.await();
-    }
-  }
-
-
-  @Test
-  public void testMockScanFullEngine() throws Exception{
-    RemoteServiceSet serviceSet = RemoteServiceSet.getLocalServiceSet();
-
-    DrillConfig config = DrillConfig.create();
-
-//    try(Drillbit bit1 = new Drillbit(config, serviceSet); DrillClient client = new DrillClient(config, serviceSet.getCoordinator());){
-    try(DrillClient client = new DrillClient(config)){
-      long A = System.nanoTime();
-//      bit1.run();
-      long B = System.nanoTime();
-      client.connect();
-      long C = System.nanoTime();
-      ParquetResultListener listener = new ParquetResultListener();
-      client.runQuery(UserProtos.QueryType.PHYSICAL, Files.toString(FileUtils.getResourceAsFile("/mock-scan.json"), Charsets.UTF_8), listener);
-      listener.await();
-      long D = System.nanoTime();
-      System.out.println(String.format("Took %f s to run query", (float)(D-C) / 1E9));
-    }
-  }
-}


[06/27] git commit: Fix VarLen getBuffers template and retain

Posted by ja...@apache.org.
Fix VarLen getBuffers template and retain


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

Branch: refs/heads/master
Commit: 0a327ede37cc3e712a6bf3729f52d0563ac78275
Parents: 6f9dadb
Author: Timothy Chen <tn...@gmail.com>
Authored: Tue Aug 13 21:06:50 2013 -0700
Committer: Timothy Chen <tn...@gmail.com>
Committed: Tue Aug 13 21:06:50 2013 -0700

----------------------------------------------------------------------
 .../codegen/ValueVectors/templates/VariableLengthVectors.java     | 3 ++-
 .../java/org/apache/drill/exec/vector/BaseDataValueVector.java    | 1 +
 2 files changed, 3 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/0a327ede/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/VariableLengthVectors.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/VariableLengthVectors.java b/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/VariableLengthVectors.java
index 4492aa9..061234c 100644
--- a/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/VariableLengthVectors.java
+++ b/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/VariableLengthVectors.java
@@ -14,6 +14,7 @@ import java.io.Closeable;
 import java.nio.ByteBuffer;
 import java.util.Random;
 
+import org.apache.commons.lang3.ArrayUtils;
 import org.apache.drill.exec.memory.BufferAllocator;
 import org.apache.drill.exec.proto.SchemaDefProtos;
 import org.apache.drill.exec.proto.UserBitShared.FieldMetadata;
@@ -107,7 +108,7 @@ public final class ${minor.class}Vector extends BaseDataValueVector implements V
 
   @Override
   public ByteBuf[] getBuffers() {
-    return new ByteBuf[]{offsetVector.data, this.data};
+    return ArrayUtils.addAll(offsetVector.getBuffers(), super.getBuffers());
   }
   
   public TransferPair getTransferPair(){

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/0a327ede/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/BaseDataValueVector.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/BaseDataValueVector.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/BaseDataValueVector.java
index 1b1e39a..4bfab47 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/BaseDataValueVector.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/BaseDataValueVector.java
@@ -34,6 +34,7 @@ abstract class BaseDataValueVector extends BaseValueVector{
   public ByteBuf[] getBuffers(){
     ByteBuf[] out = new ByteBuf[]{data};
     data.readerIndex(0);
+    data.retain();
     clear();
     return out;
   }


[16/27] cleanup comparison function tests. use a single json file, and abstract out redundant code.

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/be77d5aa/sandbox/prototype/exec/java-exec/src/test/resources/functions/longEqual.json
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/resources/functions/longEqual.json b/sandbox/prototype/exec/java-exec/src/test/resources/functions/longEqual.json
deleted file mode 100644
index 0a7827b..0000000
--- a/sandbox/prototype/exec/java-exec/src/test/resources/functions/longEqual.json
+++ /dev/null
@@ -1,35 +0,0 @@
-{
-    head:{
-        type:"APACHE_DRILL_PHYSICAL",
-        version:"1",
-        generator:{
-            type:"manual"
-        }
-    },
-	graph:[
-        {
-            @id:1,
-            pop:"mock-sub-scan",
-            url: "http://apache.org",
-            entries:[
-            	{records: 100, types: [
-            	  {name: "blue", type: "INT", mode: "REQUIRED"},
-            	  {name: "red", type: "BIGINT", mode: "REQUIRED"},
-            	  {name: "yellow", type: "FLOAT8", mode: "REQUIRED"},
-            	  {name: "green", type: "INT", mode: "REQUIRED"}
-            	]}
-            ]
-        },
-        {
-            @id:2,
-            child: 1,
-            pop:"filter",
-            expr: "red == red"
-        },
-        {
-            @id: 3,
-            child: 2,
-            pop: "screen"
-        }
-    ]
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/be77d5aa/sandbox/prototype/exec/java-exec/src/test/resources/functions/longGreaterThan.json
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/resources/functions/longGreaterThan.json b/sandbox/prototype/exec/java-exec/src/test/resources/functions/longGreaterThan.json
deleted file mode 100644
index 6d2a415..0000000
--- a/sandbox/prototype/exec/java-exec/src/test/resources/functions/longGreaterThan.json
+++ /dev/null
@@ -1,35 +0,0 @@
-{
-    head:{
-        type:"APACHE_DRILL_PHYSICAL",
-        version:"1",
-        generator:{
-            type:"manual"
-        }
-    },
-	graph:[
-        {
-            @id:1,
-            pop:"mock-sub-scan",
-            url: "http://apache.org",
-            entries:[
-            	{records: 100, types: [
-            	  {name: "blue", type: "INT", mode: "REQUIRED"},
-            	  {name: "red", type: "BIGINT", mode: "REQUIRED"},
-            	  {name: "yellow", type: "FLOAT8", mode: "REQUIRED"},
-            	  {name: "green", type: "INT", mode: "REQUIRED"}
-            	]}
-            ]
-        },
-        {
-            @id:2,
-            child: 1,
-            pop:"filter",
-            expr: "red > red"
-        },
-        {
-            @id: 3,
-            child: 2,
-            pop: "screen"
-        }
-    ]
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/be77d5aa/sandbox/prototype/exec/java-exec/src/test/resources/functions/longGreaterThanEqual.json
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/resources/functions/longGreaterThanEqual.json b/sandbox/prototype/exec/java-exec/src/test/resources/functions/longGreaterThanEqual.json
deleted file mode 100644
index 7dffbe8..0000000
--- a/sandbox/prototype/exec/java-exec/src/test/resources/functions/longGreaterThanEqual.json
+++ /dev/null
@@ -1,35 +0,0 @@
-{
-    head:{
-        type:"APACHE_DRILL_PHYSICAL",
-        version:"1",
-        generator:{
-            type:"manual"
-        }
-    },
-	graph:[
-        {
-            @id:1,
-            pop:"mock-sub-scan",
-            url: "http://apache.org",
-            entries:[
-            	{records: 100, types: [
-            	  {name: "blue", type: "INT", mode: "REQUIRED"},
-            	  {name: "red", type: "BIGINT", mode: "REQUIRED"},
-            	  {name: "yellow", type: "FLOAT8", mode: "REQUIRED"},
-            	  {name: "green", type: "INT", mode: "REQUIRED"}
-            	]}
-            ]
-        },
-        {
-            @id:2,
-            child: 1,
-            pop:"filter",
-            expr: "red >= red"
-        },
-        {
-            @id: 3,
-            child: 2,
-            pop: "screen"
-        }
-    ]
-}
\ No newline at end of file

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

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/be77d5aa/sandbox/prototype/exec/java-exec/src/test/resources/functions/longLessThanEqual.json
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/resources/functions/longLessThanEqual.json b/sandbox/prototype/exec/java-exec/src/test/resources/functions/longLessThanEqual.json
deleted file mode 100644
index 5900f73..0000000
--- a/sandbox/prototype/exec/java-exec/src/test/resources/functions/longLessThanEqual.json
+++ /dev/null
@@ -1,35 +0,0 @@
-{
-    head:{
-        type:"APACHE_DRILL_PHYSICAL",
-        version:"1",
-        generator:{
-            type:"manual"
-        }
-    },
-	graph:[
-        {
-            @id:1,
-            pop:"mock-sub-scan",
-            url: "http://apache.org",
-            entries:[
-            	{records: 100, types: [
-            	  {name: "blue", type: "INT", mode: "REQUIRED"},
-            	  {name: "red", type: "BIGINT", mode: "REQUIRED"},
-            	  {name: "yellow", type: "FLOAT8", mode: "REQUIRED"},
-            	  {name: "green", type: "INT", mode: "REQUIRED"}
-            	]}
-            ]
-        },
-        {
-            @id:2,
-            child: 1,
-            pop:"filter",
-            expr: "red <= red"
-        },
-        {
-            @id: 3,
-            child: 2,
-            pop: "screen"
-        }
-    ]
-}
\ No newline at end of file

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

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/be77d5aa/sandbox/prototype/exec/java-exec/src/test/resources/functions/nullableBigIntEqual.json
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/resources/functions/nullableBigIntEqual.json b/sandbox/prototype/exec/java-exec/src/test/resources/functions/nullableBigIntEqual.json
deleted file mode 100644
index 212c73a..0000000
--- a/sandbox/prototype/exec/java-exec/src/test/resources/functions/nullableBigIntEqual.json
+++ /dev/null
@@ -1,35 +0,0 @@
-{
-    head:{
-        type:"APACHE_DRILL_PHYSICAL",
-        version:"1",
-        generator:{
-            type:"manual"
-        }
-    },
-	graph:[
-        {
-            @id:1,
-            pop:"mock-sub-scan",
-            url: "http://apache.org",
-            entries:[
-            	{records: 100, types: [
-            	  {name: "blue", type: "INT", mode: "OPTIONAL"},
-            	  {name: "red", type: "BIGINT", mode: "OPTIONAL"},
-            	  {name: "yellow", type: "FLOAT8", mode: "REQUIRED"},
-            	  {name: "green", type: "INT", mode: "REQUIRED"}
-            	]}
-            ]
-        },
-        {
-            @id:2,
-            child: 1,
-            pop:"filter",
-            expr: "red == red"
-        },
-        {
-            @id: 3,
-            child: 2,
-            pop: "screen"
-        }
-    ]
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/be77d5aa/sandbox/prototype/exec/java-exec/src/test/resources/functions/nullableBigIntGreaterThan.json
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/resources/functions/nullableBigIntGreaterThan.json b/sandbox/prototype/exec/java-exec/src/test/resources/functions/nullableBigIntGreaterThan.json
deleted file mode 100644
index c1d42fa..0000000
--- a/sandbox/prototype/exec/java-exec/src/test/resources/functions/nullableBigIntGreaterThan.json
+++ /dev/null
@@ -1,35 +0,0 @@
-{
-    head:{
-        type:"APACHE_DRILL_PHYSICAL",
-        version:"1",
-        generator:{
-            type:"manual"
-        }
-    },
-	graph:[
-        {
-            @id:1,
-            pop:"mock-sub-scan",
-            url: "http://apache.org",
-            entries:[
-            	{records: 100, types: [
-            	  {name: "blue", type: "INT", mode: "OPTIONAL"},
-            	  {name: "red", type: "BIGINT", mode: "OPTIONAL"},
-            	  {name: "yellow", type: "FLOAT8", mode: "REQUIRED"},
-            	  {name: "green", type: "INT", mode: "OPTIONAL"}
-            	]}
-            ]
-        },
-        {
-            @id:2,
-            child: 1,
-            pop:"filter",
-            expr: "red > red"
-        },
-        {
-            @id: 3,
-            child: 2,
-            pop: "screen"
-        }
-    ]
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/be77d5aa/sandbox/prototype/exec/java-exec/src/test/resources/functions/nullableBigIntGreaterThanEqual.json
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/resources/functions/nullableBigIntGreaterThanEqual.json b/sandbox/prototype/exec/java-exec/src/test/resources/functions/nullableBigIntGreaterThanEqual.json
deleted file mode 100644
index c560d0b..0000000
--- a/sandbox/prototype/exec/java-exec/src/test/resources/functions/nullableBigIntGreaterThanEqual.json
+++ /dev/null
@@ -1,35 +0,0 @@
-{
-    head:{
-        type:"APACHE_DRILL_PHYSICAL",
-        version:"1",
-        generator:{
-            type:"manual"
-        }
-    },
-	graph:[
-        {
-            @id:1,
-            pop:"mock-sub-scan",
-            url: "http://apache.org",
-            entries:[
-            	{records: 100, types: [
-            	  {name: "blue", type: "INT", mode: "OPTIONAL"},
-            	  {name: "red", type: "BIGINT", mode: "OPTIONAL"},
-            	  {name: "yellow", type: "FLOAT8", mode: "REQUIRED"},
-            	  {name: "green", type: "INT", mode: "OPTIONAL"}
-            	]}
-            ]
-        },
-        {
-            @id:2,
-            child: 1,
-            pop:"filter",
-            expr: "red >= red"
-        },
-        {
-            @id: 3,
-            child: 2,
-            pop: "screen"
-        }
-    ]
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/be77d5aa/sandbox/prototype/exec/java-exec/src/test/resources/functions/nullableBigIntLessThan.json
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/resources/functions/nullableBigIntLessThan.json b/sandbox/prototype/exec/java-exec/src/test/resources/functions/nullableBigIntLessThan.json
deleted file mode 100644
index 2cf2869..0000000
--- a/sandbox/prototype/exec/java-exec/src/test/resources/functions/nullableBigIntLessThan.json
+++ /dev/null
@@ -1,35 +0,0 @@
-{
-    head:{
-        type:"APACHE_DRILL_PHYSICAL",
-        version:"1",
-        generator:{
-            type:"manual"
-        }
-    },
-	graph:[
-        {
-            @id:1,
-            pop:"mock-sub-scan",
-            url: "http://apache.org",
-            entries:[
-            	{records: 100, types: [
-            	  {name: "blue", type: "INT", mode: "OPTIONAL"},
-            	  {name: "red", type: "BIGINT", mode: "OPTIONAL"},
-            	  {name: "yellow", type: "FLOAT8", mode: "REQUIRED"},
-            	  {name: "green", type: "INT", mode: "OPTIONAL"}
-            	]}
-            ]
-        },
-        {
-            @id:2,
-            child: 1,
-            pop:"filter",
-            expr: "red < (red + 1)"
-        },
-        {
-            @id: 3,
-            child: 2,
-            pop: "screen"
-        }
-    ]
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/be77d5aa/sandbox/prototype/exec/java-exec/src/test/resources/functions/nullableBigIntLessThanEqual.json
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/resources/functions/nullableBigIntLessThanEqual.json b/sandbox/prototype/exec/java-exec/src/test/resources/functions/nullableBigIntLessThanEqual.json
deleted file mode 100644
index 16b19d3..0000000
--- a/sandbox/prototype/exec/java-exec/src/test/resources/functions/nullableBigIntLessThanEqual.json
+++ /dev/null
@@ -1,35 +0,0 @@
-{
-    head:{
-        type:"APACHE_DRILL_PHYSICAL",
-        version:"1",
-        generator:{
-            type:"manual"
-        }
-    },
-	graph:[
-        {
-            @id:1,
-            pop:"mock-sub-scan",
-            url: "http://apache.org",
-            entries:[
-            	{records: 100, types: [
-            	  {name: "blue", type: "INT", mode: "OPTIONAL"},
-            	  {name: "red", type: "BIGINT", mode: "OPTIONAL"},
-            	  {name: "yellow", type: "FLOAT8", mode: "REQUIRED"},
-            	  {name: "green", type: "INT", mode: "OPTIONAL"}
-            	]}
-            ]
-        },
-        {
-            @id:2,
-            child: 1,
-            pop:"filter",
-            expr: "red <= (red + 1)"
-        },
-        {
-            @id: 3,
-            child: 2,
-            pop: "screen"
-        }
-    ]
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/be77d5aa/sandbox/prototype/exec/java-exec/src/test/resources/functions/nullableBigIntNotEqual.json
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/resources/functions/nullableBigIntNotEqual.json b/sandbox/prototype/exec/java-exec/src/test/resources/functions/nullableBigIntNotEqual.json
deleted file mode 100644
index 2e9778a..0000000
--- a/sandbox/prototype/exec/java-exec/src/test/resources/functions/nullableBigIntNotEqual.json
+++ /dev/null
@@ -1,35 +0,0 @@
-{
-    head:{
-        type:"APACHE_DRILL_PHYSICAL",
-        version:"1",
-        generator:{
-            type:"manual"
-        }
-    },
-	graph:[
-        {
-            @id:1,
-            pop:"mock-sub-scan",
-            url: "http://apache.org",
-            entries:[
-            	{records: 100, types: [
-            	  {name: "blue", type: "INT", mode: "OPTIONAL"},
-            	  {name: "red", type: "BIGINT", mode: "OPTIONAL"},
-            	  {name: "yellow", type: "FLOAT8", mode: "REQUIRED"},
-            	  {name: "green", type: "BIGINT", mode: "OPTIONAL"}
-            	]}
-            ]
-        },
-        {
-            @id:2,
-            child: 1,
-            pop:"filter",
-            expr: "red != (green + red)"
-        },
-        {
-            @id: 3,
-            child: 2,
-            pop: "screen"
-        }
-    ]
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/be77d5aa/sandbox/prototype/exec/java-exec/src/test/resources/functions/nullableIntEqual.json
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/resources/functions/nullableIntEqual.json b/sandbox/prototype/exec/java-exec/src/test/resources/functions/nullableIntEqual.json
deleted file mode 100644
index ba9cba8..0000000
--- a/sandbox/prototype/exec/java-exec/src/test/resources/functions/nullableIntEqual.json
+++ /dev/null
@@ -1,35 +0,0 @@
-{
-    head:{
-        type:"APACHE_DRILL_PHYSICAL",
-        version:"1",
-        generator:{
-            type:"manual"
-        }
-    },
-	graph:[
-        {
-            @id:1,
-            pop:"mock-sub-scan",
-            url: "http://apache.org",
-            entries:[
-            	{records: 100, types: [
-            	  {name: "blue", type: "INT", mode: "OPTIONAL"},
-            	  {name: "red", type: "BIGINT", mode: "REQUIRED"},
-            	  {name: "yellow", type: "FLOAT8", mode: "REQUIRED"},
-            	  {name: "green", type: "INT", mode: "REQUIRED"}
-            	]}
-            ]
-        },
-        {
-            @id:2,
-            child: 1,
-            pop:"filter",
-            expr: "blue == blue"
-        },
-        {
-            @id: 3,
-            child: 2,
-            pop: "screen"
-        }
-    ]
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/be77d5aa/sandbox/prototype/exec/java-exec/src/test/resources/functions/nullableIntGreaterThan.json
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/resources/functions/nullableIntGreaterThan.json b/sandbox/prototype/exec/java-exec/src/test/resources/functions/nullableIntGreaterThan.json
deleted file mode 100644
index 6a3fc33..0000000
--- a/sandbox/prototype/exec/java-exec/src/test/resources/functions/nullableIntGreaterThan.json
+++ /dev/null
@@ -1,35 +0,0 @@
-{
-    head:{
-        type:"APACHE_DRILL_PHYSICAL",
-        version:"1",
-        generator:{
-            type:"manual"
-        }
-    },
-	graph:[
-        {
-            @id:1,
-            pop:"mock-sub-scan",
-            url: "http://apache.org",
-            entries:[
-            	{records: 100, types: [
-            	  {name: "blue", type: "INT", mode: "OPTIONAL"},
-            	  {name: "red", type: "BIGINT", mode: "OPTIONAL"},
-            	  {name: "yellow", type: "FLOAT8", mode: "REQUIRED"},
-            	  {name: "green", type: "INT", mode: "OPTIONAL"}
-            	]}
-            ]
-        },
-        {
-            @id:2,
-            child: 1,
-            pop:"filter",
-            expr: "blue > blue"
-        },
-        {
-            @id: 3,
-            child: 2,
-            pop: "screen"
-        }
-    ]
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/be77d5aa/sandbox/prototype/exec/java-exec/src/test/resources/functions/nullableIntGreaterThanEqual.json
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/resources/functions/nullableIntGreaterThanEqual.json b/sandbox/prototype/exec/java-exec/src/test/resources/functions/nullableIntGreaterThanEqual.json
deleted file mode 100644
index daa34f9..0000000
--- a/sandbox/prototype/exec/java-exec/src/test/resources/functions/nullableIntGreaterThanEqual.json
+++ /dev/null
@@ -1,35 +0,0 @@
-{
-    head:{
-        type:"APACHE_DRILL_PHYSICAL",
-        version:"1",
-        generator:{
-            type:"manual"
-        }
-    },
-	graph:[
-        {
-            @id:1,
-            pop:"mock-sub-scan",
-            url: "http://apache.org",
-            entries:[
-            	{records: 100, types: [
-            	  {name: "blue", type: "INT", mode: "OPTIONAL"},
-            	  {name: "red", type: "BIGINT", mode: "OPTIONAL"},
-            	  {name: "yellow", type: "FLOAT8", mode: "REQUIRED"},
-            	  {name: "green", type: "INT", mode: "OPTIONAL"}
-            	]}
-            ]
-        },
-        {
-            @id:2,
-            child: 1,
-            pop:"filter",
-            expr: "blue >= blue"
-        },
-        {
-            @id: 3,
-            child: 2,
-            pop: "screen"
-        }
-    ]
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/be77d5aa/sandbox/prototype/exec/java-exec/src/test/resources/functions/nullableIntLessThan.json
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/resources/functions/nullableIntLessThan.json b/sandbox/prototype/exec/java-exec/src/test/resources/functions/nullableIntLessThan.json
deleted file mode 100644
index 0186e17..0000000
--- a/sandbox/prototype/exec/java-exec/src/test/resources/functions/nullableIntLessThan.json
+++ /dev/null
@@ -1,35 +0,0 @@
-{
-    head:{
-        type:"APACHE_DRILL_PHYSICAL",
-        version:"1",
-        generator:{
-            type:"manual"
-        }
-    },
-	graph:[
-        {
-            @id:1,
-            pop:"mock-sub-scan",
-            url: "http://apache.org",
-            entries:[
-            	{records: 100, types: [
-            	  {name: "blue", type: "INT", mode: "OPTIONAL"},
-            	  {name: "red", type: "BIGINT", mode: "OPTIONAL"},
-            	  {name: "yellow", type: "FLOAT8", mode: "REQUIRED"},
-            	  {name: "green", type: "INT", mode: "OPTIONAL"}
-            	]}
-            ]
-        },
-        {
-            @id:2,
-            child: 1,
-            pop:"filter",
-            expr: "blue < blue"
-        },
-        {
-            @id: 3,
-            child: 2,
-            pop: "screen"
-        }
-    ]
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/be77d5aa/sandbox/prototype/exec/java-exec/src/test/resources/functions/nullableIntLessThanEqual.json
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/resources/functions/nullableIntLessThanEqual.json b/sandbox/prototype/exec/java-exec/src/test/resources/functions/nullableIntLessThanEqual.json
deleted file mode 100644
index 2ee40eb..0000000
--- a/sandbox/prototype/exec/java-exec/src/test/resources/functions/nullableIntLessThanEqual.json
+++ /dev/null
@@ -1,35 +0,0 @@
-{
-    head:{
-        type:"APACHE_DRILL_PHYSICAL",
-        version:"1",
-        generator:{
-            type:"manual"
-        }
-    },
-	graph:[
-        {
-            @id:1,
-            pop:"mock-sub-scan",
-            url: "http://apache.org",
-            entries:[
-            	{records: 100, types: [
-            	  {name: "blue", type: "INT", mode: "OPTIONAL"},
-            	  {name: "red", type: "BIGINT", mode: "OPTIONAL"},
-            	  {name: "yellow", type: "FLOAT8", mode: "REQUIRED"},
-            	  {name: "green", type: "INT", mode: "OPTIONAL"}
-            	]}
-            ]
-        },
-        {
-            @id:2,
-            child: 1,
-            pop:"filter",
-            expr: "blue <= blue"
-        },
-        {
-            @id: 3,
-            child: 2,
-            pop: "screen"
-        }
-    ]
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/be77d5aa/sandbox/prototype/exec/java-exec/src/test/resources/functions/nullableIntNotEqual.json
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/resources/functions/nullableIntNotEqual.json b/sandbox/prototype/exec/java-exec/src/test/resources/functions/nullableIntNotEqual.json
deleted file mode 100644
index 494f938..0000000
--- a/sandbox/prototype/exec/java-exec/src/test/resources/functions/nullableIntNotEqual.json
+++ /dev/null
@@ -1,35 +0,0 @@
-{
-    head:{
-        type:"APACHE_DRILL_PHYSICAL",
-        version:"1",
-        generator:{
-            type:"manual"
-        }
-    },
-	graph:[
-        {
-            @id:1,
-            pop:"mock-sub-scan",
-            url: "http://apache.org",
-            entries:[
-            	{records: 100, types: [
-            	  {name: "blue", type: "INT", mode: "OPTIONAL"},
-            	  {name: "red", type: "BIGINT", mode: "OPTIONAL"},
-            	  {name: "yellow", type: "FLOAT8", mode: "REQUIRED"},
-            	  {name: "green", type: "INT", mode: "OPTIONAL"}
-            	]}
-            ]
-        },
-        {
-            @id:2,
-            child: 1,
-            pop:"filter",
-            expr: "blue != (green + blue)"
-        },
-        {
-            @id: 3,
-            child: 2,
-            pop: "screen"
-        }
-    ]
-}
\ No newline at end of file


[03/27] git commit: Fix value vectors for e2e scan json pop

Posted by ja...@apache.org.
Fix value vectors for e2e scan json pop


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

Branch: refs/heads/master
Commit: a15f5b19c7362793590d670ba1b932f6f8dfda69
Parents: 38ab96f
Author: Timothy Chen <tn...@gmail.com>
Authored: Sat Aug 10 01:32:13 2013 -0700
Committer: Timothy Chen <tn...@gmail.com>
Committed: Sat Aug 10 01:32:13 2013 -0700

----------------------------------------------------------------------
 .../templates/NullableValueVectors.java         |  5 +++++
 .../templates/RepeatedValueVectors.java         | 12 +++++-----
 .../apache/drill/exec/store/VectorHolder.java   |  8 ++++++-
 .../drill/exec/vector/RepeatedMutator.java      | 23 ++++++++++++++++++++
 .../physical/impl/TestSimpleFragmentRun.java    |  7 ++++--
 .../resources/physical_json_scan_test1.json     |  2 +-
 6 files changed, 46 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/a15f5b19/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/NullableValueVectors.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/NullableValueVectors.java b/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/NullableValueVectors.java
index ca222df..976c984 100644
--- a/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/NullableValueVectors.java
+++ b/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/NullableValueVectors.java
@@ -58,7 +58,12 @@ public final class ${className} extends BaseValueVector implements <#if type.maj
   
   @Override
   public ByteBuf[] getBuffers() {
+    <#if type.major == "VarLen">
+    ByteBuf[] valueBuffers = values.getBuffers();
+    return new ByteBuf[]{bits.data, valueBuffers[0], valueBuffers[1]};
+    <#else>
     return new ByteBuf[]{bits.data, values.data};
+    </#if>
   }
   
   @Override

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/a15f5b19/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/RepeatedValueVectors.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/RepeatedValueVectors.java b/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/RepeatedValueVectors.java
index f4a7049..5def096 100644
--- a/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/RepeatedValueVectors.java
+++ b/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/RepeatedValueVectors.java
@@ -249,12 +249,16 @@ import com.google.common.collect.Lists;
     }
   }
   
-  public final class Mutator implements ValueVector.Mutator {
+  public final class Mutator implements RepeatedMutator {
 
     
     private Mutator(){
     }
 
+    public void startNewGroup(int index) {
+      offsets.getMutator().set(index+1, offsets.getAccessor().get(index));
+    }
+
     /**
      * Add an element to the given record index.  This is similar to the set() method in other
      * value vectors, except that it permits setting multiple values for a single record.
@@ -264,18 +268,12 @@ import com.google.common.collect.Lists;
      */
     public void add(int index, <#if type.major == "VarLen">byte[]<#elseif (type.width < 4)>int<#else>${minor.javaType!type.javaType}</#if> value) {
       int nextOffset = offsets.getAccessor().get(index+1);
-      if (index > 0 && nextOffset == 0) {
-        nextOffset = offsets.getAccessor().get(index);
-      }
       values.getMutator().set(nextOffset, value);
       offsets.getMutator().set(index+1, nextOffset+1);
     }
 
     public void add(int index, ${minor.class}Holder holder){
       int nextOffset = offsets.getAccessor().get(index+1);
-      if (index > 0 && nextOffset == 0) {
-        nextOffset = offsets.getAccessor().get(index);
-      }
       values.getMutator().set(nextOffset, holder);
       offsets.getMutator().set(index+1, nextOffset+1);
     }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/a15f5b19/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/VectorHolder.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/VectorHolder.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/VectorHolder.java
index 2c28082..be0bea8 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/VectorHolder.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/VectorHolder.java
@@ -45,7 +45,13 @@ public class VectorHolder {
   }
 
   public void setGroupCount(int groupCount) {
-    this.groupCount = groupCount;
+    if(this.groupCount < groupCount) {
+      RepeatedMutator mutator = (RepeatedMutator) vector.getMutator();
+      while(this.groupCount < groupCount) {
+        mutator.startNewGroup(this.groupCount + 1);
+        this.groupCount++;
+      }
+    }
   }
 
   public boolean hasEnoughSpace(int newLength) {

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/a15f5b19/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/RepeatedMutator.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/RepeatedMutator.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/RepeatedMutator.java
new file mode 100644
index 0000000..3e81d75
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/RepeatedMutator.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.vector;
+
+public interface RepeatedMutator extends ValueVector.Mutator {
+  public void startNewGroup(int index);
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/a15f5b19/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestSimpleFragmentRun.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestSimpleFragmentRun.java b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestSimpleFragmentRun.java
index cabe9b3..db6c437 100644
--- a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestSimpleFragmentRun.java
+++ b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestSimpleFragmentRun.java
@@ -116,7 +116,10 @@ public class TestSimpleFragmentRun extends PopUnitTestBase {
       // run query.
       bit.run();
       client.connect();
-      List<QueryResultBatch> results = client.runQuery(QueryType.PHYSICAL, Files.toString(FileUtils.getResourceAsFile("/physical_json_scan_test1.json"), Charsets.UTF_8));
+      List<QueryResultBatch> results = client.runQuery(QueryType.PHYSICAL,
+          Files.toString(FileUtils.getResourceAsFile("/physical_json_scan_test1.json"), Charsets.UTF_8)
+              .replace("#{TEST_FILE}", FileUtils.getResourceAsFile("/scan_json_test_1.json").toURI().toString())
+      );
 
       // look at records
       RecordBatchLoader batchLoader = new RecordBatchLoader(bit.getContext().getAllocator());
@@ -156,7 +159,7 @@ public class TestSimpleFragmentRun extends PopUnitTestBase {
         System.out.println();
 
 
-        for (int r = 0; i < batchLoader.getRecordCount(); r++) {
+        for (int r = 0; r < batchLoader.getRecordCount(); r++) {
           boolean first = true;
           recordCount++;
           for (VectorWrapper<?> v : batchLoader) {

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/a15f5b19/sandbox/prototype/exec/java-exec/src/test/resources/physical_json_scan_test1.json
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/resources/physical_json_scan_test1.json b/sandbox/prototype/exec/java-exec/src/test/resources/physical_json_scan_test1.json
index 91eb80c..6f08937 100644
--- a/sandbox/prototype/exec/java-exec/src/test/resources/physical_json_scan_test1.json
+++ b/sandbox/prototype/exec/java-exec/src/test/resources/physical_json_scan_test1.json
@@ -11,7 +11,7 @@
             @id:1,
             pop:"json-scan",
             entries:[
-            	{url: "file:////home/tnachen/src/incubator-drill/sandbox/prototype/exec/java-exec/src/test/resources/scan_json_test_1.json"}
+            	{url: "#{TEST_FILE}"}
             ]
         },
         {


[14/27] Updates to add subscan support to JSON

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/2884db7a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/json/JSONSubScan.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/json/JSONSubScan.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/json/JSONSubScan.java
new file mode 100644
index 0000000..fe16b3a
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/json/JSONSubScan.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.store.json;
+
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+
+import org.apache.drill.exec.physical.OperatorCost;
+import org.apache.drill.exec.physical.base.AbstractBase;
+import org.apache.drill.exec.physical.base.PhysicalOperator;
+import org.apache.drill.exec.physical.base.PhysicalVisitor;
+import org.apache.drill.exec.physical.base.Size;
+import org.apache.drill.exec.physical.base.SubScan;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.annotation.JsonTypeName;
+import com.google.common.collect.Iterators;
+
+@JsonTypeName("json-sub-scan")
+public class JSONSubScan extends AbstractBase implements SubScan{
+
+    protected final List<JSONGroupScan.ScanEntry> readEntries;
+    private final OperatorCost cost;
+    private final Size size;
+    
+    @JsonCreator
+    public JSONSubScan(@JsonProperty("entries") List<JSONGroupScan.ScanEntry> readEntries) {
+        this.readEntries = readEntries;
+        OperatorCost cost = new OperatorCost(0,0,0,0);
+        Size size = new Size(0,0);
+        for(JSONGroupScan.ScanEntry r : readEntries){
+          cost = cost.add(r.getCost());
+          size = size.add(r.getSize());
+        }
+        this.cost = cost;
+        this.size = size;
+    }
+
+    public List<JSONGroupScan.ScanEntry> getReadEntries() {
+      return readEntries;
+    }
+
+    @Override
+    public PhysicalOperator getNewWithChildren(List<PhysicalOperator> children) {
+        return new JSONSubScan(readEntries);
+    }
+
+    @Override
+    public OperatorCost getCost() {
+      return cost;
+    }
+
+    @Override
+    public Size getSize() {
+      return size;
+    }
+
+    @Override
+    public <T, X, E extends Throwable> T accept(PhysicalVisitor<T, X, E> physicalVisitor, X value) throws E {
+      return physicalVisitor.visitSubScan(this, value);
+    }
+
+    @Override
+    public Iterator<PhysicalOperator> iterator() {
+      return Iterators.emptyIterator();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/2884db7a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MockGroupScanPOP.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MockGroupScanPOP.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MockGroupScanPOP.java
new file mode 100644
index 0000000..d5f1d8f
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MockGroupScanPOP.java
@@ -0,0 +1,221 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.mock;
+
+import java.util.Collections;
+import java.util.LinkedList;
+import java.util.List;
+
+import org.apache.drill.common.types.TypeProtos.DataMode;
+import org.apache.drill.common.types.TypeProtos.MajorType;
+import org.apache.drill.common.types.TypeProtos.MinorType;
+import org.apache.drill.exec.physical.EndpointAffinity;
+import org.apache.drill.exec.physical.OperatorCost;
+import org.apache.drill.exec.physical.ReadEntry;
+import org.apache.drill.exec.physical.base.*;
+import org.apache.drill.exec.physical.base.AbstractGroupScan;
+import org.apache.drill.exec.physical.base.GroupScan;
+import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
+import org.apache.drill.exec.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 MockGroupScanPOP extends AbstractGroupScan {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(MockGroupScanPOP.class);
+
+  private final String url;
+  protected final List<MockScanEntry> readEntries;
+  private final OperatorCost cost;
+  private final Size size;
+  private  LinkedList<MockScanEntry>[] mappings;
+
+  @JsonCreator
+  public MockGroupScanPOP(@JsonProperty("url") String url, @JsonProperty("entries") List<MockScanEntry> readEntries) {
+    this.readEntries = readEntries;
+    OperatorCost cost = new OperatorCost(0,0,0,0);
+    Size size = new Size(0,0);
+    for(MockScanEntry r : readEntries){
+      cost = cost.add(r.getCost());
+      size = size.add(r.getSize());
+    }
+    this.cost = cost;
+    this.size = size;
+    this.url = url;
+  }
+
+  public String getUrl() {
+    return url;
+  }
+
+  @JsonProperty("entries")
+  public List<MockScanEntry> getReadEntries() {
+    return readEntries;
+  }
+  
+  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 SubScan 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 MockSubScanPOP(url, mappings[minorFragmentId]);
+  }
+
+  @Override
+  public int getMaxParallelizationWidth() {
+    return readEntries.size();
+  }
+
+  @Override
+  public OperatorCost getCost() {
+    return cost;
+  }
+
+  @Override
+  public Size getSize() {
+    return size;
+  }
+
+  @Override
+  @JsonIgnore
+  public PhysicalOperator getNewWithChildren(List<PhysicalOperator> children) {
+    Preconditions.checkArgument(children.isEmpty());
+    return new MockGroupScanPOP(url, readEntries);
+
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/2884db7a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MockRecordReader.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MockRecordReader.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MockRecordReader.java
new file mode 100644
index 0000000..024aa21
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MockRecordReader.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.store.mock;
+
+import org.apache.drill.common.exceptions.ExecutionSetupException;
+import org.apache.drill.common.expression.ExpressionPosition;
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.common.types.TypeProtos.MajorType;
+import org.apache.drill.exec.exception.SchemaChangeException;
+import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.physical.impl.OutputMutator;
+import org.apache.drill.exec.record.MaterializedField;
+import org.apache.drill.exec.store.RecordReader;
+import org.apache.drill.exec.store.mock.MockGroupScanPOP.MockColumn;
+import org.apache.drill.exec.store.mock.MockGroupScanPOP.MockScanEntry;
+import org.apache.drill.exec.vector.AllocationHelper;
+import org.apache.drill.exec.vector.TypeHelper;
+import org.apache.drill.exec.vector.ValueVector;
+
+public class MockRecordReader implements RecordReader {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(MockRecordReader.class);
+
+  private OutputMutator output;
+  private MockScanEntry config;
+  private FragmentContext context;
+  private ValueVector[] valueVectors;
+  private int recordsRead;
+  private int batchRecordCount;
+
+  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(String name, MajorType type, int length) {
+    assert context != null : "Context shouldn't be null.";
+    MaterializedField f = MaterializedField.create(new SchemaPath(name, ExpressionPosition.UNKNOWN), type);
+    ValueVector v;
+    v = TypeHelper.getNewVector(f, context.getAllocator());
+    AllocationHelper.allocate(v, length, 50, 4);
+    
+    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];
+      batchRecordCount = 250000 / estimateRowSize;
+
+      for (int i = 0; i < config.getTypes().length; i++) {
+        valueVectors[i] = getVector(config.getTypes()[i].getName(), config.getTypes()[i].getMajorType(), batchRecordCount);
+        output.addField(valueVectors[i]);
+      }
+      output.setNewSchema();
+    } catch (SchemaChangeException e) {
+      throw new ExecutionSetupException("Failure while setting up fields", e);
+    }
+
+  }
+
+  @Override
+  public int next() {
+    
+    int recordSetSize = Math.min(batchRecordCount, this.config.getRecords()- recordsRead);
+
+    recordsRead += recordSetSize;
+    for(ValueVector v : valueVectors){
+      AllocationHelper.allocate(v, recordSetSize, 50, 5);
+      
+      logger.debug("MockRecordReader:  Generating random data for VV of type " + v.getClass().getName());
+      ValueVector.Mutator m = v.getMutator();
+      m.setValueCount(recordSetSize);
+      m.generateTestData();
+      
+    }
+    return recordSetSize;
+  }
+
+  @Override
+  public void cleanup() {
+    for (int i = 0; i < valueVectors.length; i++) {
+      try {
+        output.removeField(valueVectors[i].getField());
+      } catch (SchemaChangeException e) {
+        logger.warn("Failure while trying to remove field.", e);
+      }
+      valueVectors[i].close();
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/2884db7a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MockScanBatchCreator.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MockScanBatchCreator.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MockScanBatchCreator.java
new file mode 100644
index 0000000..5c91e1c
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MockScanBatchCreator.java
@@ -0,0 +1,46 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.store.mock;
+
+import java.util.List;
+
+import org.apache.drill.common.exceptions.ExecutionSetupException;
+import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.physical.impl.BatchCreator;
+import org.apache.drill.exec.physical.impl.ScanBatch;
+import org.apache.drill.exec.record.RecordBatch;
+import org.apache.drill.exec.store.RecordReader;
+import org.apache.drill.exec.store.mock.MockGroupScanPOP.MockScanEntry;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+
+public class MockScanBatchCreator implements BatchCreator<MockSubScanPOP>{
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(MockScanBatchCreator.class);
+
+  @Override
+  public RecordBatch getBatch(FragmentContext context, MockSubScanPOP 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/2884db7a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MockStorageEngine.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MockStorageEngine.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MockStorageEngine.java
index 23ac2b8..1ea6958 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MockStorageEngine.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MockStorageEngine.java
@@ -22,10 +22,9 @@ import java.util.ArrayList;
 
 import org.apache.drill.common.logical.data.Scan;
 import org.apache.drill.exec.physical.base.AbstractGroupScan;
-import org.apache.drill.exec.physical.config.MockGroupScanPOP;
-import org.apache.drill.exec.physical.config.MockGroupScanPOP.MockScanEntry;
 import org.apache.drill.exec.server.DrillbitContext;
 import org.apache.drill.exec.store.AbstractStorageEngine;
+import org.apache.drill.exec.store.mock.MockGroupScanPOP.MockScanEntry;
 import org.apache.drill.storage.MockStorageEngineConfig;
 
 import com.fasterxml.jackson.core.type.TypeReference;

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/2884db7a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MockStorePOP.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MockStorePOP.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MockStorePOP.java
new file mode 100644
index 0000000..4dbcd63
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/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.store.mock;
+
+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/2884db7a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MockSubScanPOP.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MockSubScanPOP.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MockSubScanPOP.java
new file mode 100644
index 0000000..38bf337
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MockSubScanPOP.java
@@ -0,0 +1,115 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.store.mock;
+
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+
+import com.google.common.collect.Iterators;
+import org.apache.drill.common.graph.GraphVisitor;
+import org.apache.drill.common.types.TypeProtos.DataMode;
+import org.apache.drill.common.types.TypeProtos.MajorType;
+import org.apache.drill.common.types.TypeProtos.MinorType;
+import org.apache.drill.exec.physical.EndpointAffinity;
+import org.apache.drill.exec.physical.OperatorCost;
+import org.apache.drill.exec.physical.ReadEntry;
+import org.apache.drill.exec.physical.base.*;
+import org.apache.drill.exec.physical.base.AbstractGroupScan;
+import org.apache.drill.exec.physical.base.GroupScan;
+import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
+import org.apache.drill.exec.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-sub-scan")
+public class MockSubScanPOP extends AbstractBase implements SubScan {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(MockGroupScanPOP.class);
+
+  private final String url;
+  protected final List<MockGroupScanPOP.MockScanEntry> readEntries;
+  private final OperatorCost cost;
+  private final Size size;
+  private  LinkedList<MockGroupScanPOP.MockScanEntry>[] mappings;
+
+  @JsonCreator
+  public MockSubScanPOP(@JsonProperty("url") String url, @JsonProperty("entries") List<MockGroupScanPOP.MockScanEntry> readEntries) {
+    this.readEntries = readEntries;
+    OperatorCost cost = new OperatorCost(0,0,0,0);
+    Size size = new Size(0,0);
+    for(MockGroupScanPOP.MockScanEntry r : readEntries){
+      cost = cost.add(r.getCost());
+      size = size.add(r.getSize());
+    }
+    this.cost = cost;
+    this.size = size;
+    this.url = url;
+  }
+
+  public String getUrl() {
+    return url;
+  }
+
+  @JsonProperty("entries")
+  public List<MockGroupScanPOP.MockScanEntry> getReadEntries() {
+    return readEntries;
+  }
+
+  @Override
+  public Iterator<PhysicalOperator> iterator() {
+    return Iterators.emptyIterator();
+  }
+
+  @Override
+  public OperatorCost getCost() {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public Size getSize() {
+    throw new UnsupportedOperationException();
+  }
+
+  // will want to replace these two methods with an interface above for AbstractSubScan
+  @Override
+  public boolean isExecutable() {
+    return true;  //To change body of implemented methods use File | Settings | File Templates.
+  }
+
+  @Override
+  public <T, X, E extends Throwable> T accept(PhysicalVisitor<T, X, E> physicalVisitor, X value) throws E{
+    return physicalVisitor.visitSubScan(this, value);
+  }
+  // see comment above about replacing this
+
+  @Override
+  @JsonIgnore
+  public PhysicalOperator getNewWithChildren(List<PhysicalOperator> children) {
+    Preconditions.checkArgument(children.isEmpty());
+    return new MockSubScanPOP(url, readEntries);
+
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/2884db7a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ColumnReader.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ColumnReader.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ColumnReader.java
index 8b4f760..99b65e6 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ColumnReader.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ColumnReader.java
@@ -62,7 +62,7 @@ public abstract class ColumnReader {
   ColumnReader(ParquetRecordReader parentReader, int allocateSize, ColumnDescriptor descriptor, ColumnChunkMetaData columnChunkMetaData,
                boolean fixedLength, ValueVector v){
     this.parentReader = parentReader;
-    if (allocateSize > 1) valueVecHolder = new VectorHolder(allocateSize, (BaseDataValueVector) v);
+    if (allocateSize > 1) valueVecHolder = new VectorHolder(allocateSize, v);
     else valueVecHolder = new VectorHolder(5000, (BaseDataValueVector) v);
 
     columnDescriptor = descriptor;

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/2884db7a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/PageReadStatus.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/PageReadStatus.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/PageReadStatus.java
index 29d9cc7..0378960 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/PageReadStatus.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/PageReadStatus.java
@@ -36,7 +36,10 @@ public final class PageReadStatus {
   // store references to the pages that have been uncompressed, but not copied to ValueVectors yet
   Page currentPage;
   // buffer to store bytes of current page, set to max size of parquet page
-  byte[] pageDataByteArray = new byte[ParquetRecordReader.PARQUET_PAGE_MAX_SIZE];
+  // TODO: add this back once toByteArray accepts an input.  byte[] pageDataByteArray = new byte[ParquetRecordReader.PARQUET_PAGE_MAX_SIZE];
+  byte[] pageDataByteArray;
+  
+  
   PageReader pageReader;
   // read position in the current page, stored in the ByteBuf in ParquetRecordReader called bufferWithAllData
   long readPosInBytes;
@@ -103,11 +106,13 @@ public final class PageReadStatus {
     }
 
     // if the buffer holding each page's data is not large enough to hold the current page, re-allocate, with a little extra space
-    if (pageHeader.getUncompressed_page_size() > pageDataByteArray.length) {
-      pageDataByteArray = new byte[pageHeader.getUncompressed_page_size() + 100];
-    }
+//    if (pageHeader.getUncompressed_page_size() > pageDataByteArray.length) {
+//      pageDataByteArray = new byte[pageHeader.getUncompressed_page_size() + 100];
+//    }
     // TODO - would like to get this into the mainline, hopefully before alpha
-    currentPage.getBytes().toByteArray(pageDataByteArray, 0, byteLength);
+    pageDataByteArray = currentPage.getBytes().toByteArray();
+    //TODO: Fix once parquet supports buffer work or at least passing in array.
+    //pageDataByteArray = currentPage.getBytes().toByteArray(pageDataByteArray, 0, byteLength);
 
     readPosInBytes = 0;
     valuesRead = 0;

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/2884db7a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetGroupScan.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetGroupScan.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetGroupScan.java
index f4988a0..66c1550 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetGroupScan.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetGroupScan.java
@@ -26,6 +26,7 @@ import com.fasterxml.jackson.annotation.JsonProperty;
 import com.fasterxml.jackson.annotation.JsonTypeName;
 import com.fasterxml.jackson.core.type.TypeReference;
 import com.fasterxml.jackson.databind.ObjectMapper;
+
 import org.apache.drill.common.JSONOptions;
 import org.apache.drill.common.config.DrillConfig;
 import org.apache.drill.exec.exception.SetupException;
@@ -36,15 +37,17 @@ import org.apache.drill.exec.physical.ReadEntryWithPath;
 import org.apache.drill.exec.physical.base.AbstractGroupScan;
 import org.apache.drill.exec.physical.base.PhysicalOperator;
 import org.apache.drill.exec.physical.base.Size;
-import org.apache.drill.exec.physical.config.MockGroupScanPOP;
 import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
 
 import com.google.common.base.Preconditions;
+
 import org.apache.drill.exec.server.DrillbitContext;
 import org.apache.drill.exec.store.StorageEngineRegistry;
 import org.apache.drill.exec.store.AffinityCalculator;
+import org.apache.drill.exec.store.mock.MockGroupScanPOP;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+
 import parquet.hadoop.ParquetFileReader;
 import parquet.hadoop.metadata.BlockMetaData;
 import parquet.hadoop.metadata.ColumnChunkMetaData;

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/2884db7a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetScanBatchCreator.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetScanBatchCreator.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetScanBatchCreator.java
index bd63406..03fb4ec 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetScanBatchCreator.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetScanBatchCreator.java
@@ -24,17 +24,18 @@ 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.MockScanBatchCreator;
-
 import org.apache.drill.exec.physical.impl.BatchCreator;
 import org.apache.drill.exec.physical.impl.ScanBatch;
 import org.apache.drill.exec.record.RecordBatch;
 import org.apache.drill.exec.store.RecordReader;
+import org.apache.drill.exec.store.mock.MockScanBatchCreator;
 
 import com.google.common.base.Preconditions;
 import com.google.common.collect.Lists;
+
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
+
 import parquet.hadoop.ParquetFileReader;
 import parquet.hadoop.metadata.ParquetMetadata;
 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/2884db7a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/JSONRecordReaderTest.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/JSONRecordReaderTest.java b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/JSONRecordReaderTest.java
index c9d6967..2d9524d 100644
--- a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/JSONRecordReaderTest.java
+++ b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/JSONRecordReaderTest.java
@@ -11,6 +11,7 @@ import java.util.List;
 
 import com.google.common.base.Predicate;
 import com.google.common.collect.Iterables;
+
 import mockit.Expectations;
 import mockit.Injectable;
 
@@ -23,6 +24,7 @@ import org.apache.drill.exec.physical.impl.OutputMutator;
 import org.apache.drill.exec.proto.SchemaDefProtos;
 import org.apache.drill.exec.proto.UserBitShared;
 import org.apache.drill.exec.record.MaterializedField;
+import org.apache.drill.exec.store.json.JSONRecordReader;
 import org.apache.drill.exec.vector.ValueVector;
 import org.junit.Ignore;
 import org.junit.Test;

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/2884db7a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/ParquetRecordReaderTest.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/ParquetRecordReaderTest.java b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/ParquetRecordReaderTest.java
index 0e31cdd..5628f50 100644
--- a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/ParquetRecordReaderTest.java
+++ b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/ParquetRecordReaderTest.java
@@ -27,7 +27,6 @@ import java.util.Arrays;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
-import java.util.Vector;
 
 import org.apache.drill.common.config.DrillConfig;
 import org.apache.drill.common.types.TypeProtos;
@@ -72,40 +71,30 @@ public class ParquetRecordReaderTest {
     new ParquetRecordReaderTest().testMultipleRowGroupsAndReadsEvent();
   }
 
+ 
   @Test
   public void testMultipleRowGroupsAndReadsEvent() throws Exception {
     String planName = "/parquet_scan_screen.json";
-    String fileName = "/tmp/testParquetFile_many_types_3";
+    String fileName = "/tmp/parquet_test_file_many_types";
     int numberRowGroups = 20;
     int recordsPerRowGroup = 300000;
-    //TestFileGenerator.generateParquetFile(fileName, numberRowGroups, recordsPerRowGroup);
+    File f = new File(fileName);
+    if(!f.exists()) TestFileGenerator.generateParquetFile(fileName, numberRowGroups, recordsPerRowGroup);
     testParquetFullEngineLocal(planName, fileName, 2, numberRowGroups, recordsPerRowGroup);
   }
 
   private class ParquetResultListener implements UserResultsListener {
-    private Vector<QueryResultBatch> results = new Vector<QueryResultBatch>();
     private SettableFuture<Void> future = SettableFuture.create();
-    int count = 0;
     RecordBatchLoader batchLoader;
-    byte[] bytes;
 
-    int numberRowGroups;
-    int numberOfTimesRead;
     int batchCounter = 1;
-    int columnValCounter = 0;
-    int i = 0;
-    private FieldInfo currentField;
     private final HashMap<String, Long> valuesChecked = new HashMap<>();
-    private final int recordsPerRowGroup;
     private final Map<String, FieldInfo> fields;
     private final long totalRecords;
     
     ParquetResultListener(int recordsPerRowGroup, RecordBatchLoader batchLoader, int numberRowGroups, int numberOfTimesRead){
       this.batchLoader = batchLoader;
       this.fields = TestFileGenerator.getFieldMap(recordsPerRowGroup);
-      this.recordsPerRowGroup = recordsPerRowGroup;
-      this.numberRowGroups = numberRowGroups;
-      this.numberOfTimesRead = numberOfTimesRead;
       this.totalRecords = recordsPerRowGroup * numberRowGroups * numberOfTimesRead;
     }
 
@@ -120,7 +109,7 @@ public class ParquetRecordReaderTest {
       long columnValCounter = 0;
       int i = 0;
       FieldInfo currentField;
-      count += result.getHeader().getRowCount();
+
       boolean schemaChanged = false;
       try {
         schemaChanged = batchLoader.load(result.getHeader().getDef(), result.getData());
@@ -128,12 +117,11 @@ public class ParquetRecordReaderTest {
         logger.error("Failure while loading batch", e);
       }
 
-      int recordCount = 0;
       // print headers.
       if (schemaChanged) {
       } // do not believe any change is needed for when the schema changes, with the current mock scan use case
 
-      for (VectorWrapper vw : batchLoader) {
+      for (VectorWrapper<?> vw : batchLoader) {
         ValueVector vv = vw.getValueVector();
         currentField = fields.get(vv.getField().getName());
         if (VERBOSE_DEBUG){
@@ -163,7 +151,6 @@ public class ParquetRecordReaderTest {
       
       if (VERBOSE_DEBUG){
         for (i = 0; i < batchLoader.getRecordCount(); i++) {
-          recordCount++;
           if (i % 50 == 0){
             System.out.println();
             for (VectorWrapper<?> vw : batchLoader) {
@@ -298,11 +285,6 @@ public class ParquetRecordReaderTest {
 
   @SuppressWarnings("unchecked")
   private <T> void assertField(ValueVector valueVector, int index, TypeProtos.MinorType expectedMinorType, T value, String name, int parentFieldId) {
-//    UserBitShared.FieldMetadata metadata = valueVector.getMetadata();
-//    SchemaDefProtos.FieldDef def = metadata.getDef();
-//    assertEquals(expectedMinorType, def.getMajorType().getMinorType());
-//    assertEquals(name, def.getNameList().get(0).getName());
-//    assertEquals(parentFieldId, def.getParentId());
 
     if (expectedMinorType == TypeProtos.MinorType.MAP) {
       return;
@@ -339,9 +321,6 @@ public class ParquetRecordReaderTest {
     assertArrayEquals(bytes.toByteArray(), page.getBytes().toByteArray());
   }
 
-  private String getResource(String resourceName) {
-    return "resource:" + resourceName;
-  }
 
   
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/2884db7a/sandbox/prototype/exec/java-exec/src/test/resources/parquet_scan_screen.json
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/resources/parquet_scan_screen.json b/sandbox/prototype/exec/java-exec/src/test/resources/parquet_scan_screen.json
index 15d3936..29cab68 100644
--- a/sandbox/prototype/exec/java-exec/src/test/resources/parquet_scan_screen.json
+++ b/sandbox/prototype/exec/java-exec/src/test/resources/parquet_scan_screen.json
@@ -22,10 +22,10 @@
       storageengine:"parquet",
       selection: [
         {
-            path: "/tmp/testParquetFile_many_types_3"
+            path: "/tmp/parquet_test_file_many_types"
         },
         {
-            path: "/tmp/testParquetFile_many_types_3"
+            path: "/tmp/parquet_test_file_many_types"
         }
       ]
     },

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/2884db7a/sandbox/prototype/sqlparser/src/test/java/org/apache/drill/jdbc/test/FullEngineTest.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/sqlparser/src/test/java/org/apache/drill/jdbc/test/FullEngineTest.java b/sandbox/prototype/sqlparser/src/test/java/org/apache/drill/jdbc/test/FullEngineTest.java
index 79b8ef8..35a2414 100644
--- a/sandbox/prototype/sqlparser/src/test/java/org/apache/drill/jdbc/test/FullEngineTest.java
+++ b/sandbox/prototype/sqlparser/src/test/java/org/apache/drill/jdbc/test/FullEngineTest.java
@@ -3,6 +3,7 @@ package org.apache.drill.jdbc.test;
 import java.io.IOException;
 
 import org.junit.BeforeClass;
+import org.junit.Ignore;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.TestName;
@@ -12,6 +13,7 @@ import org.junit.rules.Timeout;
 import com.google.common.base.Charsets;
 import com.google.common.io.Resources;
 
+@Ignore
 public class FullEngineTest {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(FullEngineTest.class);
 


[22/27] git commit: fix deadbuf issue when flushing an outgoing batch

Posted by ja...@apache.org.
fix deadbuf issue when flushing an outgoing batch


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

Branch: refs/heads/master
Commit: a136a5bf53f87ac256d311fbbe6f1a373d193b1f
Parents: 8ffc674
Author: Ben Becker <be...@gmail.com>
Authored: Sat Aug 10 01:29:39 2013 -0700
Committer: Jacques Nadeau <ja...@apache.org>
Committed: Thu Aug 15 18:31:31 2013 -0700

----------------------------------------------------------------------
 .../exec/physical/impl/partitionsender/OutgoingRecordBatch.java    | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/a136a5bf/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/partitionsender/OutgoingRecordBatch.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/partitionsender/OutgoingRecordBatch.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/partitionsender/OutgoingRecordBatch.java
index b40ce4c..6847e5a 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/partitionsender/OutgoingRecordBatch.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/partitionsender/OutgoingRecordBatch.java
@@ -98,7 +98,7 @@ public class OutgoingRecordBatch implements RecordBatch {
     for (VectorWrapper v : vectorContainer) {
       logger.debug("Reallocating vv to capacity " + recordCapacity + " after flush. " + v.getValueVector());
       getAllocator(v.getValueVector(),
-                   TypeHelper.getNewVector(v.getField(), context.getAllocator())).alloc(recordCapacity);
+                   v.getValueVector()).alloc(recordCapacity);
     }
     if (!ok) { throw new SchemaChangeException("Flush ended NOT OK!"); }
   }


[09/27] Initial Parquet commit. Suports INT, LONG, FLOAT, DOUBLE, distributed scheduling.

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/0a2f997f/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetRecordReader.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetRecordReader.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetRecordReader.java
new file mode 100644
index 0000000..2ad7b44
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetRecordReader.java
@@ -0,0 +1,403 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.parquet;
+
+import io.netty.buffer.ByteBuf;
+import org.apache.drill.common.exceptions.DrillRuntimeException;
+import org.apache.drill.common.exceptions.ExecutionSetupException;
+import org.apache.drill.common.expression.ExpressionPosition;
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.common.types.Types;
+import org.apache.drill.exec.exception.SchemaChangeException;
+import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.physical.impl.OutputMutator;
+import org.apache.drill.exec.record.MaterializedField;
+import org.apache.drill.exec.store.RecordReader;
+import org.apache.drill.exec.vector.TypeHelper;
+import org.apache.drill.exec.vector.ValueVector;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import parquet.column.ColumnDescriptor;
+import parquet.hadoop.CodecFactoryExposer;
+import parquet.hadoop.metadata.ColumnChunkMetaData;
+import parquet.hadoop.metadata.ParquetMetadata;
+import parquet.schema.MessageType;
+import parquet.schema.PrimitiveType;
+
+import java.io.IOException;
+import java.text.SimpleDateFormat;
+import java.util.ArrayList;
+import java.util.Date;
+import java.util.List;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static com.google.common.base.Preconditions.checkNotNull;
+
+public class ParquetRecordReader implements RecordReader {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ParquetRecordReader.class);
+
+  // this value has been inflated to read in multiple value vectors at once, and then break them up into smaller vectors
+  private static final int NUMBER_OF_VECTORS = 1;
+  private static final long DEFAULT_BATCH_LENGTH = 256 * 1024 * NUMBER_OF_VECTORS; // 256kb
+  private static final long DEFAULT_BATCH_LENGTH_IN_BITS = DEFAULT_BATCH_LENGTH * 8; // 256kb
+
+  // TODO - should probably find a smarter way to set this, currently 2 megabytes
+  private static final int VAR_LEN_FIELD_LENGTH = 1024 * 1024 * 2;
+  public static final int PARQUET_PAGE_MAX_SIZE = 1024 * 1024 * 5;
+  private static final String SEPERATOR = System.getProperty("file.separator");
+
+
+  // used for clearing the last n bits of a byte
+  public static final byte[] endBitMasks = {-2, -4, -8, -16, -32, -64, -128};
+  // used for clearing the first n bits of a byte
+  public static final byte[] startBitMasks = {127, 63, 31, 15, 7, 3, 1};
+
+  private int bitWidthAllFixedFields;
+  private boolean allFieldsFixedLength;
+  private int recordsPerBatch;
+  private ByteBuf bufferWithAllData;
+  long totalRecords;
+  long rowGroupOffset;
+
+  private List<ColumnReader> columnStatuses;
+  FileSystem fileSystem;
+  private BufferAllocator allocator;
+  private long batchSize;
+  Path hadoopPath;
+  private final VarLenBinaryReader varLengthReader;
+
+  public CodecFactoryExposer getCodecFactoryExposer() {
+    return codecFactoryExposer;
+  }
+
+  private final CodecFactoryExposer codecFactoryExposer;
+
+  int rowGroupIndex;
+
+  public ParquetRecordReader(FragmentContext fragmentContext,
+                             String path, int rowGroupIndex, FileSystem fs,
+                             CodecFactoryExposer codecFactoryExposer, ParquetMetadata footer) throws ExecutionSetupException {
+    this(fragmentContext, DEFAULT_BATCH_LENGTH_IN_BITS, path, rowGroupIndex, fs, codecFactoryExposer, footer);
+  }
+
+
+  public ParquetRecordReader(FragmentContext fragmentContext, long batchSize,
+                             String path, int rowGroupIndex, FileSystem fs,
+                             CodecFactoryExposer codecFactoryExposer, ParquetMetadata footer) throws ExecutionSetupException {
+    this.allocator = fragmentContext.getAllocator();
+
+    hadoopPath = new Path(path);
+    fileSystem = fs;
+    this.codecFactoryExposer = codecFactoryExposer;
+    this.rowGroupIndex = rowGroupIndex;
+    this.batchSize = batchSize;
+
+    columnStatuses = new ArrayList<>();
+
+    totalRecords = footer.getBlocks().get(rowGroupIndex).getRowCount();
+
+    List<ColumnDescriptor> columns = footer.getFileMetaData().getSchema().getColumns();
+    allFieldsFixedLength = true;
+    ColumnDescriptor column;
+    ColumnChunkMetaData columnChunkMetaData;
+
+    // loop to add up the length of the fixed width columns and build the schema
+    for (int i = 0; i < columns.size(); ++i) {
+      column = columns.get(i);
+
+      // sum the lengths of all of the fixed length fields
+      if (column.getType() != PrimitiveType.PrimitiveTypeName.BINARY) {
+        // There is not support for the fixed binary type yet in parquet, leaving a task here as a reminder
+        // TODO - implement this when the feature is added upstream
+//          if (column.getType() == PrimitiveType.PrimitiveTypeName.FIXED_LEN_BYTE_ARRAY){
+//              byteWidthAllFixedFields += column.getType().getWidth()
+//          }
+//          else { } // the code below for the rest of the fixed length fields
+
+        bitWidthAllFixedFields += getTypeLengthInBits(column.getType());
+      } else {
+        allFieldsFixedLength = false;
+      }
+
+    }
+    rowGroupOffset = footer.getBlocks().get(rowGroupIndex).getColumns().get(0).getFirstDataPageOffset();
+
+    if (allFieldsFixedLength) {
+      recordsPerBatch = (int) Math.min(batchSize / bitWidthAllFixedFields, footer.getBlocks().get(0).getColumns().get(0).getValueCount());
+    }
+    try {
+      ArrayList<VarLenBinaryReader.VarLengthColumn> varLengthColumns = new ArrayList<>();
+      // initialize all of the column read status objects
+      boolean fieldFixedLength = false;
+      MaterializedField field;
+      for (int i = 0; i < columns.size(); ++i) {
+        column = columns.get(i);
+        columnChunkMetaData = footer.getBlocks().get(0).getColumns().get(i);
+        field = MaterializedField.create(new SchemaPath(toFieldName(column.getPath()), ExpressionPosition.UNKNOWN),
+            toMajorType(column.getType(), getDataMode(column, footer.getFileMetaData().getSchema())));
+        fieldFixedLength = column.getType() != PrimitiveType.PrimitiveTypeName.BINARY;
+        ValueVector v = TypeHelper.getNewVector(field, allocator);
+        if (column.getType() != PrimitiveType.PrimitiveTypeName.BINARY) {
+          createFixedColumnReader(fieldFixedLength, field, column, columnChunkMetaData, recordsPerBatch, v);
+        } else {
+          varLengthColumns.add(new VarLenBinaryReader.VarLengthColumn(this, -1, column, columnChunkMetaData, false, v));
+        }
+      }
+      varLengthReader = new VarLenBinaryReader(this, varLengthColumns);
+    } catch (SchemaChangeException e) {
+      throw new ExecutionSetupException(e);
+    }
+  }
+
+  public ByteBuf getBufferWithAllData() {
+    return bufferWithAllData;
+  }
+
+  public int getRowGroupIndex() {
+    return rowGroupIndex;
+  }
+
+  public int getBitWidthAllFixedFields() {
+    return bitWidthAllFixedFields;
+  }
+
+  public long getBatchSize() {
+    return batchSize;
+  }
+
+  /**
+   * @param type a fixed length type from the parquet library enum
+   * @return the length in pageDataByteArray of the type
+   */
+  public static int getTypeLengthInBits(PrimitiveType.PrimitiveTypeName type) {
+    switch (type) {
+      case INT64:   return 64;
+      case INT32:   return 32;
+      case BOOLEAN: return 1;
+      case FLOAT:   return 32;
+      case DOUBLE:  return 64;
+      case INT96:   return 96;
+      // binary and fixed length byte array
+      default:
+        throw new IllegalStateException("Length cannot be determined for type " + type);
+    }
+  }
+
+  @Override
+  public void setup(OutputMutator output) throws ExecutionSetupException {
+    long tA = System.nanoTime(), tB;
+    System.out.println( new SimpleDateFormat("mm:ss S").format(new Date()) + " :Start of ParquetRecordReader.setup");
+    output.removeAllFields();
+
+    try {
+      for (ColumnReader crs : columnStatuses) {
+        output.addField(crs.valueVecHolder.getValueVector());
+      }
+      for (VarLenBinaryReader.VarLengthColumn r : varLengthReader.columns) {
+        output.addField(r.valueVecHolder.getValueVector());
+        output.setNewSchema();
+      }
+    }catch(SchemaChangeException e) {
+      throw new ExecutionSetupException("Error setting up output mutator.", e);
+    }
+
+    // the method for reading into a ByteBuf from a stream copies all of the data into a giant buffer
+    // here we do the same thing in a loop to not initialize so much on heap
+
+    // TODO - this should be replaced by an enhancement in Hadoop 2.0 that will allow reading
+    // directly into a ByteBuf passed into the reading method
+    int totalByteLength = 0;
+    long start = 0;
+    if (rowGroupIndex == 0){
+      totalByteLength = 4;
+    }
+    else{
+      start = rowGroupOffset;
+    }
+    for (ColumnReader crs : columnStatuses){
+      totalByteLength += crs.columnChunkMetaData.getTotalSize();
+    }
+    for (VarLenBinaryReader.VarLengthColumn r : varLengthReader.columns){
+      totalByteLength += r.columnChunkMetaData.getTotalSize();
+    }
+    int bufferSize = 64*1024;
+    long totalBytesWritten = 0;
+    int validBytesInCurrentBuffer;
+    byte[] buffer = new byte[bufferSize];
+    try {
+      bufferWithAllData = allocator.buffer(totalByteLength);
+      FSDataInputStream inputStream = fileSystem.open(hadoopPath);
+      inputStream.seek(start);
+      while (totalBytesWritten < totalByteLength){
+        validBytesInCurrentBuffer = (int) Math.min(bufferSize, totalByteLength - totalBytesWritten);
+        inputStream.read(buffer, 0 , validBytesInCurrentBuffer);
+        bufferWithAllData.writeBytes(buffer, 0 , (int) validBytesInCurrentBuffer);
+        totalBytesWritten += validBytesInCurrentBuffer;
+      }
+
+    } catch (IOException e) {
+      throw new ExecutionSetupException("Error opening or reading metatdata for parquet file at location: " + hadoopPath.getName());
+    }
+    System.out.println( "Total time in method: " + ((float) (System.nanoTime() - tA) / 1e9));
+  }
+
+  private static String toFieldName(String[] paths) {
+    return join(SEPERATOR, paths);
+  }
+
+  private TypeProtos.DataMode getDataMode(ColumnDescriptor column, MessageType schema) {
+    if (schema.getColumnDescription(column.getPath()).getMaxDefinitionLevel() == 0) {
+      return TypeProtos.DataMode.REQUIRED;
+    } else {
+      return TypeProtos.DataMode.OPTIONAL;
+    }
+  }
+
+  private void resetBatch() {
+    for (ColumnReader column : columnStatuses) {
+      column.valueVecHolder.reset();
+      column.valuesReadInCurrentPass = 0;
+    }
+    for (VarLenBinaryReader.VarLengthColumn r : varLengthReader.columns){
+      r.valueVecHolder.reset();
+      r.valuesReadInCurrentPass = 0;
+    }
+  }
+
+  /**
+   * @param fixedLength
+   * @param field
+   * @param descriptor
+   * @param columnChunkMetaData
+   * @param allocateSize - the size of the vector to create
+   * @return
+   * @throws SchemaChangeException
+   */
+  private boolean createFixedColumnReader(boolean fixedLength, MaterializedField field, ColumnDescriptor descriptor,
+                                          ColumnChunkMetaData columnChunkMetaData, int allocateSize, ValueVector v)
+      throws SchemaChangeException {
+    TypeProtos.MajorType type = field.getType();
+    if (columnChunkMetaData.getType() == PrimitiveType.PrimitiveTypeName.BOOLEAN){
+      columnStatuses.add(new BitReader(this, allocateSize, descriptor, columnChunkMetaData,
+          fixedLength, v));
+    }
+    else{
+      columnStatuses.add(new FixedByteAlignedReader(this, allocateSize, descriptor, columnChunkMetaData,
+          fixedLength, v));
+    }
+    return true;
+  }
+
+ public void readAllFixedFields(long recordsToRead, ColumnReader firstColumnStatus) throws IOException {
+
+   for (ColumnReader crs : columnStatuses){
+     crs.readAllFixedFields(recordsToRead, firstColumnStatus);
+   }
+ }
+
+  @Override
+  public int next() {
+    resetBatch();
+    long recordsToRead = 0;
+    try {
+      ColumnReader firstColumnStatus = columnStatuses.iterator().next();
+      if (allFieldsFixedLength) {
+        recordsToRead = Math.min(recordsPerBatch, firstColumnStatus.columnChunkMetaData.getValueCount() - firstColumnStatus.totalValuesRead);
+      } else {
+        // arbitrary
+        recordsToRead = 8000;
+
+        // going to incorporate looking at length of values and copying the data into a single loop, hopefully it won't
+        // get too complicated
+
+        //loop through variable length data to find the maximum records that will fit in this batch
+        // this will be a bit annoying if we want to loop though row groups, columns, pages and then individual variable
+        // length values...
+        // jacques believes that variable length fields will be encoded as |length|value|length|value|...
+        // cannot find more information on this right now, will keep looking
+      }
+
+      if (allFieldsFixedLength) {
+        readAllFixedFields(recordsToRead, firstColumnStatus);
+      } else { // variable length columns
+        long fixedRecordsToRead = varLengthReader.readFields(recordsToRead, firstColumnStatus);
+        readAllFixedFields(fixedRecordsToRead, firstColumnStatus);
+      }
+
+      return firstColumnStatus.valuesReadInCurrentPass;
+    } catch (IOException e) {
+      throw new DrillRuntimeException(e);
+    }
+  }
+
+  static TypeProtos.MajorType toMajorType(PrimitiveType.PrimitiveTypeName primitiveTypeName,
+                                               TypeProtos.DataMode mode) {
+    return toMajorType(primitiveTypeName, 0, mode);
+  }
+
+  static TypeProtos.MajorType toMajorType(PrimitiveType.PrimitiveTypeName primitiveTypeName, int length,
+                                               TypeProtos.DataMode mode) {
+    switch (primitiveTypeName) {
+      case BINARY:
+        return Types.required(TypeProtos.MinorType.VARBINARY);
+      case INT64:
+        return Types.required(TypeProtos.MinorType.BIGINT);
+      case INT32:
+        return Types.required(TypeProtos.MinorType.INT);
+      case BOOLEAN:
+        return Types.required(TypeProtos.MinorType.BIT);
+      case FLOAT:
+        return Types.required(TypeProtos.MinorType.FLOAT4);
+      case DOUBLE:
+        return Types.required(TypeProtos.MinorType.FLOAT8);
+      // Both of these are not supported by the parquet library yet (7/3/13),
+      // but they are declared here for when they are implemented
+      case INT96:
+        return TypeProtos.MajorType.newBuilder().setMinorType(TypeProtos.MinorType.FIXEDBINARY).setWidth(12)
+            .setMode(mode).build();
+      case FIXED_LEN_BYTE_ARRAY:
+        checkArgument(length > 0, "A length greater than zero must be provided for a FixedBinary type.");
+        return TypeProtos.MajorType.newBuilder().setMinorType(TypeProtos.MinorType.FIXEDBINARY)
+            .setWidth(length).setMode(mode).build();
+      default:
+        throw new UnsupportedOperationException("Type not supported: " + primitiveTypeName);
+    }
+  }
+
+  static String join(String delimiter, String... str) {
+    StringBuilder builder = new StringBuilder();
+    int i = 0;
+    for (String s : str) {
+      builder.append(s);
+      if (i < str.length) {
+        builder.append(delimiter);
+      }
+      i++;
+    }
+    return builder.toString();
+  }
+
+  @Override
+  public void cleanup() {
+    columnStatuses.clear();
+    bufferWithAllData.clear();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/0a2f997f/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetRowGroupScan.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetRowGroupScan.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetRowGroupScan.java
new file mode 100644
index 0000000..1e5d203
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetRowGroupScan.java
@@ -0,0 +1,137 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.parquet;
+
+import com.fasterxml.jackson.annotation.*;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Iterators;
+import org.apache.drill.common.logical.StorageEngineConfig;
+import org.apache.drill.exec.exception.SetupException;
+import org.apache.drill.exec.physical.OperatorCost;
+import org.apache.drill.exec.physical.ReadEntryFromHDFS;
+import org.apache.drill.exec.physical.base.*;
+import org.apache.drill.exec.store.StorageEngineRegistry;
+
+import java.util.*;
+
+// Class containing information for reading a single parquet row group form HDFS
+@JsonTypeName("parquet-row-group-scan")
+public class ParquetRowGroupScan extends AbstractBase implements SubScan {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ParquetRowGroupScan.class);
+
+  public StorageEngineConfig engineConfig;
+  private ParquetStorageEngine parquetStorageEngine;
+  private List<RowGroupReadEntry> rowGroupReadEntries;
+
+  @JsonCreator
+  public ParquetRowGroupScan(@JacksonInject StorageEngineRegistry registry, @JsonProperty("engineConfig") StorageEngineConfig engineConfig,
+                             @JsonProperty("rowGroupReadEntries") LinkedList<RowGroupReadEntry> rowGroupReadEntries) throws SetupException {
+    parquetStorageEngine = (ParquetStorageEngine) registry.getEngine(engineConfig);
+    this.rowGroupReadEntries = rowGroupReadEntries;
+  }
+
+  public ParquetRowGroupScan(ParquetStorageEngine engine, ParquetStorageEngineConfig config,
+                              List<RowGroupReadEntry> rowGroupReadEntries) throws SetupException {
+    parquetStorageEngine = engine;
+    engineConfig = config;
+    this.rowGroupReadEntries = rowGroupReadEntries;
+  }
+
+  public List<RowGroupReadEntry> getRowGroupReadEntries() {
+    return rowGroupReadEntries;
+  }
+
+  public StorageEngineConfig getEngineConfig() {
+    return engineConfig;
+  }
+
+  @Override
+  public OperatorCost getCost() {
+    return null;
+  }
+
+  @Override
+  public Size getSize() {
+    return null;
+  }
+
+  @Override
+  public boolean isExecutable() {
+    return false;
+  }
+
+  @JsonIgnore
+  public ParquetStorageEngine getStorageEngine(){
+    return parquetStorageEngine;
+  }
+
+  @Override
+  public <T, X, E extends Throwable> T accept(PhysicalVisitor<T, X, E> physicalVisitor, X value) throws E {
+    return physicalVisitor.visitSubScan(this, value);
+  }
+
+  @Override
+  public PhysicalOperator getNewWithChildren(List<PhysicalOperator> children) {
+    Preconditions.checkArgument(children.isEmpty());
+    try {
+      return new ParquetRowGroupScan(parquetStorageEngine, (ParquetStorageEngineConfig) engineConfig, rowGroupReadEntries);
+    } catch (SetupException e) {
+      // TODO - handle this
+      e.printStackTrace();  //To change body of catch statement use File | Settings | File Templates.
+    }
+    return null;
+  }
+
+  @Override
+  public Iterator<PhysicalOperator> iterator() {
+    return Iterators.emptyIterator();
+  }
+
+  public static class RowGroupReadEntry extends ReadEntryFromHDFS {
+
+    private int rowGroupIndex;
+
+    @parquet.org.codehaus.jackson.annotate.JsonCreator
+    public RowGroupReadEntry(@JsonProperty("path") String path, @JsonProperty("start") long start,
+                             @JsonProperty("length") long length, @JsonProperty("rowGroupIndex") int rowGroupIndex) {
+      super(path, start, length);
+      this.rowGroupIndex = rowGroupIndex;
+    }
+
+    @Override
+    public OperatorCost getCost() {
+      return new OperatorCost(1, 2, 1, 1);
+    }
+
+    @Override
+    public Size getSize() {
+      // TODO - these values are wrong, I cannot know these until after I read a file
+      return new Size(10, 10);
+    }
+
+    @JsonIgnore
+    public RowGroupReadEntry getRowGroupReadEntry() {
+      return new RowGroupReadEntry(this.getPath(), this.getStart(), this.getLength(), this.rowGroupIndex);
+    }
+
+    public int getRowGroupIndex(){
+      return rowGroupIndex;
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/0a2f997f/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetScanBatchCreator.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetScanBatchCreator.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetScanBatchCreator.java
new file mode 100644
index 0000000..bd63406
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetScanBatchCreator.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.store.parquet;
+
+import java.io.IOException;
+import java.text.SimpleDateFormat;
+import java.util.Date;
+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.MockScanBatchCreator;
+
+import org.apache.drill.exec.physical.impl.BatchCreator;
+import org.apache.drill.exec.physical.impl.ScanBatch;
+import org.apache.drill.exec.record.RecordBatch;
+import org.apache.drill.exec.store.RecordReader;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import parquet.hadoop.ParquetFileReader;
+import parquet.hadoop.metadata.ParquetMetadata;
+
+public class ParquetScanBatchCreator implements BatchCreator<ParquetRowGroupScan>{
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(MockScanBatchCreator.class);
+
+  @Override
+  public RecordBatch getBatch(FragmentContext context, ParquetRowGroupScan rowGroupScan, List<RecordBatch> children) throws ExecutionSetupException {
+    long tA = System.nanoTime(), tB;
+    System.out.println( new SimpleDateFormat("mm:ss S").format(new Date()) + " :Start of ScanBatCreator.scanBatch");
+    Preconditions.checkArgument(children.isEmpty());
+    List<RecordReader> readers = Lists.newArrayList();
+    for(ParquetRowGroupScan.RowGroupReadEntry e : rowGroupScan.getRowGroupReadEntries()){
+      /*
+      Here we could store a map from file names to footers, to prevent re-reading the footer for each row group in a file
+      TODO - to prevent reading the footer again in the parquet record reader (it is read earlier in the ParquetStorageEngine)
+      we should add more information to the RowGroupInfo that will be populated upon the first read to
+      provide the reader with all of th file meta-data it needs
+      These fields will be added to the constructor below
+      */
+      try {
+        readers.add(
+            new ParquetRecordReader(
+                context, e.getPath(), e.getRowGroupIndex(), rowGroupScan.getStorageEngine().getFileSystem(),
+                rowGroupScan.getStorageEngine().getCodecFactoryExposer(),
+                ParquetFileReader.readFooter( rowGroupScan.getStorageEngine().getFileSystem().getConf(), new Path(e.getPath()))
+            )
+        );
+      } catch (IOException e1) {
+        throw new ExecutionSetupException(e1);
+      }
+    }
+    System.out.println( "Total time in method: " + ((float) (System.nanoTime() - tA) / 1e9));
+    return new ScanBatch(context, readers.iterator());
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/0a2f997f/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetStorageEngine.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetStorageEngine.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetStorageEngine.java
new file mode 100644
index 0000000..f070f0f
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetStorageEngine.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.store.parquet;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+
+import com.fasterxml.jackson.core.type.TypeReference;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import org.apache.drill.common.exceptions.ExecutionSetupException;
+import org.apache.drill.common.logical.data.Scan;
+import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.physical.ReadEntry;
+import org.apache.drill.exec.physical.ReadEntryWithPath;
+import org.apache.drill.exec.physical.config.MockStorageEngine;
+import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
+import org.apache.drill.exec.server.DrillbitContext;
+import org.apache.drill.exec.store.AbstractStorageEngine;
+import org.apache.drill.exec.store.RecordReader;
+
+import com.google.common.collect.ListMultimap;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.FileSystem;
+import parquet.format.converter.ParquetMetadataConverter;
+import parquet.hadoop.CodecFactoryExposer;
+import parquet.hadoop.ParquetFileReader;
+import parquet.hadoop.metadata.BlockMetaData;
+import parquet.hadoop.metadata.ColumnChunkMetaData;
+import parquet.hadoop.metadata.ParquetMetadata;
+
+public class ParquetStorageEngine extends AbstractStorageEngine{
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(MockStorageEngine.class);
+
+  private final DrillbitContext context;
+  private final ParquetStorageEngineConfig configuration;
+  private FileSystem fs;
+  private Configuration conf;
+  static final ParquetMetadataConverter parquetMetadataConverter = new ParquetMetadataConverter();
+  private CodecFactoryExposer codecFactoryExposer;
+  final ParquetMetadata footer;
+  public static final String HADOOP_DEFAULT_NAME = "fs.default.name";
+
+
+  public ParquetStorageEngine(ParquetStorageEngineConfig configuration, DrillbitContext context){
+    this.context = context;
+    this.configuration = configuration;
+    this.footer = null;
+    try {
+      this.conf = new Configuration();
+      this.conf.set(HADOOP_DEFAULT_NAME, configuration.getDfsName());
+      this.fs = FileSystem.get(conf);
+      codecFactoryExposer = new CodecFactoryExposer(conf);
+    } catch (IOException ie) {
+      throw new RuntimeException("Error setting up filesystem");
+    }
+  }
+
+  public Configuration getHadoopConfig() {
+    return this.conf;
+  }
+
+  public FileSystem getFileSystem() {
+    return this.fs;
+  }
+
+  public DrillbitContext getContext() {
+    return this.context;
+  }
+
+  @Override
+  public boolean supportsRead() {
+    return true;
+  }
+
+  @Override
+  public ParquetGroupScan getPhysicalScan(Scan scan) throws IOException {
+
+    ArrayList<ReadEntryWithPath> readEntries = scan.getSelection().getListWith(new ObjectMapper(),
+        new TypeReference<ArrayList<ReadEntryWithPath>>() {});
+
+    return new ParquetGroupScan(readEntries, this);
+  }
+
+  @Override
+  public ListMultimap<ReadEntry, DrillbitEndpoint> getReadLocations(Collection<ReadEntry> entries) {
+    return null;
+  }
+
+  @Override
+  public RecordReader getReader(FragmentContext context, ReadEntry readEntry) throws IOException {
+    return null;
+  }
+
+
+  public CodecFactoryExposer getCodecFactoryExposer() {
+    return codecFactoryExposer;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/0a2f997f/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetStorageEngineConfig.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetStorageEngineConfig.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetStorageEngineConfig.java
new file mode 100644
index 0000000..ad55f13
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetStorageEngineConfig.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.store.parquet;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.annotation.JsonTypeName;
+import org.apache.drill.common.logical.StorageEngineConfigBase;
+
+import java.util.HashMap;
+
+@JsonTypeName("parquet")
+public class ParquetStorageEngineConfig extends StorageEngineConfigBase {
+
+  public String getDfsName() {
+    return dfsName;
+  }
+
+  // information needed to identify an HDFS instance
+  private String dfsName;
+  private HashMap<String,String> map;
+
+  @JsonCreator
+  public ParquetStorageEngineConfig(@JsonProperty("dfsName") String dfsName) {
+    this.dfsName = dfsName;
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) return true;
+    if (o == null || getClass() != o.getClass()) return false;
+
+    ParquetStorageEngineConfig that = (ParquetStorageEngineConfig) o;
+
+    if (dfsName != null ? !dfsName.equals(that.dfsName) : that.dfsName != null) return false;
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    return dfsName != null ? dfsName.hashCode() : 0;
+  }
+  public void set(String key, String value) {
+    map.put(key, value);
+  }
+
+  public String get(String key) {
+    return map.get(key);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/0a2f997f/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/VarLenBinaryReader.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/VarLenBinaryReader.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/VarLenBinaryReader.java
new file mode 100644
index 0000000..08e1023
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/VarLenBinaryReader.java
@@ -0,0 +1,130 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.parquet;
+
+import org.apache.drill.exec.vector.ValueVector;
+import org.apache.drill.exec.vector.VarBinaryVector;
+import parquet.bytes.BytesUtils;
+import parquet.column.ColumnDescriptor;
+import parquet.hadoop.metadata.ColumnChunkMetaData;
+
+import java.io.IOException;
+import java.util.List;
+
+public class VarLenBinaryReader {
+
+  ParquetRecordReader parentReader;
+  final List<VarLengthColumn> columns;
+
+  public VarLenBinaryReader(ParquetRecordReader parentReader, List<VarLengthColumn> columns){
+    this.parentReader = parentReader;
+    this.columns = columns;
+  }
+
+  public static class VarLengthColumn extends ColumnReader {
+
+    VarLengthColumn(ParquetRecordReader parentReader, int allocateSize, ColumnDescriptor descriptor, ColumnChunkMetaData columnChunkMetaData, boolean fixedLength, ValueVector v) {
+      super(parentReader, allocateSize, descriptor, columnChunkMetaData, fixedLength, v);
+    }
+
+    @Override
+    protected void readField(long recordsToRead, ColumnReader firstColumnStatus) {
+      throw new UnsupportedOperationException();
+    }
+  }
+
+  /**
+   * Reads as many variable length values as possible.
+   *
+   * @param recordsToReadInThisPass - the number of records recommended for reading form the reader
+   * @param firstColumnStatus - a reference to the first column status in the parquet file to grab metatdata from
+   * @return - the number of fixed length fields that will fit in the batch
+   * @throws IOException
+   */
+  public long readFields(long recordsToReadInThisPass, ColumnReader firstColumnStatus) throws IOException {
+
+    long recordsReadInCurrentPass = 0;
+    int lengthVarFieldsInCurrentRecord;
+    boolean rowGroupFinished = false;
+    byte[] bytes;
+    VarBinaryVector currVec;
+    // write the first 0 offset
+    for (ColumnReader columnReader : columns) {
+      if (columnReader.isFixedLength) {
+        continue;
+      }
+      currVec = (VarBinaryVector) columnReader.valueVecHolder.getValueVector();
+      currVec.getAccessor().getOffsetVector().getData().writeInt(0);
+      columnReader.bytesReadInCurrentPass = 0;
+      columnReader.valuesReadInCurrentPass = 0;
+    }
+    do {
+      lengthVarFieldsInCurrentRecord = 0;
+      for (ColumnReader columnReader : columns) {
+        if (columnReader.isFixedLength) {
+          continue;
+        }
+        if (columnReader.pageReadStatus.currentPage == null
+            || columnReader.pageReadStatus.valuesRead == columnReader.pageReadStatus.currentPage.getValueCount()) {
+          columnReader.totalValuesRead += columnReader.pageReadStatus.valuesRead;
+          if (!columnReader.pageReadStatus.next()) {
+            rowGroupFinished = true;
+            break;
+          }
+        }
+        bytes = columnReader.pageReadStatus.pageDataByteArray;
+
+        // re-purposing  this field here for length in BYTES to prevent repetitive multiplication/division
+        columnReader.dataTypeLengthInBits = BytesUtils.readIntLittleEndian(bytes,
+            (int) columnReader.pageReadStatus.readPosInBytes);
+        lengthVarFieldsInCurrentRecord += columnReader.dataTypeLengthInBits;
+
+      }
+      // check that the next record will fit in the batch
+      if (rowGroupFinished || (recordsReadInCurrentPass + 1) * parentReader.getBitWidthAllFixedFields() + lengthVarFieldsInCurrentRecord * 8
+          > parentReader.getBatchSize()){
+        break;
+      }
+      else{
+        recordsReadInCurrentPass++;
+      }
+      for (ColumnReader columnReader : columns) {
+        if (columnReader.isFixedLength) {
+          continue;
+        }
+        bytes = columnReader.pageReadStatus.pageDataByteArray;
+        currVec = (VarBinaryVector) columnReader.valueVecHolder.getValueVector();
+        // again, I am re-purposing the unused field here, it is a length n BYTES, not bits
+        currVec.getAccessor().getOffsetVector().getData().writeInt((int) columnReader.bytesReadInCurrentPass  +
+            columnReader.dataTypeLengthInBits - 4 * (int) columnReader.valuesReadInCurrentPass);
+        currVec.getData().writeBytes(bytes, (int) columnReader.pageReadStatus.readPosInBytes + 4,
+            columnReader.dataTypeLengthInBits);
+        columnReader.pageReadStatus.readPosInBytes += columnReader.dataTypeLengthInBits + 4;
+        columnReader.bytesReadInCurrentPass += columnReader.dataTypeLengthInBits + 4;
+        columnReader.pageReadStatus.valuesRead++;
+        columnReader.valuesReadInCurrentPass++;
+        currVec.getMutator().setValueCount((int)recordsReadInCurrentPass);
+        // reached the end of a page
+        if ( columnReader.pageReadStatus.valuesRead == columnReader.pageReadStatus.currentPage.getValueCount()) {
+          columnReader.pageReadStatus.next();
+        }
+      }
+    } while (recordsReadInCurrentPass < recordsToReadInThisPass);
+    return recordsReadInCurrentPass;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/0a2f997f/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/BaseDataValueVector.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/BaseDataValueVector.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/BaseDataValueVector.java
index 4bfab47..54a6cb8 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/BaseDataValueVector.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/BaseDataValueVector.java
@@ -7,7 +7,7 @@ import org.apache.drill.exec.proto.UserBitShared.FieldMetadata;
 import org.apache.drill.exec.record.DeadBuf;
 import org.apache.drill.exec.record.MaterializedField;
 
-abstract class BaseDataValueVector extends BaseValueVector{
+public abstract class BaseDataValueVector extends BaseValueVector{
 
   protected ByteBuf data = DeadBuf.DEAD_BUFFER;
   protected int valueCount;
@@ -47,6 +47,10 @@ abstract class BaseDataValueVector extends BaseValueVector{
   public FieldMetadata getMetadata() {
     return null;
   }
+
+  public ByteBuf getData(){
+    return data;
+  }
   
   
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/0a2f997f/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/AbstractFragmentRunnerListener.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/AbstractFragmentRunnerListener.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/AbstractFragmentRunnerListener.java
index 9fd33b9..2c86406 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/AbstractFragmentRunnerListener.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/AbstractFragmentRunnerListener.java
@@ -91,9 +91,7 @@ public abstract class AbstractFragmentRunnerListener implements FragmentRunnerLi
   }
   
   protected abstract void statusChange(FragmentHandle handle, FragmentStatus status);
-    
-  
-  
+
   @Override
   public final void fail(FragmentHandle handle, String message, Throwable excep) {
     FragmentStatus.Builder status = getBuilder(FragmentState.FAILED);

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/0a2f997f/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/RemoteFragmentRunnerListener.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/RemoteFragmentRunnerListener.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/RemoteFragmentRunnerListener.java
new file mode 100644
index 0000000..93f1af7
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/RemoteFragmentRunnerListener.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.ops.FragmentContext;
+import org.apache.drill.exec.proto.ExecProtos.FragmentHandle;
+import org.apache.drill.exec.proto.ExecProtos.FragmentStatus;
+import org.apache.drill.exec.rpc.bit.BitTunnel;
+
+/**
+ * Informs remote node as fragment changes state.
+ */
+public class RemoteFragmentRunnerListener extends AbstractFragmentRunnerListener{
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(RemoteFragmentRunnerListener.class);
+  
+  private final BitTunnel tunnel;
+
+  public RemoteFragmentRunnerListener(FragmentContext context, BitTunnel tunnel) {
+    super(context);
+    this.tunnel = tunnel;
+  }
+  
+  
+  @Override
+  protected void statusChange(FragmentHandle handle, FragmentStatus status) {
+    logger.debug("Sending remote failure.");
+    tunnel.sendFragmentStatus(status);
+  }
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/0a2f997f/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
deleted file mode 100644
index ef7bcb1..0000000
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/RemotingFragmentRunnerListener.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.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) {
-    logger.debug("Sending remote failure.");
-    tunnel.sendFragmentStatus(status);
-  }
-  
-}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/0a2f997f/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/BitComHandlerImpl.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/BitComHandlerImpl.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/BitComHandlerImpl.java
index 2829dfd..b6e0159 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/BitComHandlerImpl.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/BitComHandlerImpl.java
@@ -48,10 +48,10 @@ 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.RemoteFragmentRunnerListener;
+import org.apache.drill.exec.work.fragment.RemoteFragmentHandler;
 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;
@@ -116,7 +116,7 @@ public class BitComHandlerImpl implements BitComHandler {
     logger.debug("Received remote fragment start instruction", fragment);
     FragmentContext context = new FragmentContext(bee.getContext(), fragment.getHandle(), null, null,new FunctionImplementationRegistry(bee.getContext().getConfig()));
     BitTunnel tunnel = bee.getContext().getBitCom().getTunnel(fragment.getForeman());
-    RemotingFragmentRunnerListener listener = new RemotingFragmentRunnerListener(context, tunnel);
+    RemoteFragmentRunnerListener listener = new RemoteFragmentRunnerListener(context, tunnel);
     try{
       FragmentRoot rootOperator = bee.getContext().getPlanReader().readFragmentOperator(fragment.getFragmentJson());
       RootExec exec = ImplCreator.getExec(context, rootOperator);

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/0a2f997f/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/Foreman.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/Foreman.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/Foreman.java
index 0a4614a..c9c23b5 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/Foreman.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/Foreman.java
@@ -166,7 +166,7 @@ public class Foreman implements Runnable, Closeable, Comparable<Object>{
       LogicalPlan logicalPlan = context.getPlanReader().readLogicalPlan(json);
       logger.debug("Logical {}", logicalPlan.unparse(DrillConfig.create()));
       PhysicalPlan physicalPlan = convert(logicalPlan);
-      logger.debug("Physical {}", new ObjectMapper().writeValueAsString(physicalPlan));
+      //logger.debug("Physical {}", new ObjectMapper().writeValueAsString(physicalPlan));
       runPhysicalPlan(physicalPlan);
     } catch (IOException e) {
       fail("Failure while parsing logical plan.", e);

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/0a2f997f/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/RemoteFragmentHandler.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/RemoteFragmentHandler.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/RemoteFragmentHandler.java
index e4d0cfc..d63b4f4 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/RemoteFragmentHandler.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/RemoteFragmentHandler.java
@@ -36,7 +36,7 @@ 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.RemoteFragmentRunnerListener;
 import org.apache.drill.exec.work.batch.IncomingBuffers;
 
 /**
@@ -58,7 +58,7 @@ public class RemoteFragmentHandler implements IncomingFragmentHandler {
       this.root = context.getPlanReader().readFragmentOperator(fragment.getFragmentJson());
       this.buffers = new IncomingBuffers(root);
       this.context = new FragmentContext(context, fragment.getHandle(), null, buffers, new FunctionImplementationRegistry(context.getConfig()));
-      this.runnerListener = new RemotingFragmentRunnerListener(this.context, foremanTunnel);
+      this.runnerListener = new RemoteFragmentRunnerListener(this.context, foremanTunnel);
       this.reader = context.getPlanReader();
       
     }catch(IOException e){

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/0a2f997f/sandbox/prototype/exec/java-exec/src/main/java/parquet/hadoop/CodecFactoryExposer.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/parquet/hadoop/CodecFactoryExposer.java b/sandbox/prototype/exec/java-exec/src/main/java/parquet/hadoop/CodecFactoryExposer.java
new file mode 100644
index 0000000..72322d5
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/parquet/hadoop/CodecFactoryExposer.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 parquet.hadoop;
+
+import com.sun.corba.se.impl.interceptors.CodecFactoryImpl;
+import org.apache.hadoop.conf.Configuration;
+import parquet.bytes.BytesInput;
+import parquet.hadoop.metadata.CompressionCodecName;
+
+import java.io.IOException;
+
+public class CodecFactoryExposer{
+
+  private CodecFactory codecFactory;
+
+  public CodecFactoryExposer(Configuration config){
+    codecFactory = new CodecFactory(config);
+  }
+
+  public CodecFactory getCodecFactory() {
+    return codecFactory;
+  }
+
+  public BytesInput decompress(BytesInput bytes, int uncompressedSize, CompressionCodecName codecName) throws IOException {
+    return codecFactory.getDecompressor(codecName).decompress(bytes, uncompressedSize);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/0a2f997f/sandbox/prototype/exec/java-exec/src/main/resources/drill-module.conf
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/resources/drill-module.conf b/sandbox/prototype/exec/java-exec/src/main/resources/drill-module.conf
index a543197..a590420 100644
--- a/sandbox/prototype/exec/java-exec/src/main/resources/drill-module.conf
+++ b/sandbox/prototype/exec/java-exec/src/main/resources/drill-module.conf
@@ -8,17 +8,18 @@ drill.exec: {
   cluster-id: "drillbits1"
   rpc: {
   	user.port : 31010,
-  	bit.port : 31011
+  	bit.port : 31011,
+  	use.ip : false
   },
   optimizer: {
     implementation: "org.apache.drill.exec.opt.IdentityOptimizer"
   },
   
   zk: {
-	connect: "localhost:2181",
+	connect: "10.10.30.52:5181",
 	root: "/drill",
 	refresh: 500,
-	timeout: 1000,
+	timeout: 5000,
   	retry: {
   	  count: 7200,
   	  delay: 500

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/0a2f997f/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestDistributedFragmentRun.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestDistributedFragmentRun.java b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestDistributedFragmentRun.java
index 1513c99..f6d83cc 100644
--- a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestDistributedFragmentRun.java
+++ b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestDistributedFragmentRun.java
@@ -21,13 +21,17 @@ import static org.junit.Assert.*;
 
 import java.util.List;
 
+import org.apache.drill.common.config.DrillConfig;
+import org.apache.drill.common.logical.LogicalPlan;
 import org.apache.drill.common.util.FileUtils;
 import org.apache.drill.exec.client.DrillClient;
 import org.apache.drill.exec.pop.PopUnitTestBase;
 import org.apache.drill.exec.proto.UserProtos.QueryType;
+import org.apache.drill.exec.record.RecordBatchLoader;
 import org.apache.drill.exec.rpc.user.QueryResultBatch;
 import org.apache.drill.exec.server.Drillbit;
 import org.apache.drill.exec.server.RemoteServiceSet;
+import org.apache.drill.exec.vector.ValueVector;
 import org.junit.Test;
 
 import com.google.common.base.Charsets;
@@ -37,7 +41,6 @@ import com.google.common.io.Files;
 public class TestDistributedFragmentRun extends PopUnitTestBase{
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestDistributedFragmentRun.class);
   
-  
   @Test 
   public void oneBitOneExchangeOneEntryRun() throws Exception{
     RemoteServiceSet serviceSet = RemoteServiceSet.getLocalServiceSet();

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/0a2f997f/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/ByteArrayUtil.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/ByteArrayUtil.java b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/ByteArrayUtil.java
new file mode 100644
index 0000000..ea1404b
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/ByteArrayUtil.java
@@ -0,0 +1,181 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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;
+
+//TODO - make sure we figure out the license on these
+public class ByteArrayUtil {
+
+  public static byte[] toByta(Object data) throws Exception {
+    if (data instanceof Integer) return toByta((int) data);
+    else if (data instanceof Double) return toByta((double) data);
+    else if (data instanceof Float) return toByta((float) data);
+    else if (data instanceof Boolean) return toByta((boolean) data);
+    else if (data instanceof Long) return toByta((long) data);
+    else throw new Exception("Cannot convert that type to a byte array.");
+  }
+
+  // found at http://www.daniweb.com/software-development/java/code/216874/primitive-types-as-byte-arrays
+  // I have modified them to switch the endianess of integers and longs
+  /* ========================= */
+  /* "primitive type --> byte[] data" Methods */
+  /* ========================= */
+  public static byte[] toByta(byte data) {
+    return new byte[]{data};
+  }
+
+  public static byte[] toByta(byte[] data) {
+    return data;
+  }
+
+  /* ========================= */
+  public static byte[] toByta(short data) {
+    return new byte[]{
+        (byte) ((data >> 8) & 0xff),
+        (byte) ((data >> 0) & 0xff),
+    };
+  }
+
+  public static byte[] toByta(short[] data) {
+    if (data == null) return null;
+    // ----------
+    byte[] byts = new byte[data.length * 2];
+    for (int i = 0; i < data.length; i++)
+      System.arraycopy(toByta(data[i]), 0, byts, i * 2, 2);
+    return byts;
+  }
+
+  /* ========================= */
+  public static byte[] toByta(char data) {
+    return new byte[]{
+        (byte) ((data >> 8) & 0xff),
+        (byte) ((data >> 0) & 0xff),
+    };
+  }
+
+  public static byte[] toByta(char[] data) {
+    if (data == null) return null;
+    // ----------
+    byte[] byts = new byte[data.length * 2];
+    for (int i = 0; i < data.length; i++)
+      System.arraycopy(toByta(data[i]), 0, byts, i * 2, 2);
+    return byts;
+  }
+
+  /* ========================= */
+  public static byte[] toByta(int data) {
+    return new byte[]{
+        (byte) ((data >> 0) & 0xff),
+        (byte) ((data >> 8) & 0xff),
+        (byte) ((data >> 16) & 0xff),
+        (byte) ((data >> 24) & 0xff),
+    };
+  }
+
+  public static byte[] toByta(int[] data) {
+    if (data == null) return null;
+    // ----------
+    byte[] byts = new byte[data.length * 4];
+    for (int i = 0; i < data.length; i++)
+      System.arraycopy(toByta(data[i]), 0, byts, i * 4, 4);
+    return byts;
+  }
+
+  /* ========================= */
+  public static byte[] toByta(long data) {
+    return new byte[]{
+        (byte) ((data >> 0) & 0xff),
+        (byte) ((data >> 8) & 0xff),
+        (byte) ((data >> 16) & 0xff),
+        (byte) ((data >> 24) & 0xff),
+        (byte) ((data >> 32) & 0xff),
+        (byte) ((data >> 40) & 0xff),
+        (byte) ((data >> 48) & 0xff),
+        (byte) ((data >> 56) & 0xff),
+    };
+  }
+
+  public static byte[] toByta(long[] data) {
+    if (data == null) return null;
+    // ----------
+    byte[] byts = new byte[data.length * 8];
+    for (int i = 0; i < data.length; i++)
+      System.arraycopy(toByta(data[i]), 0, byts, i * 8, 8);
+    return byts;
+  }
+
+  /* ========================= */
+  public static byte[] toByta(float data) {
+    return toByta(Float.floatToRawIntBits(data));
+  }
+
+  public static byte[] toByta(float[] data) {
+    if (data == null) return null;
+    // ----------
+    byte[] byts = new byte[data.length * 4];
+    for (int i = 0; i < data.length; i++)
+      System.arraycopy(toByta(data[i]), 0, byts, i * 4, 4);
+    return byts;
+  }
+
+  /* ========================= */
+  public static byte[] toByta(double data) {
+    return toByta(Double.doubleToRawLongBits(data));
+  }
+
+  public static byte[] toByta(double[] data) {
+    if (data == null) return null;
+    // ----------
+    byte[] byts = new byte[data.length * 8];
+    for (int i = 0; i < data.length; i++)
+      System.arraycopy(toByta(data[i]), 0, byts, i * 8, 8);
+    return byts;
+  }
+
+  /* ========================= */
+  public static byte[] toByta(boolean data) {
+    return new byte[]{(byte) (data ? 0x01 : 0x00)}; // bool -> {1 byte}
+  }
+
+  public static byte[] toByta(boolean[] data) {
+    // Advanced Technique: The byte array containts information
+    // about how many boolean values are involved, so the exact
+    // array is returned when later decoded.
+    // ----------
+    if (data == null) return null;
+    // ----------
+    int len = data.length;
+    byte[] lena = toByta(len); // int conversion; length array = lena
+    byte[] byts = new byte[lena.length + (len / 8) + (len % 8 != 0 ? 1 : 0)];
+    // (Above) length-array-length + sets-of-8-booleans +? byte-for-remainder
+    System.arraycopy(lena, 0, byts, 0, lena.length);
+    // ----------
+    // (Below) algorithm by Matthew Cudmore: boolean[] -> bits -> byte[]
+    for (int i = 0, j = lena.length, k = 7; i < data.length; i++) {
+      byts[j] |= (data[i] ? 1 : 0) << k--;
+      if (k < 0) {
+        j++;
+        k = 7;
+      }
+    }
+    // ----------
+    return byts;
+  }
+
+  // above utility methods found here:
+  // http://www.daniweb.com/software-development/java/code/216874/primitive-types-as-byte-arrays
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/0a2f997f/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/JSONRecordReaderTest.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/JSONRecordReaderTest.java b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/JSONRecordReaderTest.java
index 6b353ae..c9d6967 100644
--- a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/JSONRecordReaderTest.java
+++ b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/JSONRecordReaderTest.java
@@ -52,6 +52,11 @@ public class JSONRecordReaderTest {
     }
 
     @Override
+    public void removeAllFields() {
+      addFields.clear();
+    }
+
+    @Override
     public void setNewSchema() throws SchemaChangeException {
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/0a2f997f/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/MockScantTest.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/MockScantTest.java b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/MockScantTest.java
new file mode 100644
index 0000000..f9a1ecb
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/MockScantTest.java
@@ -0,0 +1,115 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.store;
+
+import com.beust.jcommander.internal.Lists;
+import com.google.common.base.Charsets;
+import com.google.common.io.Files;
+import org.apache.drill.common.config.DrillConfig;
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.common.util.FileUtils;
+import org.apache.drill.exec.client.DrillClient;
+import org.apache.drill.exec.exception.SchemaChangeException;
+import org.apache.drill.exec.physical.impl.OutputMutator;
+import org.apache.drill.exec.proto.UserProtos;
+import org.apache.drill.exec.record.MaterializedField;
+import org.apache.drill.exec.record.RecordBatchLoader;
+import org.apache.drill.exec.record.VectorWrapper;
+import org.apache.drill.exec.rpc.RpcException;
+import org.apache.drill.exec.rpc.user.QueryResultBatch;
+import org.apache.drill.exec.rpc.user.UserResultsListener;
+import org.apache.drill.exec.server.BootStrapContext;
+import org.apache.drill.exec.server.Drillbit;
+import org.apache.drill.exec.server.RemoteServiceSet;
+import org.apache.drill.exec.store.parquet.ParquetStorageEngine;
+import org.apache.drill.exec.vector.BaseDataValueVector;
+import org.apache.drill.exec.vector.ValueVector;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.junit.Test;
+import parquet.bytes.BytesInput;
+import parquet.column.ColumnDescriptor;
+import parquet.column.page.Page;
+import parquet.column.page.PageReadStore;
+import parquet.column.page.PageReader;
+import parquet.hadoop.Footer;
+import parquet.hadoop.ParquetFileWriter;
+import parquet.hadoop.metadata.CompressionCodecName;
+import parquet.hadoop.metadata.ParquetMetadata;
+import parquet.schema.MessageType;
+import parquet.schema.MessageTypeParser;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CountDownLatch;
+
+import static org.junit.Assert.*;
+import static parquet.column.Encoding.PLAIN;
+
+
+public class MockScantTest {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(StorageEngineRegistry.class);
+
+  private boolean VERBOSE_DEBUG = false;
+
+  private class ParquetResultListener implements UserResultsListener {
+
+    CountDownLatch latch = new CountDownLatch(1);
+    @Override
+    public void submissionFailed(RpcException ex) {
+      latch.countDown();
+    }
+
+    @Override
+    public void resultArrived(QueryResultBatch result) {
+      if(result.getHeader().getIsLastChunk()) latch.countDown();
+      result.getData().release(1);
+    }
+
+    public void await() throws Exception {
+      latch.await();
+    }
+  }
+
+
+  @Test
+  public void testMockScanFullEngine() throws Exception{
+    RemoteServiceSet serviceSet = RemoteServiceSet.getLocalServiceSet();
+
+    DrillConfig config = DrillConfig.create();
+
+//    try(Drillbit bit1 = new Drillbit(config, serviceSet); DrillClient client = new DrillClient(config, serviceSet.getCoordinator());){
+    try(DrillClient client = new DrillClient(config)){
+      long A = System.nanoTime();
+//      bit1.run();
+      long B = System.nanoTime();
+      client.connect();
+      long C = System.nanoTime();
+      ParquetResultListener listener = new ParquetResultListener();
+      client.runQuery(UserProtos.QueryType.PHYSICAL, Files.toString(FileUtils.getResourceAsFile("/mock-scan.json"), Charsets.UTF_8), listener);
+      listener.await();
+      long D = System.nanoTime();
+      System.out.println(String.format("Took %f s to run query", (float)(D-C) / 1E9));
+    }
+  }
+}


[08/27] Initial Parquet commit. Suports INT, LONG, FLOAT, DOUBLE, distributed scheduling.

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/0a2f997f/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/ParquetRecordReaderTest.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/ParquetRecordReaderTest.java b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/ParquetRecordReaderTest.java
new file mode 100644
index 0000000..75a52c5
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/ParquetRecordReaderTest.java
@@ -0,0 +1,594 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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 com.beust.jcommander.internal.Lists;
+import com.google.common.base.Charsets;
+import com.google.common.io.Files;
+
+import com.google.common.util.concurrent.SettableFuture;
+import org.apache.drill.common.config.DrillConfig;
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.common.util.FileUtils;
+import org.apache.drill.exec.client.DrillClient;
+import org.apache.drill.exec.exception.SchemaChangeException;
+import org.apache.drill.exec.physical.impl.OutputMutator;
+
+import org.apache.drill.exec.proto.UserProtos;
+import org.apache.drill.exec.record.MaterializedField;
+import org.apache.drill.exec.record.RecordBatchLoader;
+import org.apache.drill.exec.record.VectorWrapper;
+import org.apache.drill.exec.rpc.RpcException;
+import org.apache.drill.exec.rpc.user.QueryResultBatch;
+import org.apache.drill.exec.rpc.user.UserResultsListener;
+import org.apache.drill.exec.server.BootStrapContext;
+import org.apache.drill.exec.server.Drillbit;
+import org.apache.drill.exec.server.RemoteServiceSet;
+
+import org.apache.drill.exec.store.parquet.ParquetStorageEngine;
+import org.apache.drill.exec.vector.BaseDataValueVector;
+import org.apache.drill.exec.vector.ValueVector;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.junit.Test;
+import parquet.bytes.BytesInput;
+import parquet.column.ColumnDescriptor;
+import parquet.column.page.Page;
+import parquet.column.page.PageReadStore;
+import parquet.column.page.PageReader;
+import parquet.hadoop.Footer;
+
+import parquet.hadoop.ParquetFileWriter;
+import parquet.hadoop.metadata.CompressionCodecName;
+import parquet.hadoop.metadata.ParquetMetadata;
+import parquet.schema.MessageType;
+import parquet.schema.MessageTypeParser;
+
+import java.io.File;
+import java.io.IOException;
+import java.text.SimpleDateFormat;
+import java.util.*;
+
+import static org.junit.Assert.*;
+import static org.junit.Assert.assertEquals;
+import static parquet.column.Encoding.PLAIN;
+
+public class ParquetRecordReaderTest {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(StorageEngineRegistry.class);
+
+  private boolean VERBOSE_DEBUG = false;
+
+  @Test
+  public void testMultipleRowGroupsAndReads() throws Exception {
+    testParquetFullEngine(true, "/parquet_scan_screen.json", "/tmp/testParquetFile_many_types_3", 2, numberRowGroups, recordsPerRowGroup);
+  }
+
+  @Test
+  public void testMultipleRowGroupsAndReadsEvent() throws Exception {
+    testParquetFullEngineEventBased(true, "/parquet_scan_screen.json", "/tmp/testParquetFile_many_types_3", 2, numberRowGroups, recordsPerRowGroup);
+  }
+
+  int numberRowGroups = 20;
+  static int recordsPerRowGroup = 3000000;
+
+  // 10 mb per page
+  static int bytesPerPage = 1024 * 1024 * 10;
+  // { 00000001, 00000010, 00000100, 00001000, 00010000, ... }
+  byte[] bitFields = {1, 2, 4, 8, 16, 32, 64, -128};
+  static final byte allBitsTrue = -1;
+  static final byte allBitsFalse = 0;
+  static final byte[] varLen1 = {50, 51, 52, 53, 54, 55, 56};
+  static final byte[] varLen2 = {15, 14, 13, 12, 11, 10, 9, 8, 7, 6, 5, 4, 3, 2, 1};
+  static final byte[] varLen3 = {100, 99, 98};
+
+  private static class FieldInfo {
+
+    String parquetType;
+    String name;
+    int bitLength;
+    int numberOfPages;
+    Object[] values;
+    TypeProtos.MinorType type;
+
+    FieldInfo(String parquetType, String name, int bitLength, Object[] values, TypeProtos.MinorType type){
+      this.parquetType = parquetType;
+      this.name = name;
+      this.bitLength  = bitLength;
+      this.numberOfPages = Math.max(1, (int) Math.ceil(recordsPerRowGroup * bitLength / 8.0 / bytesPerPage));
+      this.values = values;
+      // generator is designed to use 3 values
+      assert values.length == 3;
+      this.type = type;
+    }
+  }
+
+  
+  private static HashMap<String, FieldInfo> fields = new HashMap<>();
+  static {
+    Object[] intVals = {-200, 100, Integer.MAX_VALUE };
+    Object[] longVals = { -5000l, 5000l, Long.MAX_VALUE};
+    Object[] floatVals = { 1.74f, Float.MAX_VALUE, Float.MIN_VALUE};
+    Object[] doubleVals = {100.45d, Double.MAX_VALUE, Double.MIN_VALUE,};
+    Object[] boolVals = {false, false, true};
+    Object[] binVals = { varLen1, varLen2, varLen3};
+    Object[] bin2Vals = { varLen3, varLen2, varLen1};
+    fields.put("integer/", new FieldInfo("int32", "integer", 32, intVals, TypeProtos.MinorType.INT));
+    fields.put("bigInt/", new FieldInfo("int64", "bigInt", 64, longVals, TypeProtos.MinorType.BIGINT));
+    fields.put("f/", new FieldInfo("float", "f", 32, floatVals, TypeProtos.MinorType.FLOAT4));
+    fields.put("d/", new FieldInfo("double", "d", 64, doubleVals, TypeProtos.MinorType.FLOAT8));
+//    fields.put("b/", new FieldInfo("binary", "b", 1, boolVals, TypeProtos.MinorType.BIT));
+    fields.put("bin/", new FieldInfo("binary", "bin", -1, binVals, TypeProtos.MinorType.VARBINARY));
+    fields.put("bin2/", new FieldInfo("binary", "bin2", -1, bin2Vals, TypeProtos.MinorType.VARBINARY));
+  }
+
+
+  private String getResource(String resourceName) {
+    return "resource:" + resourceName;
+  }
+
+  public void generateParquetFile(String filename, int numberRowGroups, int recordsPerRowGroup) throws Exception {
+
+    int currentBooleanByte = 0;
+    WrapAroundCounter booleanBitCounter = new WrapAroundCounter(7);
+
+    Configuration configuration = new Configuration();
+    configuration.set(ParquetStorageEngine.HADOOP_DEFAULT_NAME, "file:///");
+    //"message m { required int32 integer; required int64 integer64; required boolean b; required float f; required double d;}"
+
+    FileSystem fs = FileSystem.get(configuration);
+    Path path = new Path(filename);
+    if (fs.exists(path)) fs.delete(path, false);
+
+
+    String messageSchema = "message m {";
+    for (FieldInfo fieldInfo : fields.values()) {
+      messageSchema += " required " + fieldInfo.parquetType + " " + fieldInfo.name + ";";
+    }
+    // remove the last semicolon, java really needs a join method for strings...
+    // TODO - nvm apparently it requires a semicolon after every field decl, might want to file a bug
+    //messageSchema = messageSchema.substring(schemaType, messageSchema.length() - 1);
+    messageSchema += "}";
+
+    MessageType schema = MessageTypeParser.parseMessageType(messageSchema);
+
+    CompressionCodecName codec = CompressionCodecName.UNCOMPRESSED;
+    ParquetFileWriter w = new ParquetFileWriter(configuration, schema, path);
+    w.start();
+    HashMap<String, Integer> columnValuesWritten = new HashMap();
+    int valsWritten;
+    for (int k = 0; k < numberRowGroups; k++){
+      w.startBlock(1);
+
+      for (FieldInfo fieldInfo : fields.values()) {
+
+        if ( ! columnValuesWritten.containsKey(fieldInfo.name)){
+          columnValuesWritten.put((String) fieldInfo.name, 0);
+          valsWritten = 0;
+        } else {
+          valsWritten = columnValuesWritten.get(fieldInfo.name);
+        }
+
+        String[] path1 = {(String) fieldInfo.name};
+        ColumnDescriptor c1 = schema.getColumnDescription(path1);
+
+        w.startColumn(c1, recordsPerRowGroup, codec);
+        int valsPerPage = (int) Math.ceil(recordsPerRowGroup / (float) ((int) fieldInfo.numberOfPages));
+        byte[] bytes;
+        // for variable length binary fields
+        int bytesNeededToEncodeLength = 4;
+        if ((int) fieldInfo.bitLength > 0) {
+          bytes = new byte[(int) Math.ceil(valsPerPage * (int) fieldInfo.bitLength / 8.0)];
+        } else {
+          // the twelve at the end is to account for storing a 4 byte length with each value
+          int totalValLength = ((byte[]) fieldInfo.values[0]).length + ((byte[]) fieldInfo.values[1]).length + ((byte[]) fieldInfo.values[2]).length + 3 * bytesNeededToEncodeLength;
+          // used for the case where there is a number of values in this row group that is not divisible by 3
+          int leftOverBytes = 0;
+          if ( valsPerPage % 3 > 0 ) leftOverBytes += ((byte[])fieldInfo.values[1]).length + 4;
+          if ( valsPerPage % 3 > 1 ) leftOverBytes += ((byte[])fieldInfo.values[2]).length + 4;
+          bytes = new byte[valsPerPage / 3 * totalValLength + leftOverBytes];
+        }
+        int bytesPerPage = (int) (valsPerPage * ((int) fieldInfo.bitLength / 8.0));
+        int bytesWritten = 0;
+        for (int z = 0; z < (int) fieldInfo.numberOfPages; z++, bytesWritten = 0) {
+          for (int i = 0; i < valsPerPage; i++) {
+            //System.out.print(i + ", " + (i % 25 == 0 ? "\n gen " + fieldInfo.name + ": " : ""));
+            if (fieldInfo.values[0] instanceof Boolean) {
+
+              bytes[currentBooleanByte] |= bitFields[booleanBitCounter.val] & ((boolean) fieldInfo.values[valsWritten % 3]
+                  ? allBitsTrue : allBitsFalse);
+              booleanBitCounter.increment();
+              if (booleanBitCounter.val == 0) {
+                currentBooleanByte++;
+              }
+              valsWritten++;
+              if (currentBooleanByte > bytesPerPage) break;
+            } else {
+              if (fieldInfo.values[valsWritten % 3] instanceof byte[]){
+                System.arraycopy(ByteArrayUtil.toByta(((byte[])fieldInfo.values[valsWritten % 3]).length),
+                    0, bytes, bytesWritten, bytesNeededToEncodeLength);
+                System.arraycopy(fieldInfo.values[valsWritten % 3],
+                    0, bytes, bytesWritten + bytesNeededToEncodeLength, ((byte[])fieldInfo.values[valsWritten % 3]).length);
+                bytesWritten += ((byte[])fieldInfo.values[valsWritten % 3]).length + bytesNeededToEncodeLength;
+              }
+              else{
+                System.arraycopy( ByteArrayUtil.toByta(fieldInfo.values[valsWritten % 3]),
+                    0, bytes, i * ((int) fieldInfo.bitLength / 8), (int) fieldInfo.bitLength / 8);
+              }
+              valsWritten++;
+            }
+
+          }
+          w.writeDataPage((int)(recordsPerRowGroup / (int) fieldInfo.numberOfPages), bytes.length, BytesInput.from(bytes), PLAIN, PLAIN, PLAIN);
+          currentBooleanByte = 0;
+        }
+        w.endColumn();
+        columnValuesWritten.remove((String) fieldInfo.name);
+        columnValuesWritten.put((String) fieldInfo.name, valsWritten);
+      }
+
+      w.endBlock();
+    }
+    w.end(new HashMap<String, String>());
+    logger.debug("Finished generating parquet file.");
+  }
+
+  private class ParquetResultListener implements UserResultsListener {
+    private Vector<QueryResultBatch> results = new Vector<QueryResultBatch>();
+    private SettableFuture<List<QueryResultBatch>> future = SettableFuture.create();
+    int count = 0;
+    RecordBatchLoader batchLoader;
+    byte[] bytes;
+
+    int batchCounter = 1;
+    int columnValCounter = 0;
+    int i = 0;
+    FieldInfo currentField;
+    HashMap<String, Integer> valuesChecked = new HashMap();
+
+    ParquetResultListener(RecordBatchLoader batchLoader){
+      this.batchLoader = batchLoader;
+    }
+
+    @Override
+    public void submissionFailed(RpcException ex) {
+      logger.debug("Submission failed.", ex);
+      future.setException(ex);
+    }
+
+    @Override
+    public void resultArrived(QueryResultBatch result) {
+      logger.debug("result arrived in test batch listener.");
+      int columnValCounter = 0;
+      int i = 0;
+      FieldInfo currentField;
+      count += result.getHeader().getRowCount();
+      boolean schemaChanged = false;
+      try {
+        schemaChanged = batchLoader.load(result.getHeader().getDef(), result.getData());
+      } catch (SchemaChangeException e) {
+        e.printStackTrace();
+      }
+
+      int recordCount = 0;
+      // print headers.
+      if (schemaChanged) {
+      } // do not believe any change is needed for when the schema changes, with the current mock scan use case
+
+      for (VectorWrapper vw : batchLoader) {
+        ValueVector vv = vw.getValueVector();
+        currentField = fields.get(vv.getField().getName());
+        if (VERBOSE_DEBUG){
+          System.out.println("\n" + (String) currentField.name);
+        }
+        if ( ! valuesChecked.containsKey(vv.getField().getName())){
+          valuesChecked.put(vv.getField().getName(), 0);
+          columnValCounter = 0;
+        } else {
+          columnValCounter = valuesChecked.get(vv.getField().getName());
+        }
+        for (int j = 0; j < ((BaseDataValueVector)vv).getAccessor().getValueCount(); j++) {
+          if (VERBOSE_DEBUG){
+            System.out.print(vv.getAccessor().getObject(j) + ", " + (j % 25 == 0 ? "\n batch:" + batchCounter + " v:" + j + " - " : ""));
+          }
+          assertField(vv, j, (TypeProtos.MinorType) currentField.type,
+              currentField.values[columnValCounter % 3], (String) currentField.name + "/");
+          columnValCounter++;
+        }
+        if (VERBOSE_DEBUG){
+          System.out.println("\n" + ((BaseDataValueVector)vv).getAccessor().getValueCount());
+        }
+        valuesChecked.remove(vv.getField().getName());
+        valuesChecked.put(vv.getField().getName(), columnValCounter);
+      }
+
+      if (VERBOSE_DEBUG){
+        for (i = 0; i < batchLoader.getRecordCount(); i++) {
+          recordCount++;
+          if (i % 50 == 0){
+            System.out.println();
+            for (VectorWrapper vw : batchLoader) {
+              ValueVector v = vw.getValueVector();
+              System.out.print(pad(v.getField().getName(), 20) + " ");
+
+            }
+            System.out.println();
+            System.out.println();
+          }
+
+          for (VectorWrapper vw : batchLoader) {
+            ValueVector v = vw.getValueVector();
+            System.out.print(pad(v.getAccessor().getObject(i).toString(), 20) + " ");
+          }
+          System.out.println(
+
+          );
+        }
+      }
+      batchCounter++;
+      if(result.getHeader().getIsLastChunk()){
+        future.set(results);
+      }
+    }
+
+    public List<QueryResultBatch> getResults() throws RpcException{
+      try{
+        return future.get();
+      }catch(Throwable t){
+        throw RpcException.mapException(t);
+      }
+    }
+  }
+
+  // specific tests should call this method, but it is not marked as a test itself intentionally
+  public void testParquetFullEngineEventBased(boolean generateNew, String plan, String filename, int numberOfTimesRead /* specified in json plan */, int numberRowGroups, int recordsPerRowGroup) throws Exception{
+    RemoteServiceSet serviceSet = RemoteServiceSet.getLocalServiceSet();
+
+    if (generateNew) generateParquetFile(filename, numberRowGroups, recordsPerRowGroup);
+
+    DrillConfig config = DrillConfig.create();
+
+    try(Drillbit bit1 = new Drillbit(config, serviceSet); DrillClient client = new DrillClient(config, serviceSet.getCoordinator());){
+      bit1.run();
+      client.connect();
+      List<QueryResultBatch> results = client.runQuery(UserProtos.QueryType.LOGICAL, Files.toString(FileUtils.getResourceAsFile(plan), Charsets.UTF_8));
+      int count = 0;
+      RecordBatchLoader batchLoader = new RecordBatchLoader(bit1.getContext().getAllocator());
+      ParquetResultListener resultListener = new ParquetResultListener(batchLoader);
+      client.runQuery(UserProtos.QueryType.LOGICAL, Files.toString(FileUtils.getResourceAsFile(plan), Charsets.UTF_8), resultListener);
+    }
+  }
+
+
+  // specific tests should call this method, but it is not marked as a test itself intentionally
+  public void testParquetFullEngine(boolean generateNew, String plan, String filename, int numberOfTimesRead /* specified in json plan */, int numberRowGroups, int recordsPerRowGroup) throws Exception{
+    RemoteServiceSet serviceSet = RemoteServiceSet.getLocalServiceSet();
+
+    if (generateNew) generateParquetFile(filename, numberRowGroups, recordsPerRowGroup);
+
+    DrillConfig config = DrillConfig.create();
+
+    try(Drillbit bit1 = new Drillbit(config, serviceSet); DrillClient client = new DrillClient(config, serviceSet.getCoordinator())) {
+      long A = System.nanoTime();
+      bit1.run();
+      long B = System.nanoTime();
+      client.connect();
+      long C = System.nanoTime();
+      System.out.println( new SimpleDateFormat("mm:ss S").format(new Date()) + " :Start query");
+      List<QueryResultBatch> results = client.runQuery(UserProtos.QueryType.LOGICAL, Files.toString(FileUtils.getResourceAsFile("/parquet_scan_screen.json"), Charsets.UTF_8));
+//      List<QueryResultBatch> results = client.runQuery(UserProtos.QueryType.PHYSICAL, Files.toString(FileUtils.getResourceAsFile("/parquet_scan_union_screen_physical.json"), Charsets.UTF_8));
+      long D = System.nanoTime();
+      System.out.println(String.format("Took %f s to start drillbit", (float)(B-A) / 1E9));
+      System.out.println(String.format("Took %f s to connect", (float)(C-B) / 1E9));
+      System.out.println(String.format("Took %f s to run query", (float)(D-C) / 1E9));
+      //List<QueryResultBatch> results = client.runQuery(UserProtos.QueryType.PHYSICAL, Files.toString(FileUtils.getResourceAsFile("/parquet_scan_union_screen_physical.json"), Charsets.UTF_8));
+      int count = 0;
+//      RecordBatchLoader batchLoader = new RecordBatchLoader(new BootStrapContext(config).getAllocator());
+      RecordBatchLoader batchLoader = new RecordBatchLoader(bit1.getContext().getAllocator());
+      byte[] bytes;
+
+      int batchCounter = 1;
+      int columnValCounter = 0;
+      int i = 0;
+      FieldInfo currentField;
+      HashMap<String, Integer> valuesChecked = new HashMap();
+      for(QueryResultBatch b : results){
+        count += b.getHeader().getRowCount();
+        boolean schemaChanged = batchLoader.load(b.getHeader().getDef(), b.getData());
+
+        int recordCount = 0;
+        // print headers.
+        if (schemaChanged) {
+        } // do not believe any change is needed for when the schema changes, with the current mock scan use case
+
+        for (VectorWrapper vw : batchLoader) {
+          ValueVector vv = vw.getValueVector();
+          currentField = fields.get(vv.getField().getName());
+          if (VERBOSE_DEBUG){
+            System.out.println("\n" + (String) currentField.name);
+          }
+          if ( ! valuesChecked.containsKey(vv.getField().getName())){
+            valuesChecked.put(vv.getField().getName(), 0);
+            columnValCounter = 0;
+          } else {
+            columnValCounter = valuesChecked.get(vv.getField().getName());
+          }
+          for (int j = 0; j < ((BaseDataValueVector)vv).getAccessor().getValueCount(); j++) {
+            if (VERBOSE_DEBUG){
+              System.out.print(vv.getAccessor().getObject(j) + ", " + (j % 25 == 0 ? "\n batch:" + batchCounter + " v:" + j + " - " : ""));
+            }
+            assertField(vv, j, (TypeProtos.MinorType) currentField.type,
+                currentField.values[columnValCounter % 3], (String) currentField.name + "/");
+            columnValCounter++;
+          }
+          if (VERBOSE_DEBUG){
+            System.out.println("\n" + ((BaseDataValueVector)vv).getAccessor().getValueCount());
+          }
+          valuesChecked.remove(vv.getField().getName());
+          valuesChecked.put(vv.getField().getName(), columnValCounter);
+        }
+
+        if (VERBOSE_DEBUG){
+          for (i = 1; i < batchLoader.getRecordCount(); i++) {
+            recordCount++;
+            if (i % 50 == 0){
+              System.out.println();
+              for (VectorWrapper vw : batchLoader) {
+                ValueVector v = vw.getValueVector();
+                System.out.print(pad(v.getField().getName(), 20) + " ");
+
+              }
+              System.out.println();
+              System.out.println();
+            }
+
+            for (VectorWrapper vw : batchLoader) {
+              ValueVector v = vw.getValueVector();
+              System.out.print(pad(v.getAccessor().getObject(i).toString(), 20) + " ");
+            }
+            System.out.println(
+
+            );
+          }
+        }
+        batchCounter++;
+      }
+      for (String s : valuesChecked.keySet()) {
+        assertEquals("Record count incorrect for column: " + s, recordsPerRowGroup * numberRowGroups * numberOfTimesRead, (long) valuesChecked.get(s));
+      }
+      assert valuesChecked.keySet().size() > 0;
+    }
+  }
+
+  public String pad(String value, int length) {
+    return pad(value, length, " ");
+  }
+
+  public String pad(String value, int length, String with) {
+    StringBuilder result = new StringBuilder(length);
+    result.append(value);
+
+    while (result.length() < length) {
+      result.insert(0, with);
+    }
+
+    return result.toString();
+  }
+
+  class MockOutputMutator implements OutputMutator {
+    List<MaterializedField> removedFields = Lists.newArrayList();
+    List<ValueVector> addFields = Lists.newArrayList();
+
+    @Override
+    public void removeField(MaterializedField field) throws SchemaChangeException {
+      removedFields.add(field);
+    }
+
+    @Override
+    public void addField(ValueVector vector) throws SchemaChangeException {
+      addFields.add(vector);
+    }
+
+    @Override
+    public void removeAllFields() {
+      addFields.clear();
+    }
+
+    @Override
+    public void setNewSchema() throws SchemaChangeException {
+    }
+
+    List<MaterializedField> getRemovedFields() {
+      return removedFields;
+    }
+
+    List<ValueVector> getAddFields() {
+      return addFields;
+    }
+  }
+
+  private <T> void assertField(ValueVector valueVector, int index, TypeProtos.MinorType expectedMinorType, Object value, String name) {
+    assertField(valueVector, index, expectedMinorType, value, name, 0);
+  }
+
+  private <T> void assertField(ValueVector valueVector, int index, TypeProtos.MinorType expectedMinorType, T value, String name, int parentFieldId) {
+//    UserBitShared.FieldMetadata metadata = valueVector.getMetadata();
+//    SchemaDefProtos.FieldDef def = metadata.getDef();
+//    assertEquals(expectedMinorType, def.getMajorType().getMinorType());
+//    assertEquals(name, def.getNameList().get(0).getName());
+//    assertEquals(parentFieldId, def.getParentId());
+
+    if (expectedMinorType == TypeProtos.MinorType.MAP) {
+      return;
+    }
+
+    T val = (T) valueVector.getAccessor().getObject(index);
+    if (val instanceof byte[]) {
+      assertTrue(Arrays.equals((byte[]) value, (byte[]) val));
+    } else {
+      assertEquals(value, val);
+    }
+  }
+
+  private class WrapAroundCounter {
+
+    int maxVal;
+    int val;
+
+    public WrapAroundCounter(int maxVal) {
+      this.maxVal = maxVal;
+    }
+
+    public int increment() {
+      val++;
+      if (val > maxVal) {
+        val = 0;
+      }
+      return val;
+    }
+
+    public void reset() {
+      val = 0;
+    }
+
+  }
+
+  private void validateFooters(final List<Footer> metadata) {
+    logger.debug(metadata.toString());
+    assertEquals(3, metadata.size());
+    for (Footer footer : metadata) {
+      final File file = new File(footer.getFile().toUri());
+      assertTrue(file.getName(), file.getName().startsWith("part"));
+      assertTrue(file.getPath(), file.exists());
+      final ParquetMetadata parquetMetadata = footer.getParquetMetadata();
+      assertEquals(2, parquetMetadata.getBlocks().size());
+      final Map<String, String> keyValueMetaData = parquetMetadata.getFileMetaData().getKeyValueMetaData();
+      assertEquals("bar", keyValueMetaData.get("foo"));
+      assertEquals(footer.getFile().getName(), keyValueMetaData.get(footer.getFile().getName()));
+    }
+  }
+
+  private void validateContains(MessageType schema, PageReadStore pages, String[] path, int values, BytesInput bytes)
+      throws IOException {
+    PageReader pageReader = pages.getPageReader(schema.getColumnDescription(path));
+    Page page = pageReader.readPage();
+    assertEquals(values, page.getValueCount());
+    assertArrayEquals(bytes.toByteArray(), page.getBytes().toByteArray());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/0a2f997f/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/TestAffinityCalculator.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/TestAffinityCalculator.java b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/TestAffinityCalculator.java
new file mode 100644
index 0000000..cff7c86
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/TestAffinityCalculator.java
@@ -0,0 +1,229 @@
+package org.apache.drill.exec.store;
+
+import com.google.common.collect.ImmutableRangeMap;
+import com.google.common.collect.Range;
+import junit.framework.Assert;
+import mockit.Injectable;
+import mockit.NonStrictExpectations;
+import org.apache.drill.exec.physical.EndpointAffinity;
+import org.apache.drill.exec.proto.CoordinationProtos;
+import org.apache.drill.exec.server.DrillbitContext;
+import org.apache.drill.exec.store.parquet.ParquetGroupScan;
+import org.apache.drill.exec.store.parquet.ParquetRowGroupScan;
+import org.apache.drill.exec.store.parquet.ParquetStorageEngine;
+import org.apache.hadoop.fs.BlockLocation;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.junit.Before;
+import org.junit.Test;
+
+import static junit.framework.Assert.assertNull;
+import static org.junit.Assert.assertEquals;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+
+public class TestAffinityCalculator {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestAffinityCalculator.class);
+
+  String port = "1234";
+  final String path = "path";
+
+  public BlockLocation[] buildBlockLocations(String[] hosts, long blockSize) {
+    String[] names = new String[hosts.length];
+
+    for (int i = 0; i < hosts.length; i++) {
+      hosts[i] = "host" + i;
+      names[i] = "host:" + port;
+    }
+
+    BlockLocation[] blockLocations = new BlockLocation[3];
+    blockLocations[0] = new BlockLocation(new String[]{names[0], names[1], names[2]}, new String[]{hosts[0], hosts[1], hosts[2]}, 0, blockSize);
+    blockLocations[1] = new BlockLocation(new String[]{names[0], names[2], names[3]}, new String[]{hosts[0], hosts[2], hosts[3]}, blockSize, blockSize);
+    blockLocations[2] = new BlockLocation(new String[]{names[0], names[1], names[3]}, new String[]{hosts[0], hosts[1], hosts[3]}, blockSize*2, blockSize);
+
+    return blockLocations;
+  }
+
+  public BlockLocation[] buildBlockLocations2(String[] hosts, long blockSize) {
+    String[] names = new String[hosts.length];
+
+    for (int i = 0; i < hosts.length; i++) {
+      hosts[i] = "host" + i;
+      names[i] = "host:" + port;
+    }
+
+    BlockLocation[] blockLocations = new BlockLocation[4];
+    blockLocations[0] = new BlockLocation(new String[]{names[0]}, new String[]{hosts[0]}, 0, blockSize);
+    blockLocations[1] = new BlockLocation(new String[]{names[1]}, new String[]{hosts[1]}, blockSize, blockSize);
+    blockLocations[3] = new BlockLocation(new String[]{names[3]}, new String[]{hosts[3]}, blockSize*2, blockSize);
+    blockLocations[2] = new BlockLocation(new String[]{names[2]}, new String[]{hosts[2]}, blockSize*3, blockSize);
+
+    return blockLocations;
+  }
+  public void buildRowGroups(LinkedList<ParquetGroupScan.RowGroupInfo> rowGroups, int numberOfBlocks, long blockSize, int numberOfRowGroups) {
+    long rowGroupSize = numberOfBlocks * blockSize / numberOfRowGroups;
+
+    rowGroups.clear();
+
+    for (int i = 0; i < numberOfRowGroups; i++) {
+      rowGroups.add(new ParquetGroupScan.RowGroupInfo(path, (long)i*rowGroupSize, (long)rowGroupSize, i));
+    }
+  }
+
+  public LinkedList<CoordinationProtos.DrillbitEndpoint> buildEndpoints(int numberOfEndpoints) {
+    LinkedList<CoordinationProtos.DrillbitEndpoint> endPoints = new LinkedList();
+
+    for (int i = 0; i < numberOfEndpoints; i++) {
+      endPoints.add(CoordinationProtos.DrillbitEndpoint.newBuilder().setAddress("host" + i).build());
+    }
+    return endPoints;
+  }
+
+  @Test
+  public void testSetEndpointBytes(@Injectable final FileSystem fs, @Injectable final FileStatus file) throws Throwable{
+    final long blockSize = 256*1024*1024;
+    LinkedList<ParquetGroupScan.RowGroupInfo> rowGroups = new LinkedList<>();
+    int numberOfHosts = 4;
+    int numberOfBlocks = 3;
+    String port = "1234";
+    String[] hosts = new String[numberOfHosts];
+
+    final BlockLocation[] blockLocations = buildBlockLocations(hosts, blockSize);
+    final LinkedList<CoordinationProtos.DrillbitEndpoint> endPoints = buildEndpoints(numberOfHosts);
+    buildRowGroups(rowGroups, numberOfBlocks, blockSize, 3);
+
+    new NonStrictExpectations() {{
+      fs.getFileBlockLocations(file, 0, 3*blockSize); result = blockLocations;
+      fs.getFileStatus(new Path(path)); result = file;
+      file.getLen(); result = 3*blockSize;
+    }};
+
+
+    AffinityCalculator ac = new AffinityCalculator(path, fs, endPoints);
+    for (ParquetGroupScan.RowGroupInfo rowGroup : rowGroups) {
+      ac.setEndpointBytes(rowGroup);
+    }
+    ParquetGroupScan.RowGroupInfo rg = rowGroups.get(0);
+    Long b = rg.getEndpointBytes().get(endPoints.get(0));
+    assertEquals(blockSize,b.longValue());
+    b = rg.getEndpointBytes().get(endPoints.get(3));
+    assertNull(b);
+
+    buildRowGroups(rowGroups, numberOfBlocks, blockSize, 2);
+
+    ac = new AffinityCalculator(path, fs, endPoints);
+    for (ParquetGroupScan.RowGroupInfo rowGroup : rowGroups) {
+      ac.setEndpointBytes(rowGroup);
+    }
+    rg = rowGroups.get(0);
+    b = rg.getEndpointBytes().get(endPoints.get(0));
+    assertEquals(blockSize*3/2,b.longValue());
+    b = rg.getEndpointBytes().get(endPoints.get(3));
+    assertEquals(blockSize / 2, b.longValue());
+
+    buildRowGroups(rowGroups, numberOfBlocks, blockSize, 6);
+
+    ac = new AffinityCalculator(path, fs, endPoints);
+    for (ParquetGroupScan.RowGroupInfo rowGroup : rowGroups) {
+      ac.setEndpointBytes(rowGroup);
+    }
+    rg = rowGroups.get(0);
+    b = rg.getEndpointBytes().get(endPoints.get(0));
+    assertEquals(blockSize/2,b.longValue());
+    b = rg.getEndpointBytes().get(endPoints.get(3));
+    assertNull(b);
+  }
+
+  @Test
+  public void testBuildRangeMap() {
+    BlockLocation[] blocks = buildBlockLocations(new String[4], 256*1024*1024);
+    long tA = System.nanoTime();
+    ImmutableRangeMap.Builder<Long, BlockLocation> blockMapBuilder = new ImmutableRangeMap.Builder<Long,BlockLocation>();
+    for (BlockLocation block : blocks) {
+      long start = block.getOffset();
+      long end = start + block.getLength();
+      Range<Long> range = Range.closedOpen(start, end);
+      blockMapBuilder = blockMapBuilder.put(range, block);
+    }
+    ImmutableRangeMap<Long,BlockLocation> map = blockMapBuilder.build();
+    long tB = System.nanoTime();
+    System.out.println(String.format("Took %f ms to build range map", (float)(tB - tA) / 1e6));
+  }
+  /*
+  @Test
+  public void testApplyAssignments(@Injectable final DrillbitContext context, @Injectable final ParquetStorageEngine engine,
+                                   @Injectable final FileSystem fs, @Injectable final FileStatus file) throws IOException {
+
+    final long blockSize = 256*1024*1024;
+    LinkedList<ParquetGroupScan.RowGroupInfo> rowGroups = new LinkedList<>();
+    int numberOfHosts = 4;
+    int numberOfBlocks = 4;
+    String port = "1234";
+    String[] hosts = new String[numberOfHosts];
+
+    final BlockLocation[] blockLocations = buildBlockLocations2(hosts, blockSize);
+    final LinkedList<CoordinationProtos.DrillbitEndpoint> endPoints = buildEndpoints(numberOfHosts);
+
+    new NonStrictExpectations() {{
+      engine.getFileSystem(); result = fs;
+      engine.getContext(); result = context;
+      context.getBits(); result = endPoints;
+      fs.getFileBlockLocations(file, 0, 3*blockSize); result = blockLocations;
+      fs.getFileStatus(new Path(path)); result = file;
+      file.getLen(); result = 3*blockSize;
+    }};
+
+    buildRowGroups(rowGroups, numberOfBlocks, blockSize, 4);
+    ParquetGroupScan scan = new ParquetGroupScan(rowGroups, engine);
+
+    List<EndpointAffinity> affinities = scan.getOperatorAffinity();
+
+    for (EndpointAffinity affinity : affinities) {
+      CoordinationProtos.DrillbitEndpoint db = affinity.getEndpoint();
+      assertEquals((float)0.25, affinity.getAffinity(), .01);
+    }
+
+    scan.applyAssignments(endPoints);
+
+    for (int i = 0; i < endPoints.size(); i++) {
+      List<ParquetRowGroupScan.RowGroupReadEntry> rowGroupReadEntries = scan.getSpecificScan(i).getRowGroupReadEntries();
+      assertEquals(1, rowGroupReadEntries.size());
+      switch(i) {
+        case 0: assertEquals(0,rowGroupReadEntries.get(0).getRowGroupIndex());
+          break;
+        case 1: assertEquals(1,rowGroupReadEntries.get(0).getRowGroupIndex());
+          break;
+        case 2: assertEquals(3,rowGroupReadEntries.get(0).getRowGroupIndex());
+          break;
+        case 3: assertEquals(2,rowGroupReadEntries.get(0).getRowGroupIndex());
+          break;
+      }
+    }
+
+    scan.applyAssignments(endPoints.subList(2,4));
+
+    List<ParquetRowGroupScan.RowGroupReadEntry> rowGroupReadEntries = scan.getSpecificScan(0).getRowGroupReadEntries();
+    assertEquals(2, rowGroupReadEntries.size());
+    assertEquals(3,rowGroupReadEntries.get(0).getRowGroupIndex());
+
+    rowGroupReadEntries = scan.getSpecificScan(1).getRowGroupReadEntries();
+    assertEquals(2, rowGroupReadEntries.size());
+    assertEquals(2,rowGroupReadEntries.get(0).getRowGroupIndex());
+
+    LinkedList<CoordinationProtos.DrillbitEndpoint> dupList = new LinkedList<>();
+    dupList.add(endPoints.get(0));
+    dupList.add(endPoints.get(0));
+    scan.applyAssignments(dupList);
+
+    rowGroupReadEntries = scan.getSpecificScan(0).getRowGroupReadEntries();
+    assertEquals(2, rowGroupReadEntries.size());
+    rowGroupReadEntries = scan.getSpecificScan(1).getRowGroupReadEntries();
+    assertEquals(2, rowGroupReadEntries.size());
+  }
+  */
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/0a2f997f/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/TestParquetPhysicalPlan.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/TestParquetPhysicalPlan.java b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/TestParquetPhysicalPlan.java
new file mode 100644
index 0000000..e2a00f1
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/TestParquetPhysicalPlan.java
@@ -0,0 +1,56 @@
+package org.apache.drill.exec.store;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.google.common.base.Charsets;
+import com.google.common.io.Files;
+import com.google.common.io.Resources;
+import mockit.Injectable;
+import mockit.NonStrictExpectations;
+import org.apache.drill.common.config.DrillConfig;
+import org.apache.drill.common.util.FileUtils;
+import org.apache.drill.exec.client.DrillClient;
+import org.apache.drill.exec.physical.PhysicalPlan;
+import org.apache.drill.exec.planner.PhysicalPlanReader;
+import org.apache.drill.exec.proto.CoordinationProtos;
+import org.apache.drill.exec.proto.UserProtos;
+import org.apache.drill.exec.rpc.user.QueryResultBatch;
+import org.apache.drill.exec.server.Drillbit;
+import org.apache.drill.exec.server.RemoteServiceSet;
+import org.apache.drill.exec.store.parquet.ParquetGroupScan;
+import org.apache.hadoop.fs.BlockLocation;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.junit.Ignore;
+import org.junit.Test;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.charset.Charset;
+import java.util.LinkedList;
+import java.util.List;
+
+import static junit.framework.Assert.assertNull;
+import static org.junit.Assert.assertEquals;
+
+public class TestParquetPhysicalPlan {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestParquetPhysicalPlan.class);
+
+  //public String fileName = "/physical_test2.json";
+  public String fileName = "parquet_scan_union_screen_physical.json";
+
+  @Test
+  @Ignore
+  public void testParseParquetPhysicalPlan() throws Exception {
+    RemoteServiceSet serviceSet = RemoteServiceSet.getLocalServiceSet();
+    DrillConfig config = DrillConfig.create();
+
+    try(Drillbit bit1 = new Drillbit(config, serviceSet); DrillClient client = new DrillClient(config, serviceSet.getCoordinator());){
+      bit1.run();
+      client.connect();
+      List<QueryResultBatch> results = client.runQuery(UserProtos.QueryType.PHYSICAL, Resources.toString(Resources.getResource(fileName),Charsets.UTF_8));
+      System.out.println(String.format("Got %d results", results.size()));
+      client.close();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/0a2f997f/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 771a2fd..4829d34 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
@@ -20,10 +20,10 @@ drill.exec: {
   	context: "drillbit"
   },
   zk: {
-	connect: "localhost:2181",
+	connect: "10.10.30.52:5181",
 	root: "/drill",
 	refresh: 500,
-	timeout: 1000,
+	timeout: 5000,
 	retry: {
 	  count: 7200,
 	  delay: 500

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/0a2f997f/sandbox/prototype/exec/java-exec/src/test/resources/filter/test1.json
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/resources/filter/test1.json b/sandbox/prototype/exec/java-exec/src/test/resources/filter/test1.json
index c9b367f..4f82145 100644
--- a/sandbox/prototype/exec/java-exec/src/test/resources/filter/test1.json
+++ b/sandbox/prototype/exec/java-exec/src/test/resources/filter/test1.json
@@ -9,7 +9,7 @@
 	graph:[
         {
             @id:1,
-            pop:"mock-scan",
+            pop:"mock-sub-scan",
             url: "http://apache.org",
             entries:[
             	{records: 100, types: [

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/0a2f997f/sandbox/prototype/exec/java-exec/src/test/resources/functions/float4Equal.json
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/resources/functions/float4Equal.json b/sandbox/prototype/exec/java-exec/src/test/resources/functions/float4Equal.json
index 612b2b4..7ebe3dd 100644
--- a/sandbox/prototype/exec/java-exec/src/test/resources/functions/float4Equal.json
+++ b/sandbox/prototype/exec/java-exec/src/test/resources/functions/float4Equal.json
@@ -9,7 +9,7 @@
 	graph:[
         {
             @id:1,
-            pop:"mock-scan",
+            pop:"mock-sub-scan",
             url: "http://apache.org",
             entries:[
             	{records: 100, types: [

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/0a2f997f/sandbox/prototype/exec/java-exec/src/test/resources/functions/float4GreaterThan.json
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/resources/functions/float4GreaterThan.json b/sandbox/prototype/exec/java-exec/src/test/resources/functions/float4GreaterThan.json
index dcbad4c..de300d7 100644
--- a/sandbox/prototype/exec/java-exec/src/test/resources/functions/float4GreaterThan.json
+++ b/sandbox/prototype/exec/java-exec/src/test/resources/functions/float4GreaterThan.json
@@ -9,7 +9,7 @@
 	graph:[
         {
             @id:1,
-            pop:"mock-scan",
+            pop:"mock-sub-scan",
             url: "http://apache.org",
             entries:[
             	{records: 100, types: [

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/0a2f997f/sandbox/prototype/exec/java-exec/src/test/resources/functions/float4GreaterThanEqual.json
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/resources/functions/float4GreaterThanEqual.json b/sandbox/prototype/exec/java-exec/src/test/resources/functions/float4GreaterThanEqual.json
index b5c0d54..51f0a13 100644
--- a/sandbox/prototype/exec/java-exec/src/test/resources/functions/float4GreaterThanEqual.json
+++ b/sandbox/prototype/exec/java-exec/src/test/resources/functions/float4GreaterThanEqual.json
@@ -9,7 +9,7 @@
 	graph:[
         {
             @id:1,
-            pop:"mock-scan",
+            pop:"mock-sub-scan",
             url: "http://apache.org",
             entries:[
             	{records: 100, types: [

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/0a2f997f/sandbox/prototype/exec/java-exec/src/test/resources/functions/float4LessThan.json
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/resources/functions/float4LessThan.json b/sandbox/prototype/exec/java-exec/src/test/resources/functions/float4LessThan.json
index 06aafd9..95ef169 100644
--- a/sandbox/prototype/exec/java-exec/src/test/resources/functions/float4LessThan.json
+++ b/sandbox/prototype/exec/java-exec/src/test/resources/functions/float4LessThan.json
@@ -9,7 +9,7 @@
 	graph:[
         {
             @id:1,
-            pop:"mock-scan",
+            pop:"mock-sub-scan",
             url: "http://apache.org",
             entries:[
             	{records: 100, types: [

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/0a2f997f/sandbox/prototype/exec/java-exec/src/test/resources/functions/float4LessThanEqual.json
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/resources/functions/float4LessThanEqual.json b/sandbox/prototype/exec/java-exec/src/test/resources/functions/float4LessThanEqual.json
index 92a27ac..b58d3b6 100644
--- a/sandbox/prototype/exec/java-exec/src/test/resources/functions/float4LessThanEqual.json
+++ b/sandbox/prototype/exec/java-exec/src/test/resources/functions/float4LessThanEqual.json
@@ -9,7 +9,7 @@
 	graph:[
         {
             @id:1,
-            pop:"mock-scan",
+            pop:"mock-sub-scan",
             url: "http://apache.org",
             entries:[
             	{records: 100, types: [

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/0a2f997f/sandbox/prototype/exec/java-exec/src/test/resources/functions/float4NotEqual.json
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/resources/functions/float4NotEqual.json b/sandbox/prototype/exec/java-exec/src/test/resources/functions/float4NotEqual.json
index 0b97545..ba001d4 100644
--- a/sandbox/prototype/exec/java-exec/src/test/resources/functions/float4NotEqual.json
+++ b/sandbox/prototype/exec/java-exec/src/test/resources/functions/float4NotEqual.json
@@ -9,7 +9,7 @@
 	graph:[
         {
             @id:1,
-            pop:"mock-scan",
+            pop:"mock-sub-scan",
             url: "http://apache.org",
             entries:[
             	{records: 100, types: [

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/0a2f997f/sandbox/prototype/exec/java-exec/src/test/resources/functions/float8Equal.json
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/resources/functions/float8Equal.json b/sandbox/prototype/exec/java-exec/src/test/resources/functions/float8Equal.json
index 1aa93ea..9edfd51 100644
--- a/sandbox/prototype/exec/java-exec/src/test/resources/functions/float8Equal.json
+++ b/sandbox/prototype/exec/java-exec/src/test/resources/functions/float8Equal.json
@@ -9,7 +9,7 @@
 	graph:[
         {
             @id:1,
-            pop:"mock-scan",
+            pop:"mock-sub-scan",
             url: "http://apache.org",
             entries:[
             	{records: 100, types: [

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/0a2f997f/sandbox/prototype/exec/java-exec/src/test/resources/functions/float8GreaterThan.json
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/resources/functions/float8GreaterThan.json b/sandbox/prototype/exec/java-exec/src/test/resources/functions/float8GreaterThan.json
index ddccef1..08bb328 100644
--- a/sandbox/prototype/exec/java-exec/src/test/resources/functions/float8GreaterThan.json
+++ b/sandbox/prototype/exec/java-exec/src/test/resources/functions/float8GreaterThan.json
@@ -9,7 +9,7 @@
 	graph:[
         {
             @id:1,
-            pop:"mock-scan",
+            pop:"mock-sub-scan",
             url: "http://apache.org",
             entries:[
             	{records: 100, types: [

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/0a2f997f/sandbox/prototype/exec/java-exec/src/test/resources/functions/float8GreaterThanEqual.json
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/resources/functions/float8GreaterThanEqual.json b/sandbox/prototype/exec/java-exec/src/test/resources/functions/float8GreaterThanEqual.json
index 665759f..67afb47 100644
--- a/sandbox/prototype/exec/java-exec/src/test/resources/functions/float8GreaterThanEqual.json
+++ b/sandbox/prototype/exec/java-exec/src/test/resources/functions/float8GreaterThanEqual.json
@@ -9,7 +9,7 @@
 	graph:[
         {
             @id:1,
-            pop:"mock-scan",
+            pop:"mock-sub-scan",
             url: "http://apache.org",
             entries:[
             	{records: 100, types: [

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/0a2f997f/sandbox/prototype/exec/java-exec/src/test/resources/functions/float8LessThan.json
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/resources/functions/float8LessThan.json b/sandbox/prototype/exec/java-exec/src/test/resources/functions/float8LessThan.json
index b86b118..af485b2 100644
--- a/sandbox/prototype/exec/java-exec/src/test/resources/functions/float8LessThan.json
+++ b/sandbox/prototype/exec/java-exec/src/test/resources/functions/float8LessThan.json
@@ -9,7 +9,7 @@
 	graph:[
         {
             @id:1,
-            pop:"mock-scan",
+            pop:"mock-sub-scan",
             url: "http://apache.org",
             entries:[
             	{records: 100, types: [

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/0a2f997f/sandbox/prototype/exec/java-exec/src/test/resources/functions/float8LessThanEqual.json
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/resources/functions/float8LessThanEqual.json b/sandbox/prototype/exec/java-exec/src/test/resources/functions/float8LessThanEqual.json
index 7f622fd..6f5a21e 100644
--- a/sandbox/prototype/exec/java-exec/src/test/resources/functions/float8LessThanEqual.json
+++ b/sandbox/prototype/exec/java-exec/src/test/resources/functions/float8LessThanEqual.json
@@ -9,7 +9,7 @@
 	graph:[
         {
             @id:1,
-            pop:"mock-scan",
+            pop:"mock-sub-scan",
             url: "http://apache.org",
             entries:[
             	{records: 100, types: [

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/0a2f997f/sandbox/prototype/exec/java-exec/src/test/resources/functions/float8NotEqual.json
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/resources/functions/float8NotEqual.json b/sandbox/prototype/exec/java-exec/src/test/resources/functions/float8NotEqual.json
index fdff828..cf3d053 100644
--- a/sandbox/prototype/exec/java-exec/src/test/resources/functions/float8NotEqual.json
+++ b/sandbox/prototype/exec/java-exec/src/test/resources/functions/float8NotEqual.json
@@ -9,7 +9,7 @@
 	graph:[
         {
             @id:1,
-            pop:"mock-scan",
+            pop:"mock-sub-scan",
             url: "http://apache.org",
             entries:[
             	{records: 100, types: [

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/0a2f997f/sandbox/prototype/exec/java-exec/src/test/resources/functions/intEqual.json
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/resources/functions/intEqual.json b/sandbox/prototype/exec/java-exec/src/test/resources/functions/intEqual.json
index 88fb2eb..a1d96ec 100644
--- a/sandbox/prototype/exec/java-exec/src/test/resources/functions/intEqual.json
+++ b/sandbox/prototype/exec/java-exec/src/test/resources/functions/intEqual.json
@@ -9,7 +9,7 @@
 	graph:[
         {
             @id:1,
-            pop:"mock-scan",
+            pop:"mock-sub-scan",
             url: "http://apache.org",
             entries:[
             	{records: 100, types: [

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/0a2f997f/sandbox/prototype/exec/java-exec/src/test/resources/functions/intGreaterThan.json
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/resources/functions/intGreaterThan.json b/sandbox/prototype/exec/java-exec/src/test/resources/functions/intGreaterThan.json
index 015bf36..9e21f4b 100644
--- a/sandbox/prototype/exec/java-exec/src/test/resources/functions/intGreaterThan.json
+++ b/sandbox/prototype/exec/java-exec/src/test/resources/functions/intGreaterThan.json
@@ -9,7 +9,7 @@
 	graph:[
         {
             @id:1,
-            pop:"mock-scan",
+            pop:"mock-sub-scan",
             url: "http://apache.org",
             entries:[
             	{records: 100, types: [

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/0a2f997f/sandbox/prototype/exec/java-exec/src/test/resources/functions/intGreaterThanEqual.json
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/resources/functions/intGreaterThanEqual.json b/sandbox/prototype/exec/java-exec/src/test/resources/functions/intGreaterThanEqual.json
index 167d4ba..9692f82 100644
--- a/sandbox/prototype/exec/java-exec/src/test/resources/functions/intGreaterThanEqual.json
+++ b/sandbox/prototype/exec/java-exec/src/test/resources/functions/intGreaterThanEqual.json
@@ -9,7 +9,7 @@
 	graph:[
         {
             @id:1,
-            pop:"mock-scan",
+            pop:"mock-sub-scan",
             url: "http://apache.org",
             entries:[
             	{records: 100, types: [

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/0a2f997f/sandbox/prototype/exec/java-exec/src/test/resources/functions/intLessThan.json
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/resources/functions/intLessThan.json b/sandbox/prototype/exec/java-exec/src/test/resources/functions/intLessThan.json
index 473c08f..d2ec94a 100644
--- a/sandbox/prototype/exec/java-exec/src/test/resources/functions/intLessThan.json
+++ b/sandbox/prototype/exec/java-exec/src/test/resources/functions/intLessThan.json
@@ -9,7 +9,7 @@
 	graph:[
         {
             @id:1,
-            pop:"mock-scan",
+            pop:"mock-sub-scan",
             url: "http://apache.org",
             entries:[
             	{records: 100, types: [

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/0a2f997f/sandbox/prototype/exec/java-exec/src/test/resources/functions/intLessThanEqual.json
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/resources/functions/intLessThanEqual.json b/sandbox/prototype/exec/java-exec/src/test/resources/functions/intLessThanEqual.json
index 418e5f0..834cf7e 100644
--- a/sandbox/prototype/exec/java-exec/src/test/resources/functions/intLessThanEqual.json
+++ b/sandbox/prototype/exec/java-exec/src/test/resources/functions/intLessThanEqual.json
@@ -9,7 +9,7 @@
 	graph:[
         {
             @id:1,
-            pop:"mock-scan",
+            pop:"mock-sub-scan",
             url: "http://apache.org",
             entries:[
             	{records: 100, types: [

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/0a2f997f/sandbox/prototype/exec/java-exec/src/test/resources/functions/intNotEqual.json
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/resources/functions/intNotEqual.json b/sandbox/prototype/exec/java-exec/src/test/resources/functions/intNotEqual.json
index f0e8578..ceb2913 100644
--- a/sandbox/prototype/exec/java-exec/src/test/resources/functions/intNotEqual.json
+++ b/sandbox/prototype/exec/java-exec/src/test/resources/functions/intNotEqual.json
@@ -9,7 +9,7 @@
 	graph:[
         {
             @id:1,
-            pop:"mock-scan",
+            pop:"mock-sub-scan",
             url: "http://apache.org",
             entries:[
             	{records: 100, types: [

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/0a2f997f/sandbox/prototype/exec/java-exec/src/test/resources/functions/longEqual.json
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/resources/functions/longEqual.json b/sandbox/prototype/exec/java-exec/src/test/resources/functions/longEqual.json
index 4b0e2b8..0a7827b 100644
--- a/sandbox/prototype/exec/java-exec/src/test/resources/functions/longEqual.json
+++ b/sandbox/prototype/exec/java-exec/src/test/resources/functions/longEqual.json
@@ -9,7 +9,7 @@
 	graph:[
         {
             @id:1,
-            pop:"mock-scan",
+            pop:"mock-sub-scan",
             url: "http://apache.org",
             entries:[
             	{records: 100, types: [

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/0a2f997f/sandbox/prototype/exec/java-exec/src/test/resources/functions/longGreaterThan.json
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/resources/functions/longGreaterThan.json b/sandbox/prototype/exec/java-exec/src/test/resources/functions/longGreaterThan.json
index 830c126..6d2a415 100644
--- a/sandbox/prototype/exec/java-exec/src/test/resources/functions/longGreaterThan.json
+++ b/sandbox/prototype/exec/java-exec/src/test/resources/functions/longGreaterThan.json
@@ -9,7 +9,7 @@
 	graph:[
         {
             @id:1,
-            pop:"mock-scan",
+            pop:"mock-sub-scan",
             url: "http://apache.org",
             entries:[
             	{records: 100, types: [

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/0a2f997f/sandbox/prototype/exec/java-exec/src/test/resources/functions/longGreaterThanEqual.json
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/resources/functions/longGreaterThanEqual.json b/sandbox/prototype/exec/java-exec/src/test/resources/functions/longGreaterThanEqual.json
index 56c0dd7..7dffbe8 100644
--- a/sandbox/prototype/exec/java-exec/src/test/resources/functions/longGreaterThanEqual.json
+++ b/sandbox/prototype/exec/java-exec/src/test/resources/functions/longGreaterThanEqual.json
@@ -9,7 +9,7 @@
 	graph:[
         {
             @id:1,
-            pop:"mock-scan",
+            pop:"mock-sub-scan",
             url: "http://apache.org",
             entries:[
             	{records: 100, types: [

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/0a2f997f/sandbox/prototype/exec/java-exec/src/test/resources/functions/longLessThan.json
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/resources/functions/longLessThan.json b/sandbox/prototype/exec/java-exec/src/test/resources/functions/longLessThan.json
index 267a6c0..ace6eb0 100644
--- a/sandbox/prototype/exec/java-exec/src/test/resources/functions/longLessThan.json
+++ b/sandbox/prototype/exec/java-exec/src/test/resources/functions/longLessThan.json
@@ -9,7 +9,7 @@
 	graph:[
         {
             @id:1,
-            pop:"mock-scan",
+            pop:"mock-sub-scan",
             url: "http://apache.org",
             entries:[
             	{records: 100, types: [

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/0a2f997f/sandbox/prototype/exec/java-exec/src/test/resources/functions/longLessThanEqual.json
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/resources/functions/longLessThanEqual.json b/sandbox/prototype/exec/java-exec/src/test/resources/functions/longLessThanEqual.json
index e06d865..5900f73 100644
--- a/sandbox/prototype/exec/java-exec/src/test/resources/functions/longLessThanEqual.json
+++ b/sandbox/prototype/exec/java-exec/src/test/resources/functions/longLessThanEqual.json
@@ -9,7 +9,7 @@
 	graph:[
         {
             @id:1,
-            pop:"mock-scan",
+            pop:"mock-sub-scan",
             url: "http://apache.org",
             entries:[
             	{records: 100, types: [

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/0a2f997f/sandbox/prototype/exec/java-exec/src/test/resources/functions/longNotEqual.json
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/resources/functions/longNotEqual.json b/sandbox/prototype/exec/java-exec/src/test/resources/functions/longNotEqual.json
index 71269a6..4db69c8 100644
--- a/sandbox/prototype/exec/java-exec/src/test/resources/functions/longNotEqual.json
+++ b/sandbox/prototype/exec/java-exec/src/test/resources/functions/longNotEqual.json
@@ -9,7 +9,7 @@
 	graph:[
         {
             @id:1,
-            pop:"mock-scan",
+            pop:"mock-sub-scan",
             url: "http://apache.org",
             entries:[
             	{records: 100, types: [

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/0a2f997f/sandbox/prototype/exec/java-exec/src/test/resources/functions/nullableBigIntEqual.json
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/resources/functions/nullableBigIntEqual.json b/sandbox/prototype/exec/java-exec/src/test/resources/functions/nullableBigIntEqual.json
index eca6d15..212c73a 100644
--- a/sandbox/prototype/exec/java-exec/src/test/resources/functions/nullableBigIntEqual.json
+++ b/sandbox/prototype/exec/java-exec/src/test/resources/functions/nullableBigIntEqual.json
@@ -9,7 +9,7 @@
 	graph:[
         {
             @id:1,
-            pop:"mock-scan",
+            pop:"mock-sub-scan",
             url: "http://apache.org",
             entries:[
             	{records: 100, types: [

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/0a2f997f/sandbox/prototype/exec/java-exec/src/test/resources/functions/nullableBigIntGreaterThan.json
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/resources/functions/nullableBigIntGreaterThan.json b/sandbox/prototype/exec/java-exec/src/test/resources/functions/nullableBigIntGreaterThan.json
index 7218449..c1d42fa 100644
--- a/sandbox/prototype/exec/java-exec/src/test/resources/functions/nullableBigIntGreaterThan.json
+++ b/sandbox/prototype/exec/java-exec/src/test/resources/functions/nullableBigIntGreaterThan.json
@@ -9,7 +9,7 @@
 	graph:[
         {
             @id:1,
-            pop:"mock-scan",
+            pop:"mock-sub-scan",
             url: "http://apache.org",
             entries:[
             	{records: 100, types: [

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/0a2f997f/sandbox/prototype/exec/java-exec/src/test/resources/functions/nullableBigIntGreaterThanEqual.json
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/resources/functions/nullableBigIntGreaterThanEqual.json b/sandbox/prototype/exec/java-exec/src/test/resources/functions/nullableBigIntGreaterThanEqual.json
index ed2456a..c560d0b 100644
--- a/sandbox/prototype/exec/java-exec/src/test/resources/functions/nullableBigIntGreaterThanEqual.json
+++ b/sandbox/prototype/exec/java-exec/src/test/resources/functions/nullableBigIntGreaterThanEqual.json
@@ -9,7 +9,7 @@
 	graph:[
         {
             @id:1,
-            pop:"mock-scan",
+            pop:"mock-sub-scan",
             url: "http://apache.org",
             entries:[
             	{records: 100, types: [

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/0a2f997f/sandbox/prototype/exec/java-exec/src/test/resources/functions/nullableBigIntLessThan.json
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/resources/functions/nullableBigIntLessThan.json b/sandbox/prototype/exec/java-exec/src/test/resources/functions/nullableBigIntLessThan.json
index ebfee49..2cf2869 100644
--- a/sandbox/prototype/exec/java-exec/src/test/resources/functions/nullableBigIntLessThan.json
+++ b/sandbox/prototype/exec/java-exec/src/test/resources/functions/nullableBigIntLessThan.json
@@ -9,7 +9,7 @@
 	graph:[
         {
             @id:1,
-            pop:"mock-scan",
+            pop:"mock-sub-scan",
             url: "http://apache.org",
             entries:[
             	{records: 100, types: [

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/0a2f997f/sandbox/prototype/exec/java-exec/src/test/resources/functions/nullableBigIntLessThanEqual.json
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/resources/functions/nullableBigIntLessThanEqual.json b/sandbox/prototype/exec/java-exec/src/test/resources/functions/nullableBigIntLessThanEqual.json
index 98e6fcf..16b19d3 100644
--- a/sandbox/prototype/exec/java-exec/src/test/resources/functions/nullableBigIntLessThanEqual.json
+++ b/sandbox/prototype/exec/java-exec/src/test/resources/functions/nullableBigIntLessThanEqual.json
@@ -9,7 +9,7 @@
 	graph:[
         {
             @id:1,
-            pop:"mock-scan",
+            pop:"mock-sub-scan",
             url: "http://apache.org",
             entries:[
             	{records: 100, types: [

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/0a2f997f/sandbox/prototype/exec/java-exec/src/test/resources/functions/nullableBigIntNotEqual.json
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/resources/functions/nullableBigIntNotEqual.json b/sandbox/prototype/exec/java-exec/src/test/resources/functions/nullableBigIntNotEqual.json
index a501f50..2e9778a 100644
--- a/sandbox/prototype/exec/java-exec/src/test/resources/functions/nullableBigIntNotEqual.json
+++ b/sandbox/prototype/exec/java-exec/src/test/resources/functions/nullableBigIntNotEqual.json
@@ -9,7 +9,7 @@
 	graph:[
         {
             @id:1,
-            pop:"mock-scan",
+            pop:"mock-sub-scan",
             url: "http://apache.org",
             entries:[
             	{records: 100, types: [

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/0a2f997f/sandbox/prototype/exec/java-exec/src/test/resources/functions/nullableIntEqual.json
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/resources/functions/nullableIntEqual.json b/sandbox/prototype/exec/java-exec/src/test/resources/functions/nullableIntEqual.json
index 60a00fb..ba9cba8 100644
--- a/sandbox/prototype/exec/java-exec/src/test/resources/functions/nullableIntEqual.json
+++ b/sandbox/prototype/exec/java-exec/src/test/resources/functions/nullableIntEqual.json
@@ -9,7 +9,7 @@
 	graph:[
         {
             @id:1,
-            pop:"mock-scan",
+            pop:"mock-sub-scan",
             url: "http://apache.org",
             entries:[
             	{records: 100, types: [

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/0a2f997f/sandbox/prototype/exec/java-exec/src/test/resources/functions/nullableIntGreaterThan.json
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/resources/functions/nullableIntGreaterThan.json b/sandbox/prototype/exec/java-exec/src/test/resources/functions/nullableIntGreaterThan.json
index c6da740..6a3fc33 100644
--- a/sandbox/prototype/exec/java-exec/src/test/resources/functions/nullableIntGreaterThan.json
+++ b/sandbox/prototype/exec/java-exec/src/test/resources/functions/nullableIntGreaterThan.json
@@ -9,7 +9,7 @@
 	graph:[
         {
             @id:1,
-            pop:"mock-scan",
+            pop:"mock-sub-scan",
             url: "http://apache.org",
             entries:[
             	{records: 100, types: [

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/0a2f997f/sandbox/prototype/exec/java-exec/src/test/resources/functions/nullableIntGreaterThanEqual.json
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/resources/functions/nullableIntGreaterThanEqual.json b/sandbox/prototype/exec/java-exec/src/test/resources/functions/nullableIntGreaterThanEqual.json
index a3f373c..daa34f9 100644
--- a/sandbox/prototype/exec/java-exec/src/test/resources/functions/nullableIntGreaterThanEqual.json
+++ b/sandbox/prototype/exec/java-exec/src/test/resources/functions/nullableIntGreaterThanEqual.json
@@ -9,7 +9,7 @@
 	graph:[
         {
             @id:1,
-            pop:"mock-scan",
+            pop:"mock-sub-scan",
             url: "http://apache.org",
             entries:[
             	{records: 100, types: [

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/0a2f997f/sandbox/prototype/exec/java-exec/src/test/resources/functions/nullableIntLessThan.json
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/resources/functions/nullableIntLessThan.json b/sandbox/prototype/exec/java-exec/src/test/resources/functions/nullableIntLessThan.json
index 46eee38..0186e17 100644
--- a/sandbox/prototype/exec/java-exec/src/test/resources/functions/nullableIntLessThan.json
+++ b/sandbox/prototype/exec/java-exec/src/test/resources/functions/nullableIntLessThan.json
@@ -9,7 +9,7 @@
 	graph:[
         {
             @id:1,
-            pop:"mock-scan",
+            pop:"mock-sub-scan",
             url: "http://apache.org",
             entries:[
             	{records: 100, types: [

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/0a2f997f/sandbox/prototype/exec/java-exec/src/test/resources/functions/nullableIntLessThanEqual.json
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/resources/functions/nullableIntLessThanEqual.json b/sandbox/prototype/exec/java-exec/src/test/resources/functions/nullableIntLessThanEqual.json
index 77db318..2ee40eb 100644
--- a/sandbox/prototype/exec/java-exec/src/test/resources/functions/nullableIntLessThanEqual.json
+++ b/sandbox/prototype/exec/java-exec/src/test/resources/functions/nullableIntLessThanEqual.json
@@ -9,7 +9,7 @@
 	graph:[
         {
             @id:1,
-            pop:"mock-scan",
+            pop:"mock-sub-scan",
             url: "http://apache.org",
             entries:[
             	{records: 100, types: [

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/0a2f997f/sandbox/prototype/exec/java-exec/src/test/resources/functions/nullableIntNotEqual.json
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/resources/functions/nullableIntNotEqual.json b/sandbox/prototype/exec/java-exec/src/test/resources/functions/nullableIntNotEqual.json
index 2cefc5c..494f938 100644
--- a/sandbox/prototype/exec/java-exec/src/test/resources/functions/nullableIntNotEqual.json
+++ b/sandbox/prototype/exec/java-exec/src/test/resources/functions/nullableIntNotEqual.json
@@ -9,7 +9,7 @@
 	graph:[
         {
             @id:1,
-            pop:"mock-scan",
+            pop:"mock-sub-scan",
             url: "http://apache.org",
             entries:[
             	{records: 100, types: [

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/0a2f997f/sandbox/prototype/exec/java-exec/src/test/resources/functions/testByteSubstring.json
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/resources/functions/testByteSubstring.json b/sandbox/prototype/exec/java-exec/src/test/resources/functions/testByteSubstring.json
index 3c246c5..299bb51 100644
--- a/sandbox/prototype/exec/java-exec/src/test/resources/functions/testByteSubstring.json
+++ b/sandbox/prototype/exec/java-exec/src/test/resources/functions/testByteSubstring.json
@@ -9,7 +9,7 @@
     graph:[
         {
             @id:1,
-            pop:"mock-scan",
+            pop:"mock-sub-scan",
             url: "http://apache.org",
             entries:[
             	{records: 100, types: [

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/0a2f997f/sandbox/prototype/exec/java-exec/src/test/resources/functions/testIsNotNull.json
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/resources/functions/testIsNotNull.json b/sandbox/prototype/exec/java-exec/src/test/resources/functions/testIsNotNull.json
index f24d2c4..713d58f 100644
--- a/sandbox/prototype/exec/java-exec/src/test/resources/functions/testIsNotNull.json
+++ b/sandbox/prototype/exec/java-exec/src/test/resources/functions/testIsNotNull.json
@@ -9,7 +9,7 @@
 	graph:[
         {
             @id:1,
-            pop:"mock-scan",
+            pop:"mock-sub-scan",
             url: "http://apache.org",
             entries:[
             	{records: 100, types: [

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/0a2f997f/sandbox/prototype/exec/java-exec/src/test/resources/functions/testIsNull.json
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/resources/functions/testIsNull.json b/sandbox/prototype/exec/java-exec/src/test/resources/functions/testIsNull.json
index 850bdc5..e173adc 100644
--- a/sandbox/prototype/exec/java-exec/src/test/resources/functions/testIsNull.json
+++ b/sandbox/prototype/exec/java-exec/src/test/resources/functions/testIsNull.json
@@ -9,7 +9,7 @@
 	graph:[
         {
             @id:1,
-            pop:"mock-scan",
+            pop:"mock-sub-scan",
             url: "http://apache.org",
             entries:[
             	{records: 100, types: [

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/0a2f997f/sandbox/prototype/exec/java-exec/src/test/resources/functions/testSubstring.json
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/resources/functions/testSubstring.json b/sandbox/prototype/exec/java-exec/src/test/resources/functions/testSubstring.json
index 08a8ece..d0fe2b9 100644
--- a/sandbox/prototype/exec/java-exec/src/test/resources/functions/testSubstring.json
+++ b/sandbox/prototype/exec/java-exec/src/test/resources/functions/testSubstring.json
@@ -9,7 +9,7 @@
     graph:[
         {
             @id:1,
-            pop:"mock-scan",
+            pop:"mock-sub-scan",
             url: "http://apache.org",
             entries:[
             	{records: 100, types: [

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/0a2f997f/sandbox/prototype/exec/java-exec/src/test/resources/functions/testSubstringNegative.json
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/resources/functions/testSubstringNegative.json b/sandbox/prototype/exec/java-exec/src/test/resources/functions/testSubstringNegative.json
index 4951913..12ff0f2 100644
--- a/sandbox/prototype/exec/java-exec/src/test/resources/functions/testSubstringNegative.json
+++ b/sandbox/prototype/exec/java-exec/src/test/resources/functions/testSubstringNegative.json
@@ -9,7 +9,7 @@
     graph:[
         {
             @id:1,
-            pop:"mock-scan",
+            pop:"mock-sub-scan",
             url: "http://apache.org",
             entries:[
             	{records: 100, types: [

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

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/0a2f997f/sandbox/prototype/exec/java-exec/src/test/resources/parquet_scan_screen.json
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/resources/parquet_scan_screen.json b/sandbox/prototype/exec/java-exec/src/test/resources/parquet_scan_screen.json
new file mode 100644
index 0000000..15d3936
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/test/resources/parquet_scan_screen.json
@@ -0,0 +1,44 @@
+{
+  head:{
+    type:"APACHE_DRILL_LOGICAL",
+    version:"1",
+    generator:{
+      type:"manual",
+      info:"na"
+    }
+  },
+  storage:{
+    "parquet" :
+      {
+        "type":"parquet",
+        "dfsName" : "file:///"
+      }
+  },
+  query:[
+    {
+      @id:"1",
+      op:"scan",
+      memo:"initial_scan",
+      storageengine:"parquet",
+      selection: [
+        {
+            path: "/tmp/testParquetFile_many_types_3"
+        },
+        {
+            path: "/tmp/testParquetFile_many_types_3"
+        }
+      ]
+    },
+    {
+      @id:"2",
+      input: 1,
+      op: "store",
+      memo: "output sink",
+      target: {
+        file: "console:///stdout"
+      }
+
+    }
+
+  ]
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/0a2f997f/sandbox/prototype/exec/java-exec/src/test/resources/parquet_scan_union_screen_physical.json
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/resources/parquet_scan_union_screen_physical.json b/sandbox/prototype/exec/java-exec/src/test/resources/parquet_scan_union_screen_physical.json
new file mode 100644
index 0000000..954082c
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/test/resources/parquet_scan_union_screen_physical.json
@@ -0,0 +1,35 @@
+{
+  head : {
+    type : "APACHE_DRILL_PHYSICAL",
+    version : 1,
+    generator : {
+      type : "manual"
+    }
+  },
+  graph : [ {
+    pop : "parquet-scan",
+    @id : 1,
+    entries : [
+    {
+        path : "/tmp/testParquetFile_many_types_3"
+    },
+    {
+        path : "/tmp/testParquetFile_many_types_3"
+    }
+    ],
+    storageengine:{
+                         "type":"parquet",
+                         "dfsName" : "maprfs:///"
+                   }
+  },
+  {
+     "@id": 2,
+     "child": 1,
+     "pop": "union-exchange"
+  },
+  {
+    pop : "screen",
+    @id : 3,
+    child : 2
+  } ]
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/0a2f997f/sandbox/prototype/exec/java-exec/src/test/resources/physical_repeated_1.json
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/resources/physical_repeated_1.json b/sandbox/prototype/exec/java-exec/src/test/resources/physical_repeated_1.json
index c26be01..71eff1d 100644
--- a/sandbox/prototype/exec/java-exec/src/test/resources/physical_repeated_1.json
+++ b/sandbox/prototype/exec/java-exec/src/test/resources/physical_repeated_1.json
@@ -9,7 +9,7 @@
   graph:[
         {
             @id:1,
-            pop:"mock-scan",
+            pop:"mock-sub-scan",
             url: "http://apache.org",
             entries:[
               {records: 100, types: [

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/0a2f997f/sandbox/prototype/exec/java-exec/src/test/resources/project/test1.json
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/resources/project/test1.json b/sandbox/prototype/exec/java-exec/src/test/resources/project/test1.json
index 70a5115..2a7c935 100644
--- a/sandbox/prototype/exec/java-exec/src/test/resources/project/test1.json
+++ b/sandbox/prototype/exec/java-exec/src/test/resources/project/test1.json
@@ -9,7 +9,7 @@
 	graph:[
         {
             @id:1,
-            pop:"mock-scan",
+            pop:"mock-sub-scan",
             url: "http://apache.org",
             entries:[
             	{records: 100, types: [

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/0a2f997f/sandbox/prototype/exec/java-exec/src/test/resources/remover/test1.json
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/resources/remover/test1.json b/sandbox/prototype/exec/java-exec/src/test/resources/remover/test1.json
index 7ef8fd6..3abe476 100644
--- a/sandbox/prototype/exec/java-exec/src/test/resources/remover/test1.json
+++ b/sandbox/prototype/exec/java-exec/src/test/resources/remover/test1.json
@@ -9,7 +9,7 @@
 	graph:[
         {
             @id:1,
-            pop:"mock-scan",
+            pop:"mock-sub-scan",
             url: "http://apache.org",
             entries:[
             	{records: 100, types: [

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/0a2f997f/sandbox/prototype/exec/java-exec/src/test/resources/sort/one_key_sort.json
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/resources/sort/one_key_sort.json b/sandbox/prototype/exec/java-exec/src/test/resources/sort/one_key_sort.json
index 3bd0b71..baabcb3 100644
--- a/sandbox/prototype/exec/java-exec/src/test/resources/sort/one_key_sort.json
+++ b/sandbox/prototype/exec/java-exec/src/test/resources/sort/one_key_sort.json
@@ -9,7 +9,7 @@
 	graph:[
         {
             @id:1,
-            pop:"mock-scan",
+            pop:"mock-sub-scan",
             url: "http://apache.org",
             entries:[
             	{records: 1000000, types: [

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/0a2f997f/sandbox/prototype/exec/java-exec/src/test/resources/sort/two_key_sort.json
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/resources/sort/two_key_sort.json b/sandbox/prototype/exec/java-exec/src/test/resources/sort/two_key_sort.json
index 2394626..ab14002 100644
--- a/sandbox/prototype/exec/java-exec/src/test/resources/sort/two_key_sort.json
+++ b/sandbox/prototype/exec/java-exec/src/test/resources/sort/two_key_sort.json
@@ -9,7 +9,7 @@
 	graph:[
         {
             @id:1,
-            pop:"mock-scan",
+            pop:"mock-sub-scan",
             url: "http://apache.org",
             entries:[
             	{records: 100, types: [

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

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/0a2f997f/sandbox/prototype/exec/ref/src/main/java/org/apache/drill/exec/ref/ROPConverter.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/ref/src/main/java/org/apache/drill/exec/ref/ROPConverter.java b/sandbox/prototype/exec/ref/src/main/java/org/apache/drill/exec/ref/ROPConverter.java
index 90f3374..384af2d 100644
--- a/sandbox/prototype/exec/ref/src/main/java/org/apache/drill/exec/ref/ROPConverter.java
+++ b/sandbox/prototype/exec/ref/src/main/java/org/apache/drill/exec/ref/ROPConverter.java
@@ -35,7 +35,6 @@ import org.apache.drill.exec.ref.rops.ROP;
 import org.apache.drill.exec.ref.rops.ScanROP;
 import org.apache.drill.exec.ref.rops.StoreROP;
 import org.apache.drill.exec.ref.rops.UnionROP;
-import org.apache.drill.exec.ref.rops.UnionROP;
 import org.apache.drill.exec.ref.rse.RSERegistry;
 import org.apache.drill.exec.ref.rse.ReferenceStorageEngine;
 import org.apache.drill.exec.ref.rse.ReferenceStorageEngine.ReadEntry;
@@ -106,7 +105,7 @@ class ROPConverter {
   }
 
   private ReferenceStorageEngine getEngine(String name){
-    StorageEngineConfig config = plan.getStorageEngine(name);
+    StorageEngineConfig config = plan.getStorageEngineConfig(name);
     if(config == null) throw new SetupException(String.format("Unable to find define logical plan of name [%s].", name));
     ReferenceStorageEngine engine = engineRegistry.getEngine(config);
     return engine;
@@ -118,7 +117,7 @@ class ROPConverter {
   }
 
   public void convertSpecific(Scan scan) throws SetupException {
-    StorageEngineConfig engineConfig = plan.getStorageEngine(scan.getStorageEngine());
+    StorageEngineConfig engineConfig = plan.getStorageEngineConfig(scan.getStorageEngine());
     ReferenceStorageEngine engine = engineRegistry.getEngine(engineConfig);
     Collection<ReadEntry> readEntries;
     try {

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/0a2f997f/sandbox/prototype/exec/ref/src/main/java/org/apache/drill/exec/ref/rse/ClasspathRSE.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/ref/src/main/java/org/apache/drill/exec/ref/rse/ClasspathRSE.java b/sandbox/prototype/exec/ref/src/main/java/org/apache/drill/exec/ref/rse/ClasspathRSE.java
index aa8186d..6515a3a 100644
--- a/sandbox/prototype/exec/ref/src/main/java/org/apache/drill/exec/ref/rse/ClasspathRSE.java
+++ b/sandbox/prototype/exec/ref/src/main/java/org/apache/drill/exec/ref/rse/ClasspathRSE.java
@@ -48,6 +48,11 @@ public class ClasspathRSE extends RSEBase {
   
   @JsonTypeName("classpath")
   public static class ClasspathRSEConfig extends StorageEngineConfigBase {
+    @Override
+    public boolean equals(Object o) {
+      // if fields are added to this class this method should be changed
+      return true;
+    }
   }
   
   public static class ClasspathInputConfig implements ReadEntry{

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/0a2f997f/sandbox/prototype/exec/ref/src/main/java/org/apache/drill/exec/ref/rse/ConsoleRSE.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/ref/src/main/java/org/apache/drill/exec/ref/rse/ConsoleRSE.java b/sandbox/prototype/exec/ref/src/main/java/org/apache/drill/exec/ref/rse/ConsoleRSE.java
index 1570ea9..a914fe3 100644
--- a/sandbox/prototype/exec/ref/src/main/java/org/apache/drill/exec/ref/rse/ConsoleRSE.java
+++ b/sandbox/prototype/exec/ref/src/main/java/org/apache/drill/exec/ref/rse/ConsoleRSE.java
@@ -44,7 +44,13 @@ public class ConsoleRSE extends RSEBase {
     public ConverterType type = ConverterType.JSON;
   }
   
-  @JsonTypeName("console") public static class ConsoleRSEConfig extends StorageEngineConfigBase {}
+  @JsonTypeName("console") public static class ConsoleRSEConfig extends StorageEngineConfigBase {
+    @Override
+    public boolean equals(Object o) {
+      // if fields are added to this class this method needs to be updated
+      return true;
+    }
+  }
   
   public boolean supportsWrite() {
     return true;

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/0a2f997f/sandbox/prototype/exec/ref/src/main/java/org/apache/drill/exec/ref/rse/FileSystemRSE.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/ref/src/main/java/org/apache/drill/exec/ref/rse/FileSystemRSE.java b/sandbox/prototype/exec/ref/src/main/java/org/apache/drill/exec/ref/rse/FileSystemRSE.java
index 522191b..eece57d 100644
--- a/sandbox/prototype/exec/ref/src/main/java/org/apache/drill/exec/ref/rse/FileSystemRSE.java
+++ b/sandbox/prototype/exec/ref/src/main/java/org/apache/drill/exec/ref/rse/FileSystemRSE.java
@@ -72,6 +72,23 @@ public class FileSystemRSE extends RSEBase {
     public FileSystemRSEConfig(@JsonProperty("root") String root) {
       this.root = root;
     }
+
+    @Override
+    public boolean equals(Object o) {
+      if (this == o) return true;
+      if (o == null || getClass() != o.getClass()) return false;
+
+      FileSystemRSEConfig that = (FileSystemRSEConfig) o;
+
+      if (root != null ? !root.equals(that.root) : that.root != null) return false;
+
+      return true;
+    }
+
+    @Override
+    public int hashCode() {
+      return root != null ? root.hashCode() : 0;
+    }
   }
   
   public static class FileSystemInputConfig {


[23/27] git commit: implement hash expression evaluation

Posted by ja...@apache.org.
implement hash expression evaluation


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

Branch: refs/heads/master
Commit: 2c8094b420da075dec1cb224ff4ecb37f3d7f7f4
Parents: 98156ee
Author: Ben Becker <be...@gmail.com>
Authored: Mon Aug 12 02:27:24 2013 -0700
Committer: Jacques Nadeau <ja...@apache.org>
Committed: Thu Aug 15 18:31:31 2013 -0700

----------------------------------------------------------------------
 .../apache/drill/exec/expr/fn/impl/Hash.java    |  5 +-
 .../partitionsender/OutgoingRecordBatch.java    |  9 ++--
 .../PartitionSenderRootExec.java                | 56 +++++++++++++++-----
 .../partitionsender/PartitionerTemplate.java    |  4 +-
 .../test/resources/sender/hash_exchange.json    |  2 +-
 5 files changed, 50 insertions(+), 26 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/2c8094b4/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/Hash.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/Hash.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/Hash.java
index 45a3ee4..2ffd389 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/Hash.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/Hash.java
@@ -18,6 +18,7 @@
 
 package org.apache.drill.exec.expr.fn.impl;
 
+import com.google.common.hash.Hashing;
 import org.apache.drill.common.expression.*;
 import org.apache.drill.exec.expr.DrillFunc;
 import org.apache.drill.exec.expr.annotations.FunctionTemplate;
@@ -38,8 +39,8 @@ public class Hash implements DrillFunc {
   }
 
   public void eval() {
-    // TODO: implement hash function for various types
-    out.value = (int)in.value;
+    // TODO: implement hash function for other types
+    out.value = Hashing.murmur3_128().hashLong(in.value).asInt();
   }
 
   public static class Provider implements CallProvider{

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/2c8094b4/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/partitionsender/OutgoingRecordBatch.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/partitionsender/OutgoingRecordBatch.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/partitionsender/OutgoingRecordBatch.java
index 927cc75..bc1ef4e 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/partitionsender/OutgoingRecordBatch.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/partitionsender/OutgoingRecordBatch.java
@@ -70,8 +70,9 @@ public class OutgoingRecordBatch implements RecordBatch {
     try {
       if (recordCount == recordCapacity) flush();
     } catch (SchemaChangeException e) {
-      // TODO:
-      logger.error("Unable to flush outgoing record batch: " + e);
+      incoming.kill();
+      logger.error("Error flushing outgoing batches", e);
+      context.fail(e);
     }
   }
 
@@ -86,7 +87,6 @@ public class OutgoingRecordBatch implements RecordBatch {
    * @throws SchemaChangeException
    */
   public boolean flush() throws SchemaChangeException {
-    logger.error("Creating FragmentWritableBatch.  IsLast? " + (isLast ? " (last batch)" : ""));
     final ExecProtos.FragmentHandle handle = context.getHandle();
 
     if (recordCount != 0) {
@@ -100,9 +100,7 @@ public class OutgoingRecordBatch implements RecordBatch {
       tunnel.sendRecordBatch(statusHandler, context, writableBatch);
     } else {
       logger.debug("Flush requested on an empty outgoing record batch" + (isLast ? " (last batch)" : ""));
-
       if (isLast) {
-
         // if the last batch is empty, it must not contain any value vectors.
         vectorContainer = new VectorContainer();
 
@@ -116,7 +114,6 @@ public class OutgoingRecordBatch implements RecordBatch {
                                                                         getWritableBatch());
         tunnel.sendRecordBatch(statusHandler, context, writableBatch);
         return true;
-
       }
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/2c8094b4/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/partitionsender/PartitionSenderRootExec.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/partitionsender/PartitionSenderRootExec.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/partitionsender/PartitionSenderRootExec.java
index 476de7d..6d24e0b 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/partitionsender/PartitionSenderRootExec.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/partitionsender/PartitionSenderRootExec.java
@@ -18,6 +18,7 @@
 
 package org.apache.drill.exec.physical.impl.partitionsender;
 
+import com.google.common.collect.Lists;
 import com.sun.codemodel.*;
 import org.apache.drill.common.expression.*;
 import org.apache.drill.exec.exception.ClassTransformationException;
@@ -27,14 +28,15 @@ import org.apache.drill.exec.expr.ExpressionTreeMaterializer;
 import org.apache.drill.exec.ops.FragmentContext;
 import org.apache.drill.exec.physical.config.HashPartitionSender;
 import org.apache.drill.exec.physical.impl.RootExec;
+import org.apache.drill.exec.physical.impl.filter.Filterer;
+import org.apache.drill.exec.physical.impl.filter.ReturnValueExpression;
 import org.apache.drill.exec.proto.CoordinationProtos;
-import org.apache.drill.exec.record.RecordBatch;
-import org.apache.drill.exec.record.TypedFieldId;
-import org.apache.drill.exec.record.VectorWrapper;
+import org.apache.drill.exec.record.*;
 import org.apache.drill.exec.vector.TypeHelper;
 import org.apache.drill.exec.vector.ValueVector;
 
 import java.io.IOException;
+import java.util.List;
 
 class PartitionSenderRootExec implements RootExec {
 
@@ -122,6 +124,29 @@ class PartitionSenderRootExec implements RootExec {
     ok = false;
     incoming.kill();
   }
+  
+  private void generatePartitionFunction() throws SchemaChangeException {
+
+    LogicalExpression filterExpression = operator.getExpr();
+    final ErrorCollector collector = new ErrorCollectorImpl();
+    final CodeGenerator<Partitioner> cg = new CodeGenerator<Partitioner>(Partitioner.TEMPLATE_DEFINITION, context.getFunctionRegistry());
+
+    final LogicalExpression expr = ExpressionTreeMaterializer.materialize(filterExpression, incoming, collector);
+    if(collector.hasErrors()){
+      throw new SchemaChangeException(String.format("Failure while trying to materialize incoming schema.  Errors:\n %s.", collector.toErrorString()));
+    }
+
+    cg.addExpr(new ReturnValueExpression(expr));
+    
+    try {
+      Partitioner p = context.getImplementationClass(cg);
+      p.setup(context, incoming, outgoing);
+    } catch (ClassTransformationException | IOException e) {
+      throw new SchemaChangeException("Failure while attempting to load generated class", e);
+    }
+
+
+  }
 
   private void createPartitioner() throws SchemaChangeException {
 
@@ -131,7 +156,7 @@ class PartitionSenderRootExec implements RootExec {
     final CodeGenerator<Partitioner> cg = new CodeGenerator<Partitioner>(Partitioner.TEMPLATE_DEFINITION,
                                                                          context.getFunctionRegistry());
 
-    final LogicalExpression logicalExp = ExpressionTreeMaterializer.materialize(expr, incoming, collector);
+    final LogicalExpression materializedExpr = ExpressionTreeMaterializer.materialize(expr, incoming, collector);
     if (collector.hasErrors()) {
       throw new SchemaChangeException(String.format(
           "Failure while trying to materialize incoming schema.  Errors:\n %s.",
@@ -140,15 +165,18 @@ class PartitionSenderRootExec implements RootExec {
 
     // generate code to copy from an incoming value vector to the destination partition's outgoing value vector
     JExpression inIndex = JExpr.direct("inIndex");
-    JExpression outIndex = JExpr.direct("outIndex");
+    JExpression bucket = JExpr.direct("bucket");
     JType outgoingVectorArrayType = cg.getModel().ref(ValueVector.class).array().array();
     JType outgoingBatchArrayType = cg.getModel().ref(OutgoingRecordBatch.class).array();
-    cg.rotateBlock();
+
+    // generate evaluate expression to determine the hash
+    CodeGenerator.HoldingContainer exprHolder = cg.addExpr(materializedExpr);
+    cg.getBlock().decl(JType.parse(cg.getModel(), "int"), "bucket", exprHolder.getValue().mod(JExpr.lit(outgoing.length)));
 
     // declare and assign the array of outgoing record batches
     JVar outgoingBatches = cg.clazz.field(JMod.NONE,
-                                          outgoingBatchArrayType,
-                                          "outgoingBatches");
+        outgoingBatchArrayType,
+        "outgoingBatches");
     cg.getSetupBlock().assign(outgoingBatches, JExpr.direct("outgoing"));
 
     // declare a two-dimensional array of value vectors; batch is first dimension, ValueVector is the second
@@ -199,25 +227,25 @@ class PartitionSenderRootExec implements RootExec {
                                                        vvIn.getField().getType().getMode());
       JClass vvClass = cg.getModel().ref(vvType);
       // the following block generates calls to copyFrom(); e.g.:
-      // ((IntVector) outgoingVectors[outIndex][0]).copyFrom(inIndex,
-      //                                                     outgoingBatches[outIndex].getRecordCount(),
+      // ((IntVector) outgoingVectors[bucket][0]).copyFrom(inIndex,
+      //                                                     outgoingBatches[bucket].getRecordCount(),
       //                                                     vv1);
       cg.getBlock().add(
         ((JExpression) JExpr.cast(vvClass,
               ((JExpression)
                      outgoingVectors
-                       .component(outIndex))
+                       .component(bucket))
                        .component(JExpr.lit(fieldId))))
                        .invoke("copyFrom")
                        .arg(inIndex)
-                       .arg(((JExpression) outgoingBatches.component(outIndex)).invoke("getRecordCount"))
+                       .arg(((JExpression) outgoingBatches.component(bucket)).invoke("getRecordCount"))
                        .arg(incomingVV));
 
       ++fieldId;
     }
     // generate the OutgoingRecordBatch helper invocations
-    cg.getBlock().add(((JExpression) outgoingBatches.component(outIndex)).invoke("incRecordCount"));
-    cg.getBlock().add(((JExpression) outgoingBatches.component(outIndex)).invoke("flushIfNecessary"));
+    cg.getBlock().add(((JExpression) outgoingBatches.component(bucket)).invoke("incRecordCount"));
+    cg.getBlock().add(((JExpression) outgoingBatches.component(bucket)).invoke("flushIfNecessary"));
     try {
       // compile and setup generated code
       partitioner = context.getImplementationClassMultipleOutput(cg);

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/2c8094b4/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/partitionsender/PartitionerTemplate.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/partitionsender/PartitionerTemplate.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/partitionsender/PartitionerTemplate.java
index 4072b20..7198c3a 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/partitionsender/PartitionerTemplate.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/partitionsender/PartitionerTemplate.java
@@ -43,12 +43,10 @@ public abstract class PartitionerTemplate implements Partitioner {
     for (int recordId = 0; recordId < incoming.getRecordCount(); ++recordId) {
       // for each record
 
-      // TODO: evaluate partitioning expression
-      int partition = 0;
       // TODO: if attempting to insert too large of a value into a vector:
       //         - send the batch
       //         - reallocate (at least the size of the current value) and try again
-      doEval(recordId, partition);
+      doEval(recordId, 0);
     }
 
   }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/2c8094b4/sandbox/prototype/exec/java-exec/src/test/resources/sender/hash_exchange.json
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/resources/sender/hash_exchange.json b/sandbox/prototype/exec/java-exec/src/test/resources/sender/hash_exchange.json
index 78f3394..38116e7 100644
--- a/sandbox/prototype/exec/java-exec/src/test/resources/sender/hash_exchange.json
+++ b/sandbox/prototype/exec/java-exec/src/test/resources/sender/hash_exchange.json
@@ -28,7 +28,7 @@
             @id: 2,
             child: 1,
             pop: "hash-to-random-exchange",
-            expr: "hash(red)"
+            expr: "hash(1)"
         },
         {
              @id: 3,


[15/27] git commit: Updates to add subscan support to JSON

Posted by ja...@apache.org.
Updates to add subscan support to JSON


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

Branch: refs/heads/master
Commit: 2884db7a9162f9d348dbb88663e0da191a399c4d
Parents: 402be7e
Author: Jacques Nadeau <ja...@apache.org>
Authored: Thu Aug 15 17:57:24 2013 -0700
Committer: Jacques Nadeau <ja...@apache.org>
Committed: Thu Aug 15 17:57:24 2013 -0700

----------------------------------------------------------------------
 sandbox/prototype/exec/java-exec/pom.xml        |  15 +-
 .../apache/drill/exec/opt/BasicOptimizer.java   |   2 +-
 .../physical/config/JSONScanBatchCreator.java   |  46 --
 .../drill/exec/physical/config/JSONScanPOP.java | 114 -----
 .../exec/physical/config/MockGroupScanPOP.java  | 221 ---------
 .../exec/physical/config/MockRecordReader.java  | 118 -----
 .../physical/config/MockScanBatchCreator.java   |  46 --
 .../exec/physical/config/MockStorePOP.java      |  75 ---
 .../exec/physical/config/MockSubScanPOP.java    | 115 -----
 .../drill/exec/physical/impl/ImplCreator.java   |  56 +--
 .../drill/exec/store/JSONRecordReader.java      | 486 ------------------
 .../apache/drill/exec/store/VectorHolder.java   |  21 +-
 .../drill/exec/store/json/JSONGroupScan.java    | 145 ++++++
 .../drill/exec/store/json/JSONRecordReader.java | 489 +++++++++++++++++++
 .../exec/store/json/JSONScanBatchCreator.java   |  46 ++
 .../drill/exec/store/json/JSONSubScan.java      |  86 ++++
 .../drill/exec/store/mock/MockGroupScanPOP.java | 221 +++++++++
 .../drill/exec/store/mock/MockRecordReader.java | 118 +++++
 .../exec/store/mock/MockScanBatchCreator.java   |  46 ++
 .../exec/store/mock/MockStorageEngine.java      |   3 +-
 .../drill/exec/store/mock/MockStorePOP.java     |  75 +++
 .../drill/exec/store/mock/MockSubScanPOP.java   | 115 +++++
 .../drill/exec/store/parquet/ColumnReader.java  |   2 +-
 .../exec/store/parquet/PageReadStatus.java      |  15 +-
 .../exec/store/parquet/ParquetGroupScan.java    |   5 +-
 .../store/parquet/ParquetScanBatchCreator.java  |   5 +-
 .../drill/exec/store/JSONRecordReaderTest.java  |   2 +
 .../store/parquet/ParquetRecordReaderTest.java  |  33 +-
 .../src/test/resources/parquet_scan_screen.json |   4 +-
 .../apache/drill/jdbc/test/FullEngineTest.java  |   2 +
 30 files changed, 1416 insertions(+), 1311 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/2884db7a/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 a2e8501..b36208c 100644
--- a/sandbox/prototype/exec/java-exec/pom.xml
+++ b/sandbox/prototype/exec/java-exec/pom.xml
@@ -46,12 +46,12 @@
     <dependency>
       <groupId>com.twitter</groupId>
       <artifactId>parquet-column</artifactId>
-      <version>1.0.1-SNAPSHOT</version>
+      <version>1.0.1</version>
     </dependency>
     <dependency>
       <groupId>com.twitter</groupId>
       <artifactId>parquet-hadoop</artifactId>
-      <version>1.0.1-SNAPSHOT</version>
+      <version>1.0.1</version>
     </dependency>
     <dependency>
       <groupId>com.yammer.metrics</groupId>
@@ -106,7 +106,7 @@
     <dependency>
       <groupId>org.apache.hadoop</groupId>
       <artifactId>hadoop-core</artifactId>
-      <version>1.0.3-mapr-2.1.2.1</version>
+      <version>1.2.1</version>
       <exclusions>
         <exclusion>
           <artifactId>jets3t</artifactId>
@@ -291,12 +291,5 @@
   </build>
 
 
-  <repositories>
-    <repository>
-      <id>mapr-releases</id>
-      <url>http://repository.mapr.com/maven/</url>
-      <snapshots><enabled>false</enabled></snapshots>
-      <releases><enabled>true</enabled></releases>
-    </repository>
-  </repositories>
+
 </project>

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/2884db7a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/opt/BasicOptimizer.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/opt/BasicOptimizer.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/opt/BasicOptimizer.java
index b5eea03..2c2a342 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/opt/BasicOptimizer.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/opt/BasicOptimizer.java
@@ -28,10 +28,10 @@ import org.apache.drill.exec.exception.SetupException;
 import org.apache.drill.exec.ops.QueryContext;
 import org.apache.drill.exec.physical.PhysicalPlan;
 import org.apache.drill.exec.physical.base.PhysicalOperator;
-import org.apache.drill.exec.physical.config.MockGroupScanPOP;
 import org.apache.drill.exec.physical.config.Screen;
 import org.apache.drill.exec.physical.config.SelectionVectorRemover;
 import org.apache.drill.exec.store.StorageEngine;
+import org.apache.drill.exec.store.mock.MockGroupScanPOP;
 
 import com.fasterxml.jackson.core.type.TypeReference;
 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/2884db7a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/JSONScanBatchCreator.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/JSONScanBatchCreator.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/JSONScanBatchCreator.java
deleted file mode 100644
index f93f03b..0000000
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/JSONScanBatchCreator.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.physical.config;
-
-import com.google.common.base.Preconditions;
-import com.google.common.collect.Lists;
-import org.apache.drill.common.exceptions.ExecutionSetupException;
-import org.apache.drill.exec.ops.FragmentContext;
-import org.apache.drill.exec.physical.impl.BatchCreator;
-import org.apache.drill.exec.physical.impl.ScanBatch;
-import org.apache.drill.exec.record.RecordBatch;
-import org.apache.drill.exec.store.JSONRecordReader;
-import org.apache.drill.exec.store.RecordReader;
-
-import java.util.List;
-
-public class JSONScanBatchCreator implements BatchCreator<JSONScanPOP> {
-
-    @Override
-    public RecordBatch getBatch(FragmentContext context, JSONScanPOP config, List<RecordBatch> children) throws ExecutionSetupException {
-        Preconditions.checkArgument(children.isEmpty());
-        List<JSONScanPOP.ScanEntry> entries = config.getReadEntries();
-        List<RecordReader> readers = Lists.newArrayList();
-        for (JSONScanPOP.ScanEntry e : entries) {
-            readers.add(new JSONRecordReader(context, e.getUrl()));
-        }
-
-        return new ScanBatch(context, readers.iterator());
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/2884db7a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/JSONScanPOP.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/JSONScanPOP.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/JSONScanPOP.java
deleted file mode 100644
index 1dcf5e1..0000000
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/JSONScanPOP.java
+++ /dev/null
@@ -1,114 +0,0 @@
-/*******************************************************************************
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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 com.fasterxml.jackson.annotation.JsonCreator;
-import com.fasterxml.jackson.annotation.JsonProperty;
-import com.fasterxml.jackson.annotation.JsonTypeName;
-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;
-
-import java.io.File;
-import java.net.URI;
-import java.util.Collections;
-import java.util.LinkedList;
-import java.util.List;
-
-import static com.google.common.base.Preconditions.checkArgument;
-
-@JsonTypeName("json-scan")
-public class JSONScanPOP extends AbstractScan<JSONScanPOP.ScanEntry> {
-    private static int ESTIMATED_RECORD_SIZE = 1024; // 1kb
-
-    private LinkedList[] mappings;
-
-    @JsonCreator
-    public JSONScanPOP(@JsonProperty("entries") List<JSONScanPOP.ScanEntry> readEntries) {
-        super(readEntries);
-    }
-
-    @SuppressWarnings("unchecked")
-    @Override
-    public void applyAssignments(List<CoordinationProtos.DrillbitEndpoint> endpoints) {
-        checkArgument(endpoints.size() <= getReadEntries().size());
-
-        mappings = new LinkedList[endpoints.size()];
-
-        int i = 0;
-        for (ScanEntry e : this.getReadEntries()) {
-            if (i == endpoints.size()) i = 0;
-            LinkedList entries = mappings[i];
-            if (entries == null) {
-                entries = new LinkedList<>();
-                mappings[i] = entries;
-            }
-            entries.add(e);
-            i++;
-        }
-    }
-
-    @SuppressWarnings("unchecked")
-    @Override
-    public Scan<?> getSpecificScan(int minorFragmentId) {
-        checkArgument(minorFragmentId < mappings.length, "Mappings length [%s] should be longer than minor fragment id [%s] but it isn't.", mappings.length, minorFragmentId);
-        return new JSONScanPOP(mappings[minorFragmentId]);
-    }
-
-    @Override
-    public List<EndpointAffinity> getOperatorAffinity() {
-        return Collections.emptyList();
-    }
-
-    @Override
-    public PhysicalOperator getNewWithChildren(List<PhysicalOperator> children) {
-        return new JSONScanPOP(readEntries);
-    }
-
-    public static class ScanEntry implements ReadEntry {
-        private final String url;
-        private Size size;
-
-        @JsonCreator
-        public ScanEntry(@JsonProperty("url") String url) {
-            this.url = url;
-            long fileLength = new File(URI.create(url)).length();
-            size = new Size(fileLength / ESTIMATED_RECORD_SIZE, ESTIMATED_RECORD_SIZE);
-        }
-
-        @Override
-        public OperatorCost getCost() {
-            return new OperatorCost(1, 1, 2, 2);
-        }
-
-        @Override
-        public Size getSize() {
-            return size;
-        }
-
-        public String getUrl() {
-            return url;
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/2884db7a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockGroupScanPOP.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockGroupScanPOP.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockGroupScanPOP.java
deleted file mode 100644
index a28c7d8..0000000
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockGroupScanPOP.java
+++ /dev/null
@@ -1,221 +0,0 @@
-/*******************************************************************************
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.common.types.TypeProtos.DataMode;
-import org.apache.drill.common.types.TypeProtos.MajorType;
-import org.apache.drill.common.types.TypeProtos.MinorType;
-import org.apache.drill.exec.physical.EndpointAffinity;
-import org.apache.drill.exec.physical.OperatorCost;
-import org.apache.drill.exec.physical.ReadEntry;
-import org.apache.drill.exec.physical.base.*;
-import org.apache.drill.exec.physical.base.AbstractGroupScan;
-import org.apache.drill.exec.physical.base.GroupScan;
-import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
-import org.apache.drill.exec.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 MockGroupScanPOP extends AbstractGroupScan {
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(MockGroupScanPOP.class);
-
-  private final String url;
-  protected final List<MockScanEntry> readEntries;
-  private final OperatorCost cost;
-  private final Size size;
-  private  LinkedList<MockScanEntry>[] mappings;
-
-  @JsonCreator
-  public MockGroupScanPOP(@JsonProperty("url") String url, @JsonProperty("entries") List<MockScanEntry> readEntries) {
-    this.readEntries = readEntries;
-    OperatorCost cost = new OperatorCost(0,0,0,0);
-    Size size = new Size(0,0);
-    for(MockScanEntry r : readEntries){
-      cost = cost.add(r.getCost());
-      size = size.add(r.getSize());
-    }
-    this.cost = cost;
-    this.size = size;
-    this.url = url;
-  }
-
-  public String getUrl() {
-    return url;
-  }
-
-  @JsonProperty("entries")
-  public List<MockScanEntry> getReadEntries() {
-    return readEntries;
-  }
-  
-  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 SubScan 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 MockSubScanPOP(url, mappings[minorFragmentId]);
-  }
-
-  @Override
-  public int getMaxParallelizationWidth() {
-    return readEntries.size();
-  }
-
-  @Override
-  public OperatorCost getCost() {
-    return cost;
-  }
-
-  @Override
-  public Size getSize() {
-    return size;
-  }
-
-  @Override
-  @JsonIgnore
-  public PhysicalOperator getNewWithChildren(List<PhysicalOperator> children) {
-    Preconditions.checkArgument(children.isEmpty());
-    return new MockGroupScanPOP(url, readEntries);
-
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/2884db7a/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
deleted file mode 100644
index bd57823..0000000
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockRecordReader.java
+++ /dev/null
@@ -1,118 +0,0 @@
-/*******************************************************************************
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- ******************************************************************************/
-package org.apache.drill.exec.physical.config;
-
-import org.apache.drill.common.exceptions.ExecutionSetupException;
-import org.apache.drill.common.expression.ExpressionPosition;
-import org.apache.drill.common.expression.SchemaPath;
-import org.apache.drill.common.types.TypeProtos.MajorType;
-import org.apache.drill.exec.exception.SchemaChangeException;
-import org.apache.drill.exec.ops.FragmentContext;
-import org.apache.drill.exec.physical.config.MockGroupScanPOP.MockColumn;
-import org.apache.drill.exec.physical.config.MockGroupScanPOP.MockScanEntry;
-import org.apache.drill.exec.physical.impl.OutputMutator;
-import org.apache.drill.exec.record.MaterializedField;
-import org.apache.drill.exec.store.RecordReader;
-import org.apache.drill.exec.vector.AllocationHelper;
-import org.apache.drill.exec.vector.TypeHelper;
-import org.apache.drill.exec.vector.ValueVector;
-
-public class MockRecordReader implements RecordReader {
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(MockRecordReader.class);
-
-  private OutputMutator output;
-  private MockScanEntry config;
-  private FragmentContext context;
-  private ValueVector[] valueVectors;
-  private int recordsRead;
-  private int batchRecordCount;
-
-  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(String name, MajorType type, int length) {
-    assert context != null : "Context shouldn't be null.";
-    MaterializedField f = MaterializedField.create(new SchemaPath(name, ExpressionPosition.UNKNOWN), type);
-    ValueVector v;
-    v = TypeHelper.getNewVector(f, context.getAllocator());
-    AllocationHelper.allocate(v, length, 50, 4);
-    
-    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];
-      batchRecordCount = 250000 / estimateRowSize;
-
-      for (int i = 0; i < config.getTypes().length; i++) {
-        valueVectors[i] = getVector(config.getTypes()[i].getName(), config.getTypes()[i].getMajorType(), batchRecordCount);
-        output.addField(valueVectors[i]);
-      }
-      output.setNewSchema();
-    } catch (SchemaChangeException e) {
-      throw new ExecutionSetupException("Failure while setting up fields", e);
-    }
-
-  }
-
-  @Override
-  public int next() {
-    
-    int recordSetSize = Math.min(batchRecordCount, this.config.getRecords()- recordsRead);
-
-    recordsRead += recordSetSize;
-    for(ValueVector v : valueVectors){
-      AllocationHelper.allocate(v, recordSetSize, 50, 5);
-      
-      logger.debug("MockRecordReader:  Generating random data for VV of type " + v.getClass().getName());
-      ValueVector.Mutator m = v.getMutator();
-      m.setValueCount(recordSetSize);
-      m.generateTestData();
-      
-    }
-    return recordSetSize;
-  }
-
-  @Override
-  public void cleanup() {
-    for (int i = 0; i < valueVectors.length; i++) {
-      try {
-        output.removeField(valueVectors[i].getField());
-      } catch (SchemaChangeException e) {
-        logger.warn("Failure while trying to remove field.", e);
-      }
-      valueVectors[i].close();
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/2884db7a/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
deleted file mode 100644
index a06aaee..0000000
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockScanBatchCreator.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.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.MockGroupScanPOP.MockScanEntry;
-import org.apache.drill.exec.physical.impl.BatchCreator;
-import org.apache.drill.exec.physical.impl.ScanBatch;
-import org.apache.drill.exec.record.RecordBatch;
-import org.apache.drill.exec.store.RecordReader;
-
-import com.google.common.base.Preconditions;
-import com.google.common.collect.Lists;
-
-public class MockScanBatchCreator implements BatchCreator<MockSubScanPOP>{
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(MockScanBatchCreator.class);
-
-  @Override
-  public RecordBatch getBatch(FragmentContext context, MockSubScanPOP 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/2884db7a/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
deleted file mode 100644
index 639d0d2..0000000
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockStorePOP.java
+++ /dev/null
@@ -1,75 +0,0 @@
-/*******************************************************************************
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- ******************************************************************************/
-package org.apache.drill.exec.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/2884db7a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockSubScanPOP.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockSubScanPOP.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockSubScanPOP.java
deleted file mode 100644
index 7380617..0000000
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockSubScanPOP.java
+++ /dev/null
@@ -1,115 +0,0 @@
-/*******************************************************************************
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.Iterator;
-import java.util.LinkedList;
-import java.util.List;
-
-import com.google.common.collect.Iterators;
-import org.apache.drill.common.graph.GraphVisitor;
-import org.apache.drill.common.types.TypeProtos.DataMode;
-import org.apache.drill.common.types.TypeProtos.MajorType;
-import org.apache.drill.common.types.TypeProtos.MinorType;
-import org.apache.drill.exec.physical.EndpointAffinity;
-import org.apache.drill.exec.physical.OperatorCost;
-import org.apache.drill.exec.physical.ReadEntry;
-import org.apache.drill.exec.physical.base.*;
-import org.apache.drill.exec.physical.base.AbstractGroupScan;
-import org.apache.drill.exec.physical.base.GroupScan;
-import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
-import org.apache.drill.exec.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-sub-scan")
-public class MockSubScanPOP extends AbstractBase implements SubScan {
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(MockGroupScanPOP.class);
-
-  private final String url;
-  protected final List<MockGroupScanPOP.MockScanEntry> readEntries;
-  private final OperatorCost cost;
-  private final Size size;
-  private  LinkedList<MockGroupScanPOP.MockScanEntry>[] mappings;
-
-  @JsonCreator
-  public MockSubScanPOP(@JsonProperty("url") String url, @JsonProperty("entries") List<MockGroupScanPOP.MockScanEntry> readEntries) {
-    this.readEntries = readEntries;
-    OperatorCost cost = new OperatorCost(0,0,0,0);
-    Size size = new Size(0,0);
-    for(MockGroupScanPOP.MockScanEntry r : readEntries){
-      cost = cost.add(r.getCost());
-      size = size.add(r.getSize());
-    }
-    this.cost = cost;
-    this.size = size;
-    this.url = url;
-  }
-
-  public String getUrl() {
-    return url;
-  }
-
-  @JsonProperty("entries")
-  public List<MockGroupScanPOP.MockScanEntry> getReadEntries() {
-    return readEntries;
-  }
-
-  @Override
-  public Iterator<PhysicalOperator> iterator() {
-    return Iterators.emptyIterator();
-  }
-
-  @Override
-  public OperatorCost getCost() {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public Size getSize() {
-    throw new UnsupportedOperationException();
-  }
-
-  // will want to replace these two methods with an interface above for AbstractSubScan
-  @Override
-  public boolean isExecutable() {
-    return true;  //To change body of implemented methods use File | Settings | File Templates.
-  }
-
-  @Override
-  public <T, X, E extends Throwable> T accept(PhysicalVisitor<T, X, E> physicalVisitor, X value) throws E{
-    return physicalVisitor.visitSubScan(this, value);
-  }
-  // see comment above about replacing this
-
-  @Override
-  @JsonIgnore
-  public PhysicalOperator getNewWithChildren(List<PhysicalOperator> children) {
-    Preconditions.checkArgument(children.isEmpty());
-    return new MockSubScanPOP(url, readEntries);
-
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/2884db7a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ImplCreator.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ImplCreator.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ImplCreator.java
index 61c9383..0a329d6 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ImplCreator.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ImplCreator.java
@@ -25,30 +25,31 @@ 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.base.SubScan;
 import org.apache.drill.exec.physical.config.Filter;
-import org.apache.drill.exec.physical.config.MockScanBatchCreator;
 import org.apache.drill.exec.physical.config.Project;
 import org.apache.drill.exec.physical.config.RandomReceiver;
 import org.apache.drill.exec.physical.config.Screen;
 import org.apache.drill.exec.physical.config.SelectionVectorRemover;
 import org.apache.drill.exec.physical.config.SingleSender;
 import org.apache.drill.exec.physical.config.Sort;
-import org.apache.drill.exec.physical.base.*;
-import org.apache.drill.exec.physical.config.*;
 import org.apache.drill.exec.physical.impl.filter.FilterBatchCreator;
 import org.apache.drill.exec.physical.impl.project.ProjectBatchCreator;
 import org.apache.drill.exec.physical.impl.sort.SortBatchCreator;
 import org.apache.drill.exec.physical.impl.svremover.SVRemoverCreator;
 import org.apache.drill.exec.record.RecordBatch;
+import org.apache.drill.exec.store.json.JSONScanBatchCreator;
+import org.apache.drill.exec.store.json.JSONSubScan;
+import org.apache.drill.exec.store.mock.MockGroupScanPOP;
+import org.apache.drill.exec.store.mock.MockScanBatchCreator;
+import org.apache.drill.exec.store.mock.MockSubScanPOP;
+import org.apache.drill.exec.store.parquet.ParquetRowGroupScan;
+import org.apache.drill.exec.store.parquet.ParquetScanBatchCreator;
 
 import com.google.common.base.Preconditions;
 import com.google.common.collect.Lists;
-import org.apache.drill.exec.store.parquet.ParquetGroupScan;
-import org.apache.drill.exec.store.parquet.ParquetRowGroupScan;
-import org.apache.drill.exec.store.parquet.ParquetScanBatchCreator;
 
-public class ImplCreator extends AbstractPhysicalVisitor<RecordBatch, FragmentContext, ExecutionSetupException>{
+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();
@@ -62,9 +63,10 @@ public class ImplCreator extends AbstractPhysicalVisitor<RecordBatch, FragmentCo
   private SortBatchCreator sbc = new SortBatchCreator();
   private RootExec root = null;
 
-  private ImplCreator(){}
+  private ImplCreator() {
+  }
 
-  public RootExec getRoot(){
+  public RootExec getRoot() {
     return root;
   }
 
@@ -78,20 +80,13 @@ public class ImplCreator extends AbstractPhysicalVisitor<RecordBatch, FragmentCo
     Preconditions.checkNotNull(subScan);
     Preconditions.checkNotNull(context);
 
-    if(subScan instanceof MockSubScanPOP){
+    if (subScan instanceof MockSubScanPOP) {
       return msc.getBatch(context, (MockSubScanPOP) subScan, Collections.<RecordBatch> emptyList());
-    
-    if(scan instanceof MockScanPOP){
-      return msc.getBatch(context, (MockScanPOP) scan, Collections.<RecordBatch>emptyList());
-    } else if(scan instanceof JSONScanPOP) {
-      return new JSONScanBatchCreator().getBatch(context, (JSONScanPOP)scan, Collections.<RecordBatch>emptyList());
-    }else{
-      return super.visitScan(scan, context);  
-    }
-    else if (subScan instanceof ParquetRowGroupScan){
-      return parquetScan.getBatch(context, (ParquetRowGroupScan) subScan,  Collections.<RecordBatch> emptyList());
-    }
-    else{
+    } else if (subScan instanceof JSONSubScan) {
+      return new JSONScanBatchCreator().getBatch(context, (JSONSubScan) subScan, Collections.<RecordBatch> emptyList());
+    } else if (subScan instanceof ParquetRowGroupScan) {
+      return parquetScan.getBatch(context, (ParquetRowGroupScan) subScan, Collections.<RecordBatch> emptyList());
+    } else {
       return super.visitSubScan(subScan, context);
     }
 
@@ -99,14 +94,13 @@ public class ImplCreator extends AbstractPhysicalVisitor<RecordBatch, FragmentCo
 
   @Override
   public RecordBatch visitOp(PhysicalOperator op, FragmentContext context) throws ExecutionSetupException {
-    if(op instanceof SelectionVectorRemover){
+    if (op instanceof SelectionVectorRemover) {
       return svc.getBatch(context, (SelectionVectorRemover) op, getChildren(op, context));
-    }else{
+    } else {
       return super.visitOp(op, context);
     }
   }
 
-  
   @Override
   public RecordBatch visitSort(Sort sort, FragmentContext context) throws ExecutionSetupException {
     return sbc.getBatch(context, sort, getChildren(sort, context));
@@ -135,18 +129,20 @@ public class ImplCreator extends AbstractPhysicalVisitor<RecordBatch, FragmentCo
     return rrc.getBatch(context, op, null);
   }
 
-  private List<RecordBatch> getChildren(PhysicalOperator op, FragmentContext context) throws ExecutionSetupException{
+  private List<RecordBatch> getChildren(PhysicalOperator op, FragmentContext context) throws ExecutionSetupException {
     List<RecordBatch> children = Lists.newArrayList();
-    for(PhysicalOperator child : op){
+    for (PhysicalOperator child : op) {
       children.add(child.accept(this, context));
     }
     return children;
   }
 
-  public static RootExec getExec(FragmentContext context, FragmentRoot root) throws ExecutionSetupException{
+  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.");
+    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/2884db7a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/JSONRecordReader.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/JSONRecordReader.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/JSONRecordReader.java
deleted file mode 100644
index 8a2de63..0000000
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/JSONRecordReader.java
+++ /dev/null
@@ -1,486 +0,0 @@
-package org.apache.drill.exec.store;
-
-import com.fasterxml.jackson.core.JsonFactory;
-import com.fasterxml.jackson.core.JsonParser;
-import com.fasterxml.jackson.core.JsonToken;
-import com.google.common.base.Charsets;
-import com.google.common.collect.Iterables;
-import com.google.common.collect.Lists;
-import com.google.common.collect.Maps;
-import com.google.common.io.Files;
-import com.google.common.io.InputSupplier;
-import com.google.common.io.Resources;
-import org.apache.drill.common.exceptions.DrillRuntimeException;
-import org.apache.drill.common.exceptions.ExecutionSetupException;
-import org.apache.drill.common.expression.ExpressionPosition;
-import org.apache.drill.common.expression.SchemaPath;
-import org.apache.drill.common.types.TypeProtos.MajorType;
-import org.apache.drill.common.types.TypeProtos.MinorType;
-import org.apache.drill.common.types.Types;
-import org.apache.drill.exec.exception.SchemaChangeException;
-import org.apache.drill.exec.memory.BufferAllocator;
-import org.apache.drill.exec.ops.FragmentContext;
-import org.apache.drill.exec.physical.impl.OutputMutator;
-import org.apache.drill.exec.record.MaterializedField;
-import org.apache.drill.exec.schema.*;
-import org.apache.drill.exec.schema.json.jackson.JacksonHelper;
-import org.apache.drill.exec.vector.*;
-
-import java.io.File;
-import java.io.IOException;
-import java.io.InputStreamReader;
-import java.net.URI;
-import java.nio.charset.Charset;
-import java.util.List;
-import java.util.Map;
-
-import static com.fasterxml.jackson.core.JsonToken.*;
-
-public class JSONRecordReader implements RecordReader {
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(JSONRecordReader.class);
-  private static final int DEFAULT_LENGTH = 256 * 1024; // 256kb
-  public static final Charset UTF_8 = Charset.forName("UTF-8");
-
-  private final String inputPath;
-
-  private final Map<String, VectorHolder> valueVectorMap;
-
-  private JsonParser parser;
-  private SchemaIdGenerator generator;
-  private DiffSchema diffSchema;
-  private RecordSchema currentSchema;
-  private List<Field> removedFields;
-  private OutputMutator outputMutator;
-  private BufferAllocator allocator;
-  private int batchSize;
-
-  public JSONRecordReader(FragmentContext fragmentContext, String inputPath, int batchSize) {
-    this.inputPath = inputPath;
-    this.allocator = fragmentContext.getAllocator();
-    this.batchSize = batchSize;
-    valueVectorMap = Maps.newHashMap();
-  }
-
-  public JSONRecordReader(FragmentContext fragmentContext, String inputPath) {
-    this(fragmentContext, inputPath, DEFAULT_LENGTH);
-  }
-
-  private JsonParser getParser() {
-    return parser;
-  }
-
-  @Override
-  public void setup(OutputMutator output) throws ExecutionSetupException {
-    outputMutator = output;
-    currentSchema = new ObjectSchema();
-    diffSchema = new DiffSchema();
-    removedFields = Lists.newArrayList();
-
-    try {
-      InputSupplier<InputStreamReader> input;
-      if (inputPath.startsWith("resource:")) {
-        input = Resources.newReaderSupplier(Resources.getResource(inputPath.substring(9)), Charsets.UTF_8);
-      } else {
-        input = Files.newReaderSupplier(new File(URI.create(inputPath)), Charsets.UTF_8);
-      }
-
-      JsonFactory factory = new JsonFactory();
-      parser = factory.createJsonParser(input.getInput());
-      parser.nextToken(); // Read to the first START_OBJECT token
-      generator = new SchemaIdGenerator();
-    } catch (IOException e) {
-      throw new ExecutionSetupException(e);
-    }
-  }
-
-  @Override
-  public int next() {
-    if (parser.isClosed() || !parser.hasCurrentToken()) {
-      return 0;
-    }
-
-    resetBatch();
-
-    int nextRowIndex = 0;
-
-    try {
-      while (ReadType.OBJECT.readRecord(this, null, nextRowIndex++, 0)) {
-        parser.nextToken(); // Read to START_OBJECT token
-
-        if (!parser.hasCurrentToken()) {
-          parser.close();
-          break;
-        }
-      }
-
-      parser.nextToken();
-
-      if (!parser.hasCurrentToken()) {
-        parser.close();
-      }
-
-      // Garbage collect fields never referenced in this batch
-      for (Field field : Iterables.concat(currentSchema.removeUnreadFields(), removedFields)) {
-        diffSchema.addRemovedField(field);
-        outputMutator.removeField(field.getAsMaterializedField());
-      }
-
-      if (diffSchema.isChanged()) {
-        outputMutator.setNewSchema();
-      }
-
-
-    } catch (IOException | SchemaChangeException e) {
-      logger.error("Error reading next in Json reader", e);
-    }
-
-    for (VectorHolder holder : valueVectorMap.values()) {
-      holder.populateVectorLength();
-    }
-
-    return nextRowIndex;
-  }
-
-  private void resetBatch() {
-    for (VectorHolder value : valueVectorMap.values()) {
-      value.reset();
-    }
-
-    currentSchema.resetMarkedFields();
-    diffSchema.reset();
-    removedFields.clear();
-  }
-
-  @Override
-  public void cleanup() {
-    try {
-      parser.close();
-    } catch (IOException e) {
-      logger.warn("Error closing Json parser", e);
-    }
-  }
-
-
-  private RecordSchema getCurrentSchema() {
-    return currentSchema;
-  }
-
-  private void setCurrentSchema(RecordSchema schema) {
-    currentSchema = schema;
-  }
-
-  private List<Field> getRemovedFields() {
-    return removedFields;
-  }
-
-  public BufferAllocator getAllocator() {
-    return allocator;
-  }
-
-  public static enum ReadType {
-    ARRAY(END_ARRAY) {
-      @Override
-      public Field createField(RecordSchema parentSchema, String prefixFieldName, String fieldName, MajorType fieldType, int index) {
-        return new NamedField(parentSchema, prefixFieldName, fieldName, fieldType);
-      }
-
-      @Override
-      public RecordSchema createSchema() throws IOException {
-        return new ObjectSchema();
-      }
-    },
-    OBJECT(END_OBJECT) {
-      @Override
-      public Field createField(RecordSchema parentSchema,
-                               String prefixFieldName,
-                               String fieldName,
-                               MajorType fieldType,
-                               int index) {
-        return new NamedField(parentSchema, prefixFieldName, fieldName, fieldType);
-      }
-
-      @Override
-      public RecordSchema createSchema() throws IOException {
-        return new ObjectSchema();
-      }
-    };
-
-    private final JsonToken endObject;
-
-    ReadType(JsonToken endObject) {
-      this.endObject = endObject;
-    }
-
-    public JsonToken getEndObject() {
-      return endObject;
-    }
-
-    @SuppressWarnings("ConstantConditions")
-    public boolean readRecord(JSONRecordReader reader,
-                              String prefixFieldName,
-                              int rowIndex,
-                              int groupCount) throws IOException, SchemaChangeException {
-      JsonParser parser = reader.getParser();
-      JsonToken token = parser.nextToken();
-      JsonToken endObject = getEndObject();
-      int colIndex = 0;
-      boolean isFull = false;
-      while (token != endObject) {
-        if (token == FIELD_NAME) {
-          token = parser.nextToken();
-          continue;
-        }
-
-        String fieldName = parser.getCurrentName();
-        MajorType fieldType = JacksonHelper.getFieldType(token, this == ReadType.ARRAY);
-        ReadType readType = null;
-        switch (token) {
-          case START_ARRAY:
-            readType = ReadType.ARRAY;
-            groupCount++;
-            break;
-          case START_OBJECT:
-            readType = ReadType.OBJECT;
-            groupCount = 0;
-            break;
-        }
-
-        if (fieldType != null) { // Including nulls
-          boolean currentFieldFull = !recordData(
-              readType,
-              reader,
-              fieldType,
-              prefixFieldName,
-              fieldName,
-              rowIndex,
-              colIndex,
-              groupCount);
-
-          isFull = isFull || currentFieldFull;
-        }
-        token = parser.nextToken();
-        colIndex += 1;
-      }
-      return !isFull;
-    }
-
-    private void removeChildFields(List<Field> removedFields, Field field) {
-      RecordSchema schema = field.getAssignedSchema();
-      if (schema == null) {
-        return;
-      }
-      for (Field childField : schema.getFields()) {
-        removedFields.add(childField);
-        if (childField.hasSchema()) {
-          removeChildFields(removedFields, childField);
-        }
-      }
-    }
-
-    private boolean recordData(JSONRecordReader.ReadType readType,
-                               JSONRecordReader reader,
-                               MajorType fieldType,
-                               String prefixFieldName,
-                               String fieldName,
-                               int rowIndex,
-                               int colIndex,
-                               int groupCount) throws IOException, SchemaChangeException {
-      RecordSchema currentSchema = reader.getCurrentSchema();
-      Field field = currentSchema.getField(fieldName == null ? prefixFieldName : fieldName, colIndex);
-      boolean isFieldFound = field != null;
-      List<Field> removedFields = reader.getRemovedFields();
-      boolean newFieldLateBound = fieldType.getMinorType().equals(MinorType.LATE);
-
-      if (isFieldFound && !field.getFieldType().equals(fieldType)) {
-        boolean existingFieldLateBound = field.getFieldType().getMinorType().equals(MinorType.LATE);
-
-        if (newFieldLateBound && !existingFieldLateBound) {
-          fieldType = Types.overrideMinorType(fieldType, field.getFieldType().getMinorType());
-        } else if (!newFieldLateBound && existingFieldLateBound) {
-          field.setFieldType(Types.overrideMinorType(field.getFieldType(), fieldType.getMinorType()));
-        } else if (!newFieldLateBound && !existingFieldLateBound) {
-          if (field.hasSchema()) {
-            removeChildFields(removedFields, field);
-          }
-          removedFields.add(field);
-          currentSchema.removeField(field, colIndex);
-
-          isFieldFound = false;
-        }
-      }
-
-      if (!isFieldFound) {
-        field = createField(
-            currentSchema,
-            prefixFieldName,
-            fieldName,
-            fieldType,
-            colIndex
-        );
-
-        reader.recordNewField(field);
-        currentSchema.addField(field);
-      }
-
-      field.setRead(true);
-
-      VectorHolder holder = getOrCreateVectorHolder(reader, field);
-      if (readType != null) {
-        RecordSchema fieldSchema = field.getAssignedSchema();
-        RecordSchema newSchema = readType.createSchema();
-
-        if (readType != ReadType.ARRAY) {
-          reader.setCurrentSchema(fieldSchema);
-          if (fieldSchema == null) reader.setCurrentSchema(newSchema);
-          readType.readRecord(reader, field.getFullFieldName(), rowIndex, groupCount);
-        } else {
-          readType.readRecord(reader, field.getFullFieldName(), rowIndex, groupCount);
-        }
-
-        reader.setCurrentSchema(currentSchema);
-
-      } else if (holder != null && !newFieldLateBound && fieldType.getMinorType() != MinorType.LATE) {
-        return addValueToVector(
-            rowIndex,
-            holder,
-            JacksonHelper.getValueFromFieldType(
-                reader.getParser(),
-                fieldType.getMinorType()
-            ),
-            fieldType.getMinorType(),
-            groupCount
-        );
-      }
-
-      return true;
-    }
-
-    private static <T> boolean addValueToVector(int index, VectorHolder holder, T val, MinorType minorType, int groupCount) {
-      switch (minorType) {
-        case INT: {
-          holder.incAndCheckLength(NullableIntHolder.WIDTH * 8 + 1);
-          if (groupCount == 0) {
-            if (val != null) {
-              NullableIntVector int4 = (NullableIntVector) holder.getValueVector();
-              NullableIntVector.Mutator m = int4.getMutator();
-              m.set(index, (Integer) val);
-            }
-          } else {
-            if (val == null) {
-              throw new UnsupportedOperationException("Nullable repeated int is not supported.");
-            }
-
-            RepeatedIntVector repeatedInt4 = (RepeatedIntVector) holder.getValueVector();
-            RepeatedIntVector.Mutator m = repeatedInt4.getMutator();
-            holder.setGroupCount(index);
-            m.add(index, (Integer) val);
-          }
-
-          return holder.hasEnoughSpace(NullableIntHolder.WIDTH * 8 + 1);
-        }
-        case FLOAT4: {
-          holder.incAndCheckLength(NullableFloat4Holder.WIDTH * 8 + 1);
-          if (groupCount == 0) {
-            if (val != null) {
-              NullableFloat4Vector float4 = (NullableFloat4Vector) holder.getValueVector();
-              NullableFloat4Vector.Mutator m = float4.getMutator();
-              m.set(index, (Float) val);
-            }
-          } else {
-            if (val == null) {
-              throw new UnsupportedOperationException("Nullable repeated float is not supported.");
-            }
-
-            RepeatedFloat4Vector repeatedFloat4 = (RepeatedFloat4Vector) holder.getValueVector();
-            RepeatedFloat4Vector.Mutator m = repeatedFloat4.getMutator();
-            holder.setGroupCount(index);
-            m.add(index, (Float) val);
-          }
-          return holder.hasEnoughSpace(NullableFloat4Holder.WIDTH * 8 + 1);
-        }
-        case VARCHAR: {
-          if (val == null) {
-            return (index + 1) * 4 <= holder.getLength();
-          } else {
-            byte[] bytes = ((String) val).getBytes(UTF_8);
-            int length = bytes.length;
-            holder.incAndCheckLength(length);
-            if (groupCount == 0) {
-              NullableVarCharVector varLen4 = (NullableVarCharVector) holder.getValueVector();
-              NullableVarCharVector.Mutator m = varLen4.getMutator();
-              m.set(index, bytes);
-            } else {
-              RepeatedVarCharVector repeatedVarLen4 = (RepeatedVarCharVector) holder.getValueVector();
-              RepeatedVarCharVector.Mutator m = repeatedVarLen4.getMutator();
-              holder.setGroupCount(index);
-              m.add(index, bytes);
-            }
-            return holder.hasEnoughSpace(length + 4 + 1);
-          }
-        }
-        case BIT: {
-          holder.incAndCheckLength(NullableBitHolder.WIDTH + 1);
-          if (groupCount == 0) {
-            if (val != null) {
-              NullableBitVector bit = (NullableBitVector) holder.getValueVector();
-              NullableBitVector.Mutator m = bit.getMutator();
-              m.set(index, (Boolean) val ? 1 : 0);
-            }
-          } else {
-            if (val == null) {
-              throw new UnsupportedOperationException("Nullable repeated boolean is not supported.");
-            }
-
-            RepeatedBitVector repeatedBit = (RepeatedBitVector) holder.getValueVector();
-            RepeatedBitVector.Mutator m = repeatedBit.getMutator();
-            holder.setGroupCount(index);
-            m.add(index, (Boolean) val ? 1 : 0);
-          }
-          return holder.hasEnoughSpace(NullableBitHolder.WIDTH + 1);
-        }
-        default:
-          throw new DrillRuntimeException("Type not supported to add value. Type: " + minorType);
-      }
-    }
-
-    private VectorHolder getOrCreateVectorHolder(JSONRecordReader reader, Field field) throws SchemaChangeException {
-      return reader.getOrCreateVectorHolder(field);
-    }
-
-    public abstract RecordSchema createSchema() throws IOException;
-
-    public abstract Field createField(RecordSchema parentSchema,
-                                      String prefixFieldName,
-                                      String fieldName,
-                                      MajorType fieldType,
-                                      int index);
-  }
-
-  private void recordNewField(Field field) {
-    diffSchema.recordNewField(field);
-  }
-
-  private VectorHolder getOrCreateVectorHolder(Field field) throws SchemaChangeException {
-    String fullFieldName = field.getFullFieldName();
-    VectorHolder holder = valueVectorMap.get(fullFieldName);
-
-    if (holder == null) {
-      MajorType type = field.getFieldType();
-      MinorType minorType = type.getMinorType();
-
-      if (minorType.equals(MinorType.MAP) || minorType.equals(MinorType.LATE)) {
-        return null;
-      }
-
-      MaterializedField f = MaterializedField.create(new SchemaPath(fullFieldName, ExpressionPosition.UNKNOWN), type);
-
-      ValueVector v = TypeHelper.getNewVector(f, allocator);
-      AllocationHelper.allocate(v, batchSize, 50);
-      holder = new VectorHolder(v);
-      valueVectorMap.put(fullFieldName, holder);
-      outputMutator.addField(v);
-      return holder;
-    }
-    return holder;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/2884db7a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/VectorHolder.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/VectorHolder.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/VectorHolder.java
index 7cbea57..828ae17 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/VectorHolder.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/VectorHolder.java
@@ -27,11 +27,16 @@ public class VectorHolder {
   private ValueVector vector;
   private int currentLength;
 
-    public VectorHolder(int length, ValueVector vector) {
-        this.length = length;
-        this.vector = vector;
-        this.mutator = vector.getMutator();
-    }
+  
+  public VectorHolder(int length, ValueVector vector) {
+    this.length = length;
+    this.vector = vector;
+  }
+  
+  public VectorHolder(ValueVector vector) {
+    this.length = vector.getValueCapacity();
+    this.vector = vector;
+  }
 
   public ValueVector getValueVector() {
     return vector;
@@ -47,9 +52,9 @@ public class VectorHolder {
   }
 
   public void setGroupCount(int groupCount) {
-    if(this.groupCount < groupCount) {
+    if (this.groupCount < groupCount) {
       RepeatedMutator mutator = (RepeatedMutator) vector.getMutator();
-      while(this.groupCount < groupCount) {
+      while (this.groupCount < groupCount) {
         mutator.startNewGroup(++this.groupCount);
       }
     }
@@ -71,7 +76,7 @@ public class VectorHolder {
 
   public void populateVectorLength() {
     ValueVector.Mutator mutator = vector.getMutator();
-    if(vector instanceof RepeatedFixedWidthVector || vector instanceof RepeatedVariableWidthVector) {
+    if (vector instanceof RepeatedFixedWidthVector || vector instanceof RepeatedVariableWidthVector) {
       mutator.setValueCount(groupCount);
     } else {
       mutator.setValueCount(count);

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/2884db7a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/json/JSONGroupScan.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/json/JSONGroupScan.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/json/JSONGroupScan.java
new file mode 100644
index 0000000..ff5f474
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/json/JSONGroupScan.java
@@ -0,0 +1,145 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.json;
+
+import static com.google.common.base.Preconditions.checkArgument;
+
+import java.io.File;
+import java.net.URI;
+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.AbstractGroupScan;
+import org.apache.drill.exec.physical.base.PhysicalOperator;
+import org.apache.drill.exec.physical.base.Size;
+import org.apache.drill.exec.physical.base.SubScan;
+import org.apache.drill.exec.proto.CoordinationProtos;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.annotation.JsonTypeName;
+
+@JsonTypeName("json-scan")
+public class JSONGroupScan extends AbstractGroupScan {
+    private static int ESTIMATED_RECORD_SIZE = 1024; // 1kb
+
+    private LinkedList<JSONGroupScan.ScanEntry>[] mappings;
+    protected final List<JSONGroupScan.ScanEntry> readEntries;
+    private final OperatorCost cost;
+    private final Size size;
+    
+    @JsonCreator
+    public JSONGroupScan(@JsonProperty("entries") List<JSONGroupScan.ScanEntry> readEntries) {
+        this.readEntries = readEntries;
+        OperatorCost cost = new OperatorCost(0,0,0,0);
+        Size size = new Size(0,0);
+        for(JSONGroupScan.ScanEntry r : readEntries){
+          cost = cost.add(r.getCost());
+          size = size.add(r.getSize());
+        }
+        this.cost = cost;
+        this.size = size;
+    }
+
+    @SuppressWarnings("unchecked")
+    @Override
+    public void applyAssignments(List<CoordinationProtos.DrillbitEndpoint> endpoints) {
+        checkArgument(endpoints.size() <= readEntries.size());
+
+        mappings = new LinkedList[endpoints.size()];
+
+        int i = 0;
+        for (ScanEntry e : readEntries) {
+            if (i == endpoints.size()) i = 0;
+            LinkedList entries = mappings[i];
+            if (entries == null) {
+                entries = new LinkedList<>();
+                mappings[i] = entries;
+            }
+            entries.add(e);
+            i++;
+        }
+    }
+
+    @SuppressWarnings("unchecked")
+    @Override
+    public SubScan getSpecificScan(int minorFragmentId) {
+        checkArgument(minorFragmentId < mappings.length, "Mappings length [%s] should be longer than minor fragment id [%s] but it isn't.", mappings.length, minorFragmentId);
+        return new JSONSubScan(mappings[minorFragmentId]);
+    }
+
+    @Override
+    public List<EndpointAffinity> getOperatorAffinity() {
+        return Collections.emptyList();
+    }
+    
+    public List<JSONGroupScan.ScanEntry> getReadEntries() {
+      return readEntries;
+    }
+
+    @Override
+    public PhysicalOperator getNewWithChildren(List<PhysicalOperator> children) {
+        return new JSONGroupScan(readEntries);
+    }
+
+    public static class ScanEntry implements ReadEntry {
+        private final String url;
+        private Size size;
+
+        @JsonCreator
+        public ScanEntry(@JsonProperty("url") String url) {
+            this.url = url;
+            long fileLength = new File(URI.create(url)).length();
+            size = new Size(fileLength / ESTIMATED_RECORD_SIZE, ESTIMATED_RECORD_SIZE);
+        }
+
+        @Override
+        public OperatorCost getCost() {
+            return new OperatorCost(1, 1, 2, 2);
+        }
+
+        @Override
+        public Size getSize() {
+            return size;
+        }
+
+        public String getUrl() {
+            return url;
+        }
+    }
+
+    @Override
+    public int getMaxParallelizationWidth() {
+      return readEntries.size();
+    }
+
+    @Override
+    public OperatorCost getCost() {
+      return cost;
+    }
+
+    @Override
+    public Size getSize() {
+      return size;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/2884db7a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/json/JSONRecordReader.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/json/JSONRecordReader.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/json/JSONRecordReader.java
new file mode 100644
index 0000000..eee0fb6
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/json/JSONRecordReader.java
@@ -0,0 +1,489 @@
+package org.apache.drill.exec.store.json;
+
+import com.fasterxml.jackson.core.JsonFactory;
+import com.fasterxml.jackson.core.JsonParser;
+import com.fasterxml.jackson.core.JsonToken;
+import com.google.common.base.Charsets;
+import com.google.common.collect.Iterables;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import com.google.common.io.Files;
+import com.google.common.io.InputSupplier;
+import com.google.common.io.Resources;
+
+import org.apache.drill.common.exceptions.DrillRuntimeException;
+import org.apache.drill.common.exceptions.ExecutionSetupException;
+import org.apache.drill.common.expression.ExpressionPosition;
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.common.types.TypeProtos.MajorType;
+import org.apache.drill.common.types.TypeProtos.MinorType;
+import org.apache.drill.common.types.Types;
+import org.apache.drill.exec.exception.SchemaChangeException;
+import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.physical.impl.OutputMutator;
+import org.apache.drill.exec.record.MaterializedField;
+import org.apache.drill.exec.schema.*;
+import org.apache.drill.exec.schema.json.jackson.JacksonHelper;
+import org.apache.drill.exec.store.RecordReader;
+import org.apache.drill.exec.store.VectorHolder;
+import org.apache.drill.exec.vector.*;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.net.URI;
+import java.nio.charset.Charset;
+import java.util.List;
+import java.util.Map;
+
+import static com.fasterxml.jackson.core.JsonToken.*;
+
+public class JSONRecordReader implements RecordReader {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(JSONRecordReader.class);
+  private static final int DEFAULT_LENGTH = 256 * 1024; // 256kb
+  public static final Charset UTF_8 = Charset.forName("UTF-8");
+
+  private final String inputPath;
+
+  private final Map<String, VectorHolder> valueVectorMap;
+
+  private JsonParser parser;
+  private SchemaIdGenerator generator;
+  private DiffSchema diffSchema;
+  private RecordSchema currentSchema;
+  private List<Field> removedFields;
+  private OutputMutator outputMutator;
+  private BufferAllocator allocator;
+  private int batchSize;
+
+  public JSONRecordReader(FragmentContext fragmentContext, String inputPath, int batchSize) {
+    this.inputPath = inputPath;
+    this.allocator = fragmentContext.getAllocator();
+    this.batchSize = batchSize;
+    valueVectorMap = Maps.newHashMap();
+  }
+
+  public JSONRecordReader(FragmentContext fragmentContext, String inputPath) {
+    this(fragmentContext, inputPath, DEFAULT_LENGTH);
+  }
+
+  private JsonParser getParser() {
+    return parser;
+  }
+
+  @Override
+  public void setup(OutputMutator output) throws ExecutionSetupException {
+    outputMutator = output;
+    currentSchema = new ObjectSchema();
+    diffSchema = new DiffSchema();
+    removedFields = Lists.newArrayList();
+
+    try {
+      InputSupplier<InputStreamReader> input;
+      if (inputPath.startsWith("resource:")) {
+        input = Resources.newReaderSupplier(Resources.getResource(inputPath.substring(9)), Charsets.UTF_8);
+      } else {
+        input = Files.newReaderSupplier(new File(URI.create(inputPath)), Charsets.UTF_8);
+      }
+
+      JsonFactory factory = new JsonFactory();
+      parser = factory.createJsonParser(input.getInput());
+      parser.nextToken(); // Read to the first START_OBJECT token
+      generator = new SchemaIdGenerator();
+    } catch (IOException e) {
+      throw new ExecutionSetupException(e);
+    }
+  }
+
+  @Override
+  public int next() {
+    if (parser.isClosed() || !parser.hasCurrentToken()) {
+      return 0;
+    }
+
+    resetBatch();
+
+    int nextRowIndex = 0;
+
+    try {
+      while (ReadType.OBJECT.readRecord(this, null, nextRowIndex++, 0)) {
+        parser.nextToken(); // Read to START_OBJECT token
+
+        if (!parser.hasCurrentToken()) {
+          parser.close();
+          break;
+        }
+      }
+
+      parser.nextToken();
+
+      if (!parser.hasCurrentToken()) {
+        parser.close();
+      }
+
+      // Garbage collect fields never referenced in this batch
+      for (Field field : Iterables.concat(currentSchema.removeUnreadFields(), removedFields)) {
+        diffSchema.addRemovedField(field);
+        outputMutator.removeField(field.getAsMaterializedField());
+      }
+
+      if (diffSchema.isChanged()) {
+        outputMutator.setNewSchema();
+      }
+
+
+    } catch (IOException | SchemaChangeException e) {
+      logger.error("Error reading next in Json reader", e);
+    }
+
+    for (VectorHolder holder : valueVectorMap.values()) {
+      holder.populateVectorLength();
+    }
+
+    return nextRowIndex;
+  }
+
+  private void resetBatch() {
+    for (VectorHolder value : valueVectorMap.values()) {
+      value.reset();
+    }
+
+    currentSchema.resetMarkedFields();
+    diffSchema.reset();
+    removedFields.clear();
+  }
+
+  @Override
+  public void cleanup() {
+    try {
+      parser.close();
+    } catch (IOException e) {
+      logger.warn("Error closing Json parser", e);
+    }
+  }
+
+
+  private RecordSchema getCurrentSchema() {
+    return currentSchema;
+  }
+
+  private void setCurrentSchema(RecordSchema schema) {
+    currentSchema = schema;
+  }
+
+  private List<Field> getRemovedFields() {
+    return removedFields;
+  }
+
+  public BufferAllocator getAllocator() {
+    return allocator;
+  }
+
+  public static enum ReadType {
+    ARRAY(END_ARRAY) {
+      @Override
+      public Field createField(RecordSchema parentSchema, String prefixFieldName, String fieldName, MajorType fieldType, int index) {
+        return new NamedField(parentSchema, prefixFieldName, fieldName, fieldType);
+      }
+
+      @Override
+      public RecordSchema createSchema() throws IOException {
+        return new ObjectSchema();
+      }
+    },
+    OBJECT(END_OBJECT) {
+      @Override
+      public Field createField(RecordSchema parentSchema,
+                               String prefixFieldName,
+                               String fieldName,
+                               MajorType fieldType,
+                               int index) {
+        return new NamedField(parentSchema, prefixFieldName, fieldName, fieldType);
+      }
+
+      @Override
+      public RecordSchema createSchema() throws IOException {
+        return new ObjectSchema();
+      }
+    };
+
+    private final JsonToken endObject;
+
+    ReadType(JsonToken endObject) {
+      this.endObject = endObject;
+    }
+
+    public JsonToken getEndObject() {
+      return endObject;
+    }
+
+    @SuppressWarnings("ConstantConditions")
+    public boolean readRecord(JSONRecordReader reader,
+                              String prefixFieldName,
+                              int rowIndex,
+                              int groupCount) throws IOException, SchemaChangeException {
+      JsonParser parser = reader.getParser();
+      JsonToken token = parser.nextToken();
+      JsonToken endObject = getEndObject();
+      int colIndex = 0;
+      boolean isFull = false;
+      while (token != endObject) {
+        if (token == FIELD_NAME) {
+          token = parser.nextToken();
+          continue;
+        }
+
+        String fieldName = parser.getCurrentName();
+        MajorType fieldType = JacksonHelper.getFieldType(token, this == ReadType.ARRAY);
+        ReadType readType = null;
+        switch (token) {
+          case START_ARRAY:
+            readType = ReadType.ARRAY;
+            groupCount++;
+            break;
+          case START_OBJECT:
+            readType = ReadType.OBJECT;
+            groupCount = 0;
+            break;
+        }
+
+        if (fieldType != null) { // Including nulls
+          boolean currentFieldFull = !recordData(
+              readType,
+              reader,
+              fieldType,
+              prefixFieldName,
+              fieldName,
+              rowIndex,
+              colIndex,
+              groupCount);
+
+          isFull = isFull || currentFieldFull;
+        }
+        token = parser.nextToken();
+        colIndex += 1;
+      }
+      return !isFull;
+    }
+
+    private void removeChildFields(List<Field> removedFields, Field field) {
+      RecordSchema schema = field.getAssignedSchema();
+      if (schema == null) {
+        return;
+      }
+      for (Field childField : schema.getFields()) {
+        removedFields.add(childField);
+        if (childField.hasSchema()) {
+          removeChildFields(removedFields, childField);
+        }
+      }
+    }
+
+    private boolean recordData(JSONRecordReader.ReadType readType,
+                               JSONRecordReader reader,
+                               MajorType fieldType,
+                               String prefixFieldName,
+                               String fieldName,
+                               int rowIndex,
+                               int colIndex,
+                               int groupCount) throws IOException, SchemaChangeException {
+      RecordSchema currentSchema = reader.getCurrentSchema();
+      Field field = currentSchema.getField(fieldName == null ? prefixFieldName : fieldName, colIndex);
+      boolean isFieldFound = field != null;
+      List<Field> removedFields = reader.getRemovedFields();
+      boolean newFieldLateBound = fieldType.getMinorType().equals(MinorType.LATE);
+
+      if (isFieldFound && !field.getFieldType().equals(fieldType)) {
+        boolean existingFieldLateBound = field.getFieldType().getMinorType().equals(MinorType.LATE);
+
+        if (newFieldLateBound && !existingFieldLateBound) {
+          fieldType = Types.overrideMinorType(fieldType, field.getFieldType().getMinorType());
+        } else if (!newFieldLateBound && existingFieldLateBound) {
+          field.setFieldType(Types.overrideMinorType(field.getFieldType(), fieldType.getMinorType()));
+        } else if (!newFieldLateBound && !existingFieldLateBound) {
+          if (field.hasSchema()) {
+            removeChildFields(removedFields, field);
+          }
+          removedFields.add(field);
+          currentSchema.removeField(field, colIndex);
+
+          isFieldFound = false;
+        }
+      }
+
+      if (!isFieldFound) {
+        field = createField(
+            currentSchema,
+            prefixFieldName,
+            fieldName,
+            fieldType,
+            colIndex
+        );
+
+        reader.recordNewField(field);
+        currentSchema.addField(field);
+      }
+
+      field.setRead(true);
+
+      VectorHolder holder = getOrCreateVectorHolder(reader, field);
+      if (readType != null) {
+        RecordSchema fieldSchema = field.getAssignedSchema();
+        RecordSchema newSchema = readType.createSchema();
+
+        if (readType != ReadType.ARRAY) {
+          reader.setCurrentSchema(fieldSchema);
+          if (fieldSchema == null) reader.setCurrentSchema(newSchema);
+          readType.readRecord(reader, field.getFullFieldName(), rowIndex, groupCount);
+        } else {
+          readType.readRecord(reader, field.getFullFieldName(), rowIndex, groupCount);
+        }
+
+        reader.setCurrentSchema(currentSchema);
+
+      } else if (holder != null && !newFieldLateBound && fieldType.getMinorType() != MinorType.LATE) {
+        return addValueToVector(
+            rowIndex,
+            holder,
+            JacksonHelper.getValueFromFieldType(
+                reader.getParser(),
+                fieldType.getMinorType()
+            ),
+            fieldType.getMinorType(),
+            groupCount
+        );
+      }
+
+      return true;
+    }
+
+    private static <T> boolean addValueToVector(int index, VectorHolder holder, T val, MinorType minorType, int groupCount) {
+      switch (minorType) {
+        case INT: {
+          holder.incAndCheckLength(NullableIntHolder.WIDTH * 8 + 1);
+          if (groupCount == 0) {
+            if (val != null) {
+              NullableIntVector int4 = (NullableIntVector) holder.getValueVector();
+              NullableIntVector.Mutator m = int4.getMutator();
+              m.set(index, (Integer) val);
+            }
+          } else {
+            if (val == null) {
+              throw new UnsupportedOperationException("Nullable repeated int is not supported.");
+            }
+
+            RepeatedIntVector repeatedInt4 = (RepeatedIntVector) holder.getValueVector();
+            RepeatedIntVector.Mutator m = repeatedInt4.getMutator();
+            holder.setGroupCount(index);
+            m.add(index, (Integer) val);
+          }
+
+          return holder.hasEnoughSpace(NullableIntHolder.WIDTH * 8 + 1);
+        }
+        case FLOAT4: {
+          holder.incAndCheckLength(NullableFloat4Holder.WIDTH * 8 + 1);
+          if (groupCount == 0) {
+            if (val != null) {
+              NullableFloat4Vector float4 = (NullableFloat4Vector) holder.getValueVector();
+              NullableFloat4Vector.Mutator m = float4.getMutator();
+              m.set(index, (Float) val);
+            }
+          } else {
+            if (val == null) {
+              throw new UnsupportedOperationException("Nullable repeated float is not supported.");
+            }
+
+            RepeatedFloat4Vector repeatedFloat4 = (RepeatedFloat4Vector) holder.getValueVector();
+            RepeatedFloat4Vector.Mutator m = repeatedFloat4.getMutator();
+            holder.setGroupCount(index);
+            m.add(index, (Float) val);
+          }
+          return holder.hasEnoughSpace(NullableFloat4Holder.WIDTH * 8 + 1);
+        }
+        case VARCHAR: {
+          if (val == null) {
+            return (index + 1) * 4 <= holder.getLength();
+          } else {
+            byte[] bytes = ((String) val).getBytes(UTF_8);
+            int length = bytes.length;
+            holder.incAndCheckLength(length);
+            if (groupCount == 0) {
+              NullableVarCharVector varLen4 = (NullableVarCharVector) holder.getValueVector();
+              NullableVarCharVector.Mutator m = varLen4.getMutator();
+              m.set(index, bytes);
+            } else {
+              RepeatedVarCharVector repeatedVarLen4 = (RepeatedVarCharVector) holder.getValueVector();
+              RepeatedVarCharVector.Mutator m = repeatedVarLen4.getMutator();
+              holder.setGroupCount(index);
+              m.add(index, bytes);
+            }
+            return holder.hasEnoughSpace(length + 4 + 1);
+          }
+        }
+        case BIT: {
+          holder.incAndCheckLength(NullableBitHolder.WIDTH + 1);
+          if (groupCount == 0) {
+            if (val != null) {
+              NullableBitVector bit = (NullableBitVector) holder.getValueVector();
+              NullableBitVector.Mutator m = bit.getMutator();
+              m.set(index, (Boolean) val ? 1 : 0);
+            }
+          } else {
+            if (val == null) {
+              throw new UnsupportedOperationException("Nullable repeated boolean is not supported.");
+            }
+
+            RepeatedBitVector repeatedBit = (RepeatedBitVector) holder.getValueVector();
+            RepeatedBitVector.Mutator m = repeatedBit.getMutator();
+            holder.setGroupCount(index);
+            m.add(index, (Boolean) val ? 1 : 0);
+          }
+          return holder.hasEnoughSpace(NullableBitHolder.WIDTH + 1);
+        }
+        default:
+          throw new DrillRuntimeException("Type not supported to add value. Type: " + minorType);
+      }
+    }
+
+    private VectorHolder getOrCreateVectorHolder(JSONRecordReader reader, Field field) throws SchemaChangeException {
+      return reader.getOrCreateVectorHolder(field);
+    }
+
+    public abstract RecordSchema createSchema() throws IOException;
+
+    public abstract Field createField(RecordSchema parentSchema,
+                                      String prefixFieldName,
+                                      String fieldName,
+                                      MajorType fieldType,
+                                      int index);
+  }
+
+  private void recordNewField(Field field) {
+    diffSchema.recordNewField(field);
+  }
+
+  private VectorHolder getOrCreateVectorHolder(Field field) throws SchemaChangeException {
+    String fullFieldName = field.getFullFieldName();
+    VectorHolder holder = valueVectorMap.get(fullFieldName);
+
+    if (holder == null) {
+      MajorType type = field.getFieldType();
+      MinorType minorType = type.getMinorType();
+
+      if (minorType.equals(MinorType.MAP) || minorType.equals(MinorType.LATE)) {
+        return null;
+      }
+
+      MaterializedField f = MaterializedField.create(new SchemaPath(fullFieldName, ExpressionPosition.UNKNOWN), type);
+
+      ValueVector v = TypeHelper.getNewVector(f, allocator);
+      AllocationHelper.allocate(v, batchSize, 50);
+      holder = new VectorHolder(v);
+      valueVectorMap.put(fullFieldName, holder);
+      outputMutator.addField(v);
+      return holder;
+    }
+    return holder;
+  }
+}

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


[18/27] git commit: Implement PartitionExchange Operator: - implement Partitioner template and code generation - implement OutgoingRecordBatch to manage records in the process of being sent to another host - implement PartitionSenderCreator and add vi

Posted by ja...@apache.org.
Implement PartitionExchange Operator:
 - implement Partitioner template and code generation
 - implement OutgoingRecordBatch to manage records in the process of being sent to another host
 - implement PartitionSenderCreator and add visit method to ImplCreator
 - implement boiler plate for Hash DrillFunc
 - foreman: track intermediate nodes; not just leaf nodes.
 - code gen: implement generateMultipleOutputs() and getImplementationClassMultipleOutput(), similar to generate()
 - added FIXED_INT static output type determiner
 - fix parsing of "senders" in RandomReceiver


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

Branch: refs/heads/master
Commit: 4a10ea1324e0ce9eeedf6e4949728b9a17a01605
Parents: be77d5a
Author: Ben Becker <be...@gmail.com>
Authored: Fri Aug 9 03:01:25 2013 -0700
Committer: Jacques Nadeau <ja...@apache.org>
Committed: Thu Aug 15 18:30:04 2013 -0700

----------------------------------------------------------------------
 .../common/expression/OutputTypeDeterminer.java |   1 +
 .../apache/drill/exec/expr/CodeGenerator.java   |  28 +-
 .../apache/drill/exec/expr/fn/impl/Hash.java    |  60 +++++
 .../apache/drill/exec/ops/FragmentContext.java  |   8 +
 .../physical/config/HashPartitionSender.java    |  17 +-
 .../exec/physical/config/RandomReceiver.java    |   4 +-
 .../drill/exec/physical/impl/ImplCreator.java   |  27 +-
 .../partitionsender/OutgoingRecordBatch.java    | 267 +++++++++++++++++++
 .../partitionsender/PartitionSenderCreator.java |  41 +++
 .../PartitionSenderRootExec.java                | 212 +++++++++++++++
 .../impl/partitionsender/Partitioner.java       |  39 +++
 .../partitionsender/PartitionerEvaluator.java   |  28 ++
 .../partitionsender/PartitionerTemplate.java    |  59 ++++
 .../apache/drill/exec/work/foreman/Foreman.java |   4 +-
 .../work/foreman/RunningFragmentManager.java    |  13 +-
 .../physical/impl/TestHashToRandomExchange.java |  62 +++++
 .../test/resources/sender/hash_exchange.json    |  55 ++++
 17 files changed, 909 insertions(+), 16 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/4a10ea13/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/OutputTypeDeterminer.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/OutputTypeDeterminer.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/OutputTypeDeterminer.java
index 828f2b6..01b49d2 100644
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/OutputTypeDeterminer.java
+++ b/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/OutputTypeDeterminer.java
@@ -27,6 +27,7 @@ import org.apache.drill.common.types.Types;
 public interface OutputTypeDeterminer {
 
   public static OutputTypeDeterminer FIXED_BIT = new FixedType(Types.required(MinorType.BIT));
+  public static OutputTypeDeterminer FIXED_INT = new FixedType(Types.required(MinorType.INT));
   
   public MajorType getOutputType(List<LogicalExpression> expressions);
   

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/4a10ea13/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/CodeGenerator.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/CodeGenerator.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/CodeGenerator.java
index 9895782..c37e75d 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/CodeGenerator.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/CodeGenerator.java
@@ -11,6 +11,7 @@ import org.apache.drill.exec.compile.TemplateClassDefinition;
 import org.apache.drill.exec.exception.SchemaChangeException;
 import org.apache.drill.exec.expr.fn.FunctionImplementationRegistry;
 import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.physical.impl.partitionsender.OutgoingRecordBatch;
 import org.apache.drill.exec.record.RecordBatch;
 import org.apache.drill.exec.record.TypedFieldId;
 import org.apache.drill.exec.record.VectorWrapper;
@@ -153,7 +154,32 @@ public class CodeGenerator<T> {
     model.build(w);
     return w.getCode().toString();
   }
-  
+
+  public String generateMultipleOutputs() throws IOException{
+
+    {
+      //setup method
+      JMethod m = clazz.method(JMod.PUBLIC, model.VOID, "doSetup");
+      m.param(model._ref(FragmentContext.class), "context");
+      m.param(model._ref(RecordBatch.class), "incoming");
+      m.param(model._ref(OutgoingRecordBatch.class).array(), "outgoing");
+      m._throws(SchemaChangeException.class);
+      m.body().add(parentSetupBlock);
+    }
+
+    {
+      // eval method.
+      JType ret = definition.getEvalReturnType() == null ? model.VOID : model._ref(definition.getEvalReturnType());
+      JMethod m = clazz.method(JMod.PUBLIC, ret, "doEval");
+      m.param(model.INT, "inIndex");
+      m.param(model.INT, "outIndex");
+      m.body().add(parentEvalBlock);
+    }
+
+    SingleClassStringWriter w = new SingleClassStringWriter();
+    model.build(w);
+    return w.getCode().toString();
+  }
   
   public JCodeModel getModel() {
     return model;

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/4a10ea13/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/Hash.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/Hash.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/Hash.java
new file mode 100644
index 0000000..45a3ee4
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/Hash.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.expr.fn.impl;
+
+import org.apache.drill.common.expression.*;
+import org.apache.drill.exec.expr.DrillFunc;
+import org.apache.drill.exec.expr.annotations.FunctionTemplate;
+import org.apache.drill.exec.expr.annotations.FunctionTemplate.FunctionScope;
+import org.apache.drill.exec.expr.annotations.Output;
+import org.apache.drill.exec.expr.annotations.Param;
+import org.apache.drill.exec.record.RecordBatch;
+import org.apache.drill.exec.vector.BigIntHolder;
+import org.apache.drill.exec.vector.IntHolder;
+
+@FunctionTemplate(name = "hash", scope = FunctionScope.SIMPLE, nulls = FunctionTemplate.NullHandling.NULL_IF_NULL)
+public class Hash implements DrillFunc {
+
+  @Param BigIntHolder in;
+  @Output IntHolder out;
+
+  public void setup(RecordBatch incoming) {
+  }
+
+  public void eval() {
+    // TODO: implement hash function for various types
+    out.value = (int)in.value;
+  }
+
+  public static class Provider implements CallProvider{
+
+    @Override
+    public FunctionDefinition[] getFunctionDefintions() {
+      return new FunctionDefinition[] {
+          FunctionDefinition.simple("hash",
+                                    new ArgumentValidators.AnyTypeAllowed(1),
+                                    OutputTypeDeterminer.FIXED_INT,
+                                    "hash")
+      };
+    }
+
+  }
+}
+
+

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

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/4a10ea13/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
index 84994f6..e53db43 100644
--- 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
@@ -22,12 +22,12 @@ 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.physical.base.PhysicalVisitor;
 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 {
@@ -35,9 +35,12 @@ public class HashPartitionSender extends AbstractSender {
 
   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) {
+  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;
@@ -53,6 +56,14 @@ public class HashPartitionSender extends AbstractSender {
     return new HashPartitionSender(oppositeMajorFragmentId, child, expr, endpoints);
   }
 
+  public LogicalExpression getExpr() {
+    return expr;
+  }
+  
+  @Override
+  public <T, X, E extends Throwable> T accept(PhysicalVisitor<T, X, E> physicalVisitor, X value) throws E {
+    return physicalVisitor.visitHashPartitionSender(this, value);
+  }
   
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/4a10ea13/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/RandomReceiver.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/RandomReceiver.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/RandomReceiver.java
index 6772fb0..117da0b 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/RandomReceiver.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/RandomReceiver.java
@@ -39,12 +39,14 @@ public class RandomReceiver extends AbstractReceiver{
   private List<DrillbitEndpoint> senders;
   
   @JsonCreator
-  public RandomReceiver(@JsonProperty("sender-major-fragment") int oppositeMajorFragmentId, @JsonProperty("senders") List<DrillbitEndpoint> senders){
+  public RandomReceiver(@JsonProperty("sender-major-fragment") int oppositeMajorFragmentId,
+                        @JsonProperty("senders") List<DrillbitEndpoint> senders) {
     super(oppositeMajorFragmentId);
     this.senders = senders;
   }
   
   @Override
+  @JsonProperty("senders")
   public List<DrillbitEndpoint> getProvidingEndpoints() {
     return senders;
   }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/4a10ea13/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ImplCreator.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ImplCreator.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ImplCreator.java
index 0a329d6..8207304 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ImplCreator.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ImplCreator.java
@@ -26,7 +26,11 @@ 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.SubScan;
+import org.apache.drill.exec.physical.base.Scan;
+import org.apache.drill.exec.physical.config.HashPartitionSender;
 import org.apache.drill.exec.physical.config.Filter;
+import org.apache.drill.exec.physical.config.MockScanBatchCreator;
+import org.apache.drill.exec.physical.config.MockScanPOP;
 import org.apache.drill.exec.physical.config.Project;
 import org.apache.drill.exec.physical.config.RandomReceiver;
 import org.apache.drill.exec.physical.config.Screen;
@@ -34,6 +38,7 @@ import org.apache.drill.exec.physical.config.SelectionVectorRemover;
 import org.apache.drill.exec.physical.config.SingleSender;
 import org.apache.drill.exec.physical.config.Sort;
 import org.apache.drill.exec.physical.impl.filter.FilterBatchCreator;
+import org.apache.drill.exec.physical.impl.partitionsender.PartitionSenderCreator;
 import org.apache.drill.exec.physical.impl.project.ProjectBatchCreator;
 import org.apache.drill.exec.physical.impl.sort.SortBatchCreator;
 import org.apache.drill.exec.physical.impl.svremover.SVRemoverCreator;
@@ -49,24 +54,28 @@ import org.apache.drill.exec.store.parquet.ParquetScanBatchCreator;
 import com.google.common.base.Preconditions;
 import com.google.common.collect.Lists;
 
-public class ImplCreator extends AbstractPhysicalVisitor<RecordBatch, FragmentContext, ExecutionSetupException> {
+/**
+ * Implementation of the physical operator visitor
+ */
+public class ImplCreator extends AbstractPhysicalVisitor<RecordBatch, FragmentContext, ExecutionSetupException>{
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ImplCreator.class);
 
+  // Element creators supported by this visitor
   private MockScanBatchCreator msc = new MockScanBatchCreator();
   private ParquetScanBatchCreator parquetScan = new ParquetScanBatchCreator();
   private ScreenCreator sc = new ScreenCreator();
   private RandomReceiverCreator rrc = new RandomReceiverCreator();
+  private PartitionSenderCreator hsc = new PartitionSenderCreator();
   private SingleSenderCreator ssc = new SingleSenderCreator();
   private ProjectBatchCreator pbc = new ProjectBatchCreator();
   private FilterBatchCreator fbc = new FilterBatchCreator();
   private SVRemoverCreator svc = new SVRemoverCreator();
   private SortBatchCreator sbc = new SortBatchCreator();
   private RootExec root = null;
-
-  private ImplCreator() {
-  }
-
-  public RootExec getRoot() {
+  
+  private ImplCreator(){}
+  
+  public RootExec getRoot(){
     return root;
   }
 
@@ -114,6 +123,12 @@ public class ImplCreator extends AbstractPhysicalVisitor<RecordBatch, FragmentCo
   }
 
   @Override
+  public RecordBatch visitHashPartitionSender(HashPartitionSender op, FragmentContext context) throws ExecutionSetupException {
+    root = hsc.getRoot(context, op, getChildren(op, context));
+    return null;
+  }
+  
+  @Override
   public RecordBatch visitFilter(Filter filter, FragmentContext context) throws ExecutionSetupException {
     return fbc.getBatch(context, filter, getChildren(filter, context));
   }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/4a10ea13/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/partitionsender/OutgoingRecordBatch.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/partitionsender/OutgoingRecordBatch.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/partitionsender/OutgoingRecordBatch.java
new file mode 100644
index 0000000..4ab598c
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/partitionsender/OutgoingRecordBatch.java
@@ -0,0 +1,267 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+
+package org.apache.drill.exec.physical.impl.partitionsender;
+
+import java.util.Iterator;
+import java.util.List;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.physical.config.HashPartitionSender;
+import org.apache.drill.exec.proto.ExecProtos;
+import org.apache.drill.exec.proto.GeneralRPCProtos;
+import org.apache.drill.exec.record.*;
+import org.apache.drill.exec.record.selection.SelectionVector2;
+import org.apache.drill.exec.record.selection.SelectionVector4;
+import org.apache.drill.exec.rpc.BaseRpcOutcomeListener;
+import org.apache.drill.exec.rpc.RpcException;
+import org.apache.drill.exec.rpc.bit.BitTunnel;
+import org.apache.drill.exec.vector.*;
+import org.apache.drill.exec.work.foreman.ErrorHelper;
+
+/**
+ * OutgoingRecordBatch is a holder of value vectors which are to be sent to another host.  Thus,
+ * next() will never be called on this object.  When a record batch is ready to send (e.g. nearing size
+ * limit or schema change), call flush() to send the batch.
+ */
+public class OutgoingRecordBatch implements RecordBatch {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(OutgoingRecordBatch.class);
+
+  private BitTunnel tunnel;
+  private HashPartitionSender operator;
+  private volatile boolean ok = true;
+  private boolean isLast = false;
+  private RecordBatch incoming;
+  private FragmentContext context;
+  private BatchSchema outSchema;
+  private List<ValueVector> valueVectors;
+  private VectorContainer vectorContainer;
+  private int recordCount;
+  private int recordCapacity;
+
+  public OutgoingRecordBatch(HashPartitionSender operator, BitTunnel tunnel, RecordBatch incoming, FragmentContext context) {
+    this.incoming = incoming;
+    this.context = context;
+    this.operator = operator;
+    this.tunnel = tunnel;
+    initializeBatch();
+  }
+
+  public OutgoingRecordBatch() {  }
+
+  public void init(HashPartitionSender operator, BitTunnel tunnel, RecordBatch incoming, FragmentContext context) {
+    this.incoming = incoming;
+    this.context = context;
+    this.operator = operator;
+    this.tunnel = tunnel;
+    resetBatch();
+  }
+
+  public void flushIfNecessary() {
+    if (recordCount == recordCapacity - 1) flush();
+  }
+
+  public void incRecordCount() {
+    ++recordCount;
+  }
+  
+  public void flush() {
+    if (recordCount == 0) {
+      logger.warn("Attempted to flush an empty record batch");
+      return;
+    }
+    final ExecProtos.FragmentHandle handle = context.getHandle();
+    FragmentWritableBatch writableBatch = new FragmentWritableBatch(isLast,
+                                                                    handle.getQueryId(),
+                                                                    handle.getMajorFragmentId(),
+                                                                    handle.getMinorFragmentId(),
+                                                                    operator.getOppositeMajorFragmentId(),
+                                                                    0,
+                                                                    getWritableBatch());
+    tunnel.sendRecordBatch(statusHandler, context, writableBatch);
+
+    // reset values and reallocate the buffer for each value vector.  NOTE: the value vector is directly
+    // referenced by generated code and must not be replaced.
+    recordCount = 0;
+    for (VectorWrapper v : vectorContainer) {
+      getAllocator(TypeHelper.getNewVector(v.getField(), context.getAllocator()),
+                   v.getValueVector()).alloc(recordCapacity);
+    }
+  }
+
+  /**
+   * Create a new output schema and allocate space for value vectors based on the incoming record batch.
+   */
+  public void initializeBatch() {
+    recordCapacity = incoming.getRecordCount();
+    valueVectors = Lists.newArrayList();
+    vectorContainer = new VectorContainer();
+
+    SchemaBuilder bldr = BatchSchema.newBuilder().setSelectionVectorMode(BatchSchema.SelectionVectorMode.NONE);
+    for (VectorWrapper v : incoming) {
+
+      // add field to the output schema
+      bldr.addField(v.getField());
+
+      // allocate a new value vector
+      vectorContainer.add(v.getValueVector());
+      ValueVector outgoingVector = TypeHelper.getNewVector(v.getField(), context.getAllocator());
+      getAllocator(outgoingVector, v.getValueVector()).alloc(recordCapacity);
+      valueVectors.add(outgoingVector);
+    }
+    outSchema = bldr.build();
+  }
+
+  /**
+   * Free any existing value vectors, create new output schema, and allocate value vectors based
+   * on the incoming record batch.
+   */
+  public void resetBatch() {
+    recordCount = 0;
+    recordCapacity = 0;
+    if (valueVectors != null) {
+      for(ValueVector v : valueVectors){
+        v.close();
+      }
+    }
+    initializeBatch();
+  }
+
+  public void setIsLast() {
+    isLast = true;
+  }
+
+  @Override
+  public IterOutcome next() {
+    assert false;
+    return IterOutcome.STOP;
+  }
+
+  @Override
+  public FragmentContext getContext() {
+    return context;
+  }
+
+  @Override
+  public BatchSchema getSchema() {
+    Preconditions.checkNotNull(outSchema);
+    return outSchema;
+  }
+
+  @Override
+  public int getRecordCount() {
+    return recordCount;
+  }
+
+  @Override
+  public void kill() {
+    incoming.kill();
+  }
+
+  @Override
+  public SelectionVector2 getSelectionVector2() {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public SelectionVector4 getSelectionVector4() {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public TypedFieldId getValueVectorId(SchemaPath path) {
+    return vectorContainer.getValueVector(path);
+  }
+
+  @Override
+  public VectorWrapper<?> getValueAccessorById(int fieldId, Class<?> clazz) {
+    return vectorContainer.getVectorAccessor(fieldId, clazz);
+  }
+
+  @Override
+  public Iterator<VectorWrapper<?>> iterator() {
+    return vectorContainer.iterator();
+  }
+
+  @Override
+  public WritableBatch getWritableBatch() {
+    return WritableBatch.get(this);
+  }
+
+  private VectorAllocator getAllocator(ValueVector in, ValueVector outgoing){
+    if(outgoing instanceof FixedWidthVector){
+      return new FixedVectorAllocator((FixedWidthVector) outgoing);
+    }else if(outgoing instanceof VariableWidthVector && in instanceof VariableWidthVector){
+      return new VariableVectorAllocator( (VariableWidthVector) in, (VariableWidthVector) outgoing);
+    }else{
+      throw new UnsupportedOperationException();
+    }
+  }
+
+  private class FixedVectorAllocator implements VectorAllocator{
+    FixedWidthVector out;
+
+    public FixedVectorAllocator(FixedWidthVector out) {
+      super();
+      this.out = out;
+    }
+
+    public void alloc(int recordCount){
+      out.allocateNew(recordCount);
+      out.getMutator().setValueCount(recordCount);
+    }
+  }
+
+  private class VariableVectorAllocator implements VectorAllocator{
+    VariableWidthVector in;
+    VariableWidthVector out;
+
+    public VariableVectorAllocator(VariableWidthVector in, VariableWidthVector out) {
+      super();
+      this.in = in;
+      this.out = out;
+    }
+
+    public void alloc(int recordCount){
+      out.allocateNew(in.getByteCapacity(), recordCount);
+      out.getMutator().setValueCount(recordCount);
+    }
+  }
+
+  public interface VectorAllocator{
+    public void alloc(int recordCount);
+  }  
+  
+  private StatusHandler statusHandler = new StatusHandler();
+  private class StatusHandler extends BaseRpcOutcomeListener<GeneralRPCProtos.Ack> {
+    RpcException ex;
+
+    @Override
+    public void failed(RpcException ex) {
+      logger.error("Failure while sending data to user.", ex);
+      ErrorHelper.logAndConvertError(context.getIdentity(), "Failure while sending fragment to client.", ex, logger);
+      ok = false;
+      this.ex = ex;
+    }
+
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/4a10ea13/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/partitionsender/PartitionSenderCreator.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/partitionsender/PartitionSenderCreator.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/partitionsender/PartitionSenderCreator.java
new file mode 100644
index 0000000..a119b5b
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/partitionsender/PartitionSenderCreator.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.physical.impl.partitionsender;
+
+import org.apache.drill.common.exceptions.ExecutionSetupException;
+import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.physical.config.HashPartitionSender;
+import org.apache.drill.exec.physical.impl.RootCreator;
+import org.apache.drill.exec.physical.impl.RootExec;
+import org.apache.drill.exec.record.RecordBatch;
+import java.util.List;
+
+public class PartitionSenderCreator implements RootCreator<HashPartitionSender> {
+
+  @Override
+  public RootExec getRoot(FragmentContext context,
+                          HashPartitionSender config,
+                          List<RecordBatch> children) throws ExecutionSetupException {
+
+    assert children != null && children.size() == 1;
+    return new PartitionSenderRootExec(context, children.iterator().next(), config);
+
+  }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/4a10ea13/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/partitionsender/PartitionSenderRootExec.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/partitionsender/PartitionSenderRootExec.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/partitionsender/PartitionSenderRootExec.java
new file mode 100644
index 0000000..313ddf3
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/partitionsender/PartitionSenderRootExec.java
@@ -0,0 +1,212 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+
+package org.apache.drill.exec.physical.impl.partitionsender;
+
+import com.beust.jcommander.internal.Lists;
+import com.sun.codemodel.*;
+import org.apache.drill.common.expression.*;
+import org.apache.drill.exec.exception.ClassTransformationException;
+import org.apache.drill.exec.exception.SchemaChangeException;
+import org.apache.drill.exec.expr.CodeGenerator;
+import org.apache.drill.exec.expr.ExpressionTreeMaterializer;
+import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.physical.config.HashPartitionSender;
+import org.apache.drill.exec.physical.impl.RootExec;
+import org.apache.drill.exec.proto.CoordinationProtos;
+import org.apache.drill.exec.record.RecordBatch;
+import org.apache.drill.exec.record.TypedFieldId;
+import org.apache.drill.exec.record.VectorWrapper;
+
+import java.io.IOException;
+import java.util.List;
+
+class PartitionSenderRootExec implements RootExec {
+
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(PartitionSenderRootExec.class);
+  private RecordBatch incoming;
+  private HashPartitionSender operator;
+  private OutgoingRecordBatch[] outgoing;
+  private Partitioner partitioner;
+  private FragmentContext context;
+  private boolean ok = true;
+
+  public PartitionSenderRootExec(FragmentContext context,
+                                 RecordBatch incoming,
+                                 HashPartitionSender operator) {
+
+    this.incoming = incoming;
+    this.operator = operator;
+    this.context = context;
+    this.outgoing = new OutgoingRecordBatch[operator.getDestinations().size()];
+    int fieldId = 0;
+    for (CoordinationProtos.DrillbitEndpoint endpoint : operator.getDestinations())
+      outgoing[fieldId] = new OutgoingRecordBatch(operator,
+                             context.getCommunicator().getTunnel(endpoint),
+                             incoming,
+                             context);
+    try {
+      createPartitioner();
+    } catch (SchemaChangeException e) {
+      ok = false;
+      logger.error("Failed to create partitioning sender during query ", e);
+      context.fail(e);
+    }
+  }
+
+  @Override
+  public boolean next() {
+
+    if (!ok) {
+      stop();
+      return false;
+    }
+
+    RecordBatch.IterOutcome out = incoming.next();
+    logger.debug("Partitioner.next(): got next record batch with status {}", out);
+    switch(out){
+      case STOP:
+      case NONE:
+        // populate outgoing batches
+        if (incoming.getRecordCount() > 0)
+          partitioner.partitionBatch(incoming);
+
+        // send all pending batches
+        flushOutgoingBatches(true, false);
+        return false;
+
+      case OK_NEW_SCHEMA:
+        // send all existing batches
+        flushOutgoingBatches(false, true);
+        // update OutgoingRecordBatch's schema and value vectors
+        try {
+          createPartitioner();
+          partitioner.setup(context, incoming, outgoing);
+        } catch (SchemaChangeException e) {
+          incoming.kill();
+          logger.error("Failed to create partitioning sender during query ", e);
+          context.fail(e);
+          return false;
+        }
+      case OK:
+        partitioner.partitionBatch(incoming);
+        return true;
+      case NOT_YET:
+      default:
+        throw new IllegalStateException();
+    }
+  }
+
+  public void stop() {
+    ok = false;
+    incoming.kill();
+  }
+
+  private void createPartitioner() throws SchemaChangeException {
+
+    // set up partitioning function
+    final LogicalExpression expr = operator.getExpr();
+    final ErrorCollector collector = new ErrorCollectorImpl();
+    final CodeGenerator<Partitioner> cg = new CodeGenerator<Partitioner>(Partitioner.TEMPLATE_DEFINITION,
+                                                                         context.getFunctionRegistry());
+
+    final LogicalExpression logicalExp = ExpressionTreeMaterializer.materialize(expr, incoming, collector);
+    if (collector.hasErrors()) {
+      throw new SchemaChangeException(String.format(
+          "Failure while trying to materialize incoming schema.  Errors:\n %s.",
+          collector.toErrorString()));
+    }
+
+    // generate code to copy from an incoming value vector to the destination partition's outgoing value vector
+    int fieldId = 0;
+    JExpression inIndex = JExpr.direct("inIndex");
+    JExpression outIndex = JExpr.direct("outIndex");
+    cg.rotateBlock();
+
+    // declare array of record batches for each partition
+    JVar outgoingBatches = cg.clazz.field(JMod.NONE,
+                                          cg.getModel().ref(OutgoingRecordBatch.class).array(),
+                                          "outgoingBatches");
+
+    cg.getSetupBlock().assign(outgoingBatches, JExpr.direct("outgoing"));
+
+    // declare incoming value vectors
+    List<JVar> incomingVVs = Lists.newArrayList();
+    for (VectorWrapper<?> vvIn : incoming)
+      incomingVVs.add(cg.declareVectorValueSetupAndMember("incoming", new TypedFieldId(vvIn.getField().getType(),
+                                                                                       fieldId++,
+                                                                                       vvIn.isHyper())));
+
+    int batchId = 0;
+    fieldId = 0;
+    // generate switch statement for each destination batch
+    JSwitch switchStatement = cg.getBlock()._switch(outIndex);
+    for (OutgoingRecordBatch batch : outgoing) {
+
+      // generate case statement for this batch
+      JBlock caseBlock = switchStatement._case(JExpr.lit(batchId)).body();
+
+      for (VectorWrapper<?> vv : batch) {
+        // declare outgoing value vector and a corresponding counter
+        JVar outVV = cg.declareVectorValueSetupAndMember("outgoing[" + batchId + "]",
+                                                         new TypedFieldId(vv.getField().getType(),
+                                                                          fieldId,
+                                                                          false));
+
+        caseBlock.add(outVV.invoke("copyFrom")
+                              .arg(inIndex)
+                              .arg(JExpr.direct("outgoingBatches[" + batchId + "]").invoke("getRecordCount"))
+                              .arg(incomingVVs.get(fieldId)));
+        ++fieldId;
+      }
+      caseBlock.add(JExpr.direct("outgoingBatches[" + batchId + "]").invoke("incRecordCount"));
+      caseBlock.add(JExpr.direct("outgoingBatches[" + batchId + "]").invoke("flushIfNecessary"));
+      fieldId = 0;
+      caseBlock._break();
+      ++batchId;
+    }
+
+    try {
+      // compile and setup generated code
+      partitioner = context.getImplementationClassMultipleOutput(cg);
+      partitioner.setup(context, incoming, outgoing);
+
+    } catch (ClassTransformationException | IOException e) {
+      throw new SchemaChangeException("Failure while attempting to load generated class", e);
+    }
+  }
+
+  /**
+   * Flush each outgoing record batch, and optionally reset the state of each outgoing record
+   * batch (on schema change).  Note that the schema is updated based on incoming at the time
+   * this function is invoked.
+   *
+   * @param isLastBatch    true if this is the last incoming batch
+   * @param schemaChanged  true if the schema has changed
+   */
+  public void flushOutgoingBatches(boolean isLastBatch, boolean schemaChanged) {
+    for (OutgoingRecordBatch batch : outgoing) {
+      logger.debug("Attempting to flush all outgoing batches");
+      if (isLastBatch)
+        batch.setIsLast();
+      batch.flush();
+      if (schemaChanged)
+        batch.resetBatch();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/4a10ea13/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/partitionsender/Partitioner.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/partitionsender/Partitioner.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/partitionsender/Partitioner.java
new file mode 100644
index 0000000..e8f2ca7
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/partitionsender/Partitioner.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.physical.impl.partitionsender;
+
+import org.apache.drill.exec.compile.TemplateClassDefinition;
+import org.apache.drill.exec.exception.SchemaChangeException;
+import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.record.RecordBatch;
+
+public interface Partitioner {
+
+  public abstract void setup(FragmentContext context,
+                             RecordBatch incoming,
+                             OutgoingRecordBatch[] outgoing) throws SchemaChangeException;
+
+  public abstract void partitionBatch(RecordBatch incoming);
+
+  public static TemplateClassDefinition<Partitioner> TEMPLATE_DEFINITION =
+      new TemplateClassDefinition<>(Partitioner.class,
+                                    "org.apache.drill.exec.physical.impl.partitionsender.PartitionerTemplate",
+                                    PartitionerEvaluator.class,
+                                    null);
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/4a10ea13/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/partitionsender/PartitionerEvaluator.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/partitionsender/PartitionerEvaluator.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/partitionsender/PartitionerEvaluator.java
new file mode 100644
index 0000000..8c92fdc
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/partitionsender/PartitionerEvaluator.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.partitionsender;
+
+import org.apache.drill.exec.exception.SchemaChangeException;
+import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.record.RecordBatch;
+
+public interface PartitionerEvaluator {
+  public void doSetup(FragmentContext context, RecordBatch incoming, OutgoingRecordBatch[] outgoing) throws SchemaChangeException;
+  public void doEval(int inIndex, int outIndex);
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/4a10ea13/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/partitionsender/PartitionerTemplate.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/partitionsender/PartitionerTemplate.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/partitionsender/PartitionerTemplate.java
new file mode 100644
index 0000000..4072b20
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/partitionsender/PartitionerTemplate.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.physical.impl.partitionsender;
+
+import org.apache.drill.exec.exception.SchemaChangeException;
+import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.record.RecordBatch;
+
+public abstract class PartitionerTemplate implements Partitioner {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(PartitionerTemplate.class);
+
+  public PartitionerTemplate() throws SchemaChangeException {
+  }
+
+  @Override
+  public final void setup(FragmentContext context,
+                          RecordBatch incoming,
+                          OutgoingRecordBatch[] outgoing) throws SchemaChangeException {
+
+    doSetup(context, incoming, outgoing);
+
+  }
+
+  @Override
+  public void partitionBatch(RecordBatch incoming) {
+
+    for (int recordId = 0; recordId < incoming.getRecordCount(); ++recordId) {
+      // for each record
+
+      // TODO: evaluate partitioning expression
+      int partition = 0;
+      // TODO: if attempting to insert too large of a value into a vector:
+      //         - send the batch
+      //         - reallocate (at least the size of the current value) and try again
+      doEval(recordId, partition);
+    }
+
+  }
+
+  protected abstract void doSetup(FragmentContext context, RecordBatch incoming, OutgoingRecordBatch[] outgoing) throws SchemaChangeException;
+  protected abstract void doEval(int inIndex, int outIndex);
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/4a10ea13/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/Foreman.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/Foreman.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/Foreman.java
index c9c23b5..c1fd9e5 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/Foreman.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/Foreman.java
@@ -205,17 +205,19 @@ public class Foreman implements Runnable, Closeable, Comparable<Object>{
 
       this.context.getBitCom().getListeners().addFragmentStatusListener(work.getRootFragment().getHandle(), fragmentManager);
       List<PlanFragment> leafFragments = Lists.newArrayList();
+      List<PlanFragment> intermediateFragments = Lists.newArrayList();
 
       // store fragments in distributed grid.
       for (PlanFragment f : work.getFragments()) {
         if (f.getLeafFragment()) {
           leafFragments.add(f);
         } else {
+          intermediateFragments.add(f);
           context.getCache().storeFragment(f);
         }
       }
 
-      fragmentManager.runFragments(bee, work.getRootFragment(), work.getRootOperator(), initiatingClient, leafFragments);
+      fragmentManager.runFragments(bee, work.getRootFragment(), work.getRootOperator(), initiatingClient, leafFragments, intermediateFragments);
 
     
     } catch (ExecutionSetupException | RpcException e) {

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/4a10ea13/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/RunningFragmentManager.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/RunningFragmentManager.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/RunningFragmentManager.java
index af91a6b..da2f7c1 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/RunningFragmentManager.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/RunningFragmentManager.java
@@ -72,7 +72,7 @@ class RunningFragmentManager implements FragmentStatusListener{
     
   }
 
-  public void runFragments(WorkerBee bee, PlanFragment rootFragment, FragmentRoot rootOperator, UserClientConnection rootClient, List<PlanFragment> leafFragments) throws ExecutionSetupException{
+  public void runFragments(WorkerBee bee, PlanFragment rootFragment, FragmentRoot rootOperator, UserClientConnection rootClient, List<PlanFragment> leafFragments, List<PlanFragment> intermediateFragments) throws ExecutionSetupException{
     remainingFragmentCount.set(leafFragments.size()+1);
     queryId = rootFragment.getHandle().getQueryId();
 
@@ -93,8 +93,14 @@ class RunningFragmentManager implements FragmentStatusListener{
       }
       
     }
-    
-    // send remote fragments.
+
+    // keep track of intermediate fragments (not root or leaf)
+    for (PlanFragment f : intermediateFragments) {
+      logger.debug("Tracking intermediate remote node {} with data {}", f.getAssignment(), f.getFragmentJson());
+      map.put(f.getHandle(), new FragmentData(f.getHandle(), f.getAssignment(), false));
+    }
+
+    // send remote (leaf) fragments.
     for (PlanFragment f : leafFragments) {
       sendRemoteFragment(f);
     }
@@ -268,7 +274,6 @@ class RunningFragmentManager implements FragmentStatusListener{
       RunningFragmentManager.this.statusUpdate(status);
     }
 
-
     
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/4a10ea13/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestHashToRandomExchange.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestHashToRandomExchange.java b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestHashToRandomExchange.java
new file mode 100644
index 0000000..4129079
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestHashToRandomExchange.java
@@ -0,0 +1,62 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+
+package org.apache.drill.exec.physical.impl;
+
+import com.google.common.base.Charsets;
+import com.google.common.io.Files;
+import org.apache.drill.common.util.FileUtils;
+import org.apache.drill.exec.client.DrillClient;
+import org.apache.drill.exec.pop.PopUnitTestBase;
+import org.apache.drill.exec.proto.UserProtos;
+import org.apache.drill.exec.rpc.user.QueryResultBatch;
+import org.apache.drill.exec.server.Drillbit;
+import org.apache.drill.exec.server.RemoteServiceSet;
+import org.junit.Test;
+
+import java.nio.charset.Charset;
+import java.util.List;
+import static org.junit.Assert.assertEquals;
+
+public class TestHashToRandomExchange extends PopUnitTestBase {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestHashToRandomExchange.class);
+
+  @Test
+  public void twoBitTwoExchangeTwoEntryRun() 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(UserProtos.QueryType.PHYSICAL,
+          Files.toString(FileUtils.getResourceAsFile("/sender/hash_exchange.json"),
+              Charsets.UTF_8));
+      int count = 0;
+      for(QueryResultBatch b : results) {
+        if (b.getHeader().getRowCount() != 0)
+          count += b.getHeader().getRowCount();
+      }
+      assertEquals(100, count);
+    }
+  }
+
+}

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


[05/27] git commit: Fix getBuffers and address comments

Posted by ja...@apache.org.
Fix getBuffers and address comments


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

Branch: refs/heads/master
Commit: 6f9dadb952db3740cb187057967785d0ffca8e3e
Parents: b1e48b3
Author: Timothy Chen <tn...@gmail.com>
Authored: Tue Aug 13 20:42:53 2013 -0700
Committer: Timothy Chen <tn...@gmail.com>
Committed: Tue Aug 13 20:42:53 2013 -0700

----------------------------------------------------------------------
 .../templates/NullableValueVectors.java             |  8 ++------
 .../templates/RepeatedValueVectors.java             |  3 ++-
 .../org/apache/drill/exec/schema/DiffSchema.java    | 13 ++++++-------
 .../apache/drill/exec/store/JSONRecordReader.java   | 16 ++++++++--------
 .../org/apache/drill/exec/store/VectorHolder.java   | 12 ++++++------
 5 files changed, 24 insertions(+), 28 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/6f9dadb9/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/NullableValueVectors.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/NullableValueVectors.java b/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/NullableValueVectors.java
index 976c984..483166b 100644
--- a/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/NullableValueVectors.java
+++ b/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/NullableValueVectors.java
@@ -17,6 +17,7 @@ import io.netty.buffer.ByteBuf;
 import java.io.Closeable;
 import java.util.List;
 
+import org.apache.commons.lang3.ArrayUtils;
 import org.apache.drill.exec.memory.BufferAllocator;
 import org.apache.drill.exec.proto.SchemaDefProtos;
 import org.apache.drill.exec.proto.UserBitShared.FieldMetadata;
@@ -58,12 +59,7 @@ public final class ${className} extends BaseValueVector implements <#if type.maj
   
   @Override
   public ByteBuf[] getBuffers() {
-    <#if type.major == "VarLen">
-    ByteBuf[] valueBuffers = values.getBuffers();
-    return new ByteBuf[]{bits.data, valueBuffers[0], valueBuffers[1]};
-    <#else>
-    return new ByteBuf[]{bits.data, values.data};
-    </#if>
+    return ArrayUtils.addAll(bits.getBuffers(), values.getBuffers());
   }
   
   @Override

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/6f9dadb9/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/RepeatedValueVectors.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/RepeatedValueVectors.java b/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/RepeatedValueVectors.java
index 5def096..c1660e8 100644
--- a/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/RepeatedValueVectors.java
+++ b/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/RepeatedValueVectors.java
@@ -17,6 +17,7 @@ import java.io.Closeable;
 import java.util.Random;
 import java.util.Vector;
 
+import org.apache.commons.lang3.ArrayUtils;
 import org.apache.drill.exec.memory.BufferAllocator;
 import org.apache.drill.exec.proto.SchemaDefProtos;
 import org.apache.drill.exec.proto.UserBitShared.FieldMetadata;
@@ -176,7 +177,7 @@ import com.google.common.collect.Lists;
   </#if>
 
   public ByteBuf[] getBuffers() {
-    return new ByteBuf[]{offsets.data, values.data};
+    return ArrayUtils.addAll(offsets.getBuffers(), values.getBuffers());
   }
 
   public void clear(){

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/6f9dadb9/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/schema/DiffSchema.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/schema/DiffSchema.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/schema/DiffSchema.java
index 68c3e12..81a396e 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/schema/DiffSchema.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/schema/DiffSchema.java
@@ -19,14 +19,13 @@
 package org.apache.drill.exec.schema;
 
 import com.google.common.collect.Lists;
-import org.apache.drill.exec.physical.impl.OutputMutator;
 
 import java.util.List;
 
 public class DiffSchema {
     List<Field> addedFields;
     List<Field> removedFields;
-    boolean hasChanged = false;
+    boolean changed = false;
 
     public DiffSchema() {
         this.addedFields = Lists.newArrayList();
@@ -35,22 +34,22 @@ public class DiffSchema {
 
     public void recordNewField(Field field) {
         addedFields.add(field);
-        hasChanged = true;
+        changed = true;
     }
 
     public void reset() {
         addedFields.clear();
         removedFields.clear();
-        hasChanged = false;
+        changed = false;
     }
 
     public void addRemovedField(Field field) {
         removedFields.add(field);
-        hasChanged = true;
+        changed = true;
     }
 
-    public boolean isHasChanged() {
-        return hasChanged;
+    public boolean isChanged() {
+        return changed;
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/6f9dadb9/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/JSONRecordReader.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/JSONRecordReader.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/JSONRecordReader.java
index 21b8c1b..8a2de63 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/JSONRecordReader.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/JSONRecordReader.java
@@ -125,7 +125,7 @@ public class JSONRecordReader implements RecordReader {
         outputMutator.removeField(field.getAsMaterializedField());
       }
 
-      if (diffSchema.isHasChanged()) {
+      if (diffSchema.isChanged()) {
         outputMutator.setNewSchema();
       }
 
@@ -358,7 +358,7 @@ public class JSONRecordReader implements RecordReader {
     private static <T> boolean addValueToVector(int index, VectorHolder holder, T val, MinorType minorType, int groupCount) {
       switch (minorType) {
         case INT: {
-          holder.incAndCheckLength(32 + 1);
+          holder.incAndCheckLength(NullableIntHolder.WIDTH * 8 + 1);
           if (groupCount == 0) {
             if (val != null) {
               NullableIntVector int4 = (NullableIntVector) holder.getValueVector();
@@ -376,10 +376,10 @@ public class JSONRecordReader implements RecordReader {
             m.add(index, (Integer) val);
           }
 
-          return holder.hasEnoughSpace(32 + 1);
+          return holder.hasEnoughSpace(NullableIntHolder.WIDTH * 8 + 1);
         }
         case FLOAT4: {
-          holder.incAndCheckLength(32 + 1);
+          holder.incAndCheckLength(NullableFloat4Holder.WIDTH * 8 + 1);
           if (groupCount == 0) {
             if (val != null) {
               NullableFloat4Vector float4 = (NullableFloat4Vector) holder.getValueVector();
@@ -396,7 +396,7 @@ public class JSONRecordReader implements RecordReader {
             holder.setGroupCount(index);
             m.add(index, (Float) val);
           }
-          return holder.hasEnoughSpace(32 + 1);
+          return holder.hasEnoughSpace(NullableFloat4Holder.WIDTH * 8 + 1);
         }
         case VARCHAR: {
           if (val == null) {
@@ -419,7 +419,7 @@ public class JSONRecordReader implements RecordReader {
           }
         }
         case BIT: {
-          holder.incAndCheckLength(1 + 1);
+          holder.incAndCheckLength(NullableBitHolder.WIDTH + 1);
           if (groupCount == 0) {
             if (val != null) {
               NullableBitVector bit = (NullableBitVector) holder.getValueVector();
@@ -436,7 +436,7 @@ public class JSONRecordReader implements RecordReader {
             holder.setGroupCount(index);
             m.add(index, (Boolean) val ? 1 : 0);
           }
-          return holder.hasEnoughSpace(1 + 1);
+          return holder.hasEnoughSpace(NullableBitHolder.WIDTH + 1);
         }
         default:
           throw new DrillRuntimeException("Type not supported to add value. Type: " + minorType);
@@ -476,7 +476,7 @@ public class JSONRecordReader implements RecordReader {
 
       ValueVector v = TypeHelper.getNewVector(f, allocator);
       AllocationHelper.allocate(v, batchSize, 50);
-      holder = new VectorHolder(batchSize, v);
+      holder = new VectorHolder(v);
       valueVectorMap.put(fullFieldName, holder);
       outputMutator.addField(v);
       return holder;

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/6f9dadb9/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/VectorHolder.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/VectorHolder.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/VectorHolder.java
index be0bea8..d2ad72a 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/VectorHolder.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/VectorHolder.java
@@ -27,8 +27,8 @@ public class VectorHolder {
   private ValueVector vector;
   private int currentLength;
 
-  VectorHolder(int length, ValueVector vector) {
-    this.length = length;
+  VectorHolder(ValueVector vector) {
+    this.length = vector.getValueCapacity();
     this.vector = vector;
   }
 
@@ -38,18 +38,18 @@ public class VectorHolder {
 
   public void incAndCheckLength(int newLength) {
     if (!hasEnoughSpace(newLength)) {
-      throw new BatchExceededException(length, currentLength + newLength);
+      throw new BatchExceededException(length, vector.getBufferSize() + newLength);
     }
-    count += 1;
+
     currentLength += newLength;
+    count += 1;
   }
 
   public void setGroupCount(int groupCount) {
     if(this.groupCount < groupCount) {
       RepeatedMutator mutator = (RepeatedMutator) vector.getMutator();
       while(this.groupCount < groupCount) {
-        mutator.startNewGroup(this.groupCount + 1);
-        this.groupCount++;
+        mutator.startNewGroup(++this.groupCount);
       }
     }
   }


[21/27] git commit: Fix issue where redundant batches showing up after being partitioned and sent.

Posted by ja...@apache.org.
Fix issue where redundant batches showing up after being partitioned and sent.


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

Branch: refs/heads/master
Commit: 98156ee70d0f6b04c24c7a1de442d5744f0fd63d
Parents: add8c72
Author: Ben Becker <be...@gmail.com>
Authored: Mon Aug 12 00:48:06 2013 -0700
Committer: Jacques Nadeau <ja...@apache.org>
Committed: Thu Aug 15 18:31:31 2013 -0700

----------------------------------------------------------------------
 .../physical/impl/partitionsender/PartitionSenderRootExec.java | 6 +++---
 1 file changed, 3 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/98156ee7/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/partitionsender/PartitionSenderRootExec.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/partitionsender/PartitionSenderRootExec.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/partitionsender/PartitionSenderRootExec.java
index b2ca64e..476de7d 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/partitionsender/PartitionSenderRootExec.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/partitionsender/PartitionSenderRootExec.java
@@ -213,11 +213,11 @@ class PartitionSenderRootExec implements RootExec {
                        .arg(((JExpression) outgoingBatches.component(outIndex)).invoke("getRecordCount"))
                        .arg(incomingVV));
 
-      // generate the OutgoingRecordBatch helper invocations
-      cg.getBlock().add(((JExpression) outgoingBatches.component(outIndex)).invoke("incRecordCount"));
-      cg.getBlock().add(((JExpression) outgoingBatches.component(outIndex)).invoke("flushIfNecessary"));
       ++fieldId;
     }
+    // generate the OutgoingRecordBatch helper invocations
+    cg.getBlock().add(((JExpression) outgoingBatches.component(outIndex)).invoke("incRecordCount"));
+    cg.getBlock().add(((JExpression) outgoingBatches.component(outIndex)).invoke("flushIfNecessary"));
     try {
       // compile and setup generated code
       partitioner = context.getImplementationClassMultipleOutput(cg);


[19/27] git commit: prevent NPE in recordLoader. still need to handle the last batch correctly.

Posted by ja...@apache.org.
prevent NPE in recordLoader.  still need to handle the last batch correctly.


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

Branch: refs/heads/master
Commit: 93121cbf168f63881ad93d126e2fd9306a51f64a
Parents: a136a5b
Author: Ben Becker <be...@gmail.com>
Authored: Sat Aug 10 12:44:59 2013 -0700
Committer: Jacques Nadeau <ja...@apache.org>
Committed: Thu Aug 15 18:31:31 2013 -0700

----------------------------------------------------------------------
 .../impl/partitionsender/OutgoingRecordBatch.java        |  9 ++++++---
 .../impl/partitionsender/PartitionSenderRootExec.java    | 11 +++++++++--
 2 files changed, 15 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/93121cbf/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/partitionsender/OutgoingRecordBatch.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/partitionsender/OutgoingRecordBatch.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/partitionsender/OutgoingRecordBatch.java
index 6847e5a..6eff778 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/partitionsender/OutgoingRecordBatch.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/partitionsender/OutgoingRecordBatch.java
@@ -79,9 +79,12 @@ public class OutgoingRecordBatch implements RecordBatch {
   
   public void flush() throws SchemaChangeException {
     if (recordCount == 0) {
-      logger.warn("Attempted to flush an empty record batch");
+      // TODO:  recordCount of 0 with isLast causes recordLoader to throw an NPE.  Probably
+      //        need to send notification rather than an actual batch.
+      logger.warn("Attempted to flush an empty record batch" + (isLast ? " (last batch)" : ""));
+      return;
     }
-    logger.debug("Flushing record batch.  count is: " + recordCount + ", capacity is " + recordCapacity);
+
     final ExecProtos.FragmentHandle handle = context.getHandle();
     FragmentWritableBatch writableBatch = new FragmentWritableBatch(isLast,
                                                                     handle.getQueryId(),
@@ -90,7 +93,7 @@ public class OutgoingRecordBatch implements RecordBatch {
                                                                     operator.getOppositeMajorFragmentId(),
                                                                     0,
                                                                     getWritableBatch());
-     tunnel.sendRecordBatch(statusHandler, context, writableBatch);
+    tunnel.sendRecordBatch(statusHandler, context, writableBatch);
 
     // reset values and reallocate the buffer for each value vector.  NOTE: the value vector is directly
     // referenced by generated code and must not be replaced.

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/93121cbf/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/partitionsender/PartitionSenderRootExec.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/partitionsender/PartitionSenderRootExec.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/partitionsender/PartitionSenderRootExec.java
index aa25c96..293a711 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/partitionsender/PartitionSenderRootExec.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/partitionsender/PartitionSenderRootExec.java
@@ -86,15 +86,20 @@ class PartitionSenderRootExec implements RootExec {
         if (incoming.getRecordCount() > 0)
           partitioner.partitionBatch(incoming);
 
-        // send all pending batches
         try {
-          flushOutgoingBatches(true, false);
+          // send any pending batches
+          for (OutgoingRecordBatch batch : outgoing) {
+            batch.setIsLast();
+            batch.flush();
+          }
         } catch (SchemaChangeException e) {
           incoming.kill();
           logger.error("Error while creating partitioning sender or flushing outgoing batches", e);
           context.fail(e);
           return false;
         }
+        context.batchesCompleted.inc(1);
+        context.recordsCompleted.inc(incoming.getRecordCount());
         return false;
 
       case OK_NEW_SCHEMA:
@@ -111,6 +116,8 @@ class PartitionSenderRootExec implements RootExec {
         }
       case OK:
         partitioner.partitionBatch(incoming);
+        context.batchesCompleted.inc(1);
+        context.recordsCompleted.inc(incoming.getRecordCount());
         return true;
       case NOT_YET:
       default:


[12/27] fixes for memory management and rpc throttling

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/402be7e0/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/ParquetRecordReaderTest.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/ParquetRecordReaderTest.java b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/ParquetRecordReaderTest.java
deleted file mode 100644
index 75a52c5..0000000
--- a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/ParquetRecordReaderTest.java
+++ /dev/null
@@ -1,594 +0,0 @@
-/*******************************************************************************
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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 com.beust.jcommander.internal.Lists;
-import com.google.common.base.Charsets;
-import com.google.common.io.Files;
-
-import com.google.common.util.concurrent.SettableFuture;
-import org.apache.drill.common.config.DrillConfig;
-import org.apache.drill.common.types.TypeProtos;
-import org.apache.drill.common.util.FileUtils;
-import org.apache.drill.exec.client.DrillClient;
-import org.apache.drill.exec.exception.SchemaChangeException;
-import org.apache.drill.exec.physical.impl.OutputMutator;
-
-import org.apache.drill.exec.proto.UserProtos;
-import org.apache.drill.exec.record.MaterializedField;
-import org.apache.drill.exec.record.RecordBatchLoader;
-import org.apache.drill.exec.record.VectorWrapper;
-import org.apache.drill.exec.rpc.RpcException;
-import org.apache.drill.exec.rpc.user.QueryResultBatch;
-import org.apache.drill.exec.rpc.user.UserResultsListener;
-import org.apache.drill.exec.server.BootStrapContext;
-import org.apache.drill.exec.server.Drillbit;
-import org.apache.drill.exec.server.RemoteServiceSet;
-
-import org.apache.drill.exec.store.parquet.ParquetStorageEngine;
-import org.apache.drill.exec.vector.BaseDataValueVector;
-import org.apache.drill.exec.vector.ValueVector;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.junit.Test;
-import parquet.bytes.BytesInput;
-import parquet.column.ColumnDescriptor;
-import parquet.column.page.Page;
-import parquet.column.page.PageReadStore;
-import parquet.column.page.PageReader;
-import parquet.hadoop.Footer;
-
-import parquet.hadoop.ParquetFileWriter;
-import parquet.hadoop.metadata.CompressionCodecName;
-import parquet.hadoop.metadata.ParquetMetadata;
-import parquet.schema.MessageType;
-import parquet.schema.MessageTypeParser;
-
-import java.io.File;
-import java.io.IOException;
-import java.text.SimpleDateFormat;
-import java.util.*;
-
-import static org.junit.Assert.*;
-import static org.junit.Assert.assertEquals;
-import static parquet.column.Encoding.PLAIN;
-
-public class ParquetRecordReaderTest {
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(StorageEngineRegistry.class);
-
-  private boolean VERBOSE_DEBUG = false;
-
-  @Test
-  public void testMultipleRowGroupsAndReads() throws Exception {
-    testParquetFullEngine(true, "/parquet_scan_screen.json", "/tmp/testParquetFile_many_types_3", 2, numberRowGroups, recordsPerRowGroup);
-  }
-
-  @Test
-  public void testMultipleRowGroupsAndReadsEvent() throws Exception {
-    testParquetFullEngineEventBased(true, "/parquet_scan_screen.json", "/tmp/testParquetFile_many_types_3", 2, numberRowGroups, recordsPerRowGroup);
-  }
-
-  int numberRowGroups = 20;
-  static int recordsPerRowGroup = 3000000;
-
-  // 10 mb per page
-  static int bytesPerPage = 1024 * 1024 * 10;
-  // { 00000001, 00000010, 00000100, 00001000, 00010000, ... }
-  byte[] bitFields = {1, 2, 4, 8, 16, 32, 64, -128};
-  static final byte allBitsTrue = -1;
-  static final byte allBitsFalse = 0;
-  static final byte[] varLen1 = {50, 51, 52, 53, 54, 55, 56};
-  static final byte[] varLen2 = {15, 14, 13, 12, 11, 10, 9, 8, 7, 6, 5, 4, 3, 2, 1};
-  static final byte[] varLen3 = {100, 99, 98};
-
-  private static class FieldInfo {
-
-    String parquetType;
-    String name;
-    int bitLength;
-    int numberOfPages;
-    Object[] values;
-    TypeProtos.MinorType type;
-
-    FieldInfo(String parquetType, String name, int bitLength, Object[] values, TypeProtos.MinorType type){
-      this.parquetType = parquetType;
-      this.name = name;
-      this.bitLength  = bitLength;
-      this.numberOfPages = Math.max(1, (int) Math.ceil(recordsPerRowGroup * bitLength / 8.0 / bytesPerPage));
-      this.values = values;
-      // generator is designed to use 3 values
-      assert values.length == 3;
-      this.type = type;
-    }
-  }
-
-  
-  private static HashMap<String, FieldInfo> fields = new HashMap<>();
-  static {
-    Object[] intVals = {-200, 100, Integer.MAX_VALUE };
-    Object[] longVals = { -5000l, 5000l, Long.MAX_VALUE};
-    Object[] floatVals = { 1.74f, Float.MAX_VALUE, Float.MIN_VALUE};
-    Object[] doubleVals = {100.45d, Double.MAX_VALUE, Double.MIN_VALUE,};
-    Object[] boolVals = {false, false, true};
-    Object[] binVals = { varLen1, varLen2, varLen3};
-    Object[] bin2Vals = { varLen3, varLen2, varLen1};
-    fields.put("integer/", new FieldInfo("int32", "integer", 32, intVals, TypeProtos.MinorType.INT));
-    fields.put("bigInt/", new FieldInfo("int64", "bigInt", 64, longVals, TypeProtos.MinorType.BIGINT));
-    fields.put("f/", new FieldInfo("float", "f", 32, floatVals, TypeProtos.MinorType.FLOAT4));
-    fields.put("d/", new FieldInfo("double", "d", 64, doubleVals, TypeProtos.MinorType.FLOAT8));
-//    fields.put("b/", new FieldInfo("binary", "b", 1, boolVals, TypeProtos.MinorType.BIT));
-    fields.put("bin/", new FieldInfo("binary", "bin", -1, binVals, TypeProtos.MinorType.VARBINARY));
-    fields.put("bin2/", new FieldInfo("binary", "bin2", -1, bin2Vals, TypeProtos.MinorType.VARBINARY));
-  }
-
-
-  private String getResource(String resourceName) {
-    return "resource:" + resourceName;
-  }
-
-  public void generateParquetFile(String filename, int numberRowGroups, int recordsPerRowGroup) throws Exception {
-
-    int currentBooleanByte = 0;
-    WrapAroundCounter booleanBitCounter = new WrapAroundCounter(7);
-
-    Configuration configuration = new Configuration();
-    configuration.set(ParquetStorageEngine.HADOOP_DEFAULT_NAME, "file:///");
-    //"message m { required int32 integer; required int64 integer64; required boolean b; required float f; required double d;}"
-
-    FileSystem fs = FileSystem.get(configuration);
-    Path path = new Path(filename);
-    if (fs.exists(path)) fs.delete(path, false);
-
-
-    String messageSchema = "message m {";
-    for (FieldInfo fieldInfo : fields.values()) {
-      messageSchema += " required " + fieldInfo.parquetType + " " + fieldInfo.name + ";";
-    }
-    // remove the last semicolon, java really needs a join method for strings...
-    // TODO - nvm apparently it requires a semicolon after every field decl, might want to file a bug
-    //messageSchema = messageSchema.substring(schemaType, messageSchema.length() - 1);
-    messageSchema += "}";
-
-    MessageType schema = MessageTypeParser.parseMessageType(messageSchema);
-
-    CompressionCodecName codec = CompressionCodecName.UNCOMPRESSED;
-    ParquetFileWriter w = new ParquetFileWriter(configuration, schema, path);
-    w.start();
-    HashMap<String, Integer> columnValuesWritten = new HashMap();
-    int valsWritten;
-    for (int k = 0; k < numberRowGroups; k++){
-      w.startBlock(1);
-
-      for (FieldInfo fieldInfo : fields.values()) {
-
-        if ( ! columnValuesWritten.containsKey(fieldInfo.name)){
-          columnValuesWritten.put((String) fieldInfo.name, 0);
-          valsWritten = 0;
-        } else {
-          valsWritten = columnValuesWritten.get(fieldInfo.name);
-        }
-
-        String[] path1 = {(String) fieldInfo.name};
-        ColumnDescriptor c1 = schema.getColumnDescription(path1);
-
-        w.startColumn(c1, recordsPerRowGroup, codec);
-        int valsPerPage = (int) Math.ceil(recordsPerRowGroup / (float) ((int) fieldInfo.numberOfPages));
-        byte[] bytes;
-        // for variable length binary fields
-        int bytesNeededToEncodeLength = 4;
-        if ((int) fieldInfo.bitLength > 0) {
-          bytes = new byte[(int) Math.ceil(valsPerPage * (int) fieldInfo.bitLength / 8.0)];
-        } else {
-          // the twelve at the end is to account for storing a 4 byte length with each value
-          int totalValLength = ((byte[]) fieldInfo.values[0]).length + ((byte[]) fieldInfo.values[1]).length + ((byte[]) fieldInfo.values[2]).length + 3 * bytesNeededToEncodeLength;
-          // used for the case where there is a number of values in this row group that is not divisible by 3
-          int leftOverBytes = 0;
-          if ( valsPerPage % 3 > 0 ) leftOverBytes += ((byte[])fieldInfo.values[1]).length + 4;
-          if ( valsPerPage % 3 > 1 ) leftOverBytes += ((byte[])fieldInfo.values[2]).length + 4;
-          bytes = new byte[valsPerPage / 3 * totalValLength + leftOverBytes];
-        }
-        int bytesPerPage = (int) (valsPerPage * ((int) fieldInfo.bitLength / 8.0));
-        int bytesWritten = 0;
-        for (int z = 0; z < (int) fieldInfo.numberOfPages; z++, bytesWritten = 0) {
-          for (int i = 0; i < valsPerPage; i++) {
-            //System.out.print(i + ", " + (i % 25 == 0 ? "\n gen " + fieldInfo.name + ": " : ""));
-            if (fieldInfo.values[0] instanceof Boolean) {
-
-              bytes[currentBooleanByte] |= bitFields[booleanBitCounter.val] & ((boolean) fieldInfo.values[valsWritten % 3]
-                  ? allBitsTrue : allBitsFalse);
-              booleanBitCounter.increment();
-              if (booleanBitCounter.val == 0) {
-                currentBooleanByte++;
-              }
-              valsWritten++;
-              if (currentBooleanByte > bytesPerPage) break;
-            } else {
-              if (fieldInfo.values[valsWritten % 3] instanceof byte[]){
-                System.arraycopy(ByteArrayUtil.toByta(((byte[])fieldInfo.values[valsWritten % 3]).length),
-                    0, bytes, bytesWritten, bytesNeededToEncodeLength);
-                System.arraycopy(fieldInfo.values[valsWritten % 3],
-                    0, bytes, bytesWritten + bytesNeededToEncodeLength, ((byte[])fieldInfo.values[valsWritten % 3]).length);
-                bytesWritten += ((byte[])fieldInfo.values[valsWritten % 3]).length + bytesNeededToEncodeLength;
-              }
-              else{
-                System.arraycopy( ByteArrayUtil.toByta(fieldInfo.values[valsWritten % 3]),
-                    0, bytes, i * ((int) fieldInfo.bitLength / 8), (int) fieldInfo.bitLength / 8);
-              }
-              valsWritten++;
-            }
-
-          }
-          w.writeDataPage((int)(recordsPerRowGroup / (int) fieldInfo.numberOfPages), bytes.length, BytesInput.from(bytes), PLAIN, PLAIN, PLAIN);
-          currentBooleanByte = 0;
-        }
-        w.endColumn();
-        columnValuesWritten.remove((String) fieldInfo.name);
-        columnValuesWritten.put((String) fieldInfo.name, valsWritten);
-      }
-
-      w.endBlock();
-    }
-    w.end(new HashMap<String, String>());
-    logger.debug("Finished generating parquet file.");
-  }
-
-  private class ParquetResultListener implements UserResultsListener {
-    private Vector<QueryResultBatch> results = new Vector<QueryResultBatch>();
-    private SettableFuture<List<QueryResultBatch>> future = SettableFuture.create();
-    int count = 0;
-    RecordBatchLoader batchLoader;
-    byte[] bytes;
-
-    int batchCounter = 1;
-    int columnValCounter = 0;
-    int i = 0;
-    FieldInfo currentField;
-    HashMap<String, Integer> valuesChecked = new HashMap();
-
-    ParquetResultListener(RecordBatchLoader batchLoader){
-      this.batchLoader = batchLoader;
-    }
-
-    @Override
-    public void submissionFailed(RpcException ex) {
-      logger.debug("Submission failed.", ex);
-      future.setException(ex);
-    }
-
-    @Override
-    public void resultArrived(QueryResultBatch result) {
-      logger.debug("result arrived in test batch listener.");
-      int columnValCounter = 0;
-      int i = 0;
-      FieldInfo currentField;
-      count += result.getHeader().getRowCount();
-      boolean schemaChanged = false;
-      try {
-        schemaChanged = batchLoader.load(result.getHeader().getDef(), result.getData());
-      } catch (SchemaChangeException e) {
-        e.printStackTrace();
-      }
-
-      int recordCount = 0;
-      // print headers.
-      if (schemaChanged) {
-      } // do not believe any change is needed for when the schema changes, with the current mock scan use case
-
-      for (VectorWrapper vw : batchLoader) {
-        ValueVector vv = vw.getValueVector();
-        currentField = fields.get(vv.getField().getName());
-        if (VERBOSE_DEBUG){
-          System.out.println("\n" + (String) currentField.name);
-        }
-        if ( ! valuesChecked.containsKey(vv.getField().getName())){
-          valuesChecked.put(vv.getField().getName(), 0);
-          columnValCounter = 0;
-        } else {
-          columnValCounter = valuesChecked.get(vv.getField().getName());
-        }
-        for (int j = 0; j < ((BaseDataValueVector)vv).getAccessor().getValueCount(); j++) {
-          if (VERBOSE_DEBUG){
-            System.out.print(vv.getAccessor().getObject(j) + ", " + (j % 25 == 0 ? "\n batch:" + batchCounter + " v:" + j + " - " : ""));
-          }
-          assertField(vv, j, (TypeProtos.MinorType) currentField.type,
-              currentField.values[columnValCounter % 3], (String) currentField.name + "/");
-          columnValCounter++;
-        }
-        if (VERBOSE_DEBUG){
-          System.out.println("\n" + ((BaseDataValueVector)vv).getAccessor().getValueCount());
-        }
-        valuesChecked.remove(vv.getField().getName());
-        valuesChecked.put(vv.getField().getName(), columnValCounter);
-      }
-
-      if (VERBOSE_DEBUG){
-        for (i = 0; i < batchLoader.getRecordCount(); i++) {
-          recordCount++;
-          if (i % 50 == 0){
-            System.out.println();
-            for (VectorWrapper vw : batchLoader) {
-              ValueVector v = vw.getValueVector();
-              System.out.print(pad(v.getField().getName(), 20) + " ");
-
-            }
-            System.out.println();
-            System.out.println();
-          }
-
-          for (VectorWrapper vw : batchLoader) {
-            ValueVector v = vw.getValueVector();
-            System.out.print(pad(v.getAccessor().getObject(i).toString(), 20) + " ");
-          }
-          System.out.println(
-
-          );
-        }
-      }
-      batchCounter++;
-      if(result.getHeader().getIsLastChunk()){
-        future.set(results);
-      }
-    }
-
-    public List<QueryResultBatch> getResults() throws RpcException{
-      try{
-        return future.get();
-      }catch(Throwable t){
-        throw RpcException.mapException(t);
-      }
-    }
-  }
-
-  // specific tests should call this method, but it is not marked as a test itself intentionally
-  public void testParquetFullEngineEventBased(boolean generateNew, String plan, String filename, int numberOfTimesRead /* specified in json plan */, int numberRowGroups, int recordsPerRowGroup) throws Exception{
-    RemoteServiceSet serviceSet = RemoteServiceSet.getLocalServiceSet();
-
-    if (generateNew) generateParquetFile(filename, numberRowGroups, recordsPerRowGroup);
-
-    DrillConfig config = DrillConfig.create();
-
-    try(Drillbit bit1 = new Drillbit(config, serviceSet); DrillClient client = new DrillClient(config, serviceSet.getCoordinator());){
-      bit1.run();
-      client.connect();
-      List<QueryResultBatch> results = client.runQuery(UserProtos.QueryType.LOGICAL, Files.toString(FileUtils.getResourceAsFile(plan), Charsets.UTF_8));
-      int count = 0;
-      RecordBatchLoader batchLoader = new RecordBatchLoader(bit1.getContext().getAllocator());
-      ParquetResultListener resultListener = new ParquetResultListener(batchLoader);
-      client.runQuery(UserProtos.QueryType.LOGICAL, Files.toString(FileUtils.getResourceAsFile(plan), Charsets.UTF_8), resultListener);
-    }
-  }
-
-
-  // specific tests should call this method, but it is not marked as a test itself intentionally
-  public void testParquetFullEngine(boolean generateNew, String plan, String filename, int numberOfTimesRead /* specified in json plan */, int numberRowGroups, int recordsPerRowGroup) throws Exception{
-    RemoteServiceSet serviceSet = RemoteServiceSet.getLocalServiceSet();
-
-    if (generateNew) generateParquetFile(filename, numberRowGroups, recordsPerRowGroup);
-
-    DrillConfig config = DrillConfig.create();
-
-    try(Drillbit bit1 = new Drillbit(config, serviceSet); DrillClient client = new DrillClient(config, serviceSet.getCoordinator())) {
-      long A = System.nanoTime();
-      bit1.run();
-      long B = System.nanoTime();
-      client.connect();
-      long C = System.nanoTime();
-      System.out.println( new SimpleDateFormat("mm:ss S").format(new Date()) + " :Start query");
-      List<QueryResultBatch> results = client.runQuery(UserProtos.QueryType.LOGICAL, Files.toString(FileUtils.getResourceAsFile("/parquet_scan_screen.json"), Charsets.UTF_8));
-//      List<QueryResultBatch> results = client.runQuery(UserProtos.QueryType.PHYSICAL, Files.toString(FileUtils.getResourceAsFile("/parquet_scan_union_screen_physical.json"), Charsets.UTF_8));
-      long D = System.nanoTime();
-      System.out.println(String.format("Took %f s to start drillbit", (float)(B-A) / 1E9));
-      System.out.println(String.format("Took %f s to connect", (float)(C-B) / 1E9));
-      System.out.println(String.format("Took %f s to run query", (float)(D-C) / 1E9));
-      //List<QueryResultBatch> results = client.runQuery(UserProtos.QueryType.PHYSICAL, Files.toString(FileUtils.getResourceAsFile("/parquet_scan_union_screen_physical.json"), Charsets.UTF_8));
-      int count = 0;
-//      RecordBatchLoader batchLoader = new RecordBatchLoader(new BootStrapContext(config).getAllocator());
-      RecordBatchLoader batchLoader = new RecordBatchLoader(bit1.getContext().getAllocator());
-      byte[] bytes;
-
-      int batchCounter = 1;
-      int columnValCounter = 0;
-      int i = 0;
-      FieldInfo currentField;
-      HashMap<String, Integer> valuesChecked = new HashMap();
-      for(QueryResultBatch b : results){
-        count += b.getHeader().getRowCount();
-        boolean schemaChanged = batchLoader.load(b.getHeader().getDef(), b.getData());
-
-        int recordCount = 0;
-        // print headers.
-        if (schemaChanged) {
-        } // do not believe any change is needed for when the schema changes, with the current mock scan use case
-
-        for (VectorWrapper vw : batchLoader) {
-          ValueVector vv = vw.getValueVector();
-          currentField = fields.get(vv.getField().getName());
-          if (VERBOSE_DEBUG){
-            System.out.println("\n" + (String) currentField.name);
-          }
-          if ( ! valuesChecked.containsKey(vv.getField().getName())){
-            valuesChecked.put(vv.getField().getName(), 0);
-            columnValCounter = 0;
-          } else {
-            columnValCounter = valuesChecked.get(vv.getField().getName());
-          }
-          for (int j = 0; j < ((BaseDataValueVector)vv).getAccessor().getValueCount(); j++) {
-            if (VERBOSE_DEBUG){
-              System.out.print(vv.getAccessor().getObject(j) + ", " + (j % 25 == 0 ? "\n batch:" + batchCounter + " v:" + j + " - " : ""));
-            }
-            assertField(vv, j, (TypeProtos.MinorType) currentField.type,
-                currentField.values[columnValCounter % 3], (String) currentField.name + "/");
-            columnValCounter++;
-          }
-          if (VERBOSE_DEBUG){
-            System.out.println("\n" + ((BaseDataValueVector)vv).getAccessor().getValueCount());
-          }
-          valuesChecked.remove(vv.getField().getName());
-          valuesChecked.put(vv.getField().getName(), columnValCounter);
-        }
-
-        if (VERBOSE_DEBUG){
-          for (i = 1; i < batchLoader.getRecordCount(); i++) {
-            recordCount++;
-            if (i % 50 == 0){
-              System.out.println();
-              for (VectorWrapper vw : batchLoader) {
-                ValueVector v = vw.getValueVector();
-                System.out.print(pad(v.getField().getName(), 20) + " ");
-
-              }
-              System.out.println();
-              System.out.println();
-            }
-
-            for (VectorWrapper vw : batchLoader) {
-              ValueVector v = vw.getValueVector();
-              System.out.print(pad(v.getAccessor().getObject(i).toString(), 20) + " ");
-            }
-            System.out.println(
-
-            );
-          }
-        }
-        batchCounter++;
-      }
-      for (String s : valuesChecked.keySet()) {
-        assertEquals("Record count incorrect for column: " + s, recordsPerRowGroup * numberRowGroups * numberOfTimesRead, (long) valuesChecked.get(s));
-      }
-      assert valuesChecked.keySet().size() > 0;
-    }
-  }
-
-  public String pad(String value, int length) {
-    return pad(value, length, " ");
-  }
-
-  public String pad(String value, int length, String with) {
-    StringBuilder result = new StringBuilder(length);
-    result.append(value);
-
-    while (result.length() < length) {
-      result.insert(0, with);
-    }
-
-    return result.toString();
-  }
-
-  class MockOutputMutator implements OutputMutator {
-    List<MaterializedField> removedFields = Lists.newArrayList();
-    List<ValueVector> addFields = Lists.newArrayList();
-
-    @Override
-    public void removeField(MaterializedField field) throws SchemaChangeException {
-      removedFields.add(field);
-    }
-
-    @Override
-    public void addField(ValueVector vector) throws SchemaChangeException {
-      addFields.add(vector);
-    }
-
-    @Override
-    public void removeAllFields() {
-      addFields.clear();
-    }
-
-    @Override
-    public void setNewSchema() throws SchemaChangeException {
-    }
-
-    List<MaterializedField> getRemovedFields() {
-      return removedFields;
-    }
-
-    List<ValueVector> getAddFields() {
-      return addFields;
-    }
-  }
-
-  private <T> void assertField(ValueVector valueVector, int index, TypeProtos.MinorType expectedMinorType, Object value, String name) {
-    assertField(valueVector, index, expectedMinorType, value, name, 0);
-  }
-
-  private <T> void assertField(ValueVector valueVector, int index, TypeProtos.MinorType expectedMinorType, T value, String name, int parentFieldId) {
-//    UserBitShared.FieldMetadata metadata = valueVector.getMetadata();
-//    SchemaDefProtos.FieldDef def = metadata.getDef();
-//    assertEquals(expectedMinorType, def.getMajorType().getMinorType());
-//    assertEquals(name, def.getNameList().get(0).getName());
-//    assertEquals(parentFieldId, def.getParentId());
-
-    if (expectedMinorType == TypeProtos.MinorType.MAP) {
-      return;
-    }
-
-    T val = (T) valueVector.getAccessor().getObject(index);
-    if (val instanceof byte[]) {
-      assertTrue(Arrays.equals((byte[]) value, (byte[]) val));
-    } else {
-      assertEquals(value, val);
-    }
-  }
-
-  private class WrapAroundCounter {
-
-    int maxVal;
-    int val;
-
-    public WrapAroundCounter(int maxVal) {
-      this.maxVal = maxVal;
-    }
-
-    public int increment() {
-      val++;
-      if (val > maxVal) {
-        val = 0;
-      }
-      return val;
-    }
-
-    public void reset() {
-      val = 0;
-    }
-
-  }
-
-  private void validateFooters(final List<Footer> metadata) {
-    logger.debug(metadata.toString());
-    assertEquals(3, metadata.size());
-    for (Footer footer : metadata) {
-      final File file = new File(footer.getFile().toUri());
-      assertTrue(file.getName(), file.getName().startsWith("part"));
-      assertTrue(file.getPath(), file.exists());
-      final ParquetMetadata parquetMetadata = footer.getParquetMetadata();
-      assertEquals(2, parquetMetadata.getBlocks().size());
-      final Map<String, String> keyValueMetaData = parquetMetadata.getFileMetaData().getKeyValueMetaData();
-      assertEquals("bar", keyValueMetaData.get("foo"));
-      assertEquals(footer.getFile().getName(), keyValueMetaData.get(footer.getFile().getName()));
-    }
-  }
-
-  private void validateContains(MessageType schema, PageReadStore pages, String[] path, int values, BytesInput bytes)
-      throws IOException {
-    PageReader pageReader = pages.getPageReader(schema.getColumnDescription(path));
-    Page page = pageReader.readPage();
-    assertEquals(values, page.getValueCount());
-    assertArrayEquals(bytes.toByteArray(), page.getBytes().toByteArray());
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/402be7e0/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/ParquetRecordReaderTest.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/ParquetRecordReaderTest.java b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/ParquetRecordReaderTest.java
new file mode 100644
index 0000000..0e31cdd
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/ParquetRecordReaderTest.java
@@ -0,0 +1,347 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.parquet;
+
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Vector;
+
+import org.apache.drill.common.config.DrillConfig;
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.common.util.FileUtils;
+import org.apache.drill.exec.client.DrillClient;
+import org.apache.drill.exec.exception.SchemaChangeException;
+import org.apache.drill.exec.physical.impl.OutputMutator;
+import org.apache.drill.exec.proto.UserProtos;
+import org.apache.drill.exec.record.MaterializedField;
+import org.apache.drill.exec.record.RecordBatchLoader;
+import org.apache.drill.exec.record.VectorWrapper;
+import org.apache.drill.exec.rpc.RpcException;
+import org.apache.drill.exec.rpc.user.QueryResultBatch;
+import org.apache.drill.exec.rpc.user.UserResultsListener;
+import org.apache.drill.exec.server.Drillbit;
+import org.apache.drill.exec.server.RemoteServiceSet;
+import org.apache.drill.exec.store.parquet.TestFileGenerator.FieldInfo;
+import org.apache.drill.exec.vector.BaseDataValueVector;
+import org.apache.drill.exec.vector.ValueVector;
+import org.junit.Test;
+
+import parquet.bytes.BytesInput;
+import parquet.column.page.Page;
+import parquet.column.page.PageReadStore;
+import parquet.column.page.PageReader;
+import parquet.hadoop.Footer;
+import parquet.hadoop.metadata.ParquetMetadata;
+import parquet.schema.MessageType;
+
+import com.google.common.base.Charsets;
+import com.google.common.collect.Lists;
+import com.google.common.io.Files;
+import com.google.common.util.concurrent.SettableFuture;
+
+public class ParquetRecordReaderTest {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ParquetRecordReaderTest.class);
+
+  private boolean VERBOSE_DEBUG = false;
+
+  
+  public static void main(String[] args) throws Exception{
+    new ParquetRecordReaderTest().testMultipleRowGroupsAndReadsEvent();
+  }
+
+  @Test
+  public void testMultipleRowGroupsAndReadsEvent() throws Exception {
+    String planName = "/parquet_scan_screen.json";
+    String fileName = "/tmp/testParquetFile_many_types_3";
+    int numberRowGroups = 20;
+    int recordsPerRowGroup = 300000;
+    //TestFileGenerator.generateParquetFile(fileName, numberRowGroups, recordsPerRowGroup);
+    testParquetFullEngineLocal(planName, fileName, 2, numberRowGroups, recordsPerRowGroup);
+  }
+
+  private class ParquetResultListener implements UserResultsListener {
+    private Vector<QueryResultBatch> results = new Vector<QueryResultBatch>();
+    private SettableFuture<Void> future = SettableFuture.create();
+    int count = 0;
+    RecordBatchLoader batchLoader;
+    byte[] bytes;
+
+    int numberRowGroups;
+    int numberOfTimesRead;
+    int batchCounter = 1;
+    int columnValCounter = 0;
+    int i = 0;
+    private FieldInfo currentField;
+    private final HashMap<String, Long> valuesChecked = new HashMap<>();
+    private final int recordsPerRowGroup;
+    private final Map<String, FieldInfo> fields;
+    private final long totalRecords;
+    
+    ParquetResultListener(int recordsPerRowGroup, RecordBatchLoader batchLoader, int numberRowGroups, int numberOfTimesRead){
+      this.batchLoader = batchLoader;
+      this.fields = TestFileGenerator.getFieldMap(recordsPerRowGroup);
+      this.recordsPerRowGroup = recordsPerRowGroup;
+      this.numberRowGroups = numberRowGroups;
+      this.numberOfTimesRead = numberOfTimesRead;
+      this.totalRecords = recordsPerRowGroup * numberRowGroups * numberOfTimesRead;
+    }
+
+    @Override
+    public void submissionFailed(RpcException ex) {
+      logger.debug("Submission failed.", ex);
+      future.setException(ex);
+    }
+
+    @Override
+    public void resultArrived(QueryResultBatch result) {
+      long columnValCounter = 0;
+      int i = 0;
+      FieldInfo currentField;
+      count += result.getHeader().getRowCount();
+      boolean schemaChanged = false;
+      try {
+        schemaChanged = batchLoader.load(result.getHeader().getDef(), result.getData());
+      } catch (SchemaChangeException e) {
+        logger.error("Failure while loading batch", e);
+      }
+
+      int recordCount = 0;
+      // print headers.
+      if (schemaChanged) {
+      } // do not believe any change is needed for when the schema changes, with the current mock scan use case
+
+      for (VectorWrapper vw : batchLoader) {
+        ValueVector vv = vw.getValueVector();
+        currentField = fields.get(vv.getField().getName());
+        if (VERBOSE_DEBUG){
+          System.out.println("\n" + (String) currentField.name);
+        }
+        if ( ! valuesChecked.containsKey(vv.getField().getName())){
+          valuesChecked.put(vv.getField().getName(), (long) 0);
+          columnValCounter = 0;
+        } else {
+          columnValCounter = valuesChecked.get(vv.getField().getName());
+        }
+        for (int j = 0; j < ((BaseDataValueVector)vv).getAccessor().getValueCount(); j++) {
+          if (VERBOSE_DEBUG){
+            System.out.print(vv.getAccessor().getObject(j) + ", " + (j % 25 == 0 ? "\n batch:" + batchCounter + " v:" + j + " - " : ""));
+          }
+          assertField(vv, j, (TypeProtos.MinorType) currentField.type,
+              currentField.values[(int) (columnValCounter % 3)], (String) currentField.name + "/");
+          columnValCounter++;
+        }
+        if (VERBOSE_DEBUG){
+          System.out.println("\n" + ((BaseDataValueVector)vv).getAccessor().getValueCount());
+        }
+        valuesChecked.remove(vv.getField().getName());
+        valuesChecked.put(vv.getField().getName(), columnValCounter);
+      }
+      
+      
+      if (VERBOSE_DEBUG){
+        for (i = 0; i < batchLoader.getRecordCount(); i++) {
+          recordCount++;
+          if (i % 50 == 0){
+            System.out.println();
+            for (VectorWrapper<?> vw : batchLoader) {
+              ValueVector v = vw.getValueVector();
+              System.out.print(pad(v.getField().getName(), 20) + " ");
+            }
+            System.out.println();
+            System.out.println();
+          }
+
+          for (VectorWrapper<?> vw : batchLoader) {
+            ValueVector v = vw.getValueVector();
+            System.out.print(pad(v.getAccessor().getObject(i).toString(), 20) + " ");
+          }
+          System.out.println(
+
+          );
+        }
+      }
+
+      for(VectorWrapper<?> vw : batchLoader){
+        vw.release();
+      }
+      result.release();
+      
+      batchCounter++;
+      if(result.getHeader().getIsLastChunk()){
+        for (String s : valuesChecked.keySet()) {
+          assertEquals("Record count incorrect for column: " + s, totalRecords, (long) valuesChecked.get(s));
+        }
+        
+        assert valuesChecked.keySet().size() > 0;
+        future.set(null);
+      }
+    }
+
+    public void get() throws RpcException{
+      try{
+        future.get();
+        return;
+      }catch(Throwable t){
+        throw RpcException.mapException(t);
+      }
+    }
+  }
+
+  
+  public void testParquetFullEngineRemote(String plan, String filename, int numberOfTimesRead /* specified in json plan */, int numberOfRowGroups, int recordsPerRowGroup) throws Exception{
+    
+    DrillConfig config = DrillConfig.create();
+
+    try(DrillClient client = new DrillClient(config);){
+      client.connect();
+      RecordBatchLoader batchLoader = new RecordBatchLoader(client.getAllocator());
+      ParquetResultListener resultListener = new ParquetResultListener(recordsPerRowGroup, batchLoader, numberOfRowGroups, numberOfTimesRead);
+      client.runQuery(UserProtos.QueryType.LOGICAL, Files.toString(FileUtils.getResourceAsFile(plan), Charsets.UTF_8), resultListener);
+      resultListener.get();
+    }
+    
+  }
+  
+  // specific tests should call this method, but it is not marked as a test itself intentionally
+  public void testParquetFullEngineLocal(String plan, String filename, int numberOfTimesRead /* specified in json plan */, int numberOfRowGroups, int recordsPerRowGroup) throws Exception{
+    
+    RemoteServiceSet serviceSet = RemoteServiceSet.getLocalServiceSet();
+
+    DrillConfig config = DrillConfig.create();
+
+    try(Drillbit bit1 = new Drillbit(config, serviceSet); DrillClient client = new DrillClient(config, serviceSet.getCoordinator());){
+      bit1.run();
+      client.connect();
+      RecordBatchLoader batchLoader = new RecordBatchLoader(client.getAllocator());
+      ParquetResultListener resultListener = new ParquetResultListener(recordsPerRowGroup, batchLoader, numberOfRowGroups, numberOfTimesRead);
+      client.runQuery(UserProtos.QueryType.LOGICAL, Files.toString(FileUtils.getResourceAsFile(plan), Charsets.UTF_8), resultListener);
+      resultListener.get();
+    }
+    
+  }
+
+
+
+  public String pad(String value, int length) {
+    return pad(value, length, " ");
+  }
+
+  public String pad(String value, int length, String with) {
+    StringBuilder result = new StringBuilder(length);
+    result.append(value);
+
+    while (result.length() < length) {
+      result.insert(0, with);
+    }
+
+    return result.toString();
+  }
+
+  class MockOutputMutator implements OutputMutator {
+    List<MaterializedField> removedFields = Lists.newArrayList();
+    List<ValueVector> addFields = Lists.newArrayList();
+
+    @Override
+    public void removeField(MaterializedField field) throws SchemaChangeException {
+      removedFields.add(field);
+    }
+
+    @Override
+    public void addField(ValueVector vector) throws SchemaChangeException {
+      addFields.add(vector);
+    }
+
+    @Override
+    public void removeAllFields() {
+      addFields.clear();
+    }
+
+    @Override
+    public void setNewSchema() throws SchemaChangeException {
+    }
+
+    List<MaterializedField> getRemovedFields() {
+      return removedFields;
+    }
+
+    List<ValueVector> getAddFields() {
+      return addFields;
+    }
+  }
+
+  private <T> void assertField(ValueVector valueVector, int index, TypeProtos.MinorType expectedMinorType, Object value, String name) {
+    assertField(valueVector, index, expectedMinorType, value, name, 0);
+  }
+
+  @SuppressWarnings("unchecked")
+  private <T> void assertField(ValueVector valueVector, int index, TypeProtos.MinorType expectedMinorType, T value, String name, int parentFieldId) {
+//    UserBitShared.FieldMetadata metadata = valueVector.getMetadata();
+//    SchemaDefProtos.FieldDef def = metadata.getDef();
+//    assertEquals(expectedMinorType, def.getMajorType().getMinorType());
+//    assertEquals(name, def.getNameList().get(0).getName());
+//    assertEquals(parentFieldId, def.getParentId());
+
+    if (expectedMinorType == TypeProtos.MinorType.MAP) {
+      return;
+    }
+    
+    T val = (T) valueVector.getAccessor().getObject(index);
+    if (val instanceof byte[]) {
+      assertTrue(Arrays.equals((byte[]) value, (byte[]) val));
+    } else {
+      assertEquals(value, val);
+    }
+  }
+
+  private void validateFooters(final List<Footer> metadata) {
+    logger.debug(metadata.toString());
+    assertEquals(3, metadata.size());
+    for (Footer footer : metadata) {
+      final File file = new File(footer.getFile().toUri());
+      assertTrue(file.getName(), file.getName().startsWith("part"));
+      assertTrue(file.getPath(), file.exists());
+      final ParquetMetadata parquetMetadata = footer.getParquetMetadata();
+      assertEquals(2, parquetMetadata.getBlocks().size());
+      final Map<String, String> keyValueMetaData = parquetMetadata.getFileMetaData().getKeyValueMetaData();
+      assertEquals("bar", keyValueMetaData.get("foo"));
+      assertEquals(footer.getFile().getName(), keyValueMetaData.get(footer.getFile().getName()));
+    }
+  }
+
+  private void validateContains(MessageType schema, PageReadStore pages, String[] path, int values, BytesInput bytes)
+      throws IOException {
+    PageReader pageReader = pages.getPageReader(schema.getColumnDescription(path));
+    Page page = pageReader.readPage();
+    assertEquals(values, page.getValueCount());
+    assertArrayEquals(bytes.toByteArray(), page.getBytes().toByteArray());
+  }
+
+  private String getResource(String resourceName) {
+    return "resource:" + resourceName;
+  }
+
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/402be7e0/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/TestFileGenerator.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/TestFileGenerator.java b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/TestFileGenerator.java
new file mode 100644
index 0000000..72f9123
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/TestFileGenerator.java
@@ -0,0 +1,210 @@
+package org.apache.drill.exec.store.parquet;
+
+import static parquet.column.Encoding.PLAIN;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.exec.store.ByteArrayUtil;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+
+import parquet.bytes.BytesInput;
+import parquet.column.ColumnDescriptor;
+import parquet.hadoop.ParquetFileWriter;
+import parquet.hadoop.metadata.CompressionCodecName;
+import parquet.schema.MessageType;
+import parquet.schema.MessageTypeParser;
+
+public class TestFileGenerator {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestFileGenerator.class);
+
+  
+
+  // 10 mb per page
+  static int bytesPerPage = 1024 * 1024 * 1;
+  // { 00000001, 00000010, 00000100, 00001000, 00010000, ... }
+  static byte[] bitFields = { 1, 2, 4, 8, 16, 32, 64, -128 };
+  static final byte allBitsTrue = -1;
+  static final byte allBitsFalse = 0;
+  static final byte[] varLen1 = { 50, 51, 52, 53, 54, 55, 56 };
+  static final byte[] varLen2 = { 15, 14, 13, 12, 11, 10, 9, 8, 7, 6, 5, 4, 3, 2, 1 };
+  static final byte[] varLen3 = { 100, 99, 98 };
+
+  static final Object[] intVals = { -200, 100, Integer.MAX_VALUE };
+  static final Object[] longVals = { -5000l, 5000l, Long.MAX_VALUE };
+  static final Object[] floatVals = { 1.74f, Float.MAX_VALUE, Float.MIN_VALUE };
+  static final Object[] doubleVals = { 100.45d, Double.MAX_VALUE, Double.MIN_VALUE, };
+  static final Object[] boolVals = { false, false, true };
+  static final Object[] binVals = { varLen1, varLen2, varLen3 };
+  static final Object[] bin2Vals = { varLen3, varLen2, varLen1 };
+
+  static class FieldInfo {
+
+    String parquetType;
+    String name;
+    int bitLength;
+    int numberOfPages;
+    Object[] values;
+    TypeProtos.MinorType type;
+
+    FieldInfo(int recordsPerRowGroup, String parquetType, String name, int bitLength, Object[] values, TypeProtos.MinorType type) {
+      this.parquetType = parquetType;
+      this.name = name;
+      this.bitLength = bitLength;
+      this.numberOfPages = Math.max(1, (int) Math.ceil(recordsPerRowGroup * bitLength / 8.0 / bytesPerPage));
+      this.values = values;
+      // generator is designed to use 3 values
+      assert values.length == 3;
+      this.type = type;
+    }
+  }
+
+  private static class WrapAroundCounter {
+
+    int maxVal;
+    int val;
+
+    public WrapAroundCounter(int maxVal) {
+      this.maxVal = maxVal;
+    }
+
+    public int increment() {
+      val++;
+      if (val > maxVal) {
+        val = 0;
+      }
+      return val;
+    }
+
+    public void reset() {
+      val = 0;
+    }
+
+  }
+
+  public static HashMap<String, FieldInfo> getFieldMap(int recordsPerRowGroup) {
+    HashMap<String, FieldInfo> fields = new HashMap<>();
+    fields.put("integer/", new FieldInfo(recordsPerRowGroup, "int32", "integer", 32, intVals, TypeProtos.MinorType.INT));
+    fields.put("bigInt/", new FieldInfo(recordsPerRowGroup, "int64", "bigInt", 64, longVals, TypeProtos.MinorType.BIGINT));
+    fields.put("f/", new FieldInfo(recordsPerRowGroup, "float", "f", 32, floatVals, TypeProtos.MinorType.FLOAT4));
+    fields.put("d/", new FieldInfo(recordsPerRowGroup, "double", "d", 64, doubleVals, TypeProtos.MinorType.FLOAT8));
+    // fields.put("b/", new FieldInfo("binary", "b", 1, boolVals, TypeProtos.MinorType.BIT));
+    fields.put("bin/", new FieldInfo(recordsPerRowGroup, "binary", "bin", -1, binVals, TypeProtos.MinorType.VARBINARY));
+    fields.put("bin2/", new FieldInfo(recordsPerRowGroup, "binary", "bin2", -1, bin2Vals, TypeProtos.MinorType.VARBINARY));
+    return fields;
+  }
+
+  public static void generateParquetFile(String filename, int numberRowGroups, int recordsPerRowGroup) throws Exception {
+    final Map<String, FieldInfo> fields = getFieldMap(recordsPerRowGroup);
+
+    int currentBooleanByte = 0;
+    WrapAroundCounter booleanBitCounter = new WrapAroundCounter(7);
+    Configuration configuration = new Configuration();
+    configuration.set(ParquetStorageEngine.HADOOP_DEFAULT_NAME, "file:///");
+    // "message m { required int32 integer; required int64 integer64; required boolean b; required float f; required double d;}"
+
+    FileSystem fs = FileSystem.get(configuration);
+    Path path = new Path(filename);
+    if (fs.exists(path))
+      fs.delete(path, false);
+
+    String messageSchema = "message m {";
+    for (FieldInfo fieldInfo : fields.values()) {
+      messageSchema += " required " + fieldInfo.parquetType + " " + fieldInfo.name + ";";
+    }
+    // remove the last semicolon, java really needs a join method for strings...
+    // TODO - nvm apparently it requires a semicolon after every field decl, might want to file a bug
+    // messageSchema = messageSchema.substring(schemaType, messageSchema.length() - 1);
+    messageSchema += "}";
+
+    MessageType schema = MessageTypeParser.parseMessageType(messageSchema);
+
+    CompressionCodecName codec = CompressionCodecName.UNCOMPRESSED;
+    ParquetFileWriter w = new ParquetFileWriter(configuration, schema, path);
+    w.start();
+    HashMap<String, Integer> columnValuesWritten = new HashMap();
+    int valsWritten;
+    for (int k = 0; k < numberRowGroups; k++) {
+      w.startBlock(1);
+
+      for (FieldInfo fieldInfo : fields.values()) {
+
+        if (!columnValuesWritten.containsKey(fieldInfo.name)) {
+          columnValuesWritten.put((String) fieldInfo.name, 0);
+          valsWritten = 0;
+        } else {
+          valsWritten = columnValuesWritten.get(fieldInfo.name);
+        }
+
+        String[] path1 = { (String) fieldInfo.name };
+        ColumnDescriptor c1 = schema.getColumnDescription(path1);
+
+        w.startColumn(c1, recordsPerRowGroup, codec);
+        int valsPerPage = (int) Math.ceil(recordsPerRowGroup / (float) ((int) fieldInfo.numberOfPages));
+        byte[] bytes;
+        // for variable length binary fields
+        int bytesNeededToEncodeLength = 4;
+        if ((int) fieldInfo.bitLength > 0) {
+          bytes = new byte[(int) Math.ceil(valsPerPage * (int) fieldInfo.bitLength / 8.0)];
+        } else {
+          // the twelve at the end is to account for storing a 4 byte length with each value
+          int totalValLength = ((byte[]) fieldInfo.values[0]).length + ((byte[]) fieldInfo.values[1]).length
+              + ((byte[]) fieldInfo.values[2]).length + 3 * bytesNeededToEncodeLength;
+          // used for the case where there is a number of values in this row group that is not divisible by 3
+          int leftOverBytes = 0;
+          if (valsPerPage % 3 > 0)
+            leftOverBytes += ((byte[]) fieldInfo.values[1]).length + 4;
+          if (valsPerPage % 3 > 1)
+            leftOverBytes += ((byte[]) fieldInfo.values[2]).length + 4;
+          bytes = new byte[valsPerPage / 3 * totalValLength + leftOverBytes];
+        }
+        int bytesPerPage = (int) (valsPerPage * ((int) fieldInfo.bitLength / 8.0));
+        int bytesWritten = 0;
+        for (int z = 0; z < (int) fieldInfo.numberOfPages; z++, bytesWritten = 0) {
+          for (int i = 0; i < valsPerPage; i++) {
+            // System.out.print(i + ", " + (i % 25 == 0 ? "\n gen " + fieldInfo.name + ": " : ""));
+            if (fieldInfo.values[0] instanceof Boolean) {
+
+              bytes[currentBooleanByte] |= bitFields[booleanBitCounter.val]
+                  & ((boolean) fieldInfo.values[valsWritten % 3] ? allBitsTrue : allBitsFalse);
+              booleanBitCounter.increment();
+              if (booleanBitCounter.val == 0) {
+                currentBooleanByte++;
+              }
+              valsWritten++;
+              if (currentBooleanByte > bytesPerPage)
+                break;
+            } else {
+              if (fieldInfo.values[valsWritten % 3] instanceof byte[]) {
+                System.arraycopy(ByteArrayUtil.toByta(((byte[]) fieldInfo.values[valsWritten % 3]).length), 0, bytes,
+                    bytesWritten, bytesNeededToEncodeLength);
+                System.arraycopy(fieldInfo.values[valsWritten % 3], 0, bytes, bytesWritten + bytesNeededToEncodeLength,
+                    ((byte[]) fieldInfo.values[valsWritten % 3]).length);
+                bytesWritten += ((byte[]) fieldInfo.values[valsWritten % 3]).length + bytesNeededToEncodeLength;
+              } else {
+                System.arraycopy(ByteArrayUtil.toByta(fieldInfo.values[valsWritten % 3]), 0, bytes, i
+                    * ((int) fieldInfo.bitLength / 8), (int) fieldInfo.bitLength / 8);
+              }
+              valsWritten++;
+            }
+
+          }
+          w.writeDataPage((int) (recordsPerRowGroup / (int) fieldInfo.numberOfPages), bytes.length,
+              BytesInput.from(bytes), PLAIN, PLAIN, PLAIN);
+          currentBooleanByte = 0;
+        }
+        w.endColumn();
+        columnValuesWritten.remove((String) fieldInfo.name);
+        columnValuesWritten.put((String) fieldInfo.name, valsWritten);
+      }
+
+      w.endBlock();
+    }
+    w.end(new HashMap<String, String>());
+    logger.debug("Finished generating parquet file.");
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/402be7e0/sandbox/prototype/exec/java-exec/src/test/resources/scan_screen_logical.json
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/resources/scan_screen_logical.json b/sandbox/prototype/exec/java-exec/src/test/resources/scan_screen_logical.json
index 90ba2c1..4f44f9e 100644
--- a/sandbox/prototype/exec/java-exec/src/test/resources/scan_screen_logical.json
+++ b/sandbox/prototype/exec/java-exec/src/test/resources/scan_screen_logical.json
@@ -7,12 +7,18 @@
       info:"na"
     }
   },
+  "storage" : {
+  "mock" : {
+      "type" : "mock"
+  }
+},
+  
   query:[
     {
       @id:"1",
       op:"scan",
       memo:"initial_scan",
-      storageengine:"local-logs",
+      storageengine:"mock",
       selection: [
        {
             records : 100,

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/402be7e0/sandbox/prototype/exec/java-exec/src/test/sh/logback.xml
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/sh/logback.xml b/sandbox/prototype/exec/java-exec/src/test/sh/logback.xml
new file mode 100644
index 0000000..6f2928a
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/test/sh/logback.xml
@@ -0,0 +1,35 @@
+<?xml version="1.0" encoding="UTF-8" ?>
+<configuration>
+	<appender name="SOCKET"
+		class="de.huxhorn.lilith.logback.appender.ClassicMultiplexSocketAppender">
+		<Compressing>true</Compressing>
+		<ReconnectionDelay>10000</ReconnectionDelay>
+		<IncludeCallerData>true</IncludeCallerData>
+		<RemoteHosts>localhost</RemoteHosts>
+	</appender>
+
+	<appender name="STDOUT" class="ch.qos.logback.core.ConsoleAppender">
+		<encoder>
+			<pattern>%d{HH:mm:ss.SSS} [%thread] %-5level %logger{36} - %msg%n
+			</pattern>
+		</encoder>
+	</appender>
+
+	<logger name="org.apache.drill" additivity="false">
+		<level value="info" />
+    <appender-ref ref="STDOUT" />
+<!--     <appender-ref ref="STDOUT" /> -->
+	</logger>
+  
+  <logger name="io.netty" additivity="false">
+    <level value="info" />
+    <appender-ref ref="SOCKET" />
+<!--     <appender-ref ref="STDOUT" /> -->
+  </logger>
+
+	<root>
+		<level value="error" />
+		<appender-ref ref="STDOUT" />
+	</root>  
+
+</configuration>

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/402be7e0/sandbox/prototype/exec/java-exec/src/test/sh/runbit
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/sh/runbit b/sandbox/prototype/exec/java-exec/src/test/sh/runbit
index 31d0729..8a54006 100755
--- a/sandbox/prototype/exec/java-exec/src/test/sh/runbit
+++ b/sandbox/prototype/exec/java-exec/src/test/sh/runbit
@@ -4,6 +4,6 @@ PROJECT_ROOT=../../../
 
 mvn dependency:build-classpath -f=$PROJECT_ROOT/pom.xml -Dmdep.outputFile=target/sh/cp.txt
 CP=`cat $PROJECT_ROOT/target/sh/cp.txt`
-CP=$CP:$PROJECT_ROOT/target/classes:$PROJECT_ROOT/target/test-classes
-java -XX:MaxDirectMemorySize=8192M  -cp $CP org.apache.drill.exec.server.Drillbit
+CP=./:$CP:$PROJECT_ROOT/target/classes:$PROJECT_ROOT/target/test-classes
+java -Dlogback.configurationFile=logback.xml -XX:MaxDirectMemorySize=8192M  -cp $CP org.apache.drill.exec.server.Drillbit 
 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/402be7e0/sandbox/prototype/pom.xml
----------------------------------------------------------------------
diff --git a/sandbox/prototype/pom.xml b/sandbox/prototype/pom.xml
index 52b2250..382c5ff 100644
--- a/sandbox/prototype/pom.xml
+++ b/sandbox/prototype/pom.xml
@@ -56,6 +56,15 @@
       <plugin>
         <groupId>org.apache.maven.plugins</groupId>
         <artifactId>maven-jar-plugin</artifactId>
+        <configuration>
+          <excludes>
+            <exclude>**/logging.properties</exclude>
+            <exclude>**/logback-test.xml</exclude>
+            <exclude>**/logback.out.xml</exclude>
+            <exclude>**/logback.xml</exclude>
+          </excludes>
+        </configuration>
+        
         <executions>
           <execution>
             <goals>
@@ -64,6 +73,7 @@
           </execution>
         </executions>
       </plugin>
+      
       <plugin>
         <groupId>org.apache.maven.plugins</groupId>
         <artifactId>maven-resources-plugin</artifactId>


[26/27] git commit: Fixes for Partitioner merge

Posted by ja...@apache.org.
Fixes for Partitioner merge


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

Branch: refs/heads/master
Commit: 8cd9cacd7f116c9ba5d18b21c288eba9b14fd786
Parents: b32c9ed
Author: Jacques Nadeau <ja...@apache.org>
Authored: Thu Aug 15 18:37:07 2013 -0700
Committer: Jacques Nadeau <ja...@apache.org>
Committed: Thu Aug 15 18:37:07 2013 -0700

----------------------------------------------------------------------
 .../org/apache/drill/exec/physical/impl/ImplCreator.java    | 6 +-----
 .../drill/exec/work/RemoteFragmentRunnerListener.java       | 9 +++------
 .../org/apache/drill/exec/work/batch/BitComHandlerImpl.java | 5 +----
 3 files changed, 5 insertions(+), 15 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/8cd9cacd/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ImplCreator.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ImplCreator.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ImplCreator.java
index 8207304..2ce5c28 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ImplCreator.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ImplCreator.java
@@ -26,11 +26,8 @@ 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.SubScan;
-import org.apache.drill.exec.physical.base.Scan;
-import org.apache.drill.exec.physical.config.HashPartitionSender;
 import org.apache.drill.exec.physical.config.Filter;
-import org.apache.drill.exec.physical.config.MockScanBatchCreator;
-import org.apache.drill.exec.physical.config.MockScanPOP;
+import org.apache.drill.exec.physical.config.HashPartitionSender;
 import org.apache.drill.exec.physical.config.Project;
 import org.apache.drill.exec.physical.config.RandomReceiver;
 import org.apache.drill.exec.physical.config.Screen;
@@ -45,7 +42,6 @@ import org.apache.drill.exec.physical.impl.svremover.SVRemoverCreator;
 import org.apache.drill.exec.record.RecordBatch;
 import org.apache.drill.exec.store.json.JSONScanBatchCreator;
 import org.apache.drill.exec.store.json.JSONSubScan;
-import org.apache.drill.exec.store.mock.MockGroupScanPOP;
 import org.apache.drill.exec.store.mock.MockScanBatchCreator;
 import org.apache.drill.exec.store.mock.MockSubScanPOP;
 import org.apache.drill.exec.store.parquet.ParquetRowGroupScan;

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/8cd9cacd/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/RemoteFragmentRunnerListener.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/RemoteFragmentRunnerListener.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/RemoteFragmentRunnerListener.java
index 48d7f5d..f7d4669 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/RemoteFragmentRunnerListener.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/RemoteFragmentRunnerListener.java
@@ -20,20 +20,17 @@ 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);
+public class RemoteFragmentRunnerListener extends AbstractFragmentRunnerListener{
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(RemoteFragmentRunnerListener.class);
   
   private final BitTunnel tunnel;
 
-  public RemotingFragmentRunnerListener(FragmentContext context, BitTunnel tunnel) {
+  public RemoteFragmentRunnerListener(FragmentContext context, BitTunnel tunnel) {
     super(context);
     this.tunnel = tunnel;
   }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/8cd9cacd/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/BitComHandlerImpl.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/BitComHandlerImpl.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/BitComHandlerImpl.java
index b6e0159..5807c87 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/BitComHandlerImpl.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/BitComHandlerImpl.java
@@ -30,8 +30,6 @@ 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;
@@ -49,12 +47,11 @@ 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.RemoteFragmentRunnerListener;
-import org.apache.drill.exec.work.fragment.RemoteFragmentHandler;
 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);


[10/27] Initial Parquet commit. Suports INT, LONG, FLOAT, DOUBLE, distributed scheduling.

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/0a2f997f/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/SingleSenderCreator.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/SingleSenderCreator.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/SingleSenderCreator.java
index bad3a03..a40031e 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/SingleSenderCreator.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/SingleSenderCreator.java
@@ -54,6 +54,7 @@ public class SingleSenderCreator implements RootCreator<SingleSender>{
     
     public SingleSenderRootExec(FragmentContext context, RecordBatch batch, SingleSender config){
       this.incoming = batch;
+      assert(incoming != null);
       this.handle = context.getHandle();
       this.recMajor = config.getOppositeMajorFragmentId();
       this.tunnel = context.getCommunicator().getTunnel(config.getDestination());
@@ -74,12 +75,14 @@ public class SingleSenderCreator implements RootCreator<SingleSender>{
       case NONE:
         FragmentWritableBatch b2 = new FragmentWritableBatch(true, handle.getQueryId(), handle.getMajorFragmentId(), handle.getMinorFragmentId(), recMajor, 0, incoming.getWritableBatch());
         tunnel.sendRecordBatch(new RecordSendFailure(), context, b2);
+        b2.release();
         return false;
 
       case OK_NEW_SCHEMA:
       case OK:
         FragmentWritableBatch batch = new FragmentWritableBatch(false, handle.getQueryId(), handle.getMajorFragmentId(), handle.getMinorFragmentId(), recMajor, 0, incoming.getWritableBatch());
         tunnel.sendRecordBatch(new RecordSendFailure(), context, batch);
+        batch.release();
         return true;
 
       case NOT_YET:

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/0a2f997f/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 1148c93..b04e154 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
@@ -36,6 +36,8 @@ 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;
+import org.apache.drill.exec.server.DrillbitContext;
+import org.apache.drill.exec.store.StorageEngineRegistry;
 
 public class PhysicalPlanReader {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(PhysicalPlanReader.class);
@@ -45,7 +47,8 @@ public class PhysicalPlanReader {
   private final ObjectReader operatorReader;
   private final ObjectReader logicalPlanReader;
 
-  public PhysicalPlanReader(DrillConfig config, ObjectMapper mapper, final DrillbitEndpoint endpoint) {
+  public PhysicalPlanReader(DrillConfig config, ObjectMapper mapper, final DrillbitEndpoint endpoint,
+                            final StorageEngineRegistry engineRegistry) {
 
     // Endpoint serializer/deserializer.
     SimpleModule deserModule = new SimpleModule("PhysicalOperatorModule") //
@@ -58,6 +61,7 @@ public class PhysicalPlanReader {
     mapper.registerModule(deserModule);
     mapper.registerSubtypes(PhysicalOperatorUtil.getSubTypes(config));
     InjectableValues injectables = new InjectableValues.Std() //
+            .addValue(StorageEngineRegistry.class, engineRegistry) //
         .addValue(DrillbitEndpoint.class, endpoint); //
 
     this.mapper = mapper;
@@ -66,6 +70,14 @@ public class PhysicalPlanReader {
     this.logicalPlanReader = mapper.reader(LogicalPlan.class).with(injectables);
   }
 
+  // TODO - we do not want to storage engine registry generated here in production, this was created to keep old
+  // tests passing, this constructor should be removed and the tests should be updated to use the contstructor
+  // that takes a storage engine registry
+  @Deprecated
+  public PhysicalPlanReader(DrillConfig config, ObjectMapper mapper, final DrillbitEndpoint endpoint) {
+    this(config, mapper, endpoint, null);
+  }
+
   public String writeJson(PhysicalOperator op) throws JsonProcessingException{
     return mapper.writeValueAsString(op);
   }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/0a2f997f/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
index 4188435..d3c8cee 100644
--- 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
@@ -21,6 +21,7 @@ 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.SubScan;
 
 /**
  * Responsible for breaking a plan into its constituent Fragments.
@@ -42,7 +43,13 @@ public class MakeFragmentsVisitor extends AbstractPhysicalVisitor<Fragment, Frag
     exchange.getChild().accept(this, next);
     return value;
   }
-  
+
+  @Override
+  public Fragment visitSubScan(SubScan subScan, Fragment value) throws FragmentSetupException {
+    // TODO - implement this
+    return super.visitOp(subScan, value);
+  }
+
   @Override
   public Fragment visitOp(PhysicalOperator op, Fragment value)  throws FragmentSetupException{
 //    logger.debug("Visiting Other {}", op);

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/0a2f997f/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/Materializer.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/Materializer.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/Materializer.java
index da71271..ab91d76 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/Materializer.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/Materializer.java
@@ -22,11 +22,8 @@ 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 org.apache.drill.exec.physical.base.*;
+import org.apache.drill.exec.physical.base.GroupScan;
 
 import com.google.common.collect.Lists;
 
@@ -53,8 +50,14 @@ public class Materializer extends AbstractPhysicalVisitor<PhysicalOperator, Mate
   }
 
   @Override
-  public PhysicalOperator visitScan(Scan<?> scan, IndexedFragmentNode iNode) throws ExecutionSetupException {
-    return scan.getSpecificScan(iNode.getMinorFragmentId());
+  public PhysicalOperator visitGroupScan(GroupScan groupScan, IndexedFragmentNode iNode) throws ExecutionSetupException {
+    return groupScan.getSpecificScan(iNode.getMinorFragmentId());
+  }
+
+  @Override
+  public PhysicalOperator visitSubScan(SubScan subScan, IndexedFragmentNode value) throws ExecutionSetupException {
+    // TODO - implement this
+    return super.visitOp(subScan, value);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/0a2f997f/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/SimpleParallelizer.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/SimpleParallelizer.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/SimpleParallelizer.java
index e3bcff0..2ccb17c 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/SimpleParallelizer.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/SimpleParallelizer.java
@@ -127,9 +127,11 @@ public class SimpleParallelizer {
             .build();
 
         if (isRootNode) {
+          logger.debug("Root fragment {}", fragment);
           rootFragment = fragment;
           rootOperator = root;
         } else {
+          logger.debug("Remote fragment {}", fragment);
           fragments.add(fragment);
         }
       }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/0a2f997f/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/StatsCollector.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/StatsCollector.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/StatsCollector.java
index af8ec04..2ef5295 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/StatsCollector.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/StatsCollector.java
@@ -17,11 +17,8 @@
  ******************************************************************************/
 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.physical.base.*;
+import org.apache.drill.exec.physical.base.GroupScan;
 import org.apache.drill.exec.planner.AbstractOpWrapperVisitor;
 import org.apache.drill.exec.planner.fragment.Fragment.ExchangeFragmentPair;
 
@@ -75,10 +72,16 @@ public class StatsCollector {
     }
 
     @Override
-    public Void visitScan(Scan<?> scan, Wrapper wrapper) {
+    public Void visitGroupScan(GroupScan groupScan, Wrapper wrapper) {
       Stats stats = wrapper.getStats();      
-      stats.addMaxWidth(scan.getReadEntries().size());
-      return super.visitScan(scan, wrapper);
+      stats.addMaxWidth(groupScan.getMaxParallelizationWidth());
+      return super.visitGroupScan(groupScan, wrapper);
+    }
+
+    @Override
+    public Void visitSubScan(SubScan subScan, Wrapper value) throws RuntimeException {
+      // TODO - implement this
+      return super.visitOp(subScan, value);
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/0a2f997f/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/Wrapper.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/Wrapper.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/Wrapper.java
index 0dfcb62..d5a24b0 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/Wrapper.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/Wrapper.java
@@ -25,12 +25,10 @@ import java.util.concurrent.ThreadLocalRandom;
 
 import org.apache.commons.lang.NotImplementedException;
 import org.apache.drill.common.exceptions.PhysicalOperatorSetupException;
+import org.apache.drill.exec.exception.FragmentSetupException;
 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.physical.base.*;
+import org.apache.drill.exec.physical.base.GroupScan;
 import org.apache.drill.exec.planner.fragment.Fragment.ExchangeFragmentPair;
 import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
 
@@ -113,9 +111,15 @@ public class Wrapper {
     }
 
     @Override
-    public Void visitScan(Scan<?> scan, List<DrillbitEndpoint> value) throws PhysicalOperatorSetupException {
-      scan.applyAssignments(value);
-      return super.visitScan(scan, value);
+    public Void visitGroupScan(GroupScan groupScan, List<DrillbitEndpoint> value) throws PhysicalOperatorSetupException {
+      groupScan.applyAssignments(value);
+      return super.visitGroupScan(groupScan, value);
+    }
+
+    @Override
+    public Void visitSubScan(SubScan subScan, List<DrillbitEndpoint> value) throws PhysicalOperatorSetupException {
+      // TODO - implement this
+      return visitOp(subScan, value);
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/0a2f997f/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
index c19065d..964ef5c 100644
--- 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
@@ -55,5 +55,11 @@ public class FragmentWritableBatch{
     return header;
   }
   
+  public void release(){
+    for(ByteBuf b : buffers){
+      b.release();
+    }
+  }
+  
   
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/0a2f997f/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatchLoader.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatchLoader.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatchLoader.java
index 593c28c..4d47404 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatchLoader.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatchLoader.java
@@ -6,9 +6,9 @@
  * to you under the Apache License, Version 2.0 (the
  * "License"); you may not use this file except in compliance
  * with the License.  You may obtain a copy of the License at
- * 
+ *
  * http://www.apache.org/licenses/LICENSE-2.0
- * 
+ *
  * Unless required by applicable law or agreed to in writing, software
  * distributed under the License is distributed on an "AS IS" BASIS,
  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
@@ -41,9 +41,9 @@ public class RecordBatchLoader implements Iterable<VectorWrapper<?>>{
 
   private VectorContainer container = new VectorContainer();
   private final BufferAllocator allocator;
-  private int recordCount; 
+  private int valueCount;
   private BatchSchema schema;
-  
+
   public RecordBatchLoader(BufferAllocator allocator) {
     super();
     this.allocator = allocator;
@@ -51,18 +51,18 @@ public class RecordBatchLoader implements Iterable<VectorWrapper<?>>{
 
   /**
    * Load a record batch from a single buffer.
-   * 
+   *
    * @param def
    *          The definition for the record batch.
    * @param buf
    *          The buffer that holds the data associated with the record batch
    * @return Whether or not the schema changed since the previous load.
-   * @throws SchemaChangeException 
+   * @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;
+    this.valueCount = def.getRecordCount();
+    boolean schemaChanged = schema == null;
 
     Map<MaterializedField, ValueVector> oldFields = Maps.newHashMap();
     for(VectorWrapper<?> w : container){
@@ -73,7 +73,7 @@ public class RecordBatchLoader implements Iterable<VectorWrapper<?>>{
     VectorContainer newVectors = new VectorContainer();
 
     List<FieldMetadata> fields = def.getFieldList();
-    
+
     int bufOffset = 0;
     for (FieldMetadata fmd : fields) {
       FieldDef fieldDef = fmd.getDef();
@@ -82,23 +82,27 @@ public class RecordBatchLoader implements Iterable<VectorWrapper<?>>{
         container.add(v);
         continue;
       }
-      
+
       // if we arrive here, we didn't have a matching vector.
       schemaChanged = true;
       MaterializedField m = new MaterializedField(fieldDef);
       v = TypeHelper.getNewVector(m, allocator);
-      v.load(fmd, buf.slice(bufOffset, fmd.getBufferLength()));
+      if (fmd.getValueCount() == 0){
+        v.clear();
+      } else {
+        v.load(fmd, buf.slice(bufOffset, fmd.getBufferLength()));
+      }
       bufOffset += fmd.getBufferLength();
       newVectors.add(v);
     }
-    
+
     if(!oldFields.isEmpty()){
       schemaChanged = true;
       for(ValueVector v : oldFields.values()){
         v.close();
       }
     }
-    
+
     // rebuild the schema.
     SchemaBuilder b = BatchSchema.newBuilder();
     for(VectorWrapper<?> v : newVectors){
@@ -132,7 +136,7 @@ public class RecordBatchLoader implements Iterable<VectorWrapper<?>>{
 //  }
 
   public int getRecordCount() {
-    return recordCount;
+    return valueCount;
   }
 
   public VectorWrapper<?> getValueAccessorById(int fieldId, Class<?> clazz){
@@ -140,7 +144,7 @@ public class RecordBatchLoader implements Iterable<VectorWrapper<?>>{
   }
   
   public WritableBatch getWritableBatch(){
-    return WritableBatch.getBatchNoSVWrap(recordCount, container);
+    return WritableBatch.getBatchNoSVWrap(valueCount, container);
   }
 
   @Override
@@ -152,6 +156,6 @@ public class RecordBatchLoader implements Iterable<VectorWrapper<?>>{
     return schema;
   }
 
-  
-  
+
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/0a2f997f/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/WritableBatch.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/WritableBatch.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/WritableBatch.java
index e84bf37..cac042b 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/WritableBatch.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/WritableBatch.java
@@ -71,7 +71,6 @@ public class WritableBatch {
     List<ByteBuf> buffers = Lists.newArrayList();
     List<FieldMetadata> metadata = Lists.newArrayList();
 
-    
     for (ValueVector vv : vectors) {
       metadata.add(vv.getMetadata());
       

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/0a2f997f/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 6c0158a..8c8b6b9 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
@@ -19,6 +19,7 @@ package org.apache.drill.exec.rpc;
 
 import io.netty.buffer.ByteBuf;
 import io.netty.buffer.ByteBufOutputStream;
+import io.netty.buffer.CompositeByteBuf;
 import io.netty.channel.ChannelHandlerContext;
 import io.netty.handler.codec.MessageToMessageEncoder;
 
@@ -28,6 +29,7 @@ import java.util.List;
 import org.apache.drill.exec.proto.GeneralRPCProtos.CompleteRpcMessage;
 import org.apache.drill.exec.proto.GeneralRPCProtos.RpcHeader;
 
+import com.google.common.base.Preconditions;
 import com.google.protobuf.CodedOutputStream;
 import com.google.protobuf.WireFormat;
 
@@ -103,10 +105,16 @@ class RpcEncoder extends MessageToMessageEncoder<OutboundRpcMessage>{
         cos.writeRawVarint32(rawBodyLength);
         cos.flush(); // need to flush so that dbody goes after if cos is caching.
         
-        out.add(buf);
+        CompositeByteBuf cbb = new CompositeByteBuf(buf.alloc(), true, msg.dBodies.length + 1);
+        cbb.addComponent(buf);
+        int bufLength = buf.readableBytes();
         for(ByteBuf b : msg.dBodies){
-          out.add(b);
+          cbb.addComponent(b);
+          bufLength += b.readableBytes();
         }
+        cbb.writerIndex(bufLength);
+        out.add(cbb);
+        
         
       }else{
         cos.flush();

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/0a2f997f/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 c933594..eb160be 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
@@ -26,6 +26,7 @@ import org.apache.drill.common.logical.StorageEngineConfig;
 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.exception.SetupException;
 import org.apache.drill.exec.memory.BufferAllocator;
 import org.apache.drill.exec.planner.PhysicalPlanReader;
 import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
@@ -35,6 +36,7 @@ import org.apache.drill.exec.store.StorageEngine;
 
 import com.google.common.base.Preconditions;
 import com.yammer.metrics.MetricRegistry;
+import org.apache.drill.exec.store.StorageEngineRegistry;
 
 public class DrillbitContext {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillbitContext.class);
@@ -46,6 +48,7 @@ public class DrillbitContext {
   private final BitCom com;
   private final DistributedCache cache;
   private final DrillbitEndpoint endpoint;
+  private final StorageEngineRegistry storageEngineRegistry;
   
   public DrillbitContext(DrillbitEndpoint endpoint, BootStrapContext context, ClusterCoordinator coord, BitCom com, DistributedCache cache) {
     super();
@@ -58,7 +61,8 @@ public class DrillbitContext {
     this.com = com;
     this.cache = cache;
     this.endpoint = endpoint;
-    this.reader = new PhysicalPlanReader(context.getConfig(), context.getConfig().getMapper(), endpoint);
+    this.storageEngineRegistry = new StorageEngineRegistry(this);
+    this.reader = new PhysicalPlanReader(context.getConfig(), context.getConfig().getMapper(), endpoint, storageEngineRegistry);
   }
   
   public DrillbitEndpoint getEndpoint(){
@@ -77,8 +81,8 @@ public class DrillbitContext {
     return context.getAllocator();
   }
   
-  public StorageEngine getStorageEngine(StorageEngineConfig config){
-    throw new UnsupportedOperationException();
+  public StorageEngine getStorageEngine(StorageEngineConfig config) throws SetupException {
+    return storageEngineRegistry.getEngine(config);
   }
   
   public NioEventLoopGroup getBitLoopGroup(){

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/0a2f997f/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 b07f274..bb1f2e1 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
@@ -44,6 +44,7 @@ public class ServiceEngine implements Closeable{
   private final UserServer userServer;
   private final BitCom bitCom;
   private final DrillConfig config;
+  boolean useIP = false;
   
   public ServiceEngine(BitComHandler bitComWorker, UserWorker userWorker, BootStrapContext context){
     this.userServer = new UserServer(context.getAllocator().getUnderlyingAllocator(), new NioEventLoopGroup(1, new NamedThreadFactory("UserServer-")), userWorker);
@@ -53,8 +54,10 @@ public class ServiceEngine implements Closeable{
   
   public DrillbitEndpoint start() throws DrillbitStartupException, InterruptedException, UnknownHostException{
     int userPort = userServer.bind(config.getInt(ExecConstants.INITIAL_USER_PORT));
+    String address = useIP ?  InetAddress.getLocalHost().getHostAddress() : InetAddress.getLocalHost().getCanonicalHostName();
     DrillbitEndpoint partialEndpoint = DrillbitEndpoint.newBuilder()
-        .setAddress(InetAddress.getLocalHost().getHostAddress())
+        .setAddress(address)
+        //.setAddress("localhost")
         .setUserPort(userPort)
         .build();
 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/0a2f997f/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 80704fa..9c48052 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
@@ -24,6 +24,8 @@ import java.util.List;
 
 import org.apache.drill.common.logical.data.Scan;
 import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.physical.ReadEntry;
+import org.apache.drill.exec.physical.base.AbstractGroupScan;
 import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
 
 import com.google.common.collect.ListMultimap;
@@ -48,7 +50,7 @@ public class AbstractStorageEngine implements StorageEngine{
   }
 
   @Override
-  public Collection<ReadEntry> getReadEntries(Scan scan) throws IOException {
+  public AbstractGroupScan getPhysicalScan(Scan scan) throws IOException {
     throw new UnsupportedOperationException();
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/0a2f997f/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/AffinityCalculator.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/AffinityCalculator.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/AffinityCalculator.java
new file mode 100644
index 0000000..b4092cc
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/AffinityCalculator.java
@@ -0,0 +1,112 @@
+package org.apache.drill.exec.store;
+
+
+import com.google.common.collect.ImmutableRangeMap;
+import com.google.common.collect.Range;
+import org.apache.drill.exec.store.parquet.ParquetGroupScan;
+
+import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
+import org.apache.hadoop.fs.BlockLocation;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+
+import java.io.IOException;
+import java.util.*;
+
+public class AffinityCalculator {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(AffinityCalculator.class);
+
+
+  BlockLocation[] blocks;
+  ImmutableRangeMap<Long,BlockLocation> blockMap;
+  FileSystem fs;
+  String fileName;
+  Collection<DrillbitEndpoint> endpoints;
+  HashMap<String,DrillbitEndpoint> endPointMap;
+
+  public AffinityCalculator(String fileName, FileSystem fs, Collection<DrillbitEndpoint> endpoints) {
+    this.fs = fs;
+    this.fileName = fileName;
+    this.endpoints = endpoints;
+    buildBlockMap();
+    buildEndpointMap();
+  }
+
+  private void buildBlockMap() {
+    try {
+      FileStatus file = fs.getFileStatus(new Path(fileName));
+      long tC = System.nanoTime();
+      blocks = fs.getFileBlockLocations(file, 0 , file.getLen());
+      long tD = System.nanoTime();
+      logger.debug("Block locations: {}", blocks);
+      logger.debug("Took {} ms to get Block locations", (float)(tD - tC) / 1e6);
+    } catch (IOException ioe) { throw new RuntimeException(ioe); }
+    long tA = System.nanoTime();
+    ImmutableRangeMap.Builder<Long, BlockLocation> blockMapBuilder = new ImmutableRangeMap.Builder<Long,BlockLocation>();
+    for (BlockLocation block : blocks) {
+      long start = block.getOffset();
+      long end = start + block.getLength();
+      Range<Long> range = Range.closedOpen(start, end);
+      blockMapBuilder = blockMapBuilder.put(range, block);
+    }
+    blockMap = blockMapBuilder.build();
+    long tB = System.nanoTime();
+    logger.debug("Took {} ms to build block map", (float)(tB - tA) / 1e6);
+  }
+  /**
+   *
+   * @param entry
+   */
+  public void setEndpointBytes(ParquetGroupScan.RowGroupInfo entry) {
+    long tA = System.nanoTime();
+    HashMap<String,Long> hostMap = new HashMap<>();
+    long start = entry.getStart();
+    long end = start + entry.getLength();
+    Range<Long> entryRange = Range.closedOpen(start, end);
+    ImmutableRangeMap<Long,BlockLocation> subRangeMap = blockMap.subRangeMap(entryRange);
+    for (Map.Entry<Range<Long>,BlockLocation> e : subRangeMap.asMapOfRanges().entrySet()) {
+      String[] hosts = null;
+      Range<Long> blockRange = e.getKey();
+      try {
+        hosts = e.getValue().getHosts();
+      } catch (IOException ioe) { /*TODO Handle this exception */}
+      Range<Long> intersection = entryRange.intersection(blockRange);
+      long bytes = intersection.upperEndpoint() - intersection.lowerEndpoint();
+      for (String host : hosts) {
+        if (hostMap.containsKey(host)) {
+          hostMap.put(host, hostMap.get(host) + bytes);
+        } else {
+          hostMap.put(host, bytes);
+        }
+      }
+    }
+    HashMap<DrillbitEndpoint,Long> ebs = new HashMap();
+    try {
+      for (Map.Entry<String,Long> hostEntry : hostMap.entrySet()) {
+        String host = hostEntry.getKey();
+        Long bytes = hostEntry.getValue();
+        DrillbitEndpoint d = getDrillBitEndpoint(host);
+        if (d != null ) ebs.put(d, bytes);
+      }
+    } catch (NullPointerException n) {}
+    entry.setEndpointBytes(ebs);
+    long tB = System.nanoTime();
+    logger.debug("Took {} ms to set endpoint bytes", (float)(tB - tA) / 1e6);
+  }
+
+  private DrillbitEndpoint getDrillBitEndpoint(String hostName) {
+    return endPointMap.get(hostName);
+  }
+
+  private void buildEndpointMap() {
+    long tA = System.nanoTime();
+    endPointMap = new HashMap<String, DrillbitEndpoint>();
+    for (DrillbitEndpoint d : endpoints) {
+      String hostName = d.getAddress();
+      endPointMap.put(hostName, d);
+    }
+    long tB = System.nanoTime();
+    logger.debug("Took {} ms to build endpoint map", (float)(tB - tA) / 1e6);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/0a2f997f/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 4884b7a..b24521d 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
@@ -23,6 +23,8 @@ import java.util.List;
 
 import org.apache.drill.common.logical.data.Scan;
 import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.physical.ReadEntry;
+import org.apache.drill.exec.physical.base.AbstractGroupScan;
 import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
 
 import com.google.common.collect.ListMultimap;
@@ -40,15 +42,15 @@ public interface StorageEngine {
   public List<QueryOptimizerRule> getOptimizerRules();
 
   /**
-   * Get the set of read entries required for a particular Scan (read) node. This is somewhat analogous to traditional
-   * MapReduce. The difference is, this is the most granular split paradigm.
+   * Get the physical scan operator populated with a set of read entries required for the particular GroupScan (read) node.
+   * This is somewhat analogous to traditional MapReduce. The difference is, this is the most granular split paradigm.
    * 
    * @param scan
    *          The configured scan entries.
    * @return
    * @throws IOException
    */
-  public Collection<ReadEntry> getReadEntries(Scan scan) throws IOException;
+  public AbstractGroupScan getPhysicalScan(Scan scan) throws IOException;
 
   /**
    * Get the set of Drillbit endpoints that are available for each read entry. Note that it is possible for a read entry
@@ -104,10 +106,6 @@ public interface StorageEngine {
    */
   public RecordRecorder getWriter(FragmentContext context, WriteEntry writeEntry) throws IOException;
 
-  public interface ReadEntry {
-    public Cost getCostEstimate();
-  }
-
   public interface WriteEntry {
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/0a2f997f/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/StorageEngineRegistry.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/StorageEngineRegistry.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/StorageEngineRegistry.java
index eef878e..26504a2 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/StorageEngineRegistry.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/StorageEngineRegistry.java
@@ -38,19 +38,19 @@ public class StorageEngineRegistry {
 
   private DrillbitContext context;
   public StorageEngineRegistry(DrillbitContext context){
+    init(context.getConfig());
     this.context = context;
-    setup(context.getConfig());
   }
   
   @SuppressWarnings("unchecked")
-  public void setup(DrillConfig config){
+  public void init(DrillConfig config){
     Collection<Class<? extends StorageEngine>> engines = PathScanner.scanForImplementations(StorageEngine.class, config.getStringList(ExecConstants.STORAGE_ENGINE_SCAN_PACKAGES));
     logger.debug("Loading storage engines {}", engines);
     for(Class<? extends StorageEngine> engine: engines){
       int i =0;
       for(Constructor<?> c : engine.getConstructors()){
         Class<?>[] params = c.getParameterTypes();
-        if(params.length != 2 || params[1] == DrillbitContext.class || !StorageEngineConfig.class.isAssignableFrom(params[0])){
+        if(params.length != 2 || params[1] != DrillbitContext.class || !StorageEngineConfig.class.isAssignableFrom(params[0])){
           logger.debug("Skipping StorageEngine constructor {} for engine class {} since it doesn't implement a [constructor(StorageEngineConfig, DrillbitContext)]", c, engine);
           continue;
         }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/0a2f997f/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/VectorHolder.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/VectorHolder.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/VectorHolder.java
index d2ad72a..7cbea57 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/VectorHolder.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/VectorHolder.java
@@ -27,10 +27,11 @@ public class VectorHolder {
   private ValueVector vector;
   private int currentLength;
 
-  VectorHolder(ValueVector vector) {
-    this.length = vector.getValueCapacity();
-    this.vector = vector;
-  }
+    public VectorHolder(int length, ValueVector vector) {
+        this.length = length;
+        this.vector = vector;
+        this.mutator = vector.getMutator();
+    }
 
   public ValueVector getValueVector() {
     return vector;

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/0a2f997f/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/BitReader.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/BitReader.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/BitReader.java
new file mode 100644
index 0000000..c85d4aa
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/BitReader.java
@@ -0,0 +1,87 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.parquet;
+
+import org.apache.drill.exec.vector.BaseDataValueVector;
+import org.apache.drill.exec.vector.ValueVector;
+import parquet.column.ColumnDescriptor;
+import parquet.hadoop.metadata.ColumnChunkMetaData;
+
+public final class BitReader extends ColumnReader {
+
+  byte currentByte;
+  byte nextByte;
+
+  BitReader(ParquetRecordReader parentReader, int allocateSize, ColumnDescriptor descriptor, ColumnChunkMetaData columnChunkMetaData,
+            boolean fixedLength, ValueVector v) {
+    super(parentReader, allocateSize, descriptor, columnChunkMetaData, fixedLength, v);
+  }
+
+  @Override
+  protected void readField(long recordsToReadInThisPass, ColumnReader firstColumnStatus) {
+
+    recordsReadInThisIteration = Math.min(pageReadStatus.currentPage.getValueCount()
+        - pageReadStatus.valuesRead, recordsToReadInThisPass - valuesReadInCurrentPass);
+
+    readStartInBytes = pageReadStatus.readPosInBytes;
+    readLengthInBits = recordsReadInThisIteration * dataTypeLengthInBits;
+    readLength = (int) Math.ceil(readLengthInBits / 8.0);
+
+    bytes = pageReadStatus.pageDataByteArray;
+    // standard read, using memory mapping
+    if (pageReadStatus.bitShift == 0) {
+      ((BaseDataValueVector) valueVecHolder.getValueVector()).getData().writeBytes(bytes,
+          (int) readStartInBytes, (int) readLength);
+    } else { // read in individual values, because a bitshift is necessary with where the last page or batch ended
+
+      vectorData = ((BaseDataValueVector) valueVecHolder.getValueVector()).getData();
+      nextByte = bytes[(int) Math.max(0, Math.ceil(pageReadStatus.valuesRead / 8.0) - 1)];
+      readLengthInBits = recordsReadInThisIteration + pageReadStatus.bitShift;
+      //recordsReadInThisIteration -= (8 - pageReadStatus.bitShift);
+
+      int i = 0;
+      // read individual bytes with appropriate shifting
+      for (; i <= (int) readLength; i++) {
+        currentByte = nextByte;
+        currentByte = (byte) (currentByte >>> pageReadStatus.bitShift);
+        // mask the bits about to be added from the next byte
+        currentByte = (byte) (currentByte & ParquetRecordReader.startBitMasks[pageReadStatus.bitShift - 1]);
+        // if we are not on the last byte
+        if ((int) Math.ceil(pageReadStatus.valuesRead / 8.0) + i < pageReadStatus.byteLength) {
+          // grab the next byte from the buffer, shift and mask it, and OR it with the leftover bits
+          nextByte = bytes[(int) Math.ceil(pageReadStatus.valuesRead / 8.0) + i];
+          currentByte = (byte) (currentByte | nextByte
+              << (8 - pageReadStatus.bitShift)
+              & ParquetRecordReader.endBitMasks[8 - pageReadStatus.bitShift - 1]);
+        }
+        vectorData.setByte(valuesReadInCurrentPass / 8 + i, currentByte);
+      }
+      vectorData.setIndex(0, (valuesReadInCurrentPass / 8)
+          + (int) readLength - 1);
+      vectorData.capacity(vectorData.writerIndex() + 1);
+    }
+
+    // check if the values in this page did not end on a byte boundary, store a number of bits the next page must be
+    // shifted by to read all of the values into the vector without leaving space
+    if (readLengthInBits % 8 != 0) {
+      pageReadStatus.bitShift = (int) readLengthInBits % 8;
+    } else {
+      pageReadStatus.bitShift = 0;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/0a2f997f/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ColumnReader.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ColumnReader.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ColumnReader.java
new file mode 100644
index 0000000..8b4f760
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ColumnReader.java
@@ -0,0 +1,115 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.store.parquet;
+
+import io.netty.buffer.ByteBuf;
+import org.apache.drill.exec.store.VectorHolder;
+import org.apache.drill.exec.vector.BaseDataValueVector;
+import org.apache.drill.exec.vector.ValueVector;
+import parquet.column.ColumnDescriptor;
+import parquet.hadoop.metadata.ColumnChunkMetaData;
+import parquet.schema.PrimitiveType;
+
+import java.io.IOException;
+
+public abstract class ColumnReader {
+  // Value Vector for this column
+  VectorHolder valueVecHolder;
+  // column description from the parquet library
+  ColumnDescriptor columnDescriptor;
+  // metadata of the column, from the parquet library
+  ColumnChunkMetaData columnChunkMetaData;
+  // status information on the current page
+  PageReadStatus pageReadStatus;
+
+  long readPositionInBuffer;
+
+  int compressedSize;
+
+  // quick reference to see if the field is fixed length (as this requires an instanceof)
+  boolean isFixedLength;
+  // counter for the total number of values read from one or more pages
+  // when a batch is filled all of these values should be the same for each column
+  int totalValuesRead;
+  // counter for the values that have been read in this pass (a single call to the next() method)
+  int valuesReadInCurrentPass;
+  // length of single data value in bits, if the length is fixed
+  int dataTypeLengthInBits;
+  int bytesReadInCurrentPass;
+  ParquetRecordReader parentReader;
+
+  ByteBuf vectorData;
+
+  // variables for a single read pass
+  long readStartInBytes = 0, readLength = 0, readLengthInBits = 0, recordsReadInThisIteration = 0;
+  byte[] bytes;
+
+  ColumnReader(ParquetRecordReader parentReader, int allocateSize, ColumnDescriptor descriptor, ColumnChunkMetaData columnChunkMetaData,
+               boolean fixedLength, ValueVector v){
+    this.parentReader = parentReader;
+    if (allocateSize > 1) valueVecHolder = new VectorHolder(allocateSize, (BaseDataValueVector) v);
+    else valueVecHolder = new VectorHolder(5000, (BaseDataValueVector) v);
+
+    columnDescriptor = descriptor;
+    this.columnChunkMetaData = columnChunkMetaData;
+    isFixedLength = fixedLength;
+
+    pageReadStatus = new PageReadStatus(this, parentReader.getRowGroupIndex(), parentReader.getBufferWithAllData());
+
+    if (parentReader.getRowGroupIndex() != 0) readPositionInBuffer = columnChunkMetaData.getFirstDataPageOffset() - 4;
+    else readPositionInBuffer = columnChunkMetaData.getFirstDataPageOffset();
+
+    if (columnDescriptor.getType() != PrimitiveType.PrimitiveTypeName.BINARY) {
+      dataTypeLengthInBits = ParquetRecordReader.getTypeLengthInBits(columnDescriptor.getType());
+    }
+  }
+
+  public void readAllFixedFields(long recordsToReadInThisPass, ColumnReader firstColumnStatus) throws IOException {
+    readStartInBytes = 0;
+    readLength = 0;
+    readLengthInBits = 0;
+    recordsReadInThisIteration = 0;
+    vectorData = ((BaseDataValueVector) valueVecHolder.getValueVector()).getData();
+    do {
+      // if no page has been read, or all of the records have been read out of a page, read the next one
+      if (pageReadStatus.currentPage == null
+          || pageReadStatus.valuesRead == pageReadStatus.currentPage.getValueCount()) {
+        totalValuesRead += pageReadStatus.valuesRead;
+        if (!pageReadStatus.next()) {
+          break;
+        }
+      }
+
+      readField( recordsToReadInThisPass, firstColumnStatus);
+
+      valuesReadInCurrentPass += recordsReadInThisIteration;
+      totalValuesRead += recordsReadInThisIteration;
+      pageReadStatus.valuesRead += recordsReadInThisIteration;
+      if (readStartInBytes + readLength >= pageReadStatus.byteLength) {
+        pageReadStatus.next();
+      } else {
+        pageReadStatus.readPosInBytes = readStartInBytes + readLength;
+      }
+    }
+    while (valuesReadInCurrentPass < recordsToReadInThisPass && pageReadStatus.currentPage != null);
+    ((BaseDataValueVector) valueVecHolder.getValueVector()).getMutator().setValueCount(
+        valuesReadInCurrentPass);
+  }
+
+  protected abstract void readField(long recordsToRead, ColumnReader firstColumnStatus);
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/0a2f997f/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/FixedByteAlignedReader.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/FixedByteAlignedReader.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/FixedByteAlignedReader.java
new file mode 100644
index 0000000..355405b
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/FixedByteAlignedReader.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.store.parquet;
+
+import org.apache.drill.exec.vector.BaseDataValueVector;
+import org.apache.drill.exec.vector.ValueVector;
+import parquet.column.ColumnDescriptor;
+import parquet.hadoop.metadata.ColumnChunkMetaData;
+
+public class FixedByteAlignedReader extends ColumnReader {
+
+  FixedByteAlignedReader(ParquetRecordReader parentReader, int allocateSize, ColumnDescriptor descriptor, ColumnChunkMetaData columnChunkMetaData,
+                         boolean fixedLength, ValueVector v) {
+    super(parentReader, allocateSize, descriptor, columnChunkMetaData, fixedLength, v);
+  }
+
+  // this method is called by its superclass during a read loop
+  @Override
+  protected void readField(long recordsToReadInThisPass, ColumnReader firstColumnStatus) {
+
+    recordsReadInThisIteration = Math.min(pageReadStatus.currentPage.getValueCount()
+        - pageReadStatus.valuesRead, recordsToReadInThisPass - valuesReadInCurrentPass);
+
+    readStartInBytes = pageReadStatus.readPosInBytes;
+    readLengthInBits = recordsReadInThisIteration * dataTypeLengthInBits;
+    readLength = (int) Math.ceil(readLengthInBits / 8.0);
+
+    bytes = pageReadStatus.pageDataByteArray;
+    // vectorData is assigned by the superclass read loop method
+    vectorData.writeBytes(bytes,
+        (int) readStartInBytes, (int) readLength);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/0a2f997f/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/PageReadStatus.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/PageReadStatus.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/PageReadStatus.java
new file mode 100644
index 0000000..29d9cc7
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/PageReadStatus.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.store.parquet;
+
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.ByteBufInputStream;
+import parquet.bytes.BytesInput;
+import parquet.column.page.Page;
+import parquet.column.page.PageReader;
+import parquet.format.PageHeader;
+
+import java.io.IOException;
+
+import static parquet.format.Util.readPageHeader;
+
+// class to keep track of the read position of variable length columns
+public final class PageReadStatus {
+
+  ColumnReader parentColumnReader;
+
+  // store references to the pages that have been uncompressed, but not copied to ValueVectors yet
+  Page currentPage;
+  // buffer to store bytes of current page, set to max size of parquet page
+  byte[] pageDataByteArray = new byte[ParquetRecordReader.PARQUET_PAGE_MAX_SIZE];
+  PageReader pageReader;
+  // read position in the current page, stored in the ByteBuf in ParquetRecordReader called bufferWithAllData
+  long readPosInBytes;
+  // bit shift needed for the next page if the last one did not line up with a byte boundary
+  int bitShift;
+  // storage space for extra bits at the end of a page if they did not line up with a byte boundary
+  // prevents the need to keep the entire last page, as these pageDataByteArray need to be added to the next batch
+  //byte extraBits;
+  // the number of values read out of the last page
+  int valuesRead;
+  int byteLength;
+  int rowGroupIndex;
+  // buffer with the compressed data of an entire row group
+  ByteBuf bufferWithAllData;
+
+  PageReadStatus(ColumnReader parentStatus, int rowGroupIndex, ByteBuf bufferWithAllData){
+    this.parentColumnReader = parentStatus;
+    this.rowGroupIndex = rowGroupIndex;
+  }
+
+  /**
+   * Grab the next page.
+   *
+   * @return - if another page was present
+   * @throws java.io.IOException
+   */
+  public boolean next() throws IOException {
+
+    int shift = 0;
+    if (rowGroupIndex == 0) shift = 0;
+    else shift = 4;
+    // first ROW GROUP has a different endpoint, because there are for bytes at the beginning of the file "PAR1"
+    if (parentColumnReader.readPositionInBuffer + shift == parentColumnReader.columnChunkMetaData.getFirstDataPageOffset() + parentColumnReader.columnChunkMetaData.getTotalSize()){
+      return false;
+    }
+    // TODO - in the JIRA for parquet steven put a stack trace for an error with a row group with 3 values in it
+    // the Math.min with the end of the buffer should fix it but now I'm not getting results back, leaving it here for now
+    // because it is needed, but there might be a problem with it
+    ByteBufInputStream f = new ByteBufInputStream(parentColumnReader.parentReader.getBufferWithAllData().slice(
+        (int) parentColumnReader.readPositionInBuffer,
+        Math.min(50, parentColumnReader.parentReader.getBufferWithAllData().capacity() - (int) parentColumnReader.readPositionInBuffer)));
+    int before = f.available();
+    PageHeader pageHeader = readPageHeader(f);
+    int length = before - f.available();
+    f = new ByteBufInputStream(parentColumnReader.parentReader.getBufferWithAllData().slice(
+        (int) parentColumnReader.readPositionInBuffer + length, pageHeader.getCompressed_page_size()));
+
+    BytesInput bytesIn = parentColumnReader.parentReader.getCodecFactoryExposer()
+        .decompress(BytesInput.from(f, pageHeader.compressed_page_size), pageHeader.getUncompressed_page_size(),
+            parentColumnReader.columnChunkMetaData.getCodec());
+    currentPage = new Page(
+        bytesIn,
+        pageHeader.data_page_header.num_values,
+        pageHeader.uncompressed_page_size,
+        ParquetStorageEngine.parquetMetadataConverter.getEncoding(pageHeader.data_page_header.repetition_level_encoding),
+        ParquetStorageEngine.parquetMetadataConverter.getEncoding(pageHeader.data_page_header.definition_level_encoding),
+        ParquetStorageEngine.parquetMetadataConverter.getEncoding(pageHeader.data_page_header.encoding)
+    );
+
+    parentColumnReader.readPositionInBuffer += pageHeader.compressed_page_size + length;
+    byteLength = pageHeader.uncompressed_page_size;
+    if (currentPage == null) {
+      return false;
+    }
+
+    // if the buffer holding each page's data is not large enough to hold the current page, re-allocate, with a little extra space
+    if (pageHeader.getUncompressed_page_size() > pageDataByteArray.length) {
+      pageDataByteArray = new byte[pageHeader.getUncompressed_page_size() + 100];
+    }
+    // TODO - would like to get this into the mainline, hopefully before alpha
+    currentPage.getBytes().toByteArray(pageDataByteArray, 0, byteLength);
+
+    readPosInBytes = 0;
+    valuesRead = 0;
+    return true;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/0a2f997f/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetGroupScan.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetGroupScan.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetGroupScan.java
new file mode 100644
index 0000000..f4988a0
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetGroupScan.java
@@ -0,0 +1,357 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.parquet;
+
+import java.io.IOException;
+import java.util.*;
+
+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.fasterxml.jackson.core.type.TypeReference;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import org.apache.drill.common.JSONOptions;
+import org.apache.drill.common.config.DrillConfig;
+import org.apache.drill.exec.exception.SetupException;
+import org.apache.drill.exec.physical.EndpointAffinity;
+import org.apache.drill.exec.physical.OperatorCost;
+import org.apache.drill.exec.physical.ReadEntryFromHDFS;
+import org.apache.drill.exec.physical.ReadEntryWithPath;
+import org.apache.drill.exec.physical.base.AbstractGroupScan;
+import org.apache.drill.exec.physical.base.PhysicalOperator;
+import org.apache.drill.exec.physical.base.Size;
+import org.apache.drill.exec.physical.config.MockGroupScanPOP;
+import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
+
+import com.google.common.base.Preconditions;
+import org.apache.drill.exec.server.DrillbitContext;
+import org.apache.drill.exec.store.StorageEngineRegistry;
+import org.apache.drill.exec.store.AffinityCalculator;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import parquet.hadoop.ParquetFileReader;
+import parquet.hadoop.metadata.BlockMetaData;
+import parquet.hadoop.metadata.ColumnChunkMetaData;
+import parquet.hadoop.metadata.ParquetMetadata;
+import parquet.org.codehaus.jackson.annotate.JsonCreator;
+
+
+@JsonTypeName("parquet-scan")
+public class ParquetGroupScan extends AbstractGroupScan {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ParquetGroupScan.class);
+
+  private LinkedList<ParquetRowGroupScan.RowGroupReadEntry>[] mappings;
+  private List<RowGroupInfo> rowGroupInfos;
+
+  public List<ReadEntryWithPath> getEntries() {
+    return entries;
+  }
+
+  public ParquetStorageEngineConfig getEngineConfig() {
+    return this.engineConfig;
+  }
+
+  private List<ReadEntryWithPath> entries;
+  @JsonIgnore
+  private long totalBytes;
+  private Collection<DrillbitEndpoint> availableEndpoints;
+  private ParquetStorageEngine storageEngine;
+  private StorageEngineRegistry engineRegistry;
+  private ParquetStorageEngineConfig engineConfig;
+  private FileSystem fs;
+  private String fileName;
+  private List<EndpointAffinity> endpointAffinities;
+
+  @JsonCreator
+  public ParquetGroupScan(@JsonProperty("entries") List<ReadEntryWithPath> entries,
+                          @JsonProperty("storageengine") ParquetStorageEngineConfig storageEngineConfig,
+                          @JacksonInject StorageEngineRegistry engineRegistry
+                           )throws SetupException,IOException {
+    engineRegistry.init(DrillConfig.create());
+    this.storageEngine = (ParquetStorageEngine) engineRegistry.getEngine(storageEngineConfig);
+    this.availableEndpoints = storageEngine.getContext().getBits();
+    this.fs = storageEngine.getFileSystem();
+    this.engineConfig = storageEngineConfig;
+    this.engineRegistry = engineRegistry;
+    this.entries = entries;
+    readFooter();
+    this.fileName = rowGroupInfos.get(0).getPath();
+    calculateEndpointBytes();
+  }
+
+  public ParquetGroupScan(ArrayList<ReadEntryWithPath> entries,
+                          ParquetStorageEngine storageEngine) throws IOException {
+    this.storageEngine = storageEngine;
+    this.availableEndpoints = storageEngine.getContext().getBits();
+    this.fs = storageEngine.getFileSystem();
+    this.entries = entries;
+    readFooter();
+    this.fileName = rowGroupInfos.get(0).getPath();
+    calculateEndpointBytes();
+  }
+
+  private void readFooter() throws IOException {
+    long tA = System.nanoTime();
+    rowGroupInfos = new ArrayList();
+    long start = 0, length = 0;
+    ColumnChunkMetaData columnChunkMetaData;
+    for (ReadEntryWithPath readEntryWithPath : entries){
+      Path path = new Path(readEntryWithPath.getPath());
+
+      ParquetMetadata footer = ParquetFileReader.readFooter(this.storageEngine.getHadoopConfig(), path);
+      readEntryWithPath.getPath();
+
+      int i = 0;
+      for (BlockMetaData rowGroup : footer.getBlocks()){
+        // need to grab block information from HDFS
+        columnChunkMetaData = rowGroup.getColumns().iterator().next();
+        start = columnChunkMetaData.getFirstDataPageOffset();
+        // this field is not being populated correctly, but the column chunks know their sizes, just summing them for now
+        //end = start + rowGroup.getTotalByteSize();
+        length = 0;
+        for (ColumnChunkMetaData col : rowGroup.getColumns()){
+          length += col.getTotalSize();
+        }
+        rowGroupInfos.add(new ParquetGroupScan.RowGroupInfo(readEntryWithPath.getPath(), start, length, i));
+        logger.debug("rowGroupInfo path: {} start: {} length {}", readEntryWithPath.getPath(), start, length);
+        i++;
+      }
+    }
+    long tB = System.nanoTime();
+    logger.debug("Took {} ms to get row group infos", (float)(tB - tA) / 1E6);
+  }
+
+  private void calculateEndpointBytes() {
+    long tA = System.nanoTime();
+    AffinityCalculator ac = new AffinityCalculator(fileName, fs, availableEndpoints);
+    for (RowGroupInfo e : rowGroupInfos) {
+      ac.setEndpointBytes(e);
+      totalBytes += e.getLength();
+    }
+    long tB = System.nanoTime();
+    logger.debug("Took {} ms to calculate EndpointBytes", (float)(tB - tA) / 1E6);
+  }
+/*
+  public LinkedList<RowGroupInfo> getRowGroups() {
+    return rowGroups;
+  }
+
+  public void setRowGroups(LinkedList<RowGroupInfo> rowGroups) {
+    this.rowGroups = rowGroups;
+  }
+
+  public static class ParquetFileReadEntry {
+
+    String path;
+
+    public ParquetFileReadEntry(@JsonProperty String path){
+      this.path = path;
+    }
+  }
+  */
+
+  @JsonIgnore
+  public FileSystem getFileSystem() {
+    return this.fs;
+  }
+
+  public static class RowGroupInfo extends ReadEntryFromHDFS {
+
+    private HashMap<DrillbitEndpoint,Long> endpointBytes;
+    private long maxBytes;
+    private int rowGroupIndex;
+
+    @JsonCreator
+    public RowGroupInfo(@JsonProperty("path") String path, @JsonProperty("start") long start,
+                        @JsonProperty("length") long length, @JsonProperty("rowGroupIndex") int rowGroupIndex) {
+      super(path, start, length);
+      this.rowGroupIndex = rowGroupIndex;
+    }
+
+    @Override
+    public OperatorCost getCost() {
+      return new OperatorCost(1, 2, 1, 1);
+    }
+
+    @Override
+    public Size getSize() {
+      // TODO - these values are wrong, I cannot know these until after I read a file
+      return new Size(10, 10);
+    }
+
+    public HashMap<DrillbitEndpoint,Long> getEndpointBytes() {
+      return endpointBytes;
+    }
+
+    public void setEndpointBytes(HashMap<DrillbitEndpoint,Long> endpointBytes) {
+      this.endpointBytes = endpointBytes;
+    }
+
+    public void setMaxBytes(long bytes) {
+      this.maxBytes = bytes;
+    }
+
+    public long getMaxBytes() {
+      return maxBytes;
+    }
+
+    public ParquetRowGroupScan.RowGroupReadEntry getRowGroupReadEntry() {
+      return new ParquetRowGroupScan.RowGroupReadEntry(this.getPath(), this.getStart(), this.getLength(), this.rowGroupIndex);
+    }
+
+    public int getRowGroupIndex() {
+      return this.rowGroupIndex;
+    }
+  }
+
+  private class ParquetReadEntryComparator implements Comparator<RowGroupInfo> {
+    public int compare(RowGroupInfo e1, RowGroupInfo e2) {
+      if (e1.getMaxBytes() == e2.getMaxBytes()) return 0;
+      return (e1.getMaxBytes() > e2.getMaxBytes()) ? 1 : -1;
+    }
+  }
+
+  @Override
+  public List<EndpointAffinity> getOperatorAffinity() {
+    long tA = System.nanoTime();
+    if (this.endpointAffinities == null) {
+      HashMap<DrillbitEndpoint, Float> affinities = new HashMap<>();
+      for (RowGroupInfo entry : rowGroupInfos) {
+        for (DrillbitEndpoint d : entry.getEndpointBytes().keySet()) {
+          long bytes = entry.getEndpointBytes().get(d);
+          float affinity = (float)bytes / (float)totalBytes;
+          logger.error("RowGroup: {} Endpoint: {} Bytes: {}", entry.getRowGroupIndex(), d.getAddress(), bytes);
+          if (affinities.keySet().contains(d)) {
+            affinities.put(d, affinities.get(d) + affinity);
+          } else {
+            affinities.put(d, affinity);
+          }
+        }
+      }
+      List<EndpointAffinity> affinityList = new LinkedList<>();
+      for (DrillbitEndpoint d : affinities.keySet()) {
+        logger.debug("Endpoint {} has affinity {}", d.getAddress(), affinities.get(d).floatValue());
+        affinityList.add(new EndpointAffinity(d,affinities.get(d).floatValue()));
+      }
+      this.endpointAffinities = affinityList;
+    }
+    long tB = System.nanoTime();
+    logger.debug("Took {} ms to get operator affinity", (float)(tB - tA) / 1E6);
+    return this.endpointAffinities;
+  }
+
+
+
+
+  @Override
+  public void applyAssignments(List<DrillbitEndpoint> endpoints) {
+    long tA = System.nanoTime();
+    Preconditions.checkArgument(endpoints.size() <= rowGroupInfos.size());
+
+    int i = 0;
+    for (DrillbitEndpoint endpoint : endpoints) {
+      logger.debug("Endpoint index {}, endpoint host: {}", i++, endpoint.getAddress());
+    }
+
+    Collections.sort(rowGroupInfos, new ParquetReadEntryComparator());
+    mappings = new LinkedList[endpoints.size()];
+    LinkedList<RowGroupInfo> unassigned = scanAndAssign(endpoints, rowGroupInfos, 100, true, false);
+    LinkedList<RowGroupInfo> unassigned2 = scanAndAssign(endpoints, unassigned, 50, true, false);
+    LinkedList<RowGroupInfo> unassigned3 = scanAndAssign(endpoints, unassigned2, 25, true, false);
+    LinkedList<RowGroupInfo> unassigned4 = scanAndAssign(endpoints, unassigned3, 0, false, false);
+    LinkedList<RowGroupInfo> unassigned5 = scanAndAssign(endpoints, unassigned4, 0, false, true);
+    assert unassigned5.size() == 0 : String.format("All readEntries should be assigned by now, but some are still unassigned");
+    long tB = System.nanoTime();
+    logger.debug("Took {} ms to apply assignments ", (float)(tB - tA) / 1E6);
+  }
+
+  private LinkedList<RowGroupInfo> scanAndAssign (List<DrillbitEndpoint> endpoints, List<RowGroupInfo> rowGroups, int requiredPercentage, boolean mustContain, boolean assignAll) {
+    Collections.sort(rowGroupInfos, new ParquetReadEntryComparator());
+    LinkedList<RowGroupInfo> unassigned = new LinkedList<>();
+
+    int maxEntries = (int) (rowGroupInfos.size() / endpoints.size() * 1.5);
+
+    if (maxEntries < 1) maxEntries = 1;
+
+    int i =0;
+    for(RowGroupInfo e : rowGroups) {
+      boolean assigned = false;
+      for (int j = i; j < i + endpoints.size(); j++) {
+        DrillbitEndpoint currentEndpoint = endpoints.get(j%endpoints.size());
+        if (assignAll ||
+                (e.getEndpointBytes().size() > 0 &&
+                (e.getEndpointBytes().containsKey(currentEndpoint) || !mustContain) &&
+                (mappings[j%endpoints.size()] == null || mappings[j%endpoints.size()].size() < maxEntries) &&
+                e.getEndpointBytes().get(currentEndpoint) >= e.getMaxBytes() * requiredPercentage / 100)) {
+          LinkedList<ParquetRowGroupScan.RowGroupReadEntry> entries = mappings[j%endpoints.size()];
+          if(entries == null){
+            entries = new LinkedList<ParquetRowGroupScan.RowGroupReadEntry>();
+            mappings[j%endpoints.size()] = entries;
+          }
+          entries.add(e.getRowGroupReadEntry());
+          logger.debug("Assigned rowGroup ( {} , {} ) to endpoint {}", e.getPath(), e.getStart(), currentEndpoint.getAddress());
+          assigned = true;
+          break;
+        }
+      }
+      if (!assigned) unassigned.add(e);
+      i++;
+    }
+    return unassigned;
+  }
+
+  @Override
+  public ParquetRowGroupScan getSpecificScan(int minorFragmentId) {
+    assert minorFragmentId < mappings.length : String.format("Mappings length [%d] should be longer than minor fragment id [%d] but it isn't.", mappings.length, minorFragmentId);
+    for (ParquetRowGroupScan.RowGroupReadEntry rg : mappings[minorFragmentId]) {
+      logger.debug("minorFragmentId: {} Path: {} RowGroupIndex: {}",minorFragmentId, rg.getPath(),rg.getRowGroupIndex());
+    }
+    try {
+      return new ParquetRowGroupScan(storageEngine, engineConfig, mappings[minorFragmentId]);
+    } catch (SetupException e) {
+      e.printStackTrace(); // TODO - fix this
+    }
+    return null;
+  }
+
+  @Override
+  public int getMaxParallelizationWidth() {
+    return rowGroupInfos.size();
+  }
+
+  @Override
+  public OperatorCost getCost() {
+    return new OperatorCost(1,1,1,1);
+  }
+
+  @Override
+  public Size getSize() {
+    // TODO - this is wrong, need to populate correctly
+    return new Size(10,10);
+  }
+
+  @Override
+  @JsonIgnore
+  public PhysicalOperator getNewWithChildren(List<PhysicalOperator> children) {
+    Preconditions.checkArgument(children.isEmpty());
+    //TODO return copy of self
+    return this;
+  }
+
+}