You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@drill.apache.org by ja...@apache.org on 2015/04/15 17:56:07 UTC

[6/6] drill git commit: DRILL-2695: Add Support for large in conditions through the use of the Values operator. Update JSON reader to support reading Extended JSON. Update JSON writer to support writing extended JSON data. Update JSON reader to automatic

DRILL-2695: Add Support for large in conditions through the use of the Values operator.
Update JSON reader to support reading Extended JSON.
Update JSON writer to support writing extended JSON data.
Update JSON reader to automatically unwrap a file that includes a single top-level array (used by values).
Update Options manager to use getOption(<Type>Validator) to directly retrieve typed value.
Remove JSON rewinding
Add support for CONVERT_TO( [], 'SIMPLEJSON') to disable extended types as part of udf use.


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

Branch: refs/heads/master
Commit: 314e5a2a8f476f059153fde1b7e7da7d882db94e
Parents: 75455e8
Author: Jacques Nadeau <ja...@apache.org>
Authored: Sun Mar 8 21:37:44 2015 -0700
Committer: Jacques Nadeau <ja...@apache.org>
Committed: Wed Apr 15 08:10:35 2015 -0700

----------------------------------------------------------------------
 .../org/apache/drill/common/JSONOptions.java    |  12 +
 .../drill/common/logical/data/Constant.java     |  50 --
 .../drill/common/logical/data/Values.java       |  69 +++
 .../data/visitors/AbstractLogicalVisitor.java   |   4 +-
 .../logical/data/visitors/LogicalVisitor.java   |   4 +-
 .../org/apache/drill/common/types/Types.java    |  48 +-
 .../exec/store/mongo/MongoRecordReader.java     |   2 +-
 .../src/main/codegen/templates/BaseWriter.java  |   5 -
 .../main/codegen/templates/ComplexReaders.java  |  24 +-
 .../main/codegen/templates/ComplexWriters.java  |  61 +--
 .../templates/JsonOutputRecordWriter.java       | 119 ++---
 .../src/main/codegen/templates/ListWriters.java |  48 +-
 .../src/main/codegen/templates/MapWriters.java  |  44 +-
 .../org/apache/drill/exec/ExecConstants.java    |   3 +-
 .../drill/exec/expr/EvaluationVisitor.java      |   5 +-
 .../drill/exec/expr/fn/impl/DateUtility.java    |  13 +
 .../exec/expr/fn/impl/conv/JsonConvertFrom.java |   4 +-
 .../exec/expr/fn/impl/conv/JsonConvertTo.java   |  32 +-
 .../physical/base/AbstractPhysicalVisitor.java  |   6 +
 .../exec/physical/base/PhysicalVisitor.java     |   2 +
 .../drill/exec/physical/config/Values.java      |  71 +++
 .../physical/impl/flatten/FlattenTemplate.java  |   5 +-
 .../impl/values/ValuesBatchCreator.java         |  44 ++
 .../drill/exec/planner/logical/DrillOptiq.java  |   4 +-
 .../exec/planner/logical/DrillRuleSets.java     |   3 +
 .../exec/planner/logical/DrillValuesRel.java    | 255 ++++++++-
 .../planner/logical/ScanFieldDeterminer.java    |   4 +-
 .../drill/exec/planner/physical/ValuesPrel.java |  78 +++
 .../exec/planner/physical/ValuesPrule.java      |  48 ++
 .../exec/server/options/BaseOptionManager.java  |  56 ++
 .../server/options/FallbackOptionManager.java   |   2 +-
 .../exec/server/options/OptionManager.java      |   9 +
 .../server/options/SystemOptionManager.java     |   7 +-
 .../exec/store/easy/json/JSONFormatPlugin.java  |   2 +
 .../exec/store/easy/json/JSONRecordReader.java  | 127 +++--
 .../exec/store/easy/json/JsonProcessor.java     |   4 +-
 .../exec/store/easy/json/JsonRecordWriter.java  |  14 +-
 .../store/easy/json/RewindableUtf8Reader.java   | 157 ------
 .../easy/json/reader/BaseJsonProcessor.java     |  38 +-
 .../easy/json/reader/CountingJsonReader.java    |   4 +-
 .../exec/store/parquet2/DrillParquetReader.java |   9 -
 .../DrillParquetRecordMaterializer.java         |   4 -
 .../drill/exec/vector/RepeatedVector.java       |   2 +-
 .../drill/exec/vector/complex/WriteState.java   |  46 --
 .../exec/vector/complex/fn/BasicJsonOutput.java | 530 +++++++++++++++++++
 .../vector/complex/fn/DateOutputFormat.java     |  37 ++
 .../vector/complex/fn/ExtendedJsonOutput.java   | 183 +++++++
 .../exec/vector/complex/fn/ExtendedType.java    |  40 ++
 .../vector/complex/fn/ExtendedTypeName.java     |  29 +
 .../exec/vector/complex/fn/JsonOutput.java      | 109 ++++
 .../exec/vector/complex/fn/JsonReader.java      | 335 +++++++-----
 .../exec/vector/complex/fn/JsonWriter.java      | 104 ++--
 .../exec/vector/complex/fn/VectorOutput.java    | 295 +++++++++++
 .../exec/vector/complex/fn/WorkingBuffer.java   |  70 +++
 .../vector/complex/impl/AbstractBaseWriter.java |  21 -
 .../vector/complex/impl/ComplexWriterImpl.java  |   5 -
 .../complex/impl/RepeatedListReaderImpl.java    |   4 +-
 .../complex/impl/RepeatedMapReaderImpl.java     |   4 +-
 .../complex/impl/SingleMapReaderImpl.java       |  12 +-
 .../complex/impl/VectorContainerWriter.java     |   5 -
 .../physical/impl/filter/TestLargeInClause.java |  62 +++
 .../exec/store/json/TestJsonRecordReader.java   |   5 +
 .../complex/writer/TestExtendedTypes.java       |  57 ++
 .../vector/complex/writer/TestRepeated.java     |   3 +-
 .../src/test/resources/store/json/listdoc.json  |   4 +
 .../test/resources/vector/complex/extended.json |  41 ++
 66 files changed, 2708 insertions(+), 795 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/314e5a2a/common/src/main/java/org/apache/drill/common/JSONOptions.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/drill/common/JSONOptions.java b/common/src/main/java/org/apache/drill/common/JSONOptions.java
index 945cd92..9e0514c 100644
--- a/common/src/main/java/org/apache/drill/common/JSONOptions.java
+++ b/common/src/main/java/org/apache/drill/common/JSONOptions.java
@@ -42,7 +42,9 @@ import com.fasterxml.jackson.databind.SerializerProvider;
 import com.fasterxml.jackson.databind.annotation.JsonDeserialize;
 import com.fasterxml.jackson.databind.annotation.JsonSerialize;
 import com.fasterxml.jackson.databind.deser.std.StdDeserializer;
+import com.fasterxml.jackson.databind.node.TreeTraversingParser;
 import com.fasterxml.jackson.databind.ser.std.StdSerializer;
+import com.google.common.base.Preconditions;
 
 @JsonSerialize(using = Se.class)
 @JsonDeserialize(using = De.class)
@@ -98,6 +100,16 @@ public class JSONOptions {
     return getListWith(config.getMapper(), t);
   }
 
+  public JsonNode asNode(){
+    Preconditions.checkArgument(this.root != null, "Attempted to grab JSONOptions as JsonNode when no root node was stored.  You can only convert non-opaque JSONOptions values to JsonNodes.");
+    return root;
+  }
+
+  public JsonParser asParser(){
+    Preconditions.checkArgument(this.root != null, "Attempted to grab JSONOptions as Parser when no root node was stored.  You can only convert non-opaque JSONOptions values to parsers.");
+    return new TreeTraversingParser(root);
+  }
+
   @SuppressWarnings("unchecked")
   public <T> T getListWith(ObjectMapper mapper, TypeReference<T> t) throws IOException {
     if (opaque != null) {

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

http://git-wip-us.apache.org/repos/asf/drill/blob/314e5a2a/common/src/main/java/org/apache/drill/common/logical/data/Values.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/drill/common/logical/data/Values.java b/common/src/main/java/org/apache/drill/common/logical/data/Values.java
new file mode 100644
index 0000000..9276e55
--- /dev/null
+++ b/common/src/main/java/org/apache/drill/common/logical/data/Values.java
@@ -0,0 +1,69 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.common.logical.data;
+
+import org.apache.drill.common.JSONOptions;
+import org.apache.drill.common.logical.data.visitors.LogicalVisitor;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.annotation.JsonTypeName;
+import com.fasterxml.jackson.core.JsonLocation;
+import com.fasterxml.jackson.databind.JsonNode;
+import com.google.common.base.Preconditions;
+
+@JsonTypeName("values")
+public class Values extends SourceOperator {
+
+    private final JSONOptions content;
+
+    @JsonCreator
+    public Values(@JsonProperty("content") JSONOptions content){
+        super();
+        this.content = content;
+        Preconditions.checkNotNull(content, "content attribute is required for source operator 'constant'.");
+    }
+
+    public JSONOptions getContent() {
+        return content;
+    }
+
+    @Override
+    public <T, X, E extends Throwable> T accept(LogicalVisitor<T, X, E> logicalVisitor, X value) throws E {
+      return logicalVisitor.visitValues(this, value);
+    }
+
+    public static Builder builder(){
+      return new Builder();
+    }
+
+    public static class Builder extends AbstractBuilder<Values>{
+      private JSONOptions content;
+
+      public Builder content(JsonNode n){
+        content = new JSONOptions(n, JsonLocation.NA);
+        return this;
+      }
+
+      @Override
+      public Values build() {
+        return new Values(content);
+      }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/314e5a2a/common/src/main/java/org/apache/drill/common/logical/data/visitors/AbstractLogicalVisitor.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/drill/common/logical/data/visitors/AbstractLogicalVisitor.java b/common/src/main/java/org/apache/drill/common/logical/data/visitors/AbstractLogicalVisitor.java
index 92e370f..165ee66 100644
--- a/common/src/main/java/org/apache/drill/common/logical/data/visitors/AbstractLogicalVisitor.java
+++ b/common/src/main/java/org/apache/drill/common/logical/data/visitors/AbstractLogicalVisitor.java
@@ -17,7 +17,7 @@
  */
 package org.apache.drill.common.logical.data.visitors;
 
-import org.apache.drill.common.logical.data.Constant;
+import org.apache.drill.common.logical.data.Values;
 import org.apache.drill.common.logical.data.Filter;
 import org.apache.drill.common.logical.data.Flatten;
 import org.apache.drill.common.logical.data.GroupingAggregate;
@@ -115,7 +115,7 @@ public abstract class AbstractLogicalVisitor<T, X, E extends Throwable> implemen
     }
 
     @Override
-    public T visitConstant(Constant constant, X value) throws E {
+    public T visitValues(Values constant, X value) throws E {
        return visitOp(constant, value);
     }
 

http://git-wip-us.apache.org/repos/asf/drill/blob/314e5a2a/common/src/main/java/org/apache/drill/common/logical/data/visitors/LogicalVisitor.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/drill/common/logical/data/visitors/LogicalVisitor.java b/common/src/main/java/org/apache/drill/common/logical/data/visitors/LogicalVisitor.java
index 3a426bf..1e07dc6 100644
--- a/common/src/main/java/org/apache/drill/common/logical/data/visitors/LogicalVisitor.java
+++ b/common/src/main/java/org/apache/drill/common/logical/data/visitors/LogicalVisitor.java
@@ -18,7 +18,7 @@
 package org.apache.drill.common.logical.data.visitors;
 
 
-import org.apache.drill.common.logical.data.Constant;
+import org.apache.drill.common.logical.data.Values;
 import org.apache.drill.common.logical.data.Filter;
 import org.apache.drill.common.logical.data.Flatten;
 import org.apache.drill.common.logical.data.GroupingAggregate;
@@ -53,7 +53,7 @@ public interface LogicalVisitor<RETURN, EXTRA, EXCEP extends Throwable> {
     public RETURN visitFlatten(Flatten flatten, EXTRA value) throws EXCEP;
 
     public RETURN visitProject(Project project, EXTRA value) throws EXCEP;
-    public RETURN visitConstant(Constant constant, EXTRA value) throws EXCEP;
+    public RETURN visitValues(Values constant, EXTRA value) throws EXCEP;
     public RETURN visitOrder(Order order, EXTRA value) throws EXCEP;
     public RETURN visitJoin(Join join, EXTRA value) throws EXCEP;
     public RETURN visitLimit(Limit limit, EXTRA value) throws EXCEP;

http://git-wip-us.apache.org/repos/asf/drill/blob/314e5a2a/common/src/main/java/org/apache/drill/common/types/Types.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/drill/common/types/Types.java b/common/src/main/java/org/apache/drill/common/types/Types.java
index c41727d..cec433f 100644
--- a/common/src/main/java/org/apache/drill/common/types/Types.java
+++ b/common/src/main/java/org/apache/drill/common/types/Types.java
@@ -18,7 +18,6 @@
 package org.apache.drill.common.types;
 
 import static org.apache.drill.common.types.TypeProtos.DataMode.REPEATED;
-import static org.apache.drill.common.types.TypeProtos.MinorType.*;
 
 import org.apache.drill.common.exceptions.DrillRuntimeException;
 import org.apache.drill.common.types.TypeProtos.DataMode;
@@ -39,7 +38,7 @@ public class Types {
     UNKNOWN, NONE, EQUAL, ORDERED;
   }
 
-  public static boolean isComplex(MajorType type) {
+  public static boolean isComplex(final MajorType type) {
     switch(type.getMinorType()) {
     case LIST:
     case MAP:
@@ -49,11 +48,11 @@ public class Types {
     return false;
   }
 
-  public static boolean isRepeated(MajorType type) {
+  public static boolean isRepeated(final MajorType type) {
     return type.getMode() == REPEATED ;
   }
 
-  public static boolean isNumericType(MajorType type) {
+  public static boolean isNumericType(final MajorType type) {
     if (type.getMode() == REPEATED) {
       return false;
     }
@@ -85,7 +84,7 @@ public class Types {
   /***
    * Gets JDBC type code for given Drill RPC-/protobuf-level type.
    */
-  public static int getJdbcType(MajorType type) {
+  public static int getJdbcType(final MajorType type) {
     if (type.getMode() == DataMode.REPEATED) {
       return java.sql.Types.ARRAY;
     }
@@ -159,7 +158,7 @@ public class Types {
     }
   }
 
-  public static boolean isUnSigned(MajorType type) {
+  public static boolean isUnSigned(final MajorType type) {
     switch(type.getMinorType()) {
     case UINT1:
     case UINT2:
@@ -171,7 +170,7 @@ public class Types {
     }
 
   }
-  public static boolean usesHolderForGet(MajorType type) {
+  public static boolean usesHolderForGet(final MajorType type) {
     if (type.getMode() == REPEATED) {
       return true;
     }
@@ -199,7 +198,7 @@ public class Types {
 
   }
 
-  public static boolean isFixedWidthType(MajorType type) {
+  public static boolean isFixedWidthType(final MajorType type) {
     switch(type.getMinorType()) {
     case VARBINARY:
     case VAR16CHAR:
@@ -211,7 +210,7 @@ public class Types {
   }
 
 
-  public static boolean isStringScalarType(MajorType type) {
+  public static boolean isStringScalarType(final MajorType type) {
     if (type.getMode() == REPEATED) {
       return false;
     }
@@ -226,7 +225,7 @@ public class Types {
     }
   }
 
-  public static boolean isBytesScalarType(MajorType type) {
+  public static boolean isBytesScalarType(final MajorType type) {
     if (type.getMode() == REPEATED) {
       return false;
     }
@@ -239,7 +238,7 @@ public class Types {
     }
   }
 
-  public static Comparability getComparability(MajorType type) {
+  public static Comparability getComparability(final MajorType type) {
     if (type.getMode() == REPEATED) {
       return Comparability.NONE;
     }
@@ -261,7 +260,7 @@ public class Types {
   }
 
 
-  public static boolean softEquals(MajorType a, MajorType b, boolean allowNullSwap) {
+  public static boolean softEquals(final MajorType a, final MajorType b, final boolean allowNullSwap) {
     if (a.getMinorType() != b.getMinorType()) {
         return false;
     }
@@ -279,31 +278,31 @@ public class Types {
     return a.getMode() == b.getMode();
   }
 
-  public static boolean isLateBind(MajorType type) {
+  public static boolean isLateBind(final MajorType type) {
     return type.getMinorType() == MinorType.LATE;
   }
 
-  public static MajorType withMode(MinorType type, DataMode mode) {
+  public static MajorType withMode(final MinorType type, final DataMode mode) {
     return MajorType.newBuilder().setMode(mode).setMinorType(type).build();
   }
 
-  public static MajorType withScaleAndPrecision(MinorType type, DataMode mode, int scale, int precision) {
+  public static MajorType withScaleAndPrecision(final MinorType type, final DataMode mode, final int scale, final int precision) {
     return MajorType.newBuilder().setMinorType(type).setMode(mode).setScale(scale).setPrecision(precision).build();
   }
 
-  public static MajorType required(MinorType type) {
+  public static MajorType required(final MinorType type) {
     return MajorType.newBuilder().setMode(DataMode.REQUIRED).setMinorType(type).build();
   }
 
-  public static MajorType repeated(MinorType type) {
+  public static MajorType repeated(final MinorType type) {
     return MajorType.newBuilder().setMode(REPEATED).setMinorType(type).build();
   }
 
-  public static MajorType optional(MinorType type) {
+  public static MajorType optional(final MinorType type) {
     return MajorType.newBuilder().setMode(DataMode.OPTIONAL).setMinorType(type).build();
   }
 
-  public static MajorType overrideMinorType(MajorType originalMajorType, MinorType overrideMinorType) {
+  public static MajorType overrideMinorType(final MajorType originalMajorType, final MinorType overrideMinorType) {
     switch (originalMajorType.getMode()) {
       case REPEATED:
         return repeated(overrideMinorType);
@@ -316,11 +315,11 @@ public class Types {
     }
   }
 
-  public static MajorType overrideMode(MajorType originalMajorType, DataMode overrideMode) {
+  public static MajorType overrideMode(final MajorType originalMajorType, final DataMode overrideMode) {
     return withScaleAndPrecision(originalMajorType.getMinorType(), overrideMode, originalMajorType.getScale(), originalMajorType.getPrecision());
   }
 
-  public static MajorType getMajorTypeFromName(String typeName) {
+  public static MajorType getMajorTypeFromName(final String typeName) {
     return getMajorTypeFromName(typeName, DataMode.REQUIRED);
   }
 
@@ -376,6 +375,7 @@ public class Types {
     case "binary":
       return MinorType.VARBINARY;
     case "json":
+    case "simplejson":
       return MinorType.LATE;
     case "null":
     case "any":
@@ -385,11 +385,11 @@ public class Types {
     }
   }
 
-  public static MajorType getMajorTypeFromName(String typeName, DataMode mode) {
+  public static MajorType getMajorTypeFromName(final String typeName, final DataMode mode) {
     return withMode(getMinorTypeFromName(typeName), mode);
   }
 
-  public static String getNameOfMinorType(MinorType type) {
+  public static String getNameOfMinorType(final MinorType type) {
     switch (type) {
       case BIT:
         return "bool";
@@ -440,7 +440,7 @@ public class Types {
     }
   }
 
-  public static String toString(MajorType type) {
+  public static String toString(final MajorType type) {
     return type != null ? "MajorType[" + TextFormat.shortDebugString(type) + "]" : "null";
   }
 

http://git-wip-us.apache.org/repos/asf/drill/blob/314e5a2a/contrib/storage-mongo/src/main/java/org/apache/drill/exec/store/mongo/MongoRecordReader.java
----------------------------------------------------------------------
diff --git a/contrib/storage-mongo/src/main/java/org/apache/drill/exec/store/mongo/MongoRecordReader.java b/contrib/storage-mongo/src/main/java/org/apache/drill/exec/store/mongo/MongoRecordReader.java
index 15ef197..6eff72f 100644
--- a/contrib/storage-mongo/src/main/java/org/apache/drill/exec/store/mongo/MongoRecordReader.java
+++ b/contrib/storage-mongo/src/main/java/org/apache/drill/exec/store/mongo/MongoRecordReader.java
@@ -146,7 +146,7 @@ public class MongoRecordReader extends AbstractRecordReader {
   @Override
   public void setup(OutputMutator output) throws ExecutionSetupException {
     this.writer = new VectorContainerWriter(output);
-    this.jsonReader = new JsonReader(fragmentContext.getManagedBuffer(), Lists.newArrayList(getColumns()), enableAllTextMode);
+    this.jsonReader = new JsonReader(fragmentContext.getManagedBuffer(), Lists.newArrayList(getColumns()), enableAllTextMode, false);
     logger.info("Filters Applied : " + filters);
     logger.info("Fields Selected :" + fields);
     cursor = collection.find(filters, fields);

http://git-wip-us.apache.org/repos/asf/drill/blob/314e5a2a/exec/java-exec/src/main/codegen/templates/BaseWriter.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/codegen/templates/BaseWriter.java b/exec/java-exec/src/main/codegen/templates/BaseWriter.java
index 2ce4c3c..ada410d 100644
--- a/exec/java-exec/src/main/codegen/templates/BaseWriter.java
+++ b/exec/java-exec/src/main/codegen/templates/BaseWriter.java
@@ -30,15 +30,11 @@ package org.apache.drill.exec.vector.complex.writer;
 @SuppressWarnings("unused")
 public interface BaseWriter extends Positionable{
   FieldWriter getParent();
-  boolean ok();
-  WriteState getState();
   int getValueCapacity();
-  void resetState();
 
   public interface MapWriter extends BaseWriter{
 
     MaterializedField getField();
-    void checkValueCapacity();
 
     <#list vv.types as type><#list type.minor as minor>
     <#assign lowerName = minor.class?uncap_first />
@@ -83,7 +79,6 @@ public interface BaseWriter extends Positionable{
     void copyReader(FieldReader reader);
     MapWriter rootAsMap();
     ListWriter rootAsList();
-    boolean ok();
     
     public void setPosition(int index);
     public void setValueCount(int count);

http://git-wip-us.apache.org/repos/asf/drill/blob/314e5a2a/exec/java-exec/src/main/codegen/templates/ComplexReaders.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/codegen/templates/ComplexReaders.java b/exec/java-exec/src/main/codegen/templates/ComplexReaders.java
index cd69d0d..fa1dac4 100644
--- a/exec/java-exec/src/main/codegen/templates/ComplexReaders.java
+++ b/exec/java-exec/src/main/codegen/templates/ComplexReaders.java
@@ -79,17 +79,13 @@ public class ${nullMode}${name}ReaderImpl extends AbstractFieldReader {
   <#if mode == "Repeated">
 
   public void copyAsValue(${minor.class?cap_first}Writer writer){
-    if (writer.ok()) {
-      Repeated${minor.class?cap_first}WriterImpl impl = (Repeated${minor.class?cap_first}WriterImpl) writer;
-      impl.vector.copyFromSafe(idx(), impl.idx(), vector);
-    }
+    Repeated${minor.class?cap_first}WriterImpl impl = (Repeated${minor.class?cap_first}WriterImpl) writer;
+    impl.vector.copyFromSafe(idx(), impl.idx(), vector);
   }
   
   public void copyAsField(String name, MapWriter writer){
-    if (writer.ok()) {
-      Repeated${minor.class?cap_first}WriterImpl impl = (Repeated${minor.class?cap_first}WriterImpl)  writer.list(name).${lowerName}();
-      impl.vector.copyFromSafe(idx(), impl.idx(), vector);
-    }
+    Repeated${minor.class?cap_first}WriterImpl impl = (Repeated${minor.class?cap_first}WriterImpl)  writer.list(name).${lowerName}();
+    impl.vector.copyFromSafe(idx(), impl.idx(), vector);
   }
   
   public int size(){
@@ -115,17 +111,13 @@ public class ${nullMode}${name}ReaderImpl extends AbstractFieldReader {
   <#else>
   
   public void copyAsValue(${minor.class?cap_first}Writer writer){
-    if (writer.ok()) {
-      ${nullMode}${minor.class?cap_first}WriterImpl impl = (${nullMode}${minor.class?cap_first}WriterImpl) writer;
-      impl.vector.copyFromSafe(idx(), impl.idx(), vector);
-    }
+    ${nullMode}${minor.class?cap_first}WriterImpl impl = (${nullMode}${minor.class?cap_first}WriterImpl) writer;
+    impl.vector.copyFromSafe(idx(), impl.idx(), vector);
   }
   
   public void copyAsField(String name, MapWriter writer){
-    if (writer.ok()) {
-      ${nullMode}${minor.class?cap_first}WriterImpl impl = (${nullMode}${minor.class?cap_first}WriterImpl) writer.${lowerName}(name);
-      impl.vector.copyFromSafe(idx(), impl.idx(), vector);
-    }
+    ${nullMode}${minor.class?cap_first}WriterImpl impl = (${nullMode}${minor.class?cap_first}WriterImpl) writer.${lowerName}(name);
+    impl.vector.copyFromSafe(idx(), impl.idx(), vector);
   }
   
   public void read(Nullable${minor.class?cap_first}Holder h){

http://git-wip-us.apache.org/repos/asf/drill/blob/314e5a2a/exec/java-exec/src/main/codegen/templates/ComplexWriters.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/codegen/templates/ComplexWriters.java b/exec/java-exec/src/main/codegen/templates/ComplexWriters.java
index 576fd83..d40dfd2 100644
--- a/exec/java-exec/src/main/codegen/templates/ComplexWriters.java
+++ b/exec/java-exec/src/main/codegen/templates/ComplexWriters.java
@@ -61,12 +61,8 @@ public class ${eName}WriterImpl extends AbstractFieldWriter {
     return vector.getValueCapacity();
   }
 
-  public void checkValueCapacity() {
-    inform(vector.getValueCapacity() > idx());
-  }
-
   public void allocate(){
-    inform(vector.allocateNewSafe());
+    vector.allocateNew();
   }
   
   public void clear(){
@@ -77,76 +73,53 @@ public class ${eName}WriterImpl extends AbstractFieldWriter {
     return super.idx();
   }
   
-  protected void inform(boolean ok){
-    super.inform(ok);
-  }
-  
   <#if mode == "Repeated">
 
   public void write(${minor.class?cap_first}Holder h){
-    if(ok()){
-      // update to inform(addSafe) once available for all repeated vector types for holders.
-      mutator.addSafe(idx(), h);
-      vector.setCurrentValueCount(idx());
-    }
+    mutator.addSafe(idx(), h);
+    vector.setCurrentValueCount(idx());
   }
   
   public void write(Nullable${minor.class?cap_first}Holder h){
-    if(ok()){
-      // update to inform(addSafe) once available for all repeated vector types for holders.
-      mutator.addSafe(idx(), h);
-      vector.setCurrentValueCount(idx());
-    }
+    mutator.addSafe(idx(), h);
+    vector.setCurrentValueCount(idx());
   }
 
   <#if !(minor.class == "Decimal9" || minor.class == "Decimal18" || minor.class == "Decimal28Sparse" || minor.class == "Decimal38Sparse" || minor.class == "Decimal28Dense" || minor.class == "Decimal38Dense")>
   public void write${minor.class}(<#list fields as field>${field.type} ${field.name}<#if field_has_next>, </#if></#list>){
-    if(ok()){
-      // update to inform(setSafe) once available for all vector types for holders.
-      mutator.addSafe(idx(), <#list fields as field>${field.name}<#if field_has_next>, </#if></#list>);
-      vector.setCurrentValueCount(idx());
-    }
+    mutator.addSafe(idx(), <#list fields as field>${field.name}<#if field_has_next>, </#if></#list>);
+    vector.setCurrentValueCount(idx());
   }
   </#if>
   
   public void setPosition(int idx){
-    if (ok()){
-      super.setPosition(idx);
-      mutator.startNewGroup(idx);
-    }
+    super.setPosition(idx);
+    mutator.startNewGroup(idx);
   }
   
   
   <#else>
   
   public void write(${minor.class}Holder h){
-    if(ok()){
-      mutator.setSafe(idx(), h);
-      vector.setCurrentValueCount(idx());
-    }
+    mutator.setSafe(idx(), h);
+    vector.setCurrentValueCount(idx());
   }
   
   public void write(Nullable${minor.class}Holder h){
-    if(ok()){
-      mutator.setSafe(idx(), h);
-      vector.setCurrentValueCount(idx());
-    }
+    mutator.setSafe(idx(), h);
+    vector.setCurrentValueCount(idx());
   }
   
   <#if !(minor.class == "Decimal9" || minor.class == "Decimal18" || minor.class == "Decimal28Sparse" || minor.class == "Decimal38Sparse" || minor.class == "Decimal28Dense" || minor.class == "Decimal38Dense")>
   public void write${minor.class}(<#list fields as field>${field.type} ${field.name}<#if field_has_next>, </#if></#list>){
-    if(ok()){
-      mutator.setSafe(idx(), <#if mode == "Nullable">1, </#if><#list fields as field>${field.name}<#if field_has_next>, </#if></#list>);
-      vector.setCurrentValueCount(idx());
-    }
+    mutator.setSafe(idx(), <#if mode == "Nullable">1, </#if><#list fields as field>${field.name}<#if field_has_next>, </#if></#list>);
+    vector.setCurrentValueCount(idx());
   }
 
   <#if mode == "Nullable">
   public void writeNull(){
-    if(ok()){
-      mutator.setNull(idx());
-      vector.setCurrentValueCount(idx());
-    }
+    mutator.setNull(idx());
+    vector.setCurrentValueCount(idx());
   }
   </#if>
   </#if>

http://git-wip-us.apache.org/repos/asf/drill/blob/314e5a2a/exec/java-exec/src/main/codegen/templates/JsonOutputRecordWriter.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/codegen/templates/JsonOutputRecordWriter.java b/exec/java-exec/src/main/codegen/templates/JsonOutputRecordWriter.java
index 1d0dc9d..1bea326 100644
--- a/exec/java-exec/src/main/codegen/templates/JsonOutputRecordWriter.java
+++ b/exec/java-exec/src/main/codegen/templates/JsonOutputRecordWriter.java
@@ -16,12 +16,6 @@
  * limitations under the License.
  */
 
-import org.joda.time.DateTimeUtils;
-import parquet.io.api.Binary;
-
-import java.lang.Override;
-import java.lang.RuntimeException;
-
 <@pp.dropOutputFile />
 <@pp.changeOutputFile name="org/apache/drill/exec/store/JSONOutputRecordWriter.java" />
 <#include "/@includes/license.ftl" />
@@ -46,6 +40,8 @@ import io.netty.buffer.ByteBuf;
 import org.apache.drill.exec.memory.TopLevelAllocator;
 import org.apache.drill.exec.record.BatchSchema;
 import org.apache.drill.exec.record.MaterializedField;
+import org.apache.drill.exec.vector.complex.fn.JsonOutput;
+
 
 
 import org.apache.drill.common.types.TypeProtos;
@@ -68,7 +64,7 @@ import java.util.Map;
  */
 public abstract class JSONOutputRecordWriter extends AbstractRecordWriter implements RecordWriter {
 
-  protected JsonGenerator gen;
+  protected JsonOutput gen;
 
 <#list vv.types as type>
   <#list type.minor as minor>
@@ -92,12 +88,7 @@ public abstract class JSONOutputRecordWriter extends AbstractRecordWriter implem
 
     @Override
     public void writeField() throws IOException {
-  <#if mode.prefix == "Nullable" >
-    if (!reader.isSet()) {
-      gen.writeNull();
-      return;
-    }
-  <#elseif mode.prefix == "Repeated" >
+  <#if mode.prefix == "Repeated" >
     // empty lists are represented by simply not starting a field, rather than starting one and putting in 0 elements
     if (reader.size() == 0) {
       return;
@@ -106,57 +97,59 @@ public abstract class JSONOutputRecordWriter extends AbstractRecordWriter implem
     for (int i = 0; i < reader.size(); i++) {
   <#else>
   </#if>
-
-  <#if  minor.class == "TinyInt" ||
-        minor.class == "UInt1" ||
-        minor.class == "UInt2" ||
-        minor.class == "SmallInt" ||
-        minor.class == "Int" ||
-        minor.class == "Decimal9" ||
-        minor.class == "Float4" ||
-        minor.class == "BigInt" ||
-        minor.class == "Decimal18" ||
-        minor.class == "UInt8" ||
-        minor.class == "UInt4" ||
-        minor.class == "Float8" ||
-        minor.class == "Decimal28Sparse" ||
-        minor.class == "Decimal28Dense" ||
-        minor.class == "Decimal38Dense" ||
-        minor.class == "Decimal38Sparse">
-    <#if mode.prefix == "Repeated" >
-      gen.writeNumber(reader.read${friendlyType}(i));
-    <#else>
-      gen.writeNumber(reader.read${friendlyType}());
-    </#if>
-  <#elseif minor.class == "Date" ||
-              minor.class == "Time" ||
-              minor.class == "TimeStamp" ||
-              minor.class == "TimeTZ" ||
-              minor.class == "IntervalDay" ||
-              minor.class == "Interval" ||
-              minor.class == "VarChar" ||
-              minor.class == "Var16Char" ||
-              minor.class == "IntervalYear">
-    <#if mode.prefix == "Repeated" >
-              gen.writeString(reader.read${friendlyType}(i).toString());
-    <#else>
-      gen.writeString(reader.read${friendlyType}().toString());
-    </#if>
-  <#elseif
-        minor.class == "Bit">
-      <#if mode.prefix == "Repeated" >
-              gen.writeBoolean(reader.read${friendlyType}(i));
-      <#else>
-      gen.writeBoolean(reader.read${friendlyType}());
-      </#if>
-  <#elseif
-            minor.class == "VarBinary">
-      <#if mode.prefix == "Repeated" >
-              gen.writeBinary(reader.readByteArray(i));
-      <#else>
-      gen.writeBinary(reader.readByteArray());
-      </#if>
+  
+  <#assign typeName = minor.class >
+  
+  <#switch minor.class>
+  <#case "UInt1">
+  <#case "UInt2">
+  <#case "UInt4">
+  <#case "UInt8">
+    <#assign typeName = "unsupported">
+    <#break>
+    
+  <#case "Decimal9">
+  <#case "Decimal18">
+  <#case "Decimal28Sparse">
+  <#case "Decimal28Dense">
+  <#case "Decimal38Dense">
+  <#case "Decimal38Sparse">
+    <#assign typeName = "Decimal">
+    <#break>
+  <#case "Float4">
+    <#assign typeName = "Float">
+    <#break>
+  <#case "Float8">
+    <#assign typeName = "Double">
+    <#break>
+    
+  <#case "IntervalDay">
+  <#case "IntervalYear">
+    <#assign typeName = "Interval">
+    <#break>
+    
+  <#case "Bit">
+    <#assign typeName = "Boolean">
+    <#break>  
+
+  <#case "TimeStamp">
+    <#assign typeName = "Timestamp">
+    <#break>  
+    
+  <#case "VarBinary">
+    <#assign typeName = "Binary">
+    <#break>  
+    
+  </#switch>
+  
+  <#if typeName == "unsupported">
+    throw new UnsupportedOperationException("Unable to currently write ${minor.class} type to JSON.");
+  <#elseif mode.prefix == "Repeated" >
+    gen.write${typeName}(i, reader);
+  <#else>
+    gen.write${typeName}(reader);
   </#if>
+
   <#if mode.prefix == "Repeated">
     }
       gen.writeEndArray();

http://git-wip-us.apache.org/repos/asf/drill/blob/314e5a2a/exec/java-exec/src/main/codegen/templates/ListWriters.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/codegen/templates/ListWriters.java b/exec/java-exec/src/main/codegen/templates/ListWriters.java
index ff4c3d8..29708d7 100644
--- a/exec/java-exec/src/main/codegen/templates/ListWriters.java
+++ b/exec/java-exec/src/main/codegen/templates/ListWriters.java
@@ -46,7 +46,7 @@ public class ${mode}ListWriter extends AbstractFieldWriter{
   protected final ${containerClass} container;
   private Mode mode = Mode.INIT;
   private FieldWriter writer;
-  protected ValueVector innerVector;
+  protected RepeatedVector innerVector;
   
   <#if mode == "Repeated">private int currentChildIndex = 0;</#if>
   public ${mode}ListWriter(String name, ${containerClass} container, FieldWriter parent){
@@ -62,9 +62,12 @@ public class ${mode}ListWriter extends AbstractFieldWriter{
   }
 
   public void allocate(){
-    if(writer != null) writer.allocate();
+    if(writer != null){
+      writer.allocate();
+    }
+    
     <#if mode == "Repeated">
-    inform(container.allocateNewSafe());
+    container.allocateNew();
     </#if>
   }
   
@@ -149,27 +152,27 @@ public class ${mode}ListWriter extends AbstractFieldWriter{
     return container.getField();
   }
 
-  public void checkValueCapacity() {
-    inform(container.getValueCapacity() > idx());
-  }
-
   <#if mode == "Repeated">
+  
   public void start(){
-    if(ok()){
-      checkValueCapacity();
-      if (!ok()) return;
-      // update the repeated vector to state that there is current+1 objects.
-      RepeatedListHolder h = new RepeatedListHolder();
-      container.getAccessor().get(idx(), h);
-      if(h.start >= h.end){
-        container.getMutator().startNewGroup(idx());  
-      }
-      currentChildIndex = container.getMutator().add(idx());
-      if(currentChildIndex == -1){
-        inform(false);
-      }else{
-        if(writer != null) writer.setPosition(currentChildIndex);  
-      }
+    
+    final RepeatedListVector list = (RepeatedListVector) container;
+    final RepeatedListVector.Mutator mutator = list.getMutator();
+    
+    // make sure that the current vector can support the end position of this list.
+    if(container.getValueCapacity() <= idx()){
+      mutator.setValueCount(idx()+1);
+    }
+    
+    // update the repeated vector to state that there is current+1 objects.
+    RepeatedListHolder h = new RepeatedListHolder();
+    list.getAccessor().get(idx(), h);
+    if(h.start >= h.end){
+      mutator.startNewGroup(idx());  
+    }
+    currentChildIndex = container.getMutator().add(idx());
+    if(writer != null){
+      writer.setPosition(currentChildIndex);  
     }
   }
   
@@ -180,6 +183,7 @@ public class ${mode}ListWriter extends AbstractFieldWriter{
   }
   <#else>
   
+  
   public void setPosition(int index){
     super.setPosition(index);
     if(writer != null) writer.setPosition(index);

http://git-wip-us.apache.org/repos/asf/drill/blob/314e5a2a/exec/java-exec/src/main/codegen/templates/MapWriters.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/codegen/templates/MapWriters.java b/exec/java-exec/src/main/codegen/templates/MapWriters.java
index 4dab990..6ee8035 100644
--- a/exec/java-exec/src/main/codegen/templates/MapWriters.java
+++ b/exec/java-exec/src/main/codegen/templates/MapWriters.java
@@ -63,14 +63,6 @@ public class ${mode}MapWriter extends AbstractFieldWriter{
       return container.getField();
   }
 
-  public void checkValueCapacity(){
-    <#if mode == "Repeated">
-    if (container.getValueCapacity() <= idx()) {
-      container.reAlloc();
-    }
-    </#if>
-  }
-
   public MapWriter map(String name){
     FieldWriter writer = fields.get(name);
     if(writer == null){
@@ -85,7 +77,7 @@ public class ${mode}MapWriter extends AbstractFieldWriter{
   }
   
   public void allocate(){
-    inform(container.allocateNewSafe());
+    container.allocateNew();
     for(FieldWriter w : fields.values()){
       w.allocate();
     }
@@ -96,10 +88,8 @@ public class ${mode}MapWriter extends AbstractFieldWriter{
     for(FieldWriter w : fields.values()){
       w.clear();
     }
-    
   }
   
-  
   public ListWriter list(String name){
     FieldWriter writer = fields.get(name);
     if(writer == null){
@@ -113,24 +103,24 @@ public class ${mode}MapWriter extends AbstractFieldWriter{
 
   <#if mode == "Repeated">
   public void start(){
-    if(ok()){
-      checkValueCapacity();
-      if (!ok()) return;
       // update the repeated vector to state that there is current+1 objects.
       
-      RepeatedMapHolder h = new RepeatedMapHolder();
-      container.getAccessor().get(idx(), h);
-      if(h.start >= h.end){
-        container.getMutator().startNewGroup(idx());  
-      }
-      currentChildIndex = container.getMutator().add(idx());
-      if(currentChildIndex == -1){
-        inform(false);
-      }else{
-        for(FieldWriter w: fields.values()){
-          w.setPosition(currentChildIndex);  
-        }
-      }
+    final RepeatedMapHolder h = new RepeatedMapHolder();
+    final RepeatedMapVector map = (RepeatedMapVector) container;
+    final RepeatedMapVector.Mutator mutator = map.getMutator();
+    
+    // make sure that the current vector can support the end position of this list.
+    if(container.getValueCapacity() <= idx()){
+      mutator.setValueCount(idx()+1);
+    }
+
+    map.getAccessor().get(idx(), h);
+    if(h.start >= h.end){
+      container.getMutator().startNewGroup(idx());  
+    }
+    currentChildIndex = container.getMutator().add(idx());
+    for(FieldWriter w: fields.values()){
+      w.setPosition(currentChildIndex);  
     }
   }
   

http://git-wip-us.apache.org/repos/asf/drill/blob/314e5a2a/exec/java-exec/src/main/java/org/apache/drill/exec/ExecConstants.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/ExecConstants.java b/exec/java-exec/src/main/java/org/apache/drill/exec/ExecConstants.java
index bd93206..7d89ac9 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/ExecConstants.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/ExecConstants.java
@@ -107,7 +107,8 @@ public interface ExecConstants {
   public static OptionValidator COMPILE_SCALAR_REPLACEMENT = new BooleanValidator("exec.compile.scalar_replacement", false);
 
   public static String JSON_ALL_TEXT_MODE = "store.json.all_text_mode";
-  public static OptionValidator JSON_READER_ALL_TEXT_MODE_VALIDATOR = new BooleanValidator(JSON_ALL_TEXT_MODE, false);
+  public static BooleanValidator JSON_READER_ALL_TEXT_MODE_VALIDATOR = new BooleanValidator(JSON_ALL_TEXT_MODE, false);
+  public static final BooleanValidator JSON_EXTENDED_TYPES = new BooleanValidator("store.json.extended_types", true);
 
   /**
    * The column label (for directory levels) in results when querying files in a directory

http://git-wip-us.apache.org/repos/asf/drill/blob/314e5a2a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/EvaluationVisitor.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/EvaluationVisitor.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/EvaluationVisitor.java
index b6e3858..19aa3c9 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/EvaluationVisitor.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/EvaluationVisitor.java
@@ -310,15 +310,12 @@ public class EvaluationVisitor {
             TypeHelper.getWriterInterface(inputContainer.getMinorType(), inputContainer.getMajorType().getMode()));
         JVar writer = generator.declareClassField("writer", writerIFace);
         generator.getSetupBlock().assign(writer, JExpr._new(writerImpl).arg(vv).arg(JExpr._null()));
-        generator.getEvalBlock().add(writer.invoke("resetState"));
         generator.getEvalBlock().add(writer.invoke("setPosition").arg(outIndex));
         String copyMethod = inputContainer.isSingularRepeated() ? "copyAsValueSingle" : "copyAsValue";
         generator.getEvalBlock().add(inputContainer.getHolder().invoke(copyMethod).arg(writer));
         if (e.isSafe()) {
           HoldingContainer outputContainer = generator.declare(Types.REQUIRED_BIT);
-          JConditional ifOut = generator.getEvalBlock()._if(writer.invoke("ok"));
-          ifOut._then().assign(outputContainer.getValue(), JExpr.lit(1));
-          ifOut._else().assign(outputContainer.getValue(), JExpr.lit(0));
+          generator.getEvalBlock().assign(outputContainer.getValue(), JExpr.lit(1));
           return outputContainer;
         }
       } else {

http://git-wip-us.apache.org/repos/asf/drill/blob/314e5a2a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/DateUtility.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/DateUtility.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/DateUtility.java
index a031bee..98a0b61 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/DateUtility.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/DateUtility.java
@@ -18,6 +18,7 @@
 
 package org.apache.drill.exec.expr.fn.impl;
 
+import org.joda.time.Period;
 import org.joda.time.format.DateTimeFormat;
 import org.joda.time.format.DateTimeFormatter;
 import org.joda.time.format.DateTimeFormatterBuilder;
@@ -666,4 +667,16 @@ public class DateUtility {
         }
         return timeFormat;
     }
+
+    public static int monthsFromPeriod(Period period){
+      return (period.getYears() * yearsToMonths) + period.getMonths();
+    }
+
+    public static int millisFromPeriod(final Period period){
+      return (period.getHours() * hoursToMillis) +
+      (period.getMinutes() * minutesToMillis) +
+      (period.getSeconds() * secondsToMillis) +
+      (period.getMillis());
+    }
+
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/314e5a2a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/JsonConvertFrom.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/JsonConvertFrom.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/JsonConvertFrom.java
index c828cf4..3be2c9d 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/JsonConvertFrom.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/JsonConvertFrom.java
@@ -50,7 +50,7 @@ public class JsonConvertFrom {
     @Output ComplexWriter writer;
 
     public void setup(){
-      jsonReader = new org.apache.drill.exec.vector.complex.fn.JsonReader(buffer, false);
+      jsonReader = new org.apache.drill.exec.vector.complex.fn.JsonReader(buffer, false, false);
     }
 
     public void eval(){
@@ -76,7 +76,7 @@ public class JsonConvertFrom {
     @Output ComplexWriter writer;
 
     public void setup(){
-      jsonReader = new org.apache.drill.exec.vector.complex.fn.JsonReader(buffer, false);
+      jsonReader = new org.apache.drill.exec.vector.complex.fn.JsonReader(buffer, false, false);
     }
 
     public void eval(){

http://git-wip-us.apache.org/repos/asf/drill/blob/314e5a2a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/JsonConvertTo.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/JsonConvertTo.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/JsonConvertTo.java
index ccfae14..1d2292e 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/JsonConvertTo.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/JsonConvertTo.java
@@ -52,7 +52,37 @@ public class JsonConvertTo {
 
       java.io.ByteArrayOutputStream stream = new java.io.ByteArrayOutputStream();
       try {
-        org.apache.drill.exec.vector.complex.fn.JsonWriter jsonWriter = new org.apache.drill.exec.vector.complex.fn.JsonWriter(stream, true);
+        org.apache.drill.exec.vector.complex.fn.JsonWriter jsonWriter = new org.apache.drill.exec.vector.complex.fn.JsonWriter(stream, true, true);
+
+        jsonWriter.write(input);
+      } catch (Exception e) {
+        throw new RuntimeException(e);
+      }
+
+      byte [] bytea = stream.toByteArray();
+
+      out.buffer = buffer = buffer.reallocIfNeeded(bytea.length);
+      out.buffer.setBytes(0, bytea);
+      out.end = bytea.length;
+    }
+  }
+
+  @FunctionTemplate(name = "convert_toSIMPLEJSON", scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
+  public static class ConvertToSimpleJson implements DrillSimpleFunc{
+
+    @Param FieldReader input;
+    @Output VarBinaryHolder out;
+    @Inject DrillBuf buffer;
+
+    public void setup(){
+    }
+
+    public void eval(){
+      out.start = 0;
+
+      java.io.ByteArrayOutputStream stream = new java.io.ByteArrayOutputStream();
+      try {
+        org.apache.drill.exec.vector.complex.fn.JsonWriter jsonWriter = new org.apache.drill.exec.vector.complex.fn.JsonWriter(stream, true, false);
 
         jsonWriter.write(input);
       } catch (Exception e) {

http://git-wip-us.apache.org/repos/asf/drill/blob/314e5a2a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractPhysicalVisitor.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractPhysicalVisitor.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractPhysicalVisitor.java
index 1633912..a3b5f27 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractPhysicalVisitor.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractPhysicalVisitor.java
@@ -41,6 +41,7 @@ import org.apache.drill.exec.physical.config.Trace;
 import org.apache.drill.exec.physical.config.UnionAll;
 import org.apache.drill.exec.physical.config.UnionExchange;
 import org.apache.drill.exec.physical.config.UnorderedReceiver;
+import org.apache.drill.exec.physical.config.Values;
 import org.apache.drill.exec.physical.config.WindowPOP;
 
 public abstract class AbstractPhysicalVisitor<T, X, E extends Throwable> implements PhysicalVisitor<T, X, E> {
@@ -215,6 +216,11 @@ public abstract class AbstractPhysicalVisitor<T, X, E extends Throwable> impleme
   }
 
   @Override
+  public T visitValues(Values op, X value) throws E {
+    return visitOp(op, value);
+  }
+
+  @Override
   public T visitOp(PhysicalOperator op, X value) throws E{
     throw new UnsupportedOperationException(String.format(
         "The PhysicalVisitor of type %s does not currently support visiting the PhysicalOperator type %s.", this

http://git-wip-us.apache.org/repos/asf/drill/blob/314e5a2a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/PhysicalVisitor.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/PhysicalVisitor.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/PhysicalVisitor.java
index fde27af..ae6ff60 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/PhysicalVisitor.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/PhysicalVisitor.java
@@ -41,6 +41,7 @@ import org.apache.drill.exec.physical.config.Trace;
 import org.apache.drill.exec.physical.config.UnionAll;
 import org.apache.drill.exec.physical.config.UnionExchange;
 import org.apache.drill.exec.physical.config.UnorderedReceiver;
+import org.apache.drill.exec.physical.config.Values;
 import org.apache.drill.exec.physical.config.WindowPOP;
 
 /**
@@ -73,6 +74,7 @@ public interface PhysicalVisitor<RETURN, EXTRA, EXCEP extends Throwable> {
   public RETURN visitStreamingAggregate(StreamingAggregate agg, EXTRA value) throws EXCEP;
   public RETURN visitHashAggregate(HashAggregate agg, EXTRA value) throws EXCEP;
   public RETURN visitWriter(Writer op, EXTRA value) throws EXCEP;
+  public RETURN visitValues(Values op, EXTRA value) throws EXCEP;
   public RETURN visitOp(PhysicalOperator op, EXTRA value) throws EXCEP;
 
   public RETURN visitHashPartitionSender(HashPartitionSender op, EXTRA value) throws EXCEP;

http://git-wip-us.apache.org/repos/asf/drill/blob/314e5a2a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/Values.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/Values.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/Values.java
new file mode 100644
index 0000000..fbb069b
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/Values.java
@@ -0,0 +1,71 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.exec.physical.config;
+
+import java.util.Iterator;
+import java.util.List;
+
+import org.apache.drill.common.JSONOptions;
+import org.apache.drill.common.exceptions.ExecutionSetupException;
+import org.apache.drill.exec.physical.base.AbstractBase;
+import org.apache.drill.exec.physical.base.Leaf;
+import org.apache.drill.exec.physical.base.PhysicalOperator;
+import org.apache.drill.exec.physical.base.PhysicalVisitor;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.google.common.collect.Iterators;
+
+public class Values extends AbstractBase implements Leaf {
+
+  @SuppressWarnings("unused")
+  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(Values.class);
+
+  private final JSONOptions content;
+
+  @JsonCreator
+  public Values(@JsonProperty("content") JSONOptions content){
+    this.content = content;
+  }
+
+  public JSONOptions getContent(){
+    return content;
+  }
+
+  @Override
+  public <T, X, E extends Throwable> T accept(PhysicalVisitor<T, X, E> physicalVisitor, X value) throws E {
+    return physicalVisitor.visitValues(this, value);
+  }
+
+  @Override
+  public PhysicalOperator getNewWithChildren(List<PhysicalOperator> children) throws ExecutionSetupException {
+    assert children.isEmpty();
+    return this;
+  }
+
+  @Override
+  public int getOperatorType() {
+    return -1;
+  }
+
+  @Override
+  public Iterator<PhysicalOperator> iterator() {
+    return Iterators.emptyIterator();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/314e5a2a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/flatten/FlattenTemplate.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/flatten/FlattenTemplate.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/flatten/FlattenTemplate.java
index c5d3d93..96209a2 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/flatten/FlattenTemplate.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/flatten/FlattenTemplate.java
@@ -37,6 +37,8 @@ import org.apache.drill.exec.vector.RepeatedVector;
 public abstract class FlattenTemplate implements Flattener {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(FlattenTemplate.class);
 
+  private static final int OUTPUT_BATCH_SIZE = 4*1024;
+
   private ImmutableList<TransferPair> transfers;
   private SelectionVector2 vector2;
   private SelectionVector4 vector4;
@@ -86,9 +88,10 @@ public abstract class FlattenTemplate implements Flattener {
           for ( ; groupIndex < groupCount; groupIndex++) {
             currGroupSize = accessor.getGroupSizeAtIndex(groupIndex);
             for ( ; childIndexWithinCurrGroup < currGroupSize; childIndexWithinCurrGroup++) {
-              if (!doEval(groupIndex, firstOutputIndex)) {
+              if (firstOutputIndex == OUTPUT_BATCH_SIZE) {
                 break outer;
               }
+              doEval(groupIndex, firstOutputIndex);
               firstOutputIndex++;
               childIndex++;
             }

http://git-wip-us.apache.org/repos/asf/drill/blob/314e5a2a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/values/ValuesBatchCreator.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/values/ValuesBatchCreator.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/values/ValuesBatchCreator.java
new file mode 100644
index 0000000..d526a84
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/values/ValuesBatchCreator.java
@@ -0,0 +1,44 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.exec.physical.impl.values;
+
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.drill.common.exceptions.ExecutionSetupException;
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.physical.config.Values;
+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.easy.json.JSONRecordReader;
+
+import com.google.common.collect.Iterators;
+
+public class ValuesBatchCreator implements BatchCreator<Values> {
+  @Override
+  public RecordBatch getBatch(FragmentContext context, Values config, List<RecordBatch> children)
+      throws ExecutionSetupException {
+    assert children.isEmpty();
+
+    JSONRecordReader reader = new JSONRecordReader(context, config.getContent().asNode(), null, Collections.singletonList(SchemaPath.getSimplePath("*")));
+    return new ScanBatch(config, context, Iterators.singletonIterator((RecordReader) reader));
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/314e5a2a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillOptiq.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillOptiq.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillOptiq.java
index 5dd6ed5..b96ff3d 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillOptiq.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillOptiq.java
@@ -61,7 +61,7 @@ import com.google.common.collect.Lists;
  * Utilities for Drill's planner.
  */
 public class DrillOptiq {
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillOptiq.class);
+  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillOptiq.class);
 
   /**
    * Converts a tree of {@link RexNode} operators into a scalar expression in Drill syntax.
@@ -499,7 +499,7 @@ public class DrillOptiq {
     return new TypedNullConstant(Types.optional(type));
   }
 
-  private static boolean isLiteralNull(RexLiteral literal) {
+  public static boolean isLiteralNull(RexLiteral literal) {
     return literal.getTypeName().getName().equals("NULL");
   }
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/314e5a2a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillRuleSets.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillRuleSets.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillRuleSets.java
index 92a804e..84a0b51 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillRuleSets.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillRuleSets.java
@@ -39,6 +39,7 @@ import org.apache.drill.exec.planner.physical.ScreenPrule;
 import org.apache.drill.exec.planner.physical.SortConvertPrule;
 import org.apache.drill.exec.planner.physical.SortPrule;
 import org.apache.drill.exec.planner.physical.StreamAggPrule;
+import org.apache.drill.exec.planner.physical.ValuesPrule;
 import org.apache.drill.exec.planner.physical.WindowPrule;
 import org.apache.drill.exec.planner.physical.UnionAllPrule;
 import org.apache.drill.exec.planner.physical.WriterPrule;
@@ -141,6 +142,7 @@ public class DrillRuleSets {
       DrillMergeProjectRule.getInstance(true, RelFactories.DEFAULT_PROJECT_FACTORY, context.getFunctionRegistry()),
       RemoveDistinctAggregateRule.INSTANCE, //
       // ReduceAggregatesRule.INSTANCE, // replaced by DrillReduceAggregatesRule
+      DrillValuesRule.INSTANCE,
 
       /*
       Projection push-down related rules
@@ -206,6 +208,7 @@ public class DrillRuleSets {
     ruleList.add(WindowPrule.INSTANCE);
     ruleList.add(PushLimitToTopN.INSTANCE);
     ruleList.add(UnionAllPrule.INSTANCE);
+    ruleList.add(ValuesPrule.INSTANCE);
 
     // ruleList.add(UnionDistinctPrule.INSTANCE);
 

http://git-wip-us.apache.org/repos/asf/drill/blob/314e5a2a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillValuesRel.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillValuesRel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillValuesRel.java
index a3551e7..4fbc4d4 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillValuesRel.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillValuesRel.java
@@ -17,41 +17,276 @@
  */
 package org.apache.drill.exec.planner.logical;
 
+import static org.apache.drill.exec.planner.logical.DrillOptiq.isLiteralNull;
+
+import java.io.IOException;
+import java.math.BigDecimal;
+import java.util.GregorianCalendar;
 import java.util.List;
 
+import org.apache.drill.common.JSONOptions;
+import org.apache.drill.common.exceptions.DrillRuntimeException;
 import org.apache.drill.common.logical.data.LogicalOperator;
+import org.apache.drill.common.logical.data.Values;
+import org.apache.drill.exec.vector.complex.fn.ExtendedJsonOutput;
+import org.apache.drill.exec.vector.complex.fn.JsonOutput;
+import org.eigenbase.rel.AbstractRelNode;
 import org.eigenbase.rel.RelNode;
-import org.eigenbase.rel.ValuesRelBase;
+import org.eigenbase.rel.RelWriter;
 import org.eigenbase.relopt.RelOptCluster;
 import org.eigenbase.relopt.RelOptCost;
 import org.eigenbase.relopt.RelOptPlanner;
 import org.eigenbase.relopt.RelTraitSet;
 import org.eigenbase.reltype.RelDataType;
+import org.eigenbase.reltype.RelDataTypeField;
 import org.eigenbase.rex.RexLiteral;
+import org.eigenbase.sql.SqlExplainLevel;
+import org.eigenbase.sql.type.SqlTypeUtil;
+import org.eigenbase.util.NlsString;
+import org.eigenbase.util.Pair;
+import org.joda.time.DateTime;
+import org.joda.time.Period;
+
+import com.fasterxml.jackson.core.JsonLocation;
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.util.TokenBuffer;
+import com.google.common.base.Functions;
 
 /**
  * Values implemented in Drill.
  */
-public class DrillValuesRel extends ValuesRelBase implements DrillRel {
+public class DrillValuesRel extends AbstractRelNode implements DrillRel {
+  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillValuesRel.class);
+  private static final ObjectMapper MAPPER = new ObjectMapper();
+
+  private static final long MILLIS_IN_DAY = 1000*60*60*24;
+
+  private final JSONOptions options;
+  private final double rowCount;
+
   protected DrillValuesRel(RelOptCluster cluster, RelDataType rowType, List<List<RexLiteral>> tuples, RelTraitSet traits) {
-    super(cluster, rowType, tuples, traits);
+    super(cluster, traits);
     assert getConvention() == DRILL_LOGICAL;
+    verifyRowType(tuples, rowType);
+
+    this.rowType = rowType;
+    this.rowCount = tuples.size();
+
+    try{
+      this.options = new JSONOptions(convertToJsonNode(rowType, tuples), JsonLocation.NA);
+    }catch(IOException e){
+      throw new DrillRuntimeException("Failure while attempting to encode ValuesRel in JSON.", e);
+    }
+
+  }
+
+  private DrillValuesRel(RelOptCluster cluster, RelDataType rowType, RelTraitSet traits, JSONOptions options, double rowCount){
+    super(cluster, traits);
+    this.options = options;
+    this.rowCount = rowCount;
+    this.rowType = rowType;
+  }
+
+  private static void verifyRowType(final List<List<RexLiteral>> tuples, RelDataType rowType){
+      for (List<RexLiteral> tuple : tuples) {
+        assert (tuple.size() == rowType.getFieldCount());
+
+        for (Pair<RexLiteral, RelDataTypeField> pair : Pair.zip(tuple, rowType.getFieldList())) {
+          RexLiteral literal = (RexLiteral) pair.left;
+          RelDataType fieldType = ((RelDataTypeField) pair.right).getType();
+
+          if ((!(RexLiteral.isNullLiteral(literal)))
+              && (!(SqlTypeUtil.canAssignFrom(fieldType, literal.getType())))) {
+            throw new AssertionError("to " + fieldType + " from " + literal);
+          }
+        }
+      }
+
+  }
+
+  public RelOptCost computeSelfCost(RelOptPlanner planner) {
+    return planner.getCostFactory().makeCost(this.rowCount, 1.0d, 0.0d);
   }
 
   @Override
   public RelNode copy(RelTraitSet traitSet, List<RelNode> inputs) {
     assert inputs.isEmpty();
-    return new DrillValuesRel(getCluster(), rowType, tuples, traitSet);
+    return new DrillValuesRel(getCluster(), rowType, traitSet, options, rowCount);
   }
 
   @Override
-  public RelOptCost computeSelfCost(RelOptPlanner planner) {
-    return super.computeSelfCost(planner).multiplyBy(0.1);
+  public LogicalOperator implement(DrillImplementor implementor) {
+      return Values.builder()
+          .content(options.asNode())
+          .build();
   }
 
-  @Override
-  public LogicalOperator implement(DrillImplementor implementor) {
-    // Update when https://issues.apache.org/jira/browse/DRILL-57 fixed
-    throw new UnsupportedOperationException();
+  public JSONOptions getTuplesAsJsonOptions() throws IOException {
+    return options;
+  }
+
+  public double getRows() {
+    return rowCount;
+  }
+
+  public RelWriter explainTerms(RelWriter pw) {
+    return super.explainTerms(pw)
+        .itemIf("type", this.rowType, pw.getDetailLevel() == SqlExplainLevel.DIGEST_ATTRIBUTES)
+        .itemIf("type", this.rowType.getFieldList(), pw.nest())
+        .itemIf("tuplesCount", rowCount, pw.getDetailLevel() != SqlExplainLevel.ALL_ATTRIBUTES)
+        .itemIf("tuples", options.asNode(), pw.getDetailLevel() == SqlExplainLevel.ALL_ATTRIBUTES);
+  }
+
+  private static JsonNode convertToJsonNode(RelDataType rowType, List<List<RexLiteral>> tuples) throws IOException{
+    TokenBuffer out = new TokenBuffer(MAPPER.getFactory().getCodec(), false);
+    JsonOutput json = new ExtendedJsonOutput(out);
+    json.writeStartArray();
+    String[] fields = rowType.getFieldNames().toArray(new String[rowType.getFieldCount()]);
+
+    for(List<RexLiteral> row : tuples){
+      json.writeStartObject();
+      int i =0;
+      for(RexLiteral field : row){
+        json.writeFieldName(fields[i]);
+        writeLiteral(field, json);
+        i++;
+      }
+      json.writeEndObject();
+    }
+    json.writeEndArray();
+    json.flush();
+    return out.asParser().readValueAsTree();
+  }
+
+
+  private static void writeLiteral(RexLiteral literal, JsonOutput out) throws IOException{
+
+    switch(literal.getType().getSqlTypeName()){
+    case BIGINT:
+      if (isLiteralNull(literal)) {
+        out.writeBigIntNull();
+      }else{
+        out.writeBigInt((((BigDecimal) literal.getValue()).setScale(0, BigDecimal.ROUND_HALF_UP)).longValue());
+      }
+      return;
+
+    case BOOLEAN:
+      if (isLiteralNull(literal)) {
+        out.writeBooleanNull();
+      }else{
+        out.writeBoolean((Boolean) literal.getValue());
+      }
+      return;
+
+    case CHAR:
+      if (isLiteralNull(literal)) {
+        out.writeVarcharNull();
+      }else{
+        out.writeVarChar(((NlsString)literal.getValue()).getValue());
+      }
+      return ;
+
+    case DOUBLE:
+      if (isLiteralNull(literal)){
+        out.writeDoubleNull();
+      }else{
+        out.writeDouble(((BigDecimal) literal.getValue()).doubleValue());
+      }
+      return;
+
+    case FLOAT:
+      if (isLiteralNull(literal)) {
+        out.writeFloatNull();
+      }else{
+        out.writeFloat(((BigDecimal) literal.getValue()).floatValue());
+      }
+      return;
+
+    case INTEGER:
+      if (isLiteralNull(literal)) {
+        out.writeIntNull();
+      }else{
+        out.writeInt((((BigDecimal) literal.getValue()).setScale(0, BigDecimal.ROUND_HALF_UP)).intValue());
+      }
+      return;
+
+    case DECIMAL:
+      if (isLiteralNull(literal)) {
+        out.writeDoubleNull();
+      }else{
+        out.writeDouble(((BigDecimal) literal.getValue()).doubleValue());
+      }
+      logger.warn("Converting exact decimal into approximate decimal.  Should be fixed once decimal is implemented.");
+      return;
+
+    case VARCHAR:
+      if (isLiteralNull(literal)) {
+        out.writeVarcharNull();
+      }else{
+        out.writeVarChar( ((NlsString)literal.getValue()).getValue());
+      }
+      return;
+
+    case SYMBOL:
+      if (isLiteralNull(literal)) {
+        out.writeVarcharNull();
+      }else{
+        out.writeVarChar(literal.getValue().toString());
+      }
+      return;
+
+    case DATE:
+      if (isLiteralNull(literal)) {
+        out.writeDateNull();
+      }else{
+        out.writeDate(new DateTime((GregorianCalendar)literal.getValue()));
+      }
+      return;
+
+    case TIME:
+      if (isLiteralNull(literal)) {
+        out.writeTimeNull();
+      }else{
+        out.writeTime(new DateTime((GregorianCalendar)literal.getValue()));
+      }
+      return;
+
+    case TIMESTAMP:
+      if (isLiteralNull(literal)) {
+        out.writeTimestampNull();
+      }else{
+        out.writeTimestamp(new DateTime((GregorianCalendar)literal.getValue()));
+      }
+      return;
+
+    case INTERVAL_YEAR_MONTH:
+      if (isLiteralNull(literal)) {
+        out.writeIntervalNull();
+      }else{
+        int months = ((BigDecimal) (literal.getValue())).intValue();
+        out.writeInterval(new Period().plusMonths(months));
+      }
+      return;
+
+    case INTERVAL_DAY_TIME:
+      if (isLiteralNull(literal)) {
+        out.writeIntervalNull();
+      }else{
+        long millis = ((BigDecimal) (literal.getValue())).longValue();
+        int days = (int) (millis/MILLIS_IN_DAY);
+        millis = millis - (days * MILLIS_IN_DAY);
+        out.writeInterval(new Period().plusDays(days).plusMillis( (int) millis));
+      }
+      return;
+
+    case NULL:
+      out.writeUntypedNull();
+      return;
+
+    case ANY:
+    default:
+      throw new UnsupportedOperationException(String.format("Unable to convert the value of %s and type %s to a Drill constant expression.", literal, literal.getType().getSqlTypeName()));
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/314e5a2a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/ScanFieldDeterminer.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/ScanFieldDeterminer.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/ScanFieldDeterminer.java
index 59c65f9..1178ebd 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/ScanFieldDeterminer.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/ScanFieldDeterminer.java
@@ -25,7 +25,7 @@ import org.apache.drill.common.expression.LogicalExpression;
 import org.apache.drill.common.expression.SchemaPath;
 import org.apache.drill.common.expression.visitors.AbstractExprVisitor;
 import org.apache.drill.common.logical.LogicalPlan;
-import org.apache.drill.common.logical.data.Constant;
+import org.apache.drill.common.logical.data.Values;
 import org.apache.drill.common.logical.data.Filter;
 import org.apache.drill.common.logical.data.GroupingAggregate;
 import org.apache.drill.common.logical.data.Join;
@@ -146,7 +146,7 @@ public class ScanFieldDeterminer extends AbstractLogicalVisitor<Void, ScanFieldD
   }
 
   @Override
-  public Void visitConstant(Constant constant, FieldList value) {
+  public Void visitValues(Values constant, FieldList value) {
     return null;
   }
 

http://git-wip-us.apache.org/repos/asf/drill/blob/314e5a2a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ValuesPrel.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ValuesPrel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ValuesPrel.java
new file mode 100644
index 0000000..f896345
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ValuesPrel.java
@@ -0,0 +1,78 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.exec.planner.physical;
+
+import java.io.IOException;
+import java.util.Iterator;
+
+import org.apache.drill.common.JSONOptions;
+import org.apache.drill.exec.physical.config.Values;
+import org.apache.drill.exec.planner.physical.visitor.PrelVisitor;
+import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
+import org.eigenbase.rel.AbstractRelNode;
+import org.eigenbase.relopt.RelOptCluster;
+import org.eigenbase.relopt.RelTraitSet;
+import org.eigenbase.reltype.RelDataType;
+
+import com.google.common.collect.Iterators;
+
+public class ValuesPrel extends AbstractRelNode implements Prel {
+
+  @SuppressWarnings("unused")
+  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ValuesPrel.class);
+
+  private JSONOptions content;
+
+  public ValuesPrel(RelOptCluster cluster, RelTraitSet traitSet, RelDataType rowType, JSONOptions content) {
+    super(cluster, traitSet);
+    this.rowType = rowType;
+    this.content = content;
+  }
+
+  @Override
+  public Iterator<Prel> iterator() {
+    return Iterators.emptyIterator();
+  }
+
+  @Override
+  public Values getPhysicalOperator(PhysicalPlanCreator creator) throws IOException {
+    return new Values(content);
+  }
+
+  @Override
+  public <T, X, E extends Throwable> T accept(PrelVisitor<T, X, E> logicalVisitor, X value) throws E {
+    return logicalVisitor.visitPrel(this, value);
+  }
+
+  @Override
+  public SelectionVectorMode[] getSupportedEncodings() {
+    return SelectionVectorMode.DEFAULT;
+  }
+
+  @Override
+  public SelectionVectorMode getEncoding() {
+    return SelectionVectorMode.NONE;
+  }
+
+  @Override
+  public boolean needsFinalColumnReordering() {
+    return false;
+  }
+
+
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/314e5a2a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ValuesPrule.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ValuesPrule.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ValuesPrule.java
new file mode 100644
index 0000000..dbe45ca
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ValuesPrule.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.planner.physical;
+
+import java.io.IOException;
+
+import org.apache.drill.exec.planner.logical.DrillValuesRel;
+import org.apache.drill.exec.planner.logical.RelOptHelper;
+import org.eigenbase.relopt.RelOptRule;
+import org.eigenbase.relopt.RelOptRuleCall;
+
+public class ValuesPrule extends RelOptRule {
+  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ValuesPrule.class);
+
+  public static final ValuesPrule INSTANCE = new ValuesPrule();
+
+  private ValuesPrule(){
+    super(RelOptHelper.any(DrillValuesRel.class), "Prel.ValuesPrule");
+  }
+
+  @Override
+  public void onMatch(final RelOptRuleCall call) {
+    final DrillValuesRel rel = (DrillValuesRel) call.rel(0);
+    try{
+      call.transformTo(new ValuesPrel(rel.getCluster(), rel.getTraitSet().plus(Prel.DRILL_PHYSICAL), rel.getRowType(), rel.getTuplesAsJsonOptions()));
+    }catch(IOException e){
+      logger.warn("Failure while converting JSONOptions.", e);
+    }
+  }
+
+
+
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/314e5a2a/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/BaseOptionManager.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/BaseOptionManager.java b/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/BaseOptionManager.java
new file mode 100644
index 0000000..e15eddf
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/BaseOptionManager.java
@@ -0,0 +1,56 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.exec.server.options;
+
+import org.apache.drill.exec.server.options.TypeValidators.BooleanValidator;
+import org.apache.drill.exec.server.options.TypeValidators.DoubleValidator;
+import org.apache.drill.exec.server.options.TypeValidators.LongValidator;
+import org.apache.drill.exec.server.options.TypeValidators.StringValidator;
+
+abstract class BaseOptionManager implements OptionManager {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(BaseOptionManager.class);
+
+
+  private OptionValue getOptionSafe(OptionValidator validator){
+    OptionValue value = getOption(validator.getOptionName());
+    if(value == null){
+      throw new IllegalArgumentException(String.format("Unknown value for boolean option `%s`.", validator.getOptionName()));
+    }
+    return value;
+  }
+
+  @Override
+  public boolean getOption(BooleanValidator validator) {
+    return getOptionSafe(validator).bool_val;
+  }
+
+  @Override
+  public double getOption(DoubleValidator validator) {
+    return getOptionSafe(validator).float_val;
+  }
+
+  @Override
+  public long getOption(LongValidator validator) {
+    return getOptionSafe(validator).num_val;
+  }
+
+  @Override
+  public String getOption(StringValidator validator) {
+    return getOptionSafe(validator).string_val;
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/314e5a2a/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/FallbackOptionManager.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/FallbackOptionManager.java b/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/FallbackOptionManager.java
index 45d393c..4e90616 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/FallbackOptionManager.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/FallbackOptionManager.java
@@ -24,7 +24,7 @@ import org.eigenbase.sql.SqlLiteral;
 
 import com.google.common.collect.Iterables;
 
-public abstract class FallbackOptionManager implements OptionManager {
+public abstract class FallbackOptionManager extends BaseOptionManager {
 //  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(FallbackOptionManager.class);
 
   protected final OptionManager fallback;

http://git-wip-us.apache.org/repos/asf/drill/blob/314e5a2a/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/OptionManager.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/OptionManager.java b/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/OptionManager.java
index 4ffe9a3..0b8811a 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/OptionManager.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/OptionManager.java
@@ -18,6 +18,10 @@
 package org.apache.drill.exec.server.options;
 
 import org.apache.drill.exec.server.options.OptionValue.OptionType;
+import org.apache.drill.exec.server.options.TypeValidators.BooleanValidator;
+import org.apache.drill.exec.server.options.TypeValidators.DoubleValidator;
+import org.apache.drill.exec.server.options.TypeValidators.LongValidator;
+import org.apache.drill.exec.server.options.TypeValidators.StringValidator;
 import org.eigenbase.sql.SqlLiteral;
 
 public interface OptionManager extends Iterable<OptionValue> {
@@ -28,6 +32,11 @@ public interface OptionManager extends Iterable<OptionValue> {
   public OptionManager getSystemManager();
   public OptionList getOptionList();
 
+  public boolean getOption(BooleanValidator validator);
+  public double getOption(DoubleValidator validator);
+  public long getOption(LongValidator validator);
+  public String getOption(StringValidator validator);
+
   public interface OptionAdmin {
     public void registerOptionType(OptionValidator validator);
     public void validate(OptionValue v) throws SetOptionException;

http://git-wip-us.apache.org/repos/asf/drill/blob/314e5a2a/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/SystemOptionManager.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/SystemOptionManager.java b/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/SystemOptionManager.java
index 1a8559e..4471d4f 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/SystemOptionManager.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/SystemOptionManager.java
@@ -30,6 +30,10 @@ import org.apache.drill.exec.compile.ClassTransformer;
 import org.apache.drill.exec.compile.QueryClassLoader;
 import org.apache.drill.exec.planner.physical.PlannerSettings;
 import org.apache.drill.exec.server.options.OptionValue.OptionType;
+import org.apache.drill.exec.server.options.TypeValidators.BooleanValidator;
+import org.apache.drill.exec.server.options.TypeValidators.DoubleValidator;
+import org.apache.drill.exec.server.options.TypeValidators.LongValidator;
+import org.apache.drill.exec.server.options.TypeValidators.StringValidator;
 import org.apache.drill.exec.store.sys.PStore;
 import org.apache.drill.exec.store.sys.PStoreConfig;
 import org.apache.drill.exec.store.sys.PStoreProvider;
@@ -37,7 +41,7 @@ import org.eigenbase.sql.SqlLiteral;
 
 import com.google.common.collect.Maps;
 
-public class SystemOptionManager implements OptionManager {
+public class SystemOptionManager extends BaseOptionManager {
   private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(SystemOptionManager.class);
 
   private static final OptionValidator[] VALIDATORS = {
@@ -72,6 +76,7 @@ public class SystemOptionManager implements OptionManager {
       ExecConstants.PARQUET_VECTOR_FILL_CHECK_THRESHOLD_VALIDATOR,
       ExecConstants.PARQUET_RECORD_READER_IMPLEMENTATION_VALIDATOR,
       ExecConstants.JSON_READER_ALL_TEXT_MODE_VALIDATOR,
+      ExecConstants.JSON_EXTENDED_TYPES,
       ExecConstants.FILESYSTEM_PARTITION_COLUMN_LABEL_VALIDATOR,
       ExecConstants.MONGO_READER_ALL_TEXT_MODE_VALIDATOR,
       ExecConstants.SLICE_TARGET_OPTION,