You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@drill.apache.org by jn...@apache.org on 2017/03/02 20:59:43 UTC

[16/27] drill git commit: DRILL-5258: Access mock data definition from SQL

DRILL-5258: Access mock data definition from SQL

Extends the mock data source to allow using the full power of the mock
data source from an SQL query by referencing the JSON definition
file. See JIRA and package-info for details.

Adds a boolean data generator and a varying-length string generator.

Adds \u201cmock\u201d table stats for use in the planner.

Revisions based on code review comments

close #752


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

Branch: refs/heads/master
Commit: 974c613402604b86f9d36568b4b62c22a7a291d9
Parents: 20a374c
Author: Paul Rogers <pr...@maprtech.com>
Authored: Tue Feb 14 10:02:13 2017 -0800
Committer: Jinfeng Ni <jn...@apache.org>
Committed: Wed Mar 1 23:15:33 2017 -0800

----------------------------------------------------------------------
 .../drill/exec/physical/impl/ScanBatch.java     |   6 +-
 .../apache/drill/exec/record/SchemaUtil.java    |  23 +-
 .../apache/drill/exec/record/TypedFieldId.java  |  13 ++
 .../drill/exec/record/VectorContainer.java      |   9 +-
 .../drill/exec/store/AbstractStoragePlugin.java |  14 +-
 .../exec/store/dfs/easy/EasyGroupScan.java      |   3 +-
 .../drill/exec/store/mock/BooleanGen.java       |  42 ++++
 .../apache/drill/exec/store/mock/ColumnDef.java |  19 +-
 .../apache/drill/exec/store/mock/DateGen.java   |   2 +-
 .../store/mock/ExtendedMockRecordReader.java    |  24 +-
 .../drill/exec/store/mock/MockGroupScanPOP.java | 232 +++++--------------
 .../drill/exec/store/mock/MockRecordReader.java |   4 +-
 .../exec/store/mock/MockScanBatchCreator.java   |   7 +-
 .../exec/store/mock/MockStorageEngine.java      |  92 ++++++--
 .../drill/exec/store/mock/MockSubScanPOP.java   |  11 +-
 .../drill/exec/store/mock/MockTableDef.java     | 213 +++++++++++++++++
 .../drill/exec/store/mock/VaryingStringGen.java |  70 ++++++
 .../drill/exec/store/mock/package-info.java     |  41 +++-
 .../apache/drill/exec/util/TestUtilities.java   |   6 +-
 .../fn/interp/ExpressionInterpreterTest.java    |  15 +-
 .../physical/impl/TestConvertFunctions.java     |   5 +-
 .../impl/mergereceiver/TestMergingReceiver.java |   4 +
 .../src/test/resources/test/example-mock.json   |  16 ++
 23 files changed, 597 insertions(+), 274 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/974c6134/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScanBatch.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScanBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScanBatch.java
index ad82668..e20c394 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScanBatch.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScanBatch.java
@@ -230,7 +230,7 @@ public class ScanBatch implements CloseableRecordBatch {
       hasReadNonEmptyFile = true;
       populateImplicitVectors();
 
-      for (VectorWrapper w : container) {
+      for (VectorWrapper<?> w : container) {
         w.getValueVector().getMutator().setValueCount(recordCount);
       }
 
@@ -270,6 +270,7 @@ public class ScanBatch implements CloseableRecordBatch {
       if (implicitValues != null) {
         for (String column : implicitValues.keySet()) {
           final MaterializedField field = MaterializedField.create(column, Types.optional(MinorType.VARCHAR));
+          @SuppressWarnings("resource")
           final ValueVector v = mutator.addField(field, NullableVarCharVector.class);
           implicitVectors.put(column, v);
         }
@@ -282,6 +283,7 @@ public class ScanBatch implements CloseableRecordBatch {
   private void populateImplicitVectors() {
     if (implicitValues != null) {
       for (Map.Entry<String, String> entry : implicitValues.entrySet()) {
+        @SuppressWarnings("resource")
         final NullableVarCharVector v = (NullableVarCharVector) implicitVectors.get(entry.getKey());
         String val;
         if ((val = entry.getValue()) != null) {
@@ -325,7 +327,7 @@ public class ScanBatch implements CloseableRecordBatch {
     private boolean schemaChanged = true;
 
 
-    @SuppressWarnings("unchecked")
+    @SuppressWarnings("resource")
     @Override
     public <T extends ValueVector> T addField(MaterializedField field,
                                               Class<T> clazz) throws SchemaChangeException {

http://git-wip-us.apache.org/repos/asf/drill/blob/974c6134/exec/java-exec/src/main/java/org/apache/drill/exec/record/SchemaUtil.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/record/SchemaUtil.java b/exec/java-exec/src/main/java/org/apache/drill/exec/record/SchemaUtil.java
index d6a8a40..2fc9314 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/record/SchemaUtil.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/record/SchemaUtil.java
@@ -17,26 +17,24 @@
  */
 package org.apache.drill.exec.record;
 
-import com.google.common.base.Preconditions;
-import com.google.common.collect.Lists;
-import com.google.common.collect.Maps;
-import com.google.common.collect.Sets;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
 import org.apache.drill.common.expression.SchemaPath;
 import org.apache.drill.common.types.TypeProtos.DataMode;
 import org.apache.drill.common.types.TypeProtos.MajorType;
 import org.apache.drill.common.types.TypeProtos.MinorType;
 import org.apache.drill.common.types.Types;
 import org.apache.drill.exec.expr.TypeHelper;
-import org.apache.drill.exec.memory.BufferAllocator;
 import org.apache.drill.exec.ops.OperatorContext;
-import org.apache.drill.exec.physical.impl.sort.RecordBatchData;
-import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
 import org.apache.drill.exec.vector.ValueVector;
 import org.apache.drill.exec.vector.complex.UnionVector;
 
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import com.google.common.collect.Sets;
 
 /**
  * Utility class for dealing with changing schemas
@@ -96,6 +94,7 @@ public class SchemaUtil {
     return s;
   }
 
+  @SuppressWarnings("resource")
   private static  ValueVector coerceVector(ValueVector v, VectorContainer c, MaterializedField field,
                                            int recordCount, OperatorContext context) {
     if (v != null) {
@@ -154,13 +153,14 @@ public class SchemaUtil {
     int recordCount = in.getRecordCount();
     boolean isHyper = false;
     Map<String, Object> vectorMap = Maps.newHashMap();
-    for (VectorWrapper w : in) {
+    for (VectorWrapper<?> w : in) {
       if (w.isHyper()) {
         isHyper = true;
         final ValueVector[] vvs = w.getValueVectors();
         vectorMap.put(vvs[0].getField().getPath(), vvs);
       } else {
         assert !isHyper;
+        @SuppressWarnings("resource")
         final ValueVector v = w.getValueVector();
         vectorMap.put(v.getField().getPath(), v);
       }
@@ -183,6 +183,7 @@ public class SchemaUtil {
         }
         c.add(vvsOut);
       } else {
+        @SuppressWarnings("resource")
         final ValueVector v = (ValueVector) vectorMap.remove(field.getPath());
         c.add(coerceVector(v, c, field, recordCount, context));
       }

http://git-wip-us.apache.org/repos/asf/drill/blob/974c6134/exec/java-exec/src/main/java/org/apache/drill/exec/record/TypedFieldId.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/record/TypedFieldId.java b/exec/java-exec/src/main/java/org/apache/drill/exec/record/TypedFieldId.java
index a322f72..615c7a2 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/record/TypedFieldId.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/record/TypedFieldId.java
@@ -28,6 +28,12 @@ import org.apache.drill.exec.vector.ValueVector;
 import com.carrotsearch.hppc.IntArrayList;
 import com.google.common.base.Preconditions;
 
+/**
+ * Declares a value vector field, providing metadata about the field.
+ * Drives code generation by providing type and other structural
+ * information that determine code structure.
+ */
+
 public class TypedFieldId {
   final MajorType finalType;
   final MajorType secondaryFinal;
@@ -104,6 +110,13 @@ public class TypedFieldId {
     return intermediateType;
   }
 
+  /**
+   * Return the class for the value vector (type, mode).
+   *
+   * @return the specific, generated ValueVector subclass that
+   * stores values of the given (type, mode) combination
+   */
+
   public Class<? extends ValueVector> getIntermediateClass() {
     return (Class<? extends ValueVector>) BasicTypeHelper.getValueVectorClass(intermediateType.getMinorType(),
         intermediateType.getMode());

http://git-wip-us.apache.org/repos/asf/drill/blob/974c6134/exec/java-exec/src/main/java/org/apache/drill/exec/record/VectorContainer.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/record/VectorContainer.java b/exec/java-exec/src/main/java/org/apache/drill/exec/record/VectorContainer.java
index 96d9ba6..ceedb84 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/record/VectorContainer.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/record/VectorContainer.java
@@ -28,7 +28,6 @@ import java.util.Set;
 import org.apache.drill.common.expression.SchemaPath;
 import org.apache.drill.common.types.TypeProtos.MajorType;
 import org.apache.drill.exec.expr.TypeHelper;
-import org.apache.drill.exec.memory.BufferAllocator;
 import org.apache.drill.exec.ops.OperatorContext;
 import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
 import org.apache.drill.exec.record.selection.SelectionVector2;
@@ -117,6 +116,7 @@ public class VectorContainer implements Iterable<VectorWrapper<?>>, VectorAccess
     return addOrGet(field, null);
   }
 
+  @SuppressWarnings({ "resource", "unchecked" })
   public <T extends ValueVector> T addOrGet(final MaterializedField field, final SchemaChangeCallBack callBack) {
     final TypedFieldId id = getValueVectorId(SchemaPath.getSimplePath(field.getPath()));
     final ValueVector vector;
@@ -159,10 +159,10 @@ public class VectorContainer implements Iterable<VectorWrapper<?>>, VectorAccess
     return vc;
   }
 
-  public static VectorContainer getTransferClone(VectorAccessible incoming, VectorWrapper[] ignoreWrappers, OperatorContext oContext) {
+  public static VectorContainer getTransferClone(VectorAccessible incoming, VectorWrapper<?>[] ignoreWrappers, OperatorContext oContext) {
     Iterable<VectorWrapper<?>> wrappers = incoming;
     if (ignoreWrappers != null) {
-      final List<VectorWrapper> ignored = Lists.newArrayList(ignoreWrappers);
+      final List<VectorWrapper<?>> ignored = Lists.newArrayList(ignoreWrappers);
       final Set<VectorWrapper<?>> resultant = Sets.newLinkedHashSet(incoming);
       resultant.removeAll(ignored);
       wrappers = resultant;
@@ -184,6 +184,7 @@ public class VectorContainer implements Iterable<VectorWrapper<?>>, VectorAccess
     List<VectorWrapper<?>> canonicalWrappers = new ArrayList<VectorWrapper<?>>(original.wrappers);
     // Sort list of VectorWrapper alphabetically based on SchemaPath.
     Collections.sort(canonicalWrappers, new Comparator<VectorWrapper<?>>() {
+      @Override
       public int compare(VectorWrapper<?> v1, VectorWrapper<?> v2) {
         return v1.getField().getPath().compareTo(v2.getField().getPath());
       }
@@ -265,6 +266,7 @@ public class VectorContainer implements Iterable<VectorWrapper<?>>, VectorAccess
     throw new IllegalStateException("You attempted to remove a vector that didn't exist.");
   }
 
+  @Override
   public TypedFieldId getValueVectorId(SchemaPath path) {
     for (int i = 0; i < wrappers.size(); i++) {
       VectorWrapper<?> va = wrappers.get(i);
@@ -310,6 +312,7 @@ public class VectorContainer implements Iterable<VectorWrapper<?>>, VectorAccess
     return schema != null;
   }
 
+  @Override
   public BatchSchema getSchema() {
     Preconditions
         .checkNotNull(schema,

http://git-wip-us.apache.org/repos/asf/drill/blob/974c6134/exec/java-exec/src/main/java/org/apache/drill/exec/store/AbstractStoragePlugin.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/AbstractStoragePlugin.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/AbstractStoragePlugin.java
index fa2c450..1bd56ae 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/AbstractStoragePlugin.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/AbstractStoragePlugin.java
@@ -33,11 +33,9 @@ import com.google.common.collect.ImmutableSet;
 /** Abstract class for StorePlugin implementations.
  * See StoragePlugin for description of the interface intent and its methods.
  */
-public abstract class AbstractStoragePlugin implements StoragePlugin{
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(AbstractStoragePlugin.class);
+public abstract class AbstractStoragePlugin implements StoragePlugin {
 
-  protected AbstractStoragePlugin(){
-  }
+  protected AbstractStoragePlugin() { }
 
   @Override
   public boolean supportsRead() {
@@ -95,7 +93,6 @@ public abstract class AbstractStoragePlugin implements StoragePlugin{
     default:
       return ImmutableSet.of();
     }
-
   }
 
   @Override
@@ -109,11 +106,8 @@ public abstract class AbstractStoragePlugin implements StoragePlugin{
   }
 
   @Override
-  public void start() throws IOException {
-  }
+  public void start() throws IOException { }
 
   @Override
-  public void close() throws Exception {
-  }
-
+  public void close() throws Exception { }
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/974c6134/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/easy/EasyGroupScan.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/easy/EasyGroupScan.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/easy/EasyGroupScan.java
index 7a80db3..d60b753 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/easy/EasyGroupScan.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/easy/EasyGroupScan.java
@@ -54,7 +54,7 @@ import com.google.common.collect.ListMultimap;
 import com.google.common.collect.Lists;
 
 @JsonTypeName("fs-scan")
-public class EasyGroupScan extends AbstractFileGroupScan{
+public class EasyGroupScan extends AbstractFileGroupScan {
   private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(EasyGroupScan.class);
 
   private FileSelection selection;
@@ -127,6 +127,7 @@ public class EasyGroupScan extends AbstractFileGroupScan{
   }
 
   private void initFromSelection(FileSelection selection, EasyFormatPlugin<?> formatPlugin) throws IOException {
+    @SuppressWarnings("resource")
     final DrillFileSystem dfs = ImpersonationUtil.createFileSystem(getUserName(), formatPlugin.getFsConf());
     this.selection = selection;
     BlockMapBuilder b = new BlockMapBuilder(dfs, formatPlugin.getContext().getBits());

http://git-wip-us.apache.org/repos/asf/drill/blob/974c6134/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/BooleanGen.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/BooleanGen.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/BooleanGen.java
new file mode 100644
index 0000000..dd84f4d
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/BooleanGen.java
@@ -0,0 +1,42 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.exec.store.mock;
+
+import java.util.Random;
+
+import org.apache.drill.exec.vector.BitVector;
+import org.apache.drill.exec.vector.ValueVector;
+
+public class BooleanGen implements FieldGen {
+
+  private Random rand = new Random();
+
+  @Override
+  public void setup(ColumnDef colDef) { }
+
+  public int value() {
+    return rand.nextBoolean() ? 1 : 0;
+  }
+
+  @Override
+  public void setValue(ValueVector v, int index ) {
+    BitVector vector = (BitVector) v;
+    vector.getMutator().set(index, value());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/974c6134/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/ColumnDef.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/ColumnDef.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/ColumnDef.java
index cfaacdd..2300990 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/ColumnDef.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/ColumnDef.java
@@ -19,7 +19,7 @@ package org.apache.drill.exec.store.mock;
 
 import org.apache.drill.common.types.TypeProtos.MinorType;
 import org.apache.drill.exec.expr.TypeHelper;
-import org.apache.drill.exec.store.mock.MockGroupScanPOP.MockColumn;
+import org.apache.drill.exec.store.mock.MockTableDef.MockColumn;
 
 /**
  * Defines a column for the "enhanced" version of the mock data
@@ -37,7 +37,12 @@ public class ColumnDef {
   public ColumnDef(MockColumn mockCol) {
     this.mockCol = mockCol;
     name = mockCol.getName();
-    width = TypeHelper.getSize(mockCol.getMajorType());
+    if (mockCol.getMinorType() == MinorType.VARCHAR &&
+        mockCol.getWidth() > 0) {
+      width = mockCol.getWidth();
+    } else {
+      width = TypeHelper.getSize(mockCol.getMajorType());
+    }
     makeGenerator();
   }
 
@@ -78,6 +83,7 @@ public class ColumnDef {
     case BIGINT:
       break;
     case BIT:
+      generator = new BooleanGen();
       break;
     case DATE:
       break;
@@ -168,11 +174,6 @@ public class ColumnDef {
     name += Integer.toString(rep);
   }
 
-  public MockColumn getConfig() {
-    return mockCol;
-  }
-
-  public String getName() {
-    return name;
-  }
+  public MockColumn getConfig() { return mockCol; }
+  public String getName() { return name; }
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/974c6134/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/DateGen.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/DateGen.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/DateGen.java
index f7d53ed..100d427 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/DateGen.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/DateGen.java
@@ -25,7 +25,7 @@ import org.apache.drill.exec.vector.ValueVector;
 import org.apache.drill.exec.vector.VarCharVector;
 
 /**
- * Very simple date vaue generator that produces ISO dates
+ * Very simple date value generator that produces ISO dates
  * uniformly distributed over the last year. ISO format
  * is: 2016-12-07.
  * <p>

http://git-wip-us.apache.org/repos/asf/drill/blob/974c6134/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/ExtendedMockRecordReader.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/ExtendedMockRecordReader.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/ExtendedMockRecordReader.java
index f3804d4..ac9cb6a 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/ExtendedMockRecordReader.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/ExtendedMockRecordReader.java
@@ -31,10 +31,11 @@ import org.apache.drill.exec.expr.TypeHelper;
 import org.apache.drill.exec.ops.FragmentContext;
 import org.apache.drill.exec.ops.OperatorContext;
 import org.apache.drill.exec.physical.impl.OutputMutator;
+import org.apache.drill.exec.physical.impl.ScanBatch;
 import org.apache.drill.exec.record.MaterializedField;
 import org.apache.drill.exec.store.AbstractRecordReader;
-import org.apache.drill.exec.store.mock.MockGroupScanPOP.MockColumn;
-import org.apache.drill.exec.store.mock.MockGroupScanPOP.MockScanEntry;
+import org.apache.drill.exec.store.mock.MockTableDef.MockColumn;
+import org.apache.drill.exec.store.mock.MockTableDef.MockScanEntry;
 import org.apache.drill.exec.vector.AllocationHelper;
 import org.apache.drill.exec.vector.ValueVector;
 
@@ -55,11 +56,9 @@ public class ExtendedMockRecordReader extends AbstractRecordReader {
   private int recordsRead;
 
   private final MockScanEntry config;
-  private final FragmentContext context;
   private final ColumnDef fields[];
 
   public ExtendedMockRecordReader(FragmentContext context, MockScanEntry config) {
-    this.context = context;
     this.config = config;
 
     fields = buildColumnDefs();
@@ -76,7 +75,7 @@ public class ExtendedMockRecordReader extends AbstractRecordReader {
     Set<String> names = new HashSet<>();
     MockColumn cols[] = config.getTypes();
     for (int i = 0; i < cols.length; i++) {
-      MockColumn col = cols[i];
+      MockTableDef.MockColumn col = cols[i];
       if (names.contains(col.name)) {
         throw new IllegalArgumentException("Duplicate column name: " + col.name);
       }
@@ -95,10 +94,10 @@ public class ExtendedMockRecordReader extends AbstractRecordReader {
     return defArray;
   }
 
-  private int getEstimatedRecordSize(MockColumn[] types) {
+  private int getEstimatedRecordSize() {
     int size = 0;
     for (int i = 0; i < fields.length; i++) {
-      size += TypeHelper.getSize(fields[i].getConfig().getMajorType());
+      size += fields[i].width;
     }
     return size;
   }
@@ -106,9 +105,14 @@ public class ExtendedMockRecordReader extends AbstractRecordReader {
   @Override
   public void setup(OperatorContext context, OutputMutator output) throws ExecutionSetupException {
     try {
-      final int estimateRowSize = getEstimatedRecordSize(config.getTypes());
-      valueVectors = new ValueVector[config.getTypes().length];
-      batchRecordCount = 250000 / estimateRowSize;
+      final int estimateRowSize = getEstimatedRecordSize();
+      valueVectors = new ValueVector[fields.length];
+      int batchSize = config.getBatchSize();
+      if (batchSize == 0) {
+        batchSize = 10 * 1024 * 1024;
+      }
+      batchRecordCount = Math.max(1, batchSize / estimateRowSize);
+      batchRecordCount = Math.min(batchRecordCount, Character.MAX_VALUE);
 
       for (int i = 0; i < fields.length; i++) {
         final ColumnDef col = fields[i];

http://git-wip-us.apache.org/repos/asf/drill/blob/974c6134/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MockGroupScanPOP.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MockGroupScanPOP.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MockGroupScanPOP.java
index 2e8af42..c8082a8 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MockGroupScanPOP.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MockGroupScanPOP.java
@@ -18,7 +18,6 @@
 package org.apache.drill.exec.store.mock;
 
 import java.util.ArrayList;
-import java.util.Arrays;
 import java.util.LinkedList;
 import java.util.List;
 import java.util.regex.Matcher;
@@ -26,19 +25,21 @@ import java.util.regex.Pattern;
 
 import org.apache.drill.common.expression.SchemaPath;
 import org.apache.drill.common.types.TypeProtos.DataMode;
-import org.apache.drill.common.types.TypeProtos.MajorType;
 import org.apache.drill.common.types.TypeProtos.MinorType;
+import org.apache.drill.exec.expr.TypeHelper;
 import org.apache.drill.exec.physical.base.AbstractGroupScan;
 import org.apache.drill.exec.physical.base.GroupScan;
 import org.apache.drill.exec.physical.base.PhysicalOperator;
 import org.apache.drill.exec.physical.base.ScanStats;
+import org.apache.drill.exec.physical.base.ScanStats.GroupScanProperty;
 import org.apache.drill.exec.physical.base.SubScan;
+import org.apache.drill.exec.planner.cost.DrillCostBase;
 import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
+import org.apache.drill.exec.store.mock.MockTableDef.MockColumn;
+import org.apache.drill.exec.store.mock.MockTableDef.MockScanEntry;
 
 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;
@@ -75,20 +76,66 @@ public class MockGroupScanPOP extends AbstractGroupScan {
    */
 
   private boolean extended;
+  private ScanStats scanStats = ScanStats.TRIVIAL_TABLE;
 
   @JsonCreator
   public MockGroupScanPOP(@JsonProperty("url") String url,
-      @JsonProperty("extended") Boolean extended,
       @JsonProperty("entries") List<MockScanEntry> readEntries) {
     super((String) null);
     this.readEntries = readEntries;
     this.url = url;
-    this.extended = extended == null ? false : extended;
+
+    // Compute decent row-count stats for this mock data source so that
+    // the planner is "fooled" into thinking that this operator will do
+    // disk I/O.
+
+    int rowCount = 0;
+    int rowWidth = 0;
+
+    // Can have multiple "read entries" which simulate blocks or
+    // row groups.
+
+    for (MockScanEntry entry : readEntries) {
+      rowCount += entry.getRecords();
+      int groupRowWidth = 0;
+      if (entry.getTypes() == null) {
+        // If no columns, assume a row width.
+        groupRowWidth = 50;
+      } else {
+        // The normal case: we do have columns. Use them
+        // to compute the row width.
+
+        for (MockColumn col : entry.getTypes()) {
+          int colWidth = 0;
+          if (col.getWidthValue() == 0) {
+            // Fixed width columns
+            colWidth = TypeHelper.getSize(col.getMajorType());
+          } else {
+            // Variable width columns with a specified column
+            // width
+            colWidth = col.getWidthValue();
+          }
+
+          // Columns can repeat
+          colWidth *= col.getRepeatCount();
+          groupRowWidth += colWidth;
+        }
+      }
+
+      // Overall row width is the greatest group row width.
+
+      rowWidth = Math.max(rowWidth, groupRowWidth);
+    }
+    int dataSize = rowCount * rowWidth;
+    scanStats = new ScanStats(GroupScanProperty.EXACT_ROW_COUNT,
+                               rowCount,
+                               DrillCostBase.BASE_CPU_COST * dataSize,
+                               DrillCostBase.BYTE_DISK_READ_COST * dataSize);
   }
 
   @Override
   public ScanStats getScanStats() {
-    return ScanStats.TRIVIAL_TABLE;
+    return scanStats;
   }
 
   public String getUrl() {
@@ -100,162 +147,6 @@ public class MockGroupScanPOP extends AbstractGroupScan {
     return readEntries;
   }
 
-  /**
-   * Describes one simulated file (or block) within the logical file scan
-   * described by this group scan. Each block can have a distinct schema to test
-   * for schema changes.
-   */
-
-  public static class MockScanEntry {
-
-    private final int records;
-    private final MockColumn[] types;
-
-    @JsonCreator
-    public MockScanEntry(@JsonProperty("records") int records,
-        @JsonProperty("types") MockColumn[] types) {
-      this.records = records;
-      this.types = types;
-    }
-
-    public int getRecords() {
-      return records;
-    }
-
-    public MockColumn[] getTypes() {
-      return types;
-    }
-
-    @Override
-    public String toString() {
-      return "MockScanEntry [records=" + records + ", columns="
-          + Arrays.toString(types) + "]";
-    }
-  }
-
-  /**
-   * Meta-data description of the columns we wish to create during a simulated
-   * scan.
-   */
-
-  @JsonInclude(Include.NON_NULL)
-  public static class MockColumn {
-
-    /**
-     * Column type given as a Drill minor type (that is, a type without the
-     * extra information such as cardinality, width, etc.
-     */
-
-    @JsonProperty("type")
-    public MinorType minorType;
-    public String name;
-    public DataMode mode;
-    public Integer width;
-    public Integer precision;
-    public Integer scale;
-
-    /**
-     * The scan can request to use a specific data generator class. The name of
-     * that class appears here. The name can be a simple class name, if that
-     * class resides in this Java package. Or, it can be a fully qualified name
-     * of a class that resides elsewhere. If null, the default generator for the
-     * data type is used.
-     */
-
-    public String generator;
-
-    /**
-     * Some tests want to create a very wide row with many columns. This field
-     * eases that task: specify a value other than 1 and the data source will
-     * generate that many copies of the column, each with separately generated
-     * random values. For example, to create 20 copies of field, "foo", set
-     * repeat to 20 and the actual generated batches will contain fields
-     * foo1, foo2, ... foo20.
-     */
-
-    public Integer repeat;
-
-    @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,
-        @JsonProperty("generator") String generator,
-        @JsonProperty("repeat") Integer repeat) {
-      this.name = name;
-      this.minorType = minorType;
-      this.mode = mode;
-      this.width = width;
-      this.precision = precision;
-      this.scale = scale;
-      this.generator = generator;
-      this.repeat = repeat;
-    }
-
-    @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;
-    }
-
-    public String getGenerator() {
-      return generator;
-    }
-
-    public Integer getRepeat() {
-      return repeat;
-    }
-
-    @JsonIgnore
-    public int getRepeatCount() {
-      return repeat == null ? 1 : repeat;
-    }
-
-    @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 String toString() {
-      return "MockColumn [minorType=" + minorType + ", name=" + name + ", mode="
-          + mode + "]";
-    }
-  }
-
   @SuppressWarnings("unchecked")
   @Override
   public void applyAssignments(List<DrillbitEndpoint> endpoints) {
@@ -295,7 +186,7 @@ public class MockGroupScanPOP extends AbstractGroupScan {
   @JsonIgnore
   public PhysicalOperator getNewWithChildren(List<PhysicalOperator> children) {
     Preconditions.checkArgument(children.isEmpty());
-    return new MockGroupScanPOP(url, extended, readEntries);
+    return new MockGroupScanPOP(url, readEntries);
   }
 
   @Override
@@ -304,7 +195,7 @@ public class MockGroupScanPOP extends AbstractGroupScan {
       throw new IllegalArgumentException("No columns for mock scan");
     }
     List<MockColumn> mockCols = new ArrayList<>();
-    Pattern p = Pattern.compile("(\\w+)_([isd])(\\d*)");
+    Pattern p = Pattern.compile("(\\w+)_([isdb])(\\d*)");
     for (SchemaPath path : columns) {
       String col = path.getLastSegment().getNameSegment().getPath();
       if (col.equals("*")) {
@@ -334,21 +225,24 @@ public class MockGroupScanPOP extends AbstractGroupScan {
       case "d":
         minorType = MinorType.FLOAT8;
         break;
+      case "b":
+        minorType = MinorType.BIT;
+        break;
       default:
         throw new IllegalArgumentException(
             "Unsupported field type " + type + " for mock column " + col);
       }
-      MockColumn mockCol = new MockColumn(col, minorType, DataMode.REQUIRED,
-          width, 0, 0, null, 1);
+      MockTableDef.MockColumn mockCol = new MockColumn(
+          col, minorType, DataMode.REQUIRED, width, 0, 0, null, 1, null);
       mockCols.add(mockCol);
     }
     MockScanEntry entry = readEntries.get(0);
     MockColumn types[] = new MockColumn[mockCols.size()];
     mockCols.toArray(types);
-    MockScanEntry newEntry = new MockScanEntry(entry.records, types);
+    MockScanEntry newEntry = new MockScanEntry(entry.records, true, 0, 1, types);
     List<MockScanEntry> newEntries = new ArrayList<>();
     newEntries.add(newEntry);
-    return new MockGroupScanPOP(url, true, newEntries);
+    return new MockGroupScanPOP(url, newEntries);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/drill/blob/974c6134/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MockRecordReader.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MockRecordReader.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MockRecordReader.java
index 6f8cb39..2d9973e 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MockRecordReader.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MockRecordReader.java
@@ -29,8 +29,8 @@ import org.apache.drill.exec.ops.OperatorContext;
 import org.apache.drill.exec.physical.impl.OutputMutator;
 import org.apache.drill.exec.record.MaterializedField;
 import org.apache.drill.exec.store.AbstractRecordReader;
-import org.apache.drill.exec.store.mock.MockGroupScanPOP.MockColumn;
-import org.apache.drill.exec.store.mock.MockGroupScanPOP.MockScanEntry;
+import org.apache.drill.exec.store.mock.MockTableDef.MockColumn;
+import org.apache.drill.exec.store.mock.MockTableDef.MockScanEntry;
 import org.apache.drill.exec.vector.AllocationHelper;
 import org.apache.drill.exec.vector.ValueVector;
 

http://git-wip-us.apache.org/repos/asf/drill/blob/974c6134/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MockScanBatchCreator.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MockScanBatchCreator.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MockScanBatchCreator.java
index 9cdb7ad..9a7563a 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MockScanBatchCreator.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MockScanBatchCreator.java
@@ -25,7 +25,8 @@ 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 org.apache.drill.exec.store.mock.MockTableDef.MockScanEntry;
 
 import com.google.common.base.Preconditions;
 import com.google.common.collect.Lists;
@@ -39,8 +40,8 @@ public class MockScanBatchCreator implements BatchCreator<MockSubScanPOP> {
     Preconditions.checkArgument(children.isEmpty());
     final List<MockScanEntry> entries = config.getReadEntries();
     final List<RecordReader> readers = Lists.newArrayList();
-    for(final MockScanEntry e : entries) {
-      if ( config.isExtended( ) ) {
+    for(final MockTableDef.MockScanEntry e : entries) {
+      if ( e.isExtended( ) ) {
         readers.add(new ExtendedMockRecordReader(context, e));
       } else {
         readers.add(new MockRecordReader(context, e));

http://git-wip-us.apache.org/repos/asf/drill/blob/974c6134/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MockStorageEngine.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MockStorageEngine.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MockStorageEngine.java
index df8ee50..90644b5 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MockStorageEngine.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MockStorageEngine.java
@@ -18,6 +18,7 @@
 package org.apache.drill.exec.store.mock;
 
 import java.io.IOException;
+import java.net.URL;
 import java.util.ArrayList;
 import java.util.HashSet;
 import java.util.List;
@@ -36,11 +37,15 @@ import org.apache.drill.exec.server.DrillbitContext;
 import org.apache.drill.exec.store.AbstractSchema;
 import org.apache.drill.exec.store.AbstractStoragePlugin;
 import org.apache.drill.exec.store.SchemaConfig;
-import org.apache.drill.exec.store.mock.MockGroupScanPOP.MockScanEntry;
 
+import com.fasterxml.jackson.core.JsonParseException;
+import com.fasterxml.jackson.core.JsonParser;
 import com.fasterxml.jackson.core.type.TypeReference;
+import com.fasterxml.jackson.databind.JsonMappingException;
 import com.fasterxml.jackson.databind.ObjectMapper;
+import com.google.common.base.Charsets;
 import com.google.common.collect.ImmutableList;
+import com.google.common.io.Resources;
 
 public class MockStorageEngine extends AbstractStoragePlugin {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(MockStorageEngine.class);
@@ -57,21 +62,12 @@ public class MockStorageEngine extends AbstractStoragePlugin {
   public AbstractGroupScan getPhysicalScan(String userName, JSONOptions selection, List<SchemaPath> columns)
       throws IOException {
 
-    List<MockScanEntry> readEntries = selection.getListWith(new ObjectMapper(),
-        new TypeReference<ArrayList<MockScanEntry>>() {
+    List<MockTableDef.MockScanEntry> readEntries = selection.getListWith(new ObjectMapper(),
+        new TypeReference<ArrayList<MockTableDef.MockScanEntry>>() {
         });
 
-    // The classic (logical-plan based) and extended (SQL-based) paths
-    // come through here. If this is a SQL query, then no columns are
-    // defined in the plan.
-
     assert ! readEntries.isEmpty();
-    boolean extended = readEntries.size() == 1;
-    if (extended) {
-      MockScanEntry entry = readEntries.get(0);
-      extended = entry.getTypes() == null;
-    }
-    return new MockGroupScanPOP(null, extended, readEntries);
+    return new MockGroupScanPOP(null, readEntries);
   }
 
   @Override
@@ -89,14 +85,31 @@ public class MockStorageEngine extends AbstractStoragePlugin {
     return true;
   }
 
-//  public static class ImplicitTable extends DynamicDrillTable {
-//
-//    public ImplicitTable(StoragePlugin plugin, String storageEngineName,
-//        Object selection) {
-//      super(plugin, storageEngineName, selection);
-//    }
-//
-//  }
+  /**
+   * Resolves table names within the mock data source. Tables can be of two forms:
+   * <p>
+   * <tt><name>_<n><unit></tt>
+   * <p>
+   * Where the "name" can be anything, "n" is the number of rows, and "unit" is
+   * the units for the row count: non, K (thousand) or M (million).
+   * <p>
+   * The above form generates a table directly with no other information needed.
+   * Column names must be provided, and must be of the form:
+   * <p>
+   * <tt><name>_<type><size></tt>
+   * <p>
+   * Where the name can be anything, the type must be i (integer), d (double),
+   * b (boolean)
+   * or s (string, AKA VarChar). The length is needed only for string fields.
+   * <p>
+   * Direct tables are quick, but limited. The other option is to provide the
+   * name of a definition file:
+   * <p>
+   * <tt><fileName>.json</tt>
+   * <p>
+   * In this case, the JSON file must be a resource visible on the class path.
+   * Omit the leading slash in the resource path name.
+   */
 
   private static class MockSchema extends AbstractSchema {
 
@@ -109,6 +122,36 @@ public class MockStorageEngine extends AbstractStoragePlugin {
 
     @Override
     public Table getTable(String name) {
+      if (name.toLowerCase().endsWith(".json")) {
+        return getConfigFile(name);
+      } else {
+        return getDirectTable(name);
+      }
+    }
+
+    private Table getConfigFile(String name) {
+      final URL url = Resources.getResource(name);
+      if (url == null) {
+        throw new IllegalArgumentException(
+            "Unable to find mock table config file " + name);
+      }
+      MockTableDef mockTableDefn;
+      try {
+        String json = Resources.toString(url, Charsets.UTF_8);
+        final ObjectMapper mapper = new ObjectMapper();
+        mapper.configure(JsonParser.Feature.ALLOW_UNQUOTED_FIELD_NAMES, true);
+        mockTableDefn = mapper.readValue(json, MockTableDef.class);
+      } catch (JsonParseException e) {
+        throw new IllegalArgumentException("Unable to parse mock table definition file: " + name, e);
+      } catch (JsonMappingException e) {
+        throw new IllegalArgumentException("Unable to Jackson deserialize mock table definition file: " + name, e);
+      } catch (IOException e) {
+        throw new IllegalArgumentException("Unable to read mock table definition file: " + name, e);
+      }
+      return new DynamicDrillTable(engine, this.name, mockTableDefn.getEntries());
+    }
+
+    private Table getDirectTable(String name) {
       Pattern p = Pattern.compile("(\\w+)_(\\d+)(k|m)?", Pattern.CASE_INSENSITIVE);
       Matcher m = p.matcher(name);
       if (! m.matches()) {
@@ -118,10 +161,11 @@ public class MockStorageEngine extends AbstractStoragePlugin {
       String baseName = m.group(1);
       int n = Integer.parseInt(m.group(2));
       String unit = m.group(3);
-      if (unit.equalsIgnoreCase("K")) { n *= 1000; }
+      if (unit == null) { }
+      else if (unit.equalsIgnoreCase("K")) { n *= 1000; }
       else if (unit.equalsIgnoreCase("M")) { n *= 1_000_000; }
-      MockScanEntry entry = new MockScanEntry(n, null);
-      List<MockScanEntry> list = new ArrayList<>();
+      MockTableDef.MockScanEntry entry = new MockTableDef.MockScanEntry(n, true, 0, 1, null);
+      List<MockTableDef.MockScanEntry> list = new ArrayList<>();
       list.add(entry);
       return new DynamicDrillTable(engine, this.name, list);
     }

http://git-wip-us.apache.org/repos/asf/drill/blob/974c6134/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MockSubScanPOP.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MockSubScanPOP.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MockSubScanPOP.java
index f169f51..8e474ca 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MockSubScanPOP.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MockSubScanPOP.java
@@ -17,6 +17,7 @@
  */
 package org.apache.drill.exec.store.mock;
 
+import java.util.Collections;
 import java.util.Iterator;
 import java.util.List;
 
@@ -25,13 +26,13 @@ import org.apache.drill.exec.physical.base.PhysicalOperator;
 import org.apache.drill.exec.physical.base.PhysicalVisitor;
 import org.apache.drill.exec.physical.base.SubScan;
 import org.apache.drill.exec.proto.UserBitShared.CoreOperatorType;
+import org.apache.drill.exec.store.mock.MockTableDef.MockScanEntry;
 
 import com.fasterxml.jackson.annotation.JsonCreator;
 import com.fasterxml.jackson.annotation.JsonIgnore;
 import com.fasterxml.jackson.annotation.JsonProperty;
 import com.fasterxml.jackson.annotation.JsonTypeName;
 import com.google.common.base.Preconditions;
-import com.google.common.collect.Iterators;
 
 /**
  * Describes a physical scan operation for the mock data source. Each operator
@@ -44,7 +45,7 @@ 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;
+  protected final List<MockScanEntry> readEntries;
   private final boolean extended;
 
   /**
@@ -68,7 +69,7 @@ public class MockSubScanPOP extends AbstractBase implements SubScan {
   @JsonCreator
   public MockSubScanPOP(@JsonProperty("url") String url,
                         @JsonProperty("extended") Boolean extended,
-                        @JsonProperty("entries") List<MockGroupScanPOP.MockScanEntry> readEntries) {
+                        @JsonProperty("entries") List<MockScanEntry> readEntries) {
     this.readEntries = readEntries;
 //    OperatorCost cost = new OperatorCost(0,0,0,0);
 //    Size size = new Size(0,0);
@@ -86,13 +87,13 @@ public class MockSubScanPOP extends AbstractBase implements SubScan {
   public boolean isExtended() { return extended; }
 
   @JsonProperty("entries")
-  public List<MockGroupScanPOP.MockScanEntry> getReadEntries() {
+  public List<MockScanEntry> getReadEntries() {
     return readEntries;
   }
 
   @Override
   public Iterator<PhysicalOperator> iterator() {
-    return Iterators.emptyIterator();
+    return Collections.emptyIterator();
   }
 
   // will want to replace these two methods with an interface above for AbstractSubScan

http://git-wip-us.apache.org/repos/asf/drill/blob/974c6134/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MockTableDef.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MockTableDef.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MockTableDef.java
new file mode 100644
index 0000000..81f92b1
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MockTableDef.java
@@ -0,0 +1,213 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.Arrays;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.drill.common.types.TypeProtos.DataMode;
+import org.apache.drill.common.types.TypeProtos.MajorType;
+import org.apache.drill.common.types.TypeProtos.MinorType;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonIgnore;
+import com.fasterxml.jackson.annotation.JsonInclude;
+import com.fasterxml.jackson.annotation.JsonInclude.Include;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.annotation.JsonTypeName;
+
+/**
+ * Structure of a mock table definition file. Yes, using Jackson deserialization to parse
+ * the file is brittle, but this is for testing so we're favoring convenience
+ * over robustness.
+ */
+
+@JsonTypeName("mock-table")
+public class MockTableDef {
+  /**
+   * Describes one simulated file (or block) within the logical file scan
+   * described by this group scan. Each block can have a distinct schema to test
+   * for schema changes.
+   */
+
+  public static class MockScanEntry {
+
+    final int records;
+    final boolean extended;
+    final int batchSize;
+    final int repeat;
+    private final MockColumn[] types;
+
+    @JsonCreator
+    public MockScanEntry(@JsonProperty("records") int records,
+                         @JsonProperty("extended") Boolean extended,
+                         @JsonProperty("batchSize") Integer batchSize,
+                         @JsonProperty("repeat") Integer repeat,
+                         @JsonProperty("types") MockTableDef.MockColumn[] types) {
+      this.records = records;
+      this.types = types;
+      this.extended = (extended == null) ? false : extended;
+      this.batchSize = (batchSize == null) ? 0 : batchSize;
+      this.repeat = (repeat == null) ? 1 : repeat;
+    }
+
+    public int getRecords() { return records; }
+    public boolean isExtended() { return extended; }
+    public int getBatchSize() { return batchSize; }
+    public int getRepeat() { return repeat; }
+
+    public MockTableDef.MockColumn[] getTypes() {
+      return types;
+    }
+
+    @Override
+    public String toString() {
+      return "MockScanEntry [records=" + records + ", columns="
+          + Arrays.toString(types) + "]";
+    }
+  }
+
+  /**
+   * Meta-data description of the columns we wish to create during a simulated
+   * scan.
+   */
+
+  @JsonInclude(Include.NON_NULL)
+  public static class MockColumn {
+
+    /**
+     * Column type given as a Drill minor type (that is, a type without the
+     * extra information such as cardinality, width, etc.
+     */
+
+    @JsonProperty("type")
+    public MinorType minorType;
+    public String name;
+    public DataMode mode;
+    public Integer width;
+    public Integer precision;
+    public Integer scale;
+
+    /**
+     * The scan can request to use a specific data generator class. The name of
+     * that class appears here. The name can be a simple class name, if that
+     * class resides in this Java package. Or, it can be a fully qualified name
+     * of a class that resides elsewhere. If null, the default generator for the
+     * data type is used.
+     */
+
+    public String generator;
+
+    /**
+     * Some tests want to create a very wide row with many columns. This field
+     * eases that task: specify a value other than 1 and the data source will
+     * generate that many copies of the column, each with separately generated
+     * random values. For example, to create 20 copies of field, "foo", set
+     * repeat to 20 and the actual generated batches will contain fields
+     * foo1, foo2, ... foo20.
+     */
+
+    public Integer repeat;
+    public Map<String,Object> properties;
+
+    @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,
+                      @JsonProperty("generator") String generator,
+                      @JsonProperty("repeat") Integer repeat,
+                      @JsonProperty("properties") Map<String,Object> properties) {
+      this.name = name;
+      this.minorType = minorType;
+      this.mode = mode;
+      this.width = width;
+      this.precision = precision;
+      this.scale = scale;
+      this.generator = generator;
+      this.repeat = repeat;
+      this.properties = properties;
+    }
+
+    @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; }
+    public String getGenerator() { return generator; }
+    public Integer getRepeat() { return repeat; }
+    @JsonIgnore
+    public int getRepeatCount() { return repeat == null ? 1 : repeat; }
+    @JsonIgnore
+    public int getWidthValue() { return width == null ? 0 : width; }
+    public Map<String,Object> getProperties() { return properties; }
+
+    @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 String toString() {
+      return "MockColumn [minorType=" + minorType + ", name=" + name + ", mode="
+          + mode + "]";
+    }
+  }
+
+  private String descrip;
+  List<MockTableDef.MockScanEntry> entries;
+
+  public MockTableDef(@JsonProperty("descrip") final String descrip,
+                      @JsonProperty("entries") final List<MockTableDef.MockScanEntry> entries) {
+    this.descrip = descrip;
+    this.entries = entries;
+  }
+
+  /**
+   * Description of this data source. Ignored by the scanner, purely
+   * for the convenience of the author.
+   */
+
+  public String getDescrip() { return descrip; }
+
+  /**
+   * The set of entries that define the groups within the file. Each
+   * group can have a distinct schema; each may be read in a separate
+   * fragment.
+   * @return
+   */
+
+  public List<MockTableDef.MockScanEntry> getEntries() { return entries; }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/974c6134/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/VaryingStringGen.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/VaryingStringGen.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/VaryingStringGen.java
new file mode 100644
index 0000000..bf0dec7
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/VaryingStringGen.java
@@ -0,0 +1,70 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.exec.store.mock;
+
+import java.util.Map;
+import java.util.Random;
+
+import org.apache.drill.exec.vector.ValueVector;
+import org.apache.drill.exec.vector.VarCharVector;
+
+public class VaryingStringGen implements FieldGen {
+
+  private Random rand = new Random();
+  private int length;
+  private int span;
+  private int deltaPerSpan;
+  private int valueCount;
+
+  @Override
+  public void setup(ColumnDef colDef) {
+    length = colDef.width;
+    Map<String,Object> props = colDef.mockCol.properties;
+    span = 1000;
+    deltaPerSpan = 100;
+    if (props != null) {
+      Integer value = (Integer) props.get("span");
+      if (value != null) {
+        span = Math.max(1, value);
+      }
+      value = (Integer) props.get("delta");
+      if (value != null) {
+        deltaPerSpan = value;
+      }
+    }
+  }
+
+  public String value() {
+    if (valueCount++ >= span) {
+      valueCount = 0;
+      length = Math.max(0, length + deltaPerSpan);
+    }
+    String c = Character.toString((char) (rand.nextInt(26) + 'A'));
+    StringBuilder buf = new StringBuilder();
+    for (int i = 0;  i < length;  i++) {
+      buf.append(c);
+    }
+    return buf.toString();
+  }
+
+  @Override
+  public void setValue(ValueVector v, int index) {
+    VarCharVector vector = (VarCharVector) v;
+    vector.getMutator().setSafe(index, value().getBytes());
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/974c6134/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/package-info.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/package-info.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/package-info.java
index e99cfc5..ad4595d 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/package-info.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/package-info.java
@@ -30,19 +30,21 @@
  * </ul>
  * <h3>Classic Mode</h3>
  * Create a scan operator that looks like the following (from
- * <tt></tt>):
+ * <tt>/src/test/resources/functions/cast/two_way_implicit_cast.json</tt>,
+ * used in {@link TestReverseImplicitCast}):
  * <pre><code>
  *    graph:[
- *      {
- *        {@literal @}id:1,
- *        pop:"mock-scan",
- *        url: "http://apache.org",
- *        entries:[
- *          {records: 1000000, types: [
- *             {name: "blue", type: "INT", mode: "REQUIRED"},
- *             {name: "green", type: "INT", mode: "REQUIRED"}
- *        ]}
- *      ]
+ *        {
+ *            @id:1,
+ *            pop:"mock-scan",
+ *            url: "http://apache.org",
+ *            entries:[
+ *                {records: 1, types: [
+ *                    {name: "col1", type: "FLOAT4", mode: "REQUIRED"},
+ *                    {name: "col2", type: "FLOAT8", mode: "REQUIRED"}
+ *                ]}
+ *            ]
+ *        },
  *    }, ...
  * </code></pre>
  * Here:
@@ -60,6 +62,18 @@
  * <li>The <tt>mode</tt> is one of the supported Drill
  * {@link DataMode} names: usually <tt>OPTIONAL</tt> or <tt>REQUIRED</tt>.</li>
  * </ul>
+ * <p>
+ * Recent extensions include:
+ * <ul>
+ * <li><tt>repeat</tt> in either the "entry" or "record" elements allow
+ * repeating entries (simulating multiple blocks or row groups) and
+ * repeating fields (easily create a dozen fields of some type.)</li>
+ * <li><tt>generator</tt> in a field definition lets you specify a
+ * specific data generator (see below.)</tt>
+ * <li><tt>properties</tt> in a field definition lets you pass
+ * generator-specific values to the data generator (such as, say
+ * a minimum and maximum value.)</li>
+ * </ul>
  *
  * <h3>Enhanced Mode</h3>
  * Enhanced builds on the Classic mode to add additional capabilities.
@@ -67,7 +81,7 @@
  * is randomly generated over a wide range of values and can be
  * controlled by custom generator classes. When
  * in a physical plan, the <tt>records</tt> section has additional
- * attributes as described in {@link MockGroupScanPOP.MockColumn}:
+ * attributes as described in {@link MockTableDef.MockColumn}:
  * <ul>
  * <li>The <tt>generator</tt> lets you specify a class to generate the
  * sample data. Rules for the class name is that it can either contain
@@ -111,6 +125,9 @@
  * (multiply row count by one million), case insensitive.</li>
  * <li>Another field (not yet implemented) might specify the split count.</li>
  * </ul>
+ * <h3>Enhanced Mode with Definition File</h3>
+ * You can reference a mock data definition file directly from SQL as follows:
+ * <pre<code>SELECT * FROM `mock`.`your_defn_file.json`</code></pre>
  * <h3>Data Generators</h3>
  * The classic mode uses data generators built into each vector to generate
  * the sample data. These generators use a very simple black/white alternating

http://git-wip-us.apache.org/repos/asf/drill/blob/974c6134/exec/java-exec/src/main/java/org/apache/drill/exec/util/TestUtilities.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/util/TestUtilities.java b/exec/java-exec/src/main/java/org/apache/drill/exec/util/TestUtilities.java
index 7215d10..5498ad4 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/util/TestUtilities.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/util/TestUtilities.java
@@ -17,15 +17,15 @@
  */
 package org.apache.drill.exec.util;
 
-import com.google.common.io.Files;
+import java.io.File;
+
 import org.apache.drill.common.exceptions.ExecutionSetupException;
-import org.apache.drill.exec.server.DrillbitContext;
 import org.apache.drill.exec.store.StoragePluginRegistry;
 import org.apache.drill.exec.store.dfs.FileSystemConfig;
 import org.apache.drill.exec.store.dfs.FileSystemPlugin;
 import org.apache.drill.exec.store.dfs.WorkspaceConfig;
 
-import java.io.File;
+import com.google.common.io.Files;
 
 /**
  * This class contains utility methods to speed up tests. Some of the production code currently calls this method

http://git-wip-us.apache.org/repos/asf/drill/blob/974c6134/exec/java-exec/src/test/java/org/apache/drill/exec/fn/interp/ExpressionInterpreterTest.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/fn/interp/ExpressionInterpreterTest.java b/exec/java-exec/src/test/java/org/apache/drill/exec/fn/interp/ExpressionInterpreterTest.java
index e191d35..673bf80 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/fn/interp/ExpressionInterpreterTest.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/fn/interp/ExpressionInterpreterTest.java
@@ -42,11 +42,12 @@ import org.apache.drill.exec.record.MaterializedField;
 import org.apache.drill.exec.record.RecordBatch;
 import org.apache.drill.exec.server.Drillbit;
 import org.apache.drill.exec.server.RemoteServiceSet;
-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.mock.MockTableDef;
 import org.apache.drill.exec.vector.ValueVector;
 import org.joda.time.DateTime;
+import org.joda.time.DateTimeZone;
 import org.junit.Test;
 
 import com.google.common.collect.Lists;
@@ -123,9 +124,9 @@ public class ExpressionInterpreterTest  extends PopUnitTestBase {
     final String expressionStr = "now()";
     final BitControl.PlanFragment planFragment = BitControl.PlanFragment.getDefaultInstance();
     final QueryContextInformation queryContextInfo = planFragment.getContext();
-    final int                        timeZoneIndex = queryContextInfo.getTimeZone();
-    final org.joda.time.DateTimeZone timeZone = org.joda.time.DateTimeZone.forID(org.apache.drill.exec.expr.fn.impl.DateUtility.getTimeZone(timeZoneIndex));
-    final org.joda.time.DateTime     now = new org.joda.time.DateTime(queryContextInfo.getQueryStartTime(), timeZone);
+    final int timeZoneIndex = queryContextInfo.getTimeZone();
+    final DateTimeZone timeZone = DateTimeZone.forID(org.apache.drill.exec.expr.fn.impl.DateUtility.getTimeZone(timeZoneIndex));
+    final org.joda.time.DateTime now = new org.joda.time.DateTime(queryContextInfo.getQueryStartTime(), timeZone);
 
     final long queryStartDate = now.getMillis();
 
@@ -159,13 +160,13 @@ public class ExpressionInterpreterTest  extends PopUnitTestBase {
     // Create a mock scan batch as input for evaluation.
     assertEquals(colNames.length, colTypes.length);
 
-    final MockGroupScanPOP.MockColumn[] columns = new MockGroupScanPOP.MockColumn[colNames.length];
+    final MockTableDef.MockColumn[] columns = new MockTableDef.MockColumn[colNames.length];
 
     for (int i = 0; i < colNames.length; i++ ) {
-      columns[i] = new MockGroupScanPOP.MockColumn(colNames[i], colTypes[i].getMinorType(), colTypes[i].getMode(), 0, 0, 0, null, null);
+      columns[i] = new MockTableDef.MockColumn(colNames[i], colTypes[i].getMinorType(), colTypes[i].getMode(), 0, 0, 0, null, null, null);
     }
 
-    final MockGroupScanPOP.MockScanEntry entry = new MockGroupScanPOP.MockScanEntry(10, columns);
+    final MockTableDef.MockScanEntry entry = new MockTableDef.MockScanEntry(10, false, 0, 1, columns);
     final MockSubScanPOP scanPOP = new MockSubScanPOP("testTable", false, java.util.Collections.singletonList(entry));
 
     @SuppressWarnings("resource")

http://git-wip-us.apache.org/repos/asf/drill/blob/974c6134/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestConvertFunctions.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestConvertFunctions.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestConvertFunctions.java
index 16dd0ab..23912eb 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestConvertFunctions.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestConvertFunctions.java
@@ -31,9 +31,8 @@ import java.util.List;
 import org.apache.drill.BaseTestQuery;
 import org.apache.drill.QueryTestUtil;
 import org.apache.drill.exec.ExecConstants;
-import org.apache.drill.exec.compile.ClassTransformer;
-import org.apache.drill.exec.compile.CodeCompiler;
 import org.apache.drill.exec.compile.ClassTransformer.ScalarReplacementOption;
+import org.apache.drill.exec.compile.CodeCompiler;
 import org.apache.drill.exec.expr.fn.impl.DateUtility;
 import org.apache.drill.exec.proto.UserBitShared.QueryType;
 import org.apache.drill.exec.record.RecordBatchLoader;
@@ -588,6 +587,7 @@ public class TestConvertFunctions extends BaseTestQuery {
   public void testHadooopVInt() throws Exception {
     final int _0 = 0;
     final int _9 = 9;
+    @SuppressWarnings("resource")
     final DrillBuf buffer = getAllocator().buffer(_9);
 
     long longVal = 0;
@@ -677,6 +677,7 @@ public class TestConvertFunctions extends BaseTestQuery {
     for(QueryDataBatch result : resultList) {
       if (result.getData() != null) {
         loader.load(result.getHeader().getDef(), result.getData());
+        @SuppressWarnings("resource")
         ValueVector v = loader.iterator().next().getValueVector();
         for (int j = 0; j < v.getAccessor().getValueCount(); j++) {
           if  (v instanceof VarCharVector) {

http://git-wip-us.apache.org/repos/asf/drill/blob/974c6134/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/mergereceiver/TestMergingReceiver.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/mergereceiver/TestMergingReceiver.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/mergereceiver/TestMergingReceiver.java
index 71a5070..e4a96bd 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/mergereceiver/TestMergingReceiver.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/mergereceiver/TestMergingReceiver.java
@@ -44,6 +44,7 @@ public class TestMergingReceiver extends PopUnitTestBase {
 
   @Test
   public void twoBitTwoExchange() throws Exception {
+    @SuppressWarnings("resource")
     final RemoteServiceSet serviceSet = RemoteServiceSet.getLocalServiceSet();
 
     try (final Drillbit bit1 = new Drillbit(CONFIG, serviceSet);
@@ -72,6 +73,7 @@ public class TestMergingReceiver extends PopUnitTestBase {
 
   @Test
   public void testMultipleProvidersMixedSizes() throws Exception {
+    @SuppressWarnings("resource")
     final RemoteServiceSet serviceSet = RemoteServiceSet.getLocalServiceSet();
 
     try (final Drillbit bit1 = new Drillbit(CONFIG, serviceSet);
@@ -95,6 +97,7 @@ public class TestMergingReceiver extends PopUnitTestBase {
         count += batchRowCount;
         batchLoader.load(queryData.getDef(), b.getData());
         for (final VectorWrapper<?> vw : batchLoader) {
+          @SuppressWarnings("resource")
           final ValueVector vv = vw.getValueVector();
           final ValueVector.Accessor va = vv.getAccessor();
           final MaterializedField materializedField = vv.getField();
@@ -119,6 +122,7 @@ public class TestMergingReceiver extends PopUnitTestBase {
 
   @Test
   public void handleEmptyBatch() throws Exception {
+    @SuppressWarnings("resource")
     final RemoteServiceSet serviceSet = RemoteServiceSet.getLocalServiceSet();
 
     try (final Drillbit bit1 = new Drillbit(CONFIG, serviceSet);

http://git-wip-us.apache.org/repos/asf/drill/blob/974c6134/exec/java-exec/src/test/resources/test/example-mock.json
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/resources/test/example-mock.json b/exec/java-exec/src/test/resources/test/example-mock.json
new file mode 100644
index 0000000..a0d2d73
--- /dev/null
+++ b/exec/java-exec/src/test/resources/test/example-mock.json
@@ -0,0 +1,16 @@
+{
+    descrip: "basic example",
+    entries:[
+        {records: 10, types: [
+          {name: "blue", type: "INT", mode: "REQUIRED", repeat: 2},
+          {name: "red", type: "BIGINT", mode: "REQUIRED"},
+          {name: "green", type: "INT", mode: "REQUIRED",
+           properties: { a: 10, b: "foo" }}
+        ]},
+        {records: 10, repeat: 2, types: [
+          {name: "blue", type: "INT", mode: "REQUIRED", repeat: 2},
+          {name: "red", type: "BIGINT", mode: "REQUIRED"},
+          {name: "green", type: "INT", mode: "REQUIRED"}
+        ]}
+    ]
+}