You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@drill.apache.org by ja...@apache.org on 2014/05/13 05:46:10 UTC

[3/3] git commit: DRILL-694: Implement parquet writer.

DRILL-694: Implement parquet writer.

Enable "CREATE TABLE AS" with parquet as the output format.

Add decimal metadata support to parquet reader and writer.


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

Branch: refs/heads/master
Commit: 63b0346761692f0a4c851e7910791c028dc3cded
Parents: 14c628c
Author: Steven Phillips <sp...@maprtech.com>
Authored: Mon May 12 02:56:25 2014 -0700
Committer: Aditya Kishore <ad...@maprtech.com>
Committed: Mon May 12 18:18:43 2014 -0700

----------------------------------------------------------------------
 .../org/apache/drill/common/types/Types.java    |   4 +
 .../drill/common/util/DecimalUtility.java       |  11 +-
 exec/java-exec/pom.xml                          |   4 +-
 .../templates/EventBasedRecordWriter.java       |   6 +-
 .../templates/ParquetOutputRecordWriter.java    | 189 +++++++++++
 .../codegen/templates/ParquetTypeHelper.java    | 155 ++++++++++
 .../org/apache/drill/exec/ExecConstants.java    |   8 +
 .../server/options/SystemOptionManager.java     |   5 +-
 .../exec/store/dfs/WorkspaceSchemaFactory.java  |   9 +-
 .../drill/exec/store/parquet/BitReader.java     |   5 +-
 .../drill/exec/store/parquet/ColumnReader.java  |  14 +-
 .../store/parquet/FixedByteAlignedReader.java   |  87 +++++-
 .../exec/store/parquet/NullableBitReader.java   |   5 +-
 .../store/parquet/NullableColumnReader.java     |   5 +-
 .../parquet/NullableFixedByteAlignedReader.java |  81 ++++-
 .../exec/store/parquet/ParquetFormatPlugin.java |  48 ++-
 .../exec/store/parquet/ParquetRecordReader.java | 310 ++++++++++++++-----
 .../exec/store/parquet/ParquetRecordWriter.java | 185 +++++++++++
 .../drill/exec/store/parquet/ParquetWriter.java |  91 ++++++
 .../parquet/ParquetWriterBatchCreator.java      |  36 +++
 .../store/parquet/VarLengthColumnReaders.java   | 182 ++++++++---
 .../ColumnChunkPageWriteStoreExposer.java       |  39 +++
 .../physical/impl/writer/TestParquetWriter.java | 163 ++++++++++
 .../exec/physical/impl/writer/TestWriter.java   |  19 +-
 24 files changed, 1510 insertions(+), 151 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/63b03467/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 c6ac82e..a6ec29b 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
@@ -271,6 +271,10 @@ public class Types {
     return MajorType.newBuilder().setMode(mode).setMinorType(type).build();
   }
 
+  public static MajorType withScaleAndPrecision(MinorType type, DataMode mode, int scale, int precision) {
+    return MajorType.newBuilder().setMinorType(type).setMode(mode).setScale(scale).setPrecision(precision).build();
+  }
+  
   public static MajorType required(MinorType type){
     return MajorType.newBuilder().setMode(DataMode.REQUIRED).setMinorType(type).build();
   }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/63b03467/common/src/main/java/org/apache/drill/common/util/DecimalUtility.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/drill/common/util/DecimalUtility.java b/common/src/main/java/org/apache/drill/common/util/DecimalUtility.java
index dbfd6ac..0cacc59 100644
--- a/common/src/main/java/org/apache/drill/common/util/DecimalUtility.java
+++ b/common/src/main/java/org/apache/drill/common/util/DecimalUtility.java
@@ -17,8 +17,7 @@
  */
 package org.apache.drill.common.util;
 
-import java.math.BigDecimal;
-import java.math.BigInteger;
+import java.math.*;
 
 
 import io.netty.buffer.Unpooled;
@@ -26,6 +25,7 @@ import io.netty.buffer.ByteBuf;
 
 import java.math.BigDecimal;
 import java.math.BigInteger;
+import java.util.Arrays;
 
 public class DecimalUtility {
 
@@ -486,5 +486,12 @@ public class DecimalUtility {
       return 0;
     }
 
+
+    public static BigDecimal getBigDecimalFromByteArray(byte[] bytes, int start, int length, int scale) {
+      byte[] value = Arrays.copyOfRange(bytes, start, start + length);
+      BigInteger unscaledValue = new BigInteger(value);
+      return new BigDecimal(unscaledValue, scale);
+    }
+
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/63b03467/exec/java-exec/pom.xml
----------------------------------------------------------------------
diff --git a/exec/java-exec/pom.xml b/exec/java-exec/pom.xml
index 3e26662..e282c00 100644
--- a/exec/java-exec/pom.xml
+++ b/exec/java-exec/pom.xml
@@ -95,7 +95,7 @@
     <dependency>
       <groupId>com.twitter</groupId>
       <artifactId>parquet-column</artifactId>
-      <version>1.4.0</version>
+      <version>1.5.0-SNAPSHOT</version>
       <exclusions>
         <exclusion>
           <groupId>org.apache.hadoop</groupId>
@@ -110,7 +110,7 @@
     <dependency>
       <groupId>com.twitter</groupId>
       <artifactId>parquet-hadoop</artifactId>
-      <version>1.4.0</version>
+      <version>1.5.0-SNAPSHOT</version>
       <exclusions>
         <exclusion>
           <groupId>org.apache.hadoop</groupId>

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/63b03467/exec/java-exec/src/main/codegen/templates/EventBasedRecordWriter.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/codegen/templates/EventBasedRecordWriter.java b/exec/java-exec/src/main/codegen/templates/EventBasedRecordWriter.java
index 73b6566..b58f24c 100644
--- a/exec/java-exec/src/main/codegen/templates/EventBasedRecordWriter.java
+++ b/exec/java-exec/src/main/codegen/templates/EventBasedRecordWriter.java
@@ -91,8 +91,10 @@ public class EventBasedRecordWriter {
     fieldWriters = Lists.newArrayList();
     try {
       for (int i = 0; i < schema.getFieldCount(); i++) {
-        fieldWriters.add(i, typeClassMap.get(schema.getColumn(i).getType())
-            .getConstructor(EventBasedRecordWriter.class, int.class).newInstance(this, i));
+        MajorType mt = schema.getColumn(i).getType();
+        MajorType newMt = MajorType.newBuilder().setMinorType(mt.getMinorType()).setMode(mt.getMode()).build();
+        fieldWriters.add(i, typeClassMap.get(newMt)
+                .getConstructor(EventBasedRecordWriter.class, int.class).newInstance(this, i));
       }
     } catch(Exception e) {
       logger.error("Failed to create FieldWriter.", e);

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/63b03467/exec/java-exec/src/main/codegen/templates/ParquetOutputRecordWriter.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/codegen/templates/ParquetOutputRecordWriter.java b/exec/java-exec/src/main/codegen/templates/ParquetOutputRecordWriter.java
new file mode 100644
index 0000000..5f75c1c
--- /dev/null
+++ b/exec/java-exec/src/main/codegen/templates/ParquetOutputRecordWriter.java
@@ -0,0 +1,189 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import parquet.io.api.Binary;
+
+import java.lang.Override;
+
+<@pp.dropOutputFile />
+<@pp.changeOutputFile name="org/apache/drill/exec/store/ParquetOutputRecordWriter.java" />
+<#include "/@includes/license.ftl" />
+
+package org.apache.drill.exec.store;
+
+import com.google.common.collect.Lists;
+import org.apache.drill.common.types.TypeProtos.MinorType;
+import org.apache.drill.exec.expr.TypeHelper;
+import org.apache.drill.exec.expr.holders.*;
+import org.apache.drill.exec.record.BatchSchema;
+import org.apache.drill.exec.store.parquet.ParquetTypeHelper;
+import org.apache.drill.exec.vector.*;
+import org.apache.drill.common.util.DecimalUtility;
+import parquet.io.api.RecordConsumer;
+import parquet.schema.MessageType;
+import parquet.io.api.Binary;
+import io.netty.buffer.ByteBuf;
+
+import java.io.IOException;
+import java.lang.UnsupportedOperationException;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * Abstract implementation of RecordWriter interface which exposes interface:
+ *    {@link #writeHeader(List)}
+ *    {@link #addField(int,String)}
+ * to output the data in string format instead of implementing addField for each type holder.
+ *
+ * This is useful for text format writers such as CSV, TSV etc.
+ */
+public abstract class ParquetOutputRecordWriter implements RecordWriter {
+
+  private RecordConsumer consumer;
+  private MessageType schema;
+
+  public void setUp(MessageType schema, RecordConsumer consumer) {
+    this.schema = schema;
+    this.consumer = consumer;
+  }
+
+<#list vv.types as type>
+  <#list type.minor as minor>
+    <#list vv.modes as mode>
+  @Override
+  public void add${mode.prefix}${minor.class}Holder(int fieldId, ${mode.prefix}${minor.class}Holder valueHolder) throws IOException {
+  <#if mode.prefix == "Nullable" >
+    if (valueHolder.isSet == 0) {
+      return;
+    }
+  <#elseif mode.prefix == "Repeated" >
+    consumer.startField(schema.getFieldName(fieldId), fieldId);
+    for (int i = valueHolder.start; i < valueHolder.end; i++) {
+  </#if>
+
+  <#if  minor.class == "TinyInt" ||
+        minor.class == "UInt1" ||
+        minor.class == "UInt2" ||
+        minor.class == "SmallInt" ||
+        minor.class == "Int" ||
+        minor.class == "Time" ||
+        minor.class == "IntervalYear" ||
+        minor.class == "Decimal9" ||
+        minor.class == "UInt4">
+    <#if mode.prefix == "Repeated" >
+            consumer.addInteger(valueHolder.vector.getAccessor().get(i));
+    <#else>
+    consumer.startField(schema.getFieldName(fieldId), fieldId);
+    consumer.addInteger(valueHolder.value);
+    consumer.endField(schema.getFieldName(fieldId), fieldId);
+    </#if>
+  <#elseif
+        minor.class == "Float4">
+      <#if mode.prefix == "Repeated" >
+              consumer.addFloat(valueHolder.vector.getAccessor().get(i));
+      <#else>
+    consumer.startField(schema.getFieldName(fieldId), fieldId);
+    consumer.addFloat(valueHolder.value);
+    consumer.endField(schema.getFieldName(fieldId), fieldId);
+      </#if>
+  <#elseif
+        minor.class == "BigInt" ||
+        minor.class == "Decimal18" ||
+        minor.class == "TimeStamp" ||
+        minor.class == "Date" ||
+        minor.class == "UInt8">
+      <#if mode.prefix == "Repeated" >
+              consumer.addLong(valueHolder.vector.getAccessor().get(i));
+      <#else>
+    consumer.startField(schema.getFieldName(fieldId), fieldId);
+    consumer.addLong(valueHolder.value);
+    consumer.endField(schema.getFieldName(fieldId), fieldId);
+      </#if>
+  <#elseif
+        minor.class == "Float8">
+      <#if mode.prefix == "Repeated" >
+              consumer.addDouble(valueHolder.vector.getAccessor().get(i));
+      <#else>
+    consumer.startField(schema.getFieldName(fieldId), fieldId);
+    consumer.addDouble(valueHolder.value);
+    consumer.endField(schema.getFieldName(fieldId), fieldId);
+      </#if>
+  <#elseif
+        minor.class == "Bit">
+      <#if mode.prefix == "Repeated" >
+              consumer.addBoolean(valueHolder.vector.getAccessor().get(i) == 1);
+      <#else>
+    consumer.startField(schema.getFieldName(fieldId), fieldId);
+    consumer.addBoolean(valueHolder.value == 1);
+    consumer.endField(schema.getFieldName(fieldId), fieldId);
+      </#if>
+  <#elseif
+        minor.class == "Decimal28Sparse" ||
+        minor.class == "Decimal38Sparse">
+      <#if mode.prefix == "Repeated" >
+      <#else>
+      consumer.startField(schema.getFieldName(fieldId), fieldId);
+      byte[] bytes = DecimalUtility.getBigDecimalFromSparse(
+              valueHolder.buffer, valueHolder.start, ${minor.class}Holder.nDecimalDigits, valueHolder.scale).unscaledValue().toByteArray();
+      byte[] output = new byte[ParquetTypeHelper.getLengthForMinorType(MinorType.${minor.class?upper_case})];
+      if (valueHolder.sign) {
+        Arrays.fill(output, 0, output.length - bytes.length, (byte)0xFF);
+      } else {
+        Arrays.fill(output, 0, output.length - bytes.length, (byte)0x0);
+      }
+      System.arraycopy(bytes, 0, output, output.length - bytes.length, bytes.length);
+      consumer.addBinary(Binary.fromByteArray(output));
+      consumer.endField(schema.getFieldName(fieldId), fieldId);
+      </#if>
+  <#elseif
+        minor.class == "TimeTZ" ||
+        minor.class == "TimeStampTZ" ||
+        minor.class == "IntervalDay" ||
+        minor.class == "Interval" ||
+        minor.class == "Decimal28Dense" ||
+        minor.class == "Decimal38Dense">
+
+      <#if mode.prefix == "Repeated" >
+      <#else>
+
+      </#if>
+  <#elseif minor.class == "VarChar" || minor.class == "Var16Char" || minor.class == "VarBinary">
+    <#if mode.prefix == "Repeated">
+      ${minor.class}Holder singleHolder = new ${minor.class}Holder();
+      valueHolder.vector.getAccessor().get(i, singleHolder);
+      consumer.startField(schema.getFieldName(fieldId), fieldId);
+      consumer.addBinary(Binary.fromByteBuffer(singleHolder.buffer.nioBuffer(singleHolder.start, singleHolder.end - singleHolder.start)));
+      consumer.endField(schema.getFieldName(fieldId), fieldId);
+    <#else>
+    ByteBuf buf = valueHolder.buffer;
+    consumer.startField(schema.getFieldName(fieldId), fieldId);
+    consumer.addBinary(Binary.fromByteBuffer(valueHolder.buffer.nioBuffer(valueHolder.start, valueHolder.end - valueHolder.start)));
+    consumer.endField(schema.getFieldName(fieldId), fieldId);
+    </#if>
+  </#if>
+  <#if mode.prefix == "Repeated">
+    }
+    consumer.endField(schema.getFieldName(fieldId), fieldId);
+  </#if>
+  }
+    </#list>
+  </#list>
+</#list>
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/63b03467/exec/java-exec/src/main/codegen/templates/ParquetTypeHelper.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/codegen/templates/ParquetTypeHelper.java b/exec/java-exec/src/main/codegen/templates/ParquetTypeHelper.java
new file mode 100644
index 0000000..b268d33
--- /dev/null
+++ b/exec/java-exec/src/main/codegen/templates/ParquetTypeHelper.java
@@ -0,0 +1,155 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import parquet.format.ConvertedType;
+import parquet.schema.DecimalMetadata;
+import parquet.schema.OriginalType;
+import parquet.schema.PrimitiveType.PrimitiveTypeName;
+
+<@pp.dropOutputFile />
+<@pp.changeOutputFile name="org/apache/drill/exec/store/parquet/ParquetTypeHelper.java" />
+<#include "/@includes/license.ftl" />
+
+package org.apache.drill.exec.store.parquet;
+
+import org.apache.drill.common.types.TypeProtos.DataMode;
+import org.apache.drill.common.types.TypeProtos.MinorType;
+import org.apache.drill.exec.record.MaterializedField;
+import parquet.schema.OriginalType;
+import parquet.schema.DecimalMetadata;
+import parquet.schema.PrimitiveType.PrimitiveTypeName;
+import parquet.schema.Type.Repetition;
+
+import java.util.HashMap;
+import java.util.Map;
+
+public class ParquetTypeHelper {
+  private static Map<MinorType,PrimitiveTypeName> typeMap;
+  private static Map<DataMode,Repetition> modeMap;
+  private static Map<MinorType,OriginalType> originalTypeMap;
+
+  static {
+    typeMap = new HashMap();
+
+    <#list vv.types as type>
+    <#list type.minor as minor>
+    <#if    minor.class == "TinyInt" ||
+            minor.class == "UInt1" ||
+            minor.class == "UInt2" ||
+            minor.class == "SmallInt" ||
+            minor.class == "Int" ||
+            minor.class == "Time" ||
+            minor.class == "IntervalYear" ||
+            minor.class == "Decimal9" ||
+            minor.class == "UInt4">
+                    typeMap.put(MinorType.${minor.class?upper_case}, PrimitiveTypeName.INT32);
+    <#elseif
+            minor.class == "Float4">
+                    typeMap.put(MinorType.${minor.class?upper_case}, PrimitiveTypeName.FLOAT);
+    <#elseif
+            minor.class == "BigInt" ||
+            minor.class == "Decimal18" ||
+            minor.class == "TimeStamp" ||
+            minor.class == "Date" ||
+            minor.class == "UInt8">
+                    typeMap.put(MinorType.${minor.class?upper_case}, PrimitiveTypeName.INT64);
+    <#elseif
+            minor.class == "Float8">
+                    typeMap.put(MinorType.${minor.class?upper_case}, PrimitiveTypeName.DOUBLE);
+    <#elseif
+            minor.class == "Bit">
+                    typeMap.put(MinorType.${minor.class?upper_case}, PrimitiveTypeName.BOOLEAN);
+    <#elseif
+            minor.class == "TimeTZ" ||
+            minor.class == "TimeStampTZ" ||
+            minor.class == "IntervalDay" ||
+            minor.class == "Interval" ||
+            minor.class == "Decimal28Dense" ||
+            minor.class == "Decimal38Dense" ||
+            minor.class == "Decimal28Sparse" ||
+            minor.class == "Decimal38Sparse">
+                    typeMap.put(MinorType.${minor.class?upper_case}, PrimitiveTypeName.FIXED_LEN_BYTE_ARRAY);
+    <#elseif
+            minor.class == "VarChar" ||
+            minor.class == "Var16Char" ||
+            minor.class == "VarBinary" >
+                    typeMap.put(MinorType.${minor.class?upper_case}, PrimitiveTypeName.BINARY);
+    </#if>
+    </#list>
+    </#list>
+
+    modeMap = new HashMap();
+
+    modeMap.put(DataMode.REQUIRED, Repetition.REQUIRED);
+    modeMap.put(DataMode.OPTIONAL, Repetition.OPTIONAL);
+    modeMap.put(DataMode.REPEATED, Repetition.REPEATED);
+
+    originalTypeMap = new HashMap();
+
+    <#list vv.types as type>
+    <#list type.minor as minor>
+            <#if minor.class.startsWith("Decimal")>
+            originalTypeMap.put(MinorType.${minor.class?upper_case},OriginalType.DECIMAL);
+            </#if>
+    </#list>
+    </#list>
+            originalTypeMap.put(MinorType.VARCHAR, OriginalType.UTF8);
+            originalTypeMap.put(MinorType.DATE, OriginalType.DATE);
+            originalTypeMap.put(MinorType.TIME, OriginalType.TIME);
+            originalTypeMap.put(MinorType.TIMESTAMP, OriginalType.TIMESTAMP);
+            originalTypeMap.put(MinorType.TIMESTAMPTZ, OriginalType.TIMESTAMPTZ);
+  }
+
+  public static PrimitiveTypeName getPrimitiveTypeNameForMinorType(MinorType minorType) {
+    return typeMap.get(minorType);
+  }
+
+  public static Repetition getRepetitionForDataMode(DataMode dataMode) {
+    return modeMap.get(dataMode);
+  }
+
+  public static OriginalType getOriginalTypeForMinorType(MinorType minorType) {
+    return originalTypeMap.get(minorType);
+  }
+
+  public static DecimalMetadata getDecimalMetadataForField(MaterializedField field) {
+    switch(field.getType().getMinorType()) {
+      case DECIMAL9:
+      case DECIMAL18:
+      case DECIMAL28SPARSE:
+      case DECIMAL28DENSE:
+      case DECIMAL38SPARSE:
+      case DECIMAL38DENSE:
+        return new DecimalMetadata(field.getPrecision(), field.getScale());
+      default:
+        return null;
+    }
+  }
+
+  public static int getLengthForMinorType(MinorType minorType) {
+    switch(minorType) {
+      case DECIMAL28SPARSE:
+        return 12;
+      case DECIMAL38SPARSE:
+        return 16;
+      default:
+        return 0;
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/63b03467/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 9eee08d..238fae9 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
@@ -17,6 +17,10 @@
  */
 package org.apache.drill.exec;
 
+import org.apache.drill.exec.server.options.OptionValidator;
+import org.apache.drill.exec.server.options.TypeValidators.LongValidator;
+import org.apache.drill.exec.server.options.TypeValidators.StringValidator;
+
 public interface ExecConstants {
   public static final String ZK_RETRY_TIMES = "drill.exec.zk.retry.count";
   public static final String ZK_RETRY_DELAY = "drill.exec.zk.retry.delay";
@@ -63,5 +67,9 @@ public interface ExecConstants {
   public static final String FILESYSTEM_PARTITION_COLUMN_LABEL = "drill.exec.storage.file.partition.column.label";
   public static final String HAZELCAST_SUBNETS = "drill.exec.cache.hazel.subnets";
   public static final String TOP_LEVEL_MAX_ALLOC = "drill.exec.memory.top.max";
+  public static final String OUTPUT_FORMAT_OPTION = "store.format";
+  public static final OptionValidator OUTPUT_FORMAT_VALIDATOR = new StringValidator(OUTPUT_FORMAT_OPTION, "parquet");
+  public static final String PARQUET_BLOCK_SIZE = "parquet.block.size";
+  public static final OptionValidator PARQUET_BLOCK_SIZE_VALIDATOR = new LongValidator(PARQUET_BLOCK_SIZE, 512*1024*1024);
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/63b03467/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 cfe8e2c..21031e5 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
@@ -22,6 +22,7 @@ import java.util.Map;
 import java.util.Map.Entry;
 import java.util.concurrent.ConcurrentMap;
 
+import org.apache.drill.exec.ExecConstants;
 import org.apache.drill.exec.cache.DistributedCache;
 import org.apache.drill.exec.cache.DistributedMap;
 import org.apache.drill.exec.planner.physical.PlannerSettings;
@@ -38,7 +39,9 @@ public class SystemOptionManager implements OptionManager{
       PlannerSettings.STREAMAGG,
       PlannerSettings.HASHJOIN,
       PlannerSettings.MERGEJOIN, 
-      PlannerSettings.MULTIPHASE
+      PlannerSettings.MULTIPHASE,
+      ExecConstants.OUTPUT_FORMAT_VALIDATOR,
+      ExecConstants.PARQUET_BLOCK_SIZE_VALIDATOR
   };
 
   private DistributedMap<OptionValue> options;

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/63b03467/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/WorkspaceSchemaFactory.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/WorkspaceSchemaFactory.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/WorkspaceSchemaFactory.java
index df73ea5..72b31e6 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/WorkspaceSchemaFactory.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/WorkspaceSchemaFactory.java
@@ -26,12 +26,15 @@ import com.google.common.collect.Lists;
 import com.google.common.collect.Sets;
 import net.hydromatic.optiq.Table;
 import org.apache.drill.common.exceptions.ExecutionSetupException;
+import org.apache.drill.exec.ExecConstants;
 import org.apache.drill.exec.planner.logical.CreateTableEntry;
 import org.apache.drill.exec.planner.logical.DrillTable;
 import org.apache.drill.exec.planner.logical.DynamicDrillTable;
 import org.apache.drill.exec.planner.logical.FileSystemCreateTableEntry;
 import org.apache.drill.exec.planner.sql.ExpandingConcurrentMap;
 import org.apache.drill.exec.rpc.user.UserSession;
+import org.apache.drill.exec.server.options.OptionValidator;
+import org.apache.drill.exec.server.options.TypeValidators.StringValidator;
 import org.apache.drill.exec.store.AbstractSchema;
 import org.apache.drill.exec.store.dfs.shim.DrillFileSystem;
 import org.apache.hadoop.fs.Path;
@@ -141,9 +144,11 @@ public class WorkspaceSchemaFactory implements ExpandingConcurrentMap.MapValueFa
       return fs;
     }
 
+
     @Override
     public CreateTableEntry createNewTable(String tableName) {
-      FormatPlugin formatPlugin = plugin.getFormatPlugin(config.getStorageFormat());
+      String storage = session.getOptions().getOption(ExecConstants.OUTPUT_FORMAT_OPTION).string_val;
+      FormatPlugin formatPlugin = plugin.getFormatPlugin(storage);
       if (formatPlugin == null)
         throw new UnsupportedOperationException(
           String.format("Unsupported format '%s' in workspace '%s'", config.getStorageFormat(),
@@ -151,7 +156,7 @@ public class WorkspaceSchemaFactory implements ExpandingConcurrentMap.MapValueFa
 
       return new FileSystemCreateTableEntry(
           (FileSystemConfig) plugin.getConfig(),
-          plugin.getFormatPlugin(config.getStorageFormat()),
+          formatPlugin,
           config.getLocation() + Path.SEPARATOR + tableName);
     }
   }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/63b03467/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/BitReader.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/BitReader.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/BitReader.java
index c323222..7ae95cd 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/BitReader.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/BitReader.java
@@ -22,6 +22,7 @@ import org.apache.drill.exec.vector.BaseDataValueVector;
 import org.apache.drill.exec.vector.ValueVector;
 import parquet.column.ColumnDescriptor;
 import parquet.format.ConvertedType;
+import parquet.format.SchemaElement;
 import parquet.hadoop.metadata.ColumnChunkMetaData;
 
 final class BitReader extends ColumnReader {
@@ -31,8 +32,8 @@ final class BitReader extends ColumnReader {
   private byte[] bytes;
   
   BitReader(ParquetRecordReader parentReader, int allocateSize, ColumnDescriptor descriptor, ColumnChunkMetaData columnChunkMetaData,
-            boolean fixedLength, ValueVector v, ConvertedType convertedType) throws ExecutionSetupException {
-    super(parentReader, allocateSize, descriptor, columnChunkMetaData, fixedLength, v, convertedType);
+            boolean fixedLength, ValueVector v, SchemaElement schemaElement) throws ExecutionSetupException {
+    super(parentReader, allocateSize, descriptor, columnChunkMetaData, fixedLength, v, schemaElement);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/63b03467/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ColumnReader.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ColumnReader.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ColumnReader.java
index 196e1fd..b9faafe 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ColumnReader.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ColumnReader.java
@@ -25,8 +25,10 @@ import org.apache.drill.exec.vector.ValueVector;
 import org.apache.hadoop.fs.FSDataInputStream;
 import parquet.column.ColumnDescriptor;
 import parquet.format.ConvertedType;
+import parquet.format.SchemaElement;
 import parquet.hadoop.metadata.ColumnChunkMetaData;
 import parquet.schema.PrimitiveType;
+import parquet.schema.PrimitiveType.PrimitiveTypeName;
 
 import java.io.IOException;
 
@@ -48,7 +50,7 @@ abstract class ColumnReader<V extends ValueVector> {
   // status information on the current page
   final PageReadStatus pageReadStatus;
 
-  final ConvertedType convertedType;
+  final SchemaElement schemaElement;
 
   // quick reference to see if the field is fixed length (as this requires an instanceof)
   final boolean isFixedLength;
@@ -70,12 +72,12 @@ abstract class ColumnReader<V extends ValueVector> {
   long readStartInBytes = 0, readLength = 0, readLengthInBits = 0, recordsReadInThisIteration = 0;
 
   protected ColumnReader(ParquetRecordReader parentReader, int allocateSize, ColumnDescriptor descriptor,
-      ColumnChunkMetaData columnChunkMetaData, boolean fixedLength, V v, ConvertedType convertedType) throws ExecutionSetupException {
+      ColumnChunkMetaData columnChunkMetaData, boolean fixedLength, V v, SchemaElement schemaElement) throws ExecutionSetupException {
     this.parentReader = parentReader;
     this.columnDescriptor = descriptor;
     this.columnChunkMetaData = columnChunkMetaData;
     this.isFixedLength = fixedLength;
-    this.convertedType = convertedType;
+    this.schemaElement = schemaElement;
 
     if (allocateSize > 1) {
       valueVec =  v;
@@ -87,7 +89,11 @@ abstract class ColumnReader<V extends ValueVector> {
     this.pageReadStatus = new PageReadStatus(this, parentReader.fileSystem, parentReader.hadoopPath, columnChunkMetaData);
 
     if (columnDescriptor.getType() != PrimitiveType.PrimitiveTypeName.BINARY) {
-      dataTypeLengthInBits = ParquetRecordReader.getTypeLengthInBits(columnDescriptor.getType());
+      if (columnDescriptor.getType() == PrimitiveTypeName.FIXED_LEN_BYTE_ARRAY) {
+        dataTypeLengthInBits = columnDescriptor.getTypeLength() * 8;
+      } else {
+        dataTypeLengthInBits = ParquetRecordReader.getTypeLengthInBits(columnDescriptor.getType());
+      }
     }
 
   }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/63b03467/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/FixedByteAlignedReader.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/FixedByteAlignedReader.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/FixedByteAlignedReader.java
index 0aa18cf..574b0cb 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/FixedByteAlignedReader.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/FixedByteAlignedReader.java
@@ -18,20 +18,27 @@
 package org.apache.drill.exec.store.parquet;
 
 import org.apache.drill.common.exceptions.ExecutionSetupException;
-import org.apache.drill.exec.vector.BaseDataValueVector;
+import org.apache.drill.common.util.DecimalUtility;
+import org.apache.drill.exec.expr.holders.Decimal28SparseHolder;
+import org.apache.drill.exec.expr.holders.Decimal38SparseHolder;
+import org.apache.drill.exec.vector.Decimal28SparseVector;
+import org.apache.drill.exec.vector.Decimal38SparseVector;
 import org.apache.drill.exec.vector.ValueVector;
 import parquet.column.ColumnDescriptor;
 import parquet.format.ConvertedType;
+import parquet.format.SchemaElement;
 import parquet.hadoop.metadata.ColumnChunkMetaData;
 
+import java.math.BigDecimal;
+
 class FixedByteAlignedReader extends ColumnReader {
 
-  private byte[] bytes;
+  protected byte[] bytes;
 
   
   FixedByteAlignedReader(ParquetRecordReader parentReader, int allocateSize, ColumnDescriptor descriptor, ColumnChunkMetaData columnChunkMetaData,
-                         boolean fixedLength, ValueVector v, ConvertedType convertedType) throws ExecutionSetupException {
-    super(parentReader, allocateSize, descriptor, columnChunkMetaData, fixedLength, v, convertedType);
+                         boolean fixedLength, ValueVector v, SchemaElement schemaElement) throws ExecutionSetupException {
+    super(parentReader, allocateSize, descriptor, columnChunkMetaData, fixedLength, v, schemaElement);
   }
 
   // this method is called by its superclass during a read loop
@@ -50,4 +57,76 @@ class FixedByteAlignedReader extends ColumnReader {
     vectorData.writeBytes(bytes,
         (int) readStartInBytes, (int) readLength);
   }
+
+  public static abstract class ConvertedReader extends FixedByteAlignedReader {
+
+    protected int dataTypeLengthInBytes;
+
+    ConvertedReader(ParquetRecordReader parentReader, int allocateSize, ColumnDescriptor descriptor, ColumnChunkMetaData columnChunkMetaData,
+                           boolean fixedLength, ValueVector v, SchemaElement schemaElement) throws ExecutionSetupException {
+      super(parentReader, allocateSize, descriptor, columnChunkMetaData, fixedLength, v, schemaElement);
+    }
+
+    @Override
+    protected void readField(long recordsToReadInThisPass, ColumnReader firstColumnStatus) {
+      recordsReadInThisIteration = Math.min(pageReadStatus.currentPage.getValueCount()
+              - pageReadStatus.valuesRead, recordsToReadInThisPass - valuesReadInCurrentPass);
+
+      readStartInBytes = pageReadStatus.readPosInBytes;
+      readLengthInBits = recordsReadInThisIteration * dataTypeLengthInBits;
+      readLength = (int) Math.ceil(readLengthInBits / 8.0);
+
+      bytes = pageReadStatus.pageDataByteArray;
+
+      dataTypeLengthInBytes = (int) Math.ceil(dataTypeLengthInBits / 8.0);
+      for (int i = 0; i < recordsReadInThisIteration; i++) {
+        addNext((int)readStartInBytes + i * dataTypeLengthInBytes, i + valuesReadInCurrentPass);
+      }
+    }
+
+    /**
+     * Reads from bytes, converts, and writes to buffer
+     * @param start the index in bytes to start reading from
+     * @param index the index of the ValueVector
+     */
+    abstract void addNext(int start, int index);
+  }
+
+  public static class Decimal28Reader extends ConvertedReader {
+
+    Decimal28SparseVector decimal28Vector;
+
+    Decimal28Reader(ParquetRecordReader parentReader, int allocateSize, ColumnDescriptor descriptor, ColumnChunkMetaData columnChunkMetaData,
+                    boolean fixedLength, ValueVector v, SchemaElement schemaElement) throws ExecutionSetupException {
+      super(parentReader, allocateSize, descriptor, columnChunkMetaData, fixedLength, v, schemaElement);
+      decimal28Vector = (Decimal28SparseVector) v;
+    }
+
+    @Override
+    void addNext(int start, int index) {
+      int width = Decimal28SparseHolder.WIDTH;
+      BigDecimal intermediate = DecimalUtility.getBigDecimalFromByteArray(bytes, start, dataTypeLengthInBytes, schemaElement.getScale());
+      DecimalUtility.getSparseFromBigDecimal(intermediate, decimal28Vector.getData(), index * width, schemaElement.getScale(),
+              schemaElement.getPrecision(), Decimal28SparseHolder.nDecimalDigits);
+    }
+  }
+
+  public static class Decimal38Reader extends ConvertedReader {
+
+    Decimal38SparseVector decimal38Vector;
+
+    Decimal38Reader(ParquetRecordReader parentReader, int allocateSize, ColumnDescriptor descriptor, ColumnChunkMetaData columnChunkMetaData,
+                    boolean fixedLength, ValueVector v, SchemaElement schemaElement) throws ExecutionSetupException {
+      super(parentReader, allocateSize, descriptor, columnChunkMetaData, fixedLength, v, schemaElement);
+      decimal38Vector = (Decimal38SparseVector) v;
+    }
+
+    @Override
+    void addNext(int start, int index) {
+      int width = Decimal38SparseHolder.WIDTH;
+      BigDecimal intermediate = DecimalUtility.getBigDecimalFromByteArray(bytes, start, dataTypeLengthInBytes, schemaElement.getScale());
+      DecimalUtility.getSparseFromBigDecimal(intermediate, decimal38Vector.getData(), index * width, schemaElement.getScale(),
+              schemaElement.getPrecision(), Decimal38SparseHolder.nDecimalDigits);
+    }
+  }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/63b03467/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/NullableBitReader.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/NullableBitReader.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/NullableBitReader.java
index 22933ba..29ca30a 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/NullableBitReader.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/NullableBitReader.java
@@ -24,6 +24,7 @@ import org.apache.drill.exec.vector.NullableBitVector;
 import org.apache.drill.exec.vector.ValueVector;
 import parquet.column.ColumnDescriptor;
 import parquet.format.ConvertedType;
+import parquet.format.SchemaElement;
 import parquet.hadoop.metadata.ColumnChunkMetaData;
 
 import java.io.IOException;
@@ -39,8 +40,8 @@ import java.io.IOException;
 final class NullableBitReader extends ColumnReader {
 
   NullableBitReader(ParquetRecordReader parentReader, int allocateSize, ColumnDescriptor descriptor, ColumnChunkMetaData columnChunkMetaData,
-                    boolean fixedLength, ValueVector v, ConvertedType convertedType) throws ExecutionSetupException {
-    super(parentReader, allocateSize, descriptor, columnChunkMetaData, fixedLength, v, convertedType);
+                    boolean fixedLength, ValueVector v, SchemaElement schemaElement) throws ExecutionSetupException {
+    super(parentReader, allocateSize, descriptor, columnChunkMetaData, fixedLength, v, schemaElement);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/63b03467/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/NullableColumnReader.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/NullableColumnReader.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/NullableColumnReader.java
index 66d1c5f..6040c67 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/NullableColumnReader.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/NullableColumnReader.java
@@ -23,6 +23,7 @@ import org.apache.drill.exec.vector.NullableVectorDefinitionSetter;
 import org.apache.drill.exec.vector.ValueVector;
 import parquet.column.ColumnDescriptor;
 import parquet.format.ConvertedType;
+import parquet.format.SchemaElement;
 import parquet.hadoop.metadata.ColumnChunkMetaData;
 
 import java.io.IOException;
@@ -36,8 +37,8 @@ abstract class NullableColumnReader extends ColumnReader{
   int bitsUsed;
 
   NullableColumnReader(ParquetRecordReader parentReader, int allocateSize, ColumnDescriptor descriptor, ColumnChunkMetaData columnChunkMetaData,
-               boolean fixedLength, ValueVector v, ConvertedType convertedType) throws ExecutionSetupException {
-    super(parentReader, allocateSize, descriptor, columnChunkMetaData, fixedLength, v, convertedType);
+               boolean fixedLength, ValueVector v, SchemaElement schemaElement) throws ExecutionSetupException {
+    super(parentReader, allocateSize, descriptor, columnChunkMetaData, fixedLength, v, schemaElement);
   }
 
   public void readAllFixedFields(long recordsToReadInThisPass, ColumnReader firstColumnStatus) throws IOException {

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/63b03467/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/NullableFixedByteAlignedReader.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/NullableFixedByteAlignedReader.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/NullableFixedByteAlignedReader.java
index 038f2d7..d4416c8 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/NullableFixedByteAlignedReader.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/NullableFixedByteAlignedReader.java
@@ -18,19 +18,27 @@
 package org.apache.drill.exec.store.parquet;
 
 import org.apache.drill.common.exceptions.ExecutionSetupException;
+import org.apache.drill.common.util.DecimalUtility;
+import org.apache.drill.exec.expr.holders.NullableDecimal28SparseHolder;
+import org.apache.drill.exec.expr.holders.NullableDecimal38SparseHolder;
+import org.apache.drill.exec.vector.NullableDecimal28SparseVector;
+import org.apache.drill.exec.vector.NullableDecimal38SparseVector;
 import org.apache.drill.exec.vector.ValueVector;
 
 import parquet.column.ColumnDescriptor;
 import parquet.format.ConvertedType;
+import parquet.format.SchemaElement;
 import parquet.hadoop.metadata.ColumnChunkMetaData;
 
+import java.math.BigDecimal;
+
 class NullableFixedByteAlignedReader extends NullableColumnReader {
 
-  private byte[] bytes;
+  protected byte[] bytes;
 
   NullableFixedByteAlignedReader(ParquetRecordReader parentReader, int allocateSize, ColumnDescriptor descriptor,
-      ColumnChunkMetaData columnChunkMetaData, boolean fixedLength, ValueVector v, ConvertedType convertedType) throws ExecutionSetupException {
-    super(parentReader, allocateSize, descriptor, columnChunkMetaData, fixedLength, v, convertedType);
+      ColumnChunkMetaData columnChunkMetaData, boolean fixedLength, ValueVector v, SchemaElement schemaElement) throws ExecutionSetupException {
+    super(parentReader, allocateSize, descriptor, columnChunkMetaData, fixedLength, v, schemaElement);
   }
 
   // this method is called by its superclass during a read loop
@@ -48,4 +56,71 @@ class NullableFixedByteAlignedReader extends NullableColumnReader {
     // fill in data.
     vectorData.writeBytes(bytes, (int) readStartInBytes, (int) readLength);
   }
+
+  public static abstract class NullableConvertedReader extends NullableFixedByteAlignedReader {
+
+    protected int dataTypeLengthInBytes;
+
+    NullableConvertedReader(ParquetRecordReader parentReader, int allocateSize, ColumnDescriptor descriptor,
+                                   ColumnChunkMetaData columnChunkMetaData, boolean fixedLength, ValueVector v, SchemaElement schemaElement) throws ExecutionSetupException {
+      super(parentReader, allocateSize, descriptor, columnChunkMetaData, fixedLength, v, schemaElement);
+    }
+
+    @Override
+    protected void readField(long recordsToReadInThisPass, ColumnReader firstColumnStatus) {
+
+      this.recordsReadInThisIteration = recordsToReadInThisPass;
+
+      // set up metadata
+      this.readStartInBytes = pageReadStatus.readPosInBytes;
+      this.readLengthInBits = recordsReadInThisIteration * dataTypeLengthInBits;
+      this.readLength = (int) Math.ceil(readLengthInBits / 8.0);
+      this.bytes = pageReadStatus.pageDataByteArray;
+
+      dataTypeLengthInBytes = (int) Math.ceil(dataTypeLengthInBits / 8.0);
+      for (int i = 0; i < recordsReadInThisIteration; i++) {
+        addNext((int) readStartInBytes + i * dataTypeLengthInBytes, i + valuesReadInCurrentPass);
+      }
+    }
+
+    abstract void addNext(int start, int index);
+  }
+
+  public static class NullableDecimal28Reader extends NullableConvertedReader {
+
+    NullableDecimal28SparseVector decimal28Vector;
+
+    NullableDecimal28Reader(ParquetRecordReader parentReader, int allocateSize, ColumnDescriptor descriptor, ColumnChunkMetaData columnChunkMetaData,
+                            boolean fixedLength, ValueVector v, SchemaElement schemaElement) throws ExecutionSetupException {
+      super(parentReader, allocateSize, descriptor, columnChunkMetaData, fixedLength, v, schemaElement);
+      decimal28Vector = (NullableDecimal28SparseVector) v;
+    }
+
+    @Override
+    void addNext(int start, int index) {
+      int width = NullableDecimal28SparseHolder.WIDTH;
+      BigDecimal intermediate = DecimalUtility.getBigDecimalFromByteArray(bytes, start, dataTypeLengthInBytes, schemaElement.getScale());
+      DecimalUtility.getSparseFromBigDecimal(intermediate, decimal28Vector.getData(), index * width, schemaElement.getScale(),
+              schemaElement.getPrecision(), NullableDecimal28SparseHolder.nDecimalDigits);
+    }
+  }
+
+  public static class NullableDecimal38Reader extends NullableConvertedReader {
+
+    NullableDecimal38SparseVector decimal38Vector;
+
+    NullableDecimal38Reader(ParquetRecordReader parentReader, int allocateSize, ColumnDescriptor descriptor, ColumnChunkMetaData columnChunkMetaData,
+                            boolean fixedLength, ValueVector v, SchemaElement schemaElement) throws ExecutionSetupException {
+      super(parentReader, allocateSize, descriptor, columnChunkMetaData, fixedLength, v, schemaElement);
+      decimal38Vector = (NullableDecimal38SparseVector) v;
+    }
+
+    @Override
+    void addNext(int start, int index) {
+      int width = NullableDecimal38SparseHolder.WIDTH;
+      BigDecimal intermediate = DecimalUtility.getBigDecimalFromByteArray(bytes, start, dataTypeLengthInBytes, schemaElement.getScale());
+      DecimalUtility.getSparseFromBigDecimal(intermediate, decimal38Vector.getData(), index * width, schemaElement.getScale(),
+              schemaElement.getPrecision(), NullableDecimal38SparseHolder.nDecimalDigits);
+    }
+  }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/63b03467/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetFormatPlugin.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetFormatPlugin.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetFormatPlugin.java
index a10d30f..34287f3 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetFormatPlugin.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetFormatPlugin.java
@@ -19,26 +19,30 @@ package org.apache.drill.exec.store.parquet;
 
 import java.io.IOException;
 import java.util.List;
+import java.util.Map;
 import java.util.Set;
 import java.util.regex.Pattern;
 
+import com.google.common.collect.Maps;
+import org.apache.drill.common.exceptions.ExecutionSetupException;
 import org.apache.drill.common.expression.SchemaPath;
 import org.apache.drill.common.logical.StoragePluginConfig;
+import org.apache.drill.exec.ExecConstants;
+import org.apache.drill.exec.ops.FragmentContext;
 import org.apache.drill.exec.physical.base.AbstractWriter;
 import org.apache.drill.exec.physical.base.PhysicalOperator;
+import org.apache.drill.exec.physical.impl.WriterRecordBatch;
+import org.apache.drill.exec.proto.ExecProtos.FragmentHandle;
+import org.apache.drill.exec.record.RecordBatch;
 import org.apache.drill.exec.server.DrillbitContext;
+import org.apache.drill.exec.store.RecordWriter;
 import org.apache.drill.exec.store.StoragePluginOptimizerRule;
-import org.apache.drill.exec.store.dfs.BasicFormatMatcher;
-import org.apache.drill.exec.store.dfs.DrillPathFilter;
-import org.apache.drill.exec.store.dfs.FileSelection;
-import org.apache.drill.exec.store.dfs.FormatMatcher;
-import org.apache.drill.exec.store.dfs.FormatPlugin;
-import org.apache.drill.exec.store.dfs.FormatSelection;
-import org.apache.drill.exec.store.dfs.MagicString;
+import org.apache.drill.exec.store.dfs.*;
 import org.apache.drill.exec.store.dfs.shim.DrillFileSystem;
 import org.apache.drill.exec.store.mock.MockStorageEngine;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.PathFilter;
 
@@ -104,7 +108,35 @@ public class ParquetFormatPlugin implements FormatPlugin{
 
   @Override
   public AbstractWriter getWriter(PhysicalOperator child, String location) throws IOException {
-    throw new UnsupportedOperationException("Parquet Writer is not supported currently.");
+    return new ParquetWriter(child, location, this);
+  }
+
+  public RecordWriter getRecordWriter(FragmentContext context, ParquetWriter writer) throws IOException {
+    Map<String, String> options = Maps.newHashMap();
+
+    options.put("location", writer.getLocation());
+
+    FragmentHandle handle = context.getHandle();
+    String fragmentId = String.format("%d_%d", handle.getMajorFragmentId(), handle.getMinorFragmentId());
+    options.put("prefix", fragmentId);
+
+    options.put(FileSystem.FS_DEFAULT_NAME_KEY, ((FileSystemConfig)writer.getStorageConfig()).connection);
+
+    options.put(ExecConstants.PARQUET_BLOCK_SIZE, context.getOptions().getOption(ExecConstants.PARQUET_BLOCK_SIZE).num_val.toString());
+
+    RecordWriter recordWriter = new ParquetRecordWriter();
+    recordWriter.init(options);
+
+    return recordWriter;
+  }
+
+  public RecordBatch getWriterBatch(FragmentContext context, RecordBatch incoming, ParquetWriter writer)
+          throws ExecutionSetupException {
+    try {
+      return new WriterRecordBatch(writer, incoming, context, getRecordWriter(context, writer));
+    } catch(IOException e) {
+      throw new ExecutionSetupException(String.format("Failed to create the WriterRecordBatch. %s", e.getMessage()), e);
+    }
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/63b03467/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetRecordReader.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetRecordReader.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetRecordReader.java
index 5d28456..4ca13a5 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetRecordReader.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetRecordReader.java
@@ -32,7 +32,9 @@ import org.apache.drill.common.expression.ExpressionPosition;
 import org.apache.drill.common.expression.FieldReference;
 import org.apache.drill.common.expression.SchemaPath;
 import org.apache.drill.common.types.TypeProtos;
+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.exception.SchemaChangeException;
 import org.apache.drill.exec.expr.TypeHelper;
@@ -41,6 +43,10 @@ import org.apache.drill.exec.ops.FragmentContext;
 import org.apache.drill.exec.physical.impl.OutputMutator;
 import org.apache.drill.exec.record.MaterializedField;
 import org.apache.drill.exec.store.RecordReader;
+import org.apache.drill.exec.store.parquet.FixedByteAlignedReader.Decimal28Reader;
+import org.apache.drill.exec.store.parquet.FixedByteAlignedReader.Decimal38Reader;
+import org.apache.drill.exec.store.parquet.NullableFixedByteAlignedReader.NullableDecimal28Reader;
+import org.apache.drill.exec.store.parquet.NullableFixedByteAlignedReader.NullableDecimal38Reader;
 import org.apache.drill.exec.vector.*;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -62,6 +68,7 @@ import parquet.schema.PrimitiveType;
 import org.apache.drill.exec.store.parquet.VarLengthColumnReaders.*;
 
 import com.google.common.base.Joiner;
+import parquet.schema.PrimitiveType.PrimitiveTypeName;
 
 public class ParquetRecordReader implements RecordReader {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ParquetRecordReader.class);
@@ -191,18 +198,19 @@ public class ParquetRecordReader implements RecordReader {
 
     // TODO - figure out how to deal with this better once we add nested reading, note also look where this map is used below
     // store a map from column name to converted types if they are non-null
-    HashMap<String, ConvertedType> convertedTypes = new HashMap<>();
+    HashMap<String, SchemaElement> schemaElements = new HashMap<>();
     fileMetaData = new ParquetMetadataConverter().toParquetMetadata(ParquetFileWriter.CURRENT_VERSION, footer);
     for (SchemaElement se : fileMetaData.getSchema()) {
-      convertedTypes.put(se.getName(), se.getConverted_type());
+      schemaElements.put(se.getName(), se);
     }
 
     // loop to add up the length of the fixed width columns and build the schema
     for (int i = 0; i < columns.size(); ++i) {
       column = columns.get(i);
       logger.debug("name: " + fileMetaData.getSchema().get(i).name);
-      field = MaterializedField.create(toFieldName(column.getPath()),
-          toMajorType(column.getType(), getDataMode(column), convertedTypes.get(column.getPath()[0])));
+      SchemaElement se = schemaElements.get(column.getPath()[0]);
+      MajorType mt = toMajorType(column.getType(), se.getType_length(), getDataMode(column), se);
+      field = MaterializedField.create(toFieldName(column.getPath()),mt);
       if ( ! fieldSelected(field)){
         continue;
       }
@@ -211,12 +219,11 @@ public class ParquetRecordReader implements RecordReader {
       if (column.getType() != PrimitiveType.PrimitiveTypeName.BINARY) {
         // There is not support for the fixed binary type yet in parquet, leaving a task here as a reminder
         // TODO - implement this when the feature is added upstream
-//          if (column.getType() == PrimitiveType.PrimitiveTypeName.FIXED_LEN_BYTE_ARRAY){
-//              byteWidthAllFixedFields += column.getType().getWidth()
-//          }
-//          else { } // the code below for the rest of the fixed length fields
-
-        bitWidthAllFixedFields += getTypeLengthInBits(column.getType());
+          if (column.getType() == PrimitiveType.PrimitiveTypeName.FIXED_LEN_BYTE_ARRAY){
+              bitWidthAllFixedFields += se.getType_length() * 8;
+          } else {
+            bitWidthAllFixedFields += getTypeLengthInBits(column.getType());
+          }
       } else {
         allFieldsFixedLength = false;
       }
@@ -234,16 +241,11 @@ public class ParquetRecordReader implements RecordReader {
     else {
       recordsPerBatch = DEFAULT_RECORDS_TO_READ_IF_NOT_FIXED_WIDTH;
     }
-//    for (SchemaElement se : fileMetaData.getSchema()) {
-//      if (fieldSelected())
-//      System.out.println("convertedtype :" + se.getConverted_type());
-//      System.out.println("name:" + se.getName());
-//      System.out.println();
-//
-//    }
+
     try {
       ValueVector v;
       ConvertedType convertedType;
+      SchemaElement schemaElement;
       ArrayList<VarLengthColumn> varLengthColumns = new ArrayList<>();
       ArrayList<NullableVarLengthColumn> nullableVarLengthColumns = new ArrayList<>();
       // initialize all of the column read status objects
@@ -251,40 +253,21 @@ public class ParquetRecordReader implements RecordReader {
       for (int i = 0; i < columns.size(); ++i) {
         column = columns.get(i);
         columnChunkMetaData = footer.getBlocks().get(0).getColumns().get(i);
-        convertedType = convertedTypes.get(column.getPath()[0]);
-        MajorType type = toMajorType(column.getType(), getDataMode(column), convertedType);
-//        Preconditions.checkArgument(!columnChunkMetaData.getEncodings().contains(Encoding.PLAIN_DICTIONARY), "Dictionary Encoding not currently supported");
+        schemaElement = schemaElements.get(column.getPath()[0]);
+        convertedType = schemaElement.getConverted_type();
+        MajorType type = toMajorType(column.getType(), schemaElement.getType_length(), getDataMode(column), schemaElement);
         field = MaterializedField.create(toFieldName(column.getPath()), type);
         // the field was not requested to be read
         if ( ! fieldSelected(field)) continue;
 
-        //convertedTypes.put()
         fieldFixedLength = column.getType() != PrimitiveType.PrimitiveTypeName.BINARY;
         v = output.addField(field, (Class<? extends ValueVector>) TypeHelper.getValueVectorClass(type.getMinorType(), type.getMode()));
         if (column.getType() != PrimitiveType.PrimitiveTypeName.BINARY) {
           createFixedColumnReader(fieldFixedLength, column, columnChunkMetaData, recordsPerBatch, v,
-            convertedType);
+            schemaElement);
         } else {
-          if (column.getMaxDefinitionLevel() == 0){// column is required
-            if (convertedType == ConvertedType.UTF8) {
-              varLengthColumns.add(
-                new VarCharColumn(this, -1, column, columnChunkMetaData, false, (VarCharVector) v, convertedType));
-            } else {
-              varLengthColumns.add(
-                  new VarBinaryColumn(this, -1, column, columnChunkMetaData, false, (VarBinaryVector) v, convertedType));
-            }
-          }
-          else{
-            if (convertedType == ConvertedType.UTF8) {
-              nullableVarLengthColumns.add(
-                new NullableVarCharColumn(this, -1, column, columnChunkMetaData, false,
-                    (NullableVarCharVector) v, convertedType));
-            } else {
-              nullableVarLengthColumns.add(
-                new NullableVarBinaryColumn(this, -1, column, columnChunkMetaData, false,
-                  (NullableVarBinaryVector) v, convertedType));
-            }
-          }
+          // create a reader and add it to the appropriate list
+          getReader(this, -1, column, columnChunkMetaData, false, v, schemaElement, varLengthColumns, nullableVarLengthColumns);
         }
       }
       varLengthReader = new VarLenBinaryReader(this, varLengthColumns, nullableVarLengthColumns);
@@ -338,28 +321,41 @@ public class ParquetRecordReader implements RecordReader {
    */
   private boolean createFixedColumnReader(boolean fixedLength, ColumnDescriptor descriptor,
                                           ColumnChunkMetaData columnChunkMetaData, int allocateSize, ValueVector v,
-                                          ConvertedType convertedType)
+                                          SchemaElement schemaElement)
       throws SchemaChangeException, ExecutionSetupException {
+    ConvertedType convertedType = schemaElement.getConverted_type();
     // if the column is required
     if (descriptor.getMaxDefinitionLevel() == 0){
       if (columnChunkMetaData.getType() == PrimitiveType.PrimitiveTypeName.BOOLEAN){
         columnStatuses.add(new BitReader(this, allocateSize, descriptor, columnChunkMetaData,
-            fixedLength, v, convertedType));
-      }
-      else{
+            fixedLength, v, schemaElement));
+      } else if (columnChunkMetaData.getType() == PrimitiveTypeName.FIXED_LEN_BYTE_ARRAY && convertedType == ConvertedType.DECIMAL){
+        int length = schemaElement.type_length;
+        if (length <= 12) {
+          columnStatuses.add(new Decimal28Reader(this, allocateSize, descriptor, columnChunkMetaData, fixedLength, v, schemaElement));
+        } else if (length <= 16) {
+          columnStatuses.add(new Decimal38Reader(this, allocateSize, descriptor, columnChunkMetaData, fixedLength, v, schemaElement));
+        }
+      } else {
         columnStatuses.add(new FixedByteAlignedReader(this, allocateSize, descriptor, columnChunkMetaData,
-            fixedLength, v, convertedType));
+            fixedLength, v, schemaElement));
       }
       return true;
     }
     else { // if the column is nullable
       if (columnChunkMetaData.getType() == PrimitiveType.PrimitiveTypeName.BOOLEAN){
         columnStatuses.add(new NullableBitReader(this, allocateSize, descriptor, columnChunkMetaData,
-            fixedLength, v, convertedType));
-      }
-      else{
+            fixedLength, v, schemaElement));
+      } else if (columnChunkMetaData.getType() == PrimitiveTypeName.FIXED_LEN_BYTE_ARRAY && convertedType == ConvertedType.DECIMAL){
+        int length = schemaElement.type_length;
+        if (length <= 12) {
+          columnStatuses.add(new NullableDecimal28Reader(this, allocateSize, descriptor, columnChunkMetaData, fixedLength, v, schemaElement));
+        } else if (length <= 16) {
+          columnStatuses.add(new NullableDecimal38Reader(this, allocateSize, descriptor, columnChunkMetaData, fixedLength, v, schemaElement));
+        }
+      } else {
         columnStatuses.add(new NullableFixedByteAlignedReader(this, allocateSize, descriptor, columnChunkMetaData,
-            fixedLength, v, convertedType));
+            fixedLength, v, schemaElement));
       }
       return true;
     }
@@ -420,25 +416,57 @@ public class ParquetRecordReader implements RecordReader {
   }
 
   static TypeProtos.MajorType toMajorType(PrimitiveType.PrimitiveTypeName primitiveTypeName,
-                                               TypeProtos.DataMode mode, ConvertedType convertedType) {
-    return toMajorType(primitiveTypeName, 0, mode, convertedType);
+                                               TypeProtos.DataMode mode, SchemaElement schemaElement) {
+    return toMajorType(primitiveTypeName, 0, mode, schemaElement);
   }
 
   static TypeProtos.MajorType toMajorType(PrimitiveType.PrimitiveTypeName primitiveTypeName, int length,
-                                               TypeProtos.DataMode mode, ConvertedType convertedType) {
+                                               TypeProtos.DataMode mode, SchemaElement schemaElement) {
+    ConvertedType convertedType = schemaElement.getConverted_type();
     switch (mode) {
 
       case OPTIONAL:
         switch (primitiveTypeName) {
           case BINARY:
-            if (convertedType == ConvertedType.UTF8)
-              return Types.optional(TypeProtos.MinorType.VARCHAR);
-            else
+            if (convertedType == null) {
               return Types.optional(TypeProtos.MinorType.VARBINARY);
+            }
+            switch (convertedType) {
+              case UTF8:
+                return Types.optional(MinorType.VARCHAR);
+              case DECIMAL:
+                return Types.withScaleAndPrecision(getDecimalType(schemaElement), DataMode.OPTIONAL, schemaElement.getScale(), schemaElement.getPrecision());
+              default:
+                throw new UnsupportedOperationException(String.format("unsupported type: %s %s", primitiveTypeName, convertedType));
+            }
           case INT64:
-            return Types.optional(TypeProtos.MinorType.BIGINT);
+            if (convertedType == null) {
+              return Types.optional(TypeProtos.MinorType.BIGINT);
+            }
+            switch(convertedType) {
+              case DECIMAL:
+                return Types.withScaleAndPrecision(MinorType.DECIMAL18, DataMode.OPTIONAL, schemaElement.getScale(), schemaElement.getPrecision());
+              case FINETIME:
+                throw new UnsupportedOperationException();
+              case TIMESTAMP:
+                return Types.optional(MinorType.TIMESTAMP);
+              default:
+                throw new UnsupportedOperationException(String.format("unsupported type: %s %s", primitiveTypeName, convertedType));
+            }
           case INT32:
-            return Types.optional(TypeProtos.MinorType.INT);
+            if (convertedType == null) {
+              return Types.optional(TypeProtos.MinorType.INT);
+            }
+            switch(convertedType) {
+              case DECIMAL:
+                return Types.withScaleAndPrecision(MinorType.DECIMAL9, DataMode.OPTIONAL, schemaElement.getScale(), schemaElement.getPrecision());
+              case DATE:
+                return Types.optional(MinorType.DATE);
+              case TIME:
+                return Types.optional(MinorType.TIME);
+              default:
+                throw new UnsupportedOperationException(String.format("unsupported type: %s %s", primitiveTypeName, convertedType));
+            }
           case BOOLEAN:
             return Types.optional(TypeProtos.MinorType.BIT);
           case FLOAT:
@@ -451,23 +479,58 @@ public class ParquetRecordReader implements RecordReader {
             return TypeProtos.MajorType.newBuilder().setMinorType(TypeProtos.MinorType.FIXEDBINARY).setWidth(12)
                 .setMode(mode).build();
           case FIXED_LEN_BYTE_ARRAY:
-            checkArgument(length > 0, "A length greater than zero must be provided for a FixedBinary type.");
-            return TypeProtos.MajorType.newBuilder().setMinorType(TypeProtos.MinorType.FIXEDBINARY)
-                .setWidth(length).setMode(mode).build();
+            if (convertedType == null) {
+              checkArgument(length > 0, "A length greater than zero must be provided for a FixedBinary type.");
+              return TypeProtos.MajorType.newBuilder().setMinorType(TypeProtos.MinorType.FIXEDBINARY)
+                      .setWidth(length).setMode(mode).build();
+            } else if (convertedType == ConvertedType.DECIMAL) {
+              return Types.withScaleAndPrecision(getDecimalType(schemaElement), DataMode.OPTIONAL, schemaElement.getScale(), schemaElement.getPrecision());
+            }
           default:
             throw new UnsupportedOperationException("Type not supported: " + primitiveTypeName);
         }
       case REQUIRED:
         switch (primitiveTypeName) {
           case BINARY:
-            if (convertedType == ConvertedType.UTF8)
-              return Types.required(TypeProtos.MinorType.VARCHAR);
-            else
+            if (convertedType == null) {
               return Types.required(TypeProtos.MinorType.VARBINARY);
+            }
+            switch (convertedType) {
+              case UTF8:
+                return Types.required(MinorType.VARCHAR);
+              case DECIMAL:
+                return Types.withScaleAndPrecision(getDecimalType(schemaElement), DataMode.REQUIRED, schemaElement.getScale(), schemaElement.getPrecision());
+              default:
+                throw new UnsupportedOperationException(String.format("unsupported type: %s %s", primitiveTypeName, convertedType));
+            }
           case INT64:
-            return Types.required(TypeProtos.MinorType.BIGINT);
+            if (convertedType == null) {
+              return Types.required(MinorType.BIGINT);
+            }
+            switch(convertedType) {
+              case DECIMAL:
+                return Types.withScaleAndPrecision(MinorType.DECIMAL18, DataMode.REQUIRED, schemaElement.getScale(), schemaElement.getPrecision());
+              case FINETIME:
+                throw new UnsupportedOperationException();
+              case TIMESTAMP:
+                return Types.required(MinorType.TIMESTAMP);
+              default:
+                throw new UnsupportedOperationException(String.format("unsupported type: %s %s", primitiveTypeName, convertedType));
+            }
           case INT32:
-            return Types.required(TypeProtos.MinorType.INT);
+            if (convertedType == null) {
+              return Types.required(MinorType.INT);
+            }
+            switch(convertedType) {
+              case DECIMAL:
+                return Types.withScaleAndPrecision(MinorType.DECIMAL9, DataMode.REQUIRED, schemaElement.getScale(), schemaElement.getPrecision());
+              case DATE:
+                return Types.required(MinorType.DATE);
+              case TIME:
+                return Types.required(MinorType.TIME);
+              default:
+                throw new UnsupportedOperationException(String.format("unsupported type: %s %s", primitiveTypeName, convertedType));
+            }
           case BOOLEAN:
             return Types.required(TypeProtos.MinorType.BIT);
           case FLOAT:
@@ -480,23 +543,58 @@ public class ParquetRecordReader implements RecordReader {
             return TypeProtos.MajorType.newBuilder().setMinorType(TypeProtos.MinorType.FIXEDBINARY).setWidth(12)
                 .setMode(mode).build();
           case FIXED_LEN_BYTE_ARRAY:
-            checkArgument(length > 0, "A length greater than zero must be provided for a FixedBinary type.");
-            return TypeProtos.MajorType.newBuilder().setMinorType(TypeProtos.MinorType.FIXEDBINARY)
-                .setWidth(length).setMode(mode).build();
+            if (convertedType == null) {
+              checkArgument(length > 0, "A length greater than zero must be provided for a FixedBinary type.");
+              return TypeProtos.MajorType.newBuilder().setMinorType(TypeProtos.MinorType.FIXEDBINARY)
+                      .setWidth(length).setMode(mode).build();
+            } else if (convertedType == ConvertedType.DECIMAL) {
+              return Types.withScaleAndPrecision(getDecimalType(schemaElement), DataMode.REQUIRED, schemaElement.getScale(), schemaElement.getPrecision());
+            }
           default:
             throw new UnsupportedOperationException("Type not supported: " + primitiveTypeName);
         }
       case REPEATED:
         switch (primitiveTypeName) {
           case BINARY:
-            if (convertedType == ConvertedType.UTF8)
-              return Types.required(TypeProtos.MinorType.VARCHAR);
-            else
+            if (convertedType == null) {
               return Types.repeated(TypeProtos.MinorType.VARBINARY);
+            }
+            switch (schemaElement.getConverted_type()) {
+              case UTF8:
+                return Types.repeated(MinorType.VARCHAR);
+              case DECIMAL:
+                return Types.withScaleAndPrecision(getDecimalType(schemaElement), DataMode.REPEATED, schemaElement.getScale(), schemaElement.getPrecision());
+              default:
+                throw new UnsupportedOperationException(String.format("unsupported type: %s %s", primitiveTypeName, convertedType));
+            }
           case INT64:
-            return Types.repeated(TypeProtos.MinorType.BIGINT);
+            if (convertedType == null) {
+              return Types.repeated(MinorType.BIGINT);
+            }
+            switch(convertedType) {
+              case DECIMAL:
+                return Types.withScaleAndPrecision(MinorType.DECIMAL18, DataMode.REPEATED, schemaElement.getScale(), schemaElement.getPrecision());
+              case FINETIME:
+                throw new UnsupportedOperationException();
+              case TIMESTAMP:
+                return Types.repeated(MinorType.TIMESTAMP);
+              default:
+                throw new UnsupportedOperationException(String.format("unsupported type: %s %s", primitiveTypeName, convertedType));
+            }
           case INT32:
-            return Types.repeated(TypeProtos.MinorType.INT);
+            if (convertedType == null) {
+              return Types.repeated(MinorType.INT);
+            }
+            switch(convertedType) {
+              case DECIMAL:
+                return Types.withScaleAndPrecision(MinorType.DECIMAL9, DataMode.REPEATED, schemaElement.getScale(), schemaElement.getPrecision());
+              case DATE:
+                return Types.repeated(MinorType.DATE);
+              case TIME:
+                return Types.repeated(MinorType.TIME);
+              default:
+                throw new UnsupportedOperationException(String.format("unsupported type: %s %s", primitiveTypeName, convertedType));
+            }
           case BOOLEAN:
             return Types.repeated(TypeProtos.MinorType.BIT);
           case FLOAT:
@@ -509,9 +607,13 @@ public class ParquetRecordReader implements RecordReader {
             return TypeProtos.MajorType.newBuilder().setMinorType(TypeProtos.MinorType.FIXEDBINARY).setWidth(12)
                 .setMode(mode).build();
           case FIXED_LEN_BYTE_ARRAY:
-            checkArgument(length > 0, "A length greater than zero must be provided for a FixedBinary type.");
-            return TypeProtos.MajorType.newBuilder().setMinorType(TypeProtos.MinorType.FIXEDBINARY)
-                .setWidth(length).setMode(mode).build();
+            if (convertedType == null) {
+              checkArgument(length > 0, "A length greater than zero must be provided for a FixedBinary type.");
+              return TypeProtos.MajorType.newBuilder().setMinorType(TypeProtos.MinorType.FIXEDBINARY)
+                      .setWidth(length).setMode(mode).build();
+            } else if (convertedType == ConvertedType.DECIMAL) {
+              return Types.withScaleAndPrecision(getDecimalType(schemaElement), DataMode.REPEATED, schemaElement.getScale(), schemaElement.getPrecision());
+            }
           default:
             throw new UnsupportedOperationException("Type not supported: " + primitiveTypeName);
         }
@@ -519,6 +621,58 @@ public class ParquetRecordReader implements RecordReader {
     throw new UnsupportedOperationException("Type not supported: " + primitiveTypeName + " Mode: " + mode);
   }
 
+  private static void getReader(ParquetRecordReader parentReader, int allocateSize, ColumnDescriptor descriptor,
+                                        ColumnChunkMetaData columnChunkMetaData, boolean fixedLength, ValueVector v,
+                                        SchemaElement schemaElement, List<VarLengthColumn> varLengthColumns,
+                                        List<NullableVarLengthColumn> nullableVarLengthColumns) throws ExecutionSetupException {
+    ConvertedType convertedType = schemaElement.getConverted_type();
+    switch (descriptor.getMaxDefinitionLevel()) {
+      case 0:
+        if (convertedType == null) {
+          varLengthColumns.add(new VarBinaryColumn(parentReader, allocateSize, descriptor, columnChunkMetaData, fixedLength, (VarBinaryVector) v, schemaElement));
+          return;
+        }
+        switch (convertedType) {
+          case UTF8:
+            varLengthColumns.add(new VarCharColumn(parentReader, allocateSize, descriptor, columnChunkMetaData, fixedLength, (VarCharVector) v, schemaElement));
+            return;
+          case DECIMAL:
+            if (v instanceof Decimal28SparseVector) {
+              varLengthColumns.add(new Decimal28Column(parentReader, allocateSize, descriptor, columnChunkMetaData, fixedLength, (Decimal28SparseVector) v, schemaElement));
+              return;
+            } else if (v instanceof Decimal38SparseVector) {
+              varLengthColumns.add(new Decimal38Column(parentReader, allocateSize, descriptor, columnChunkMetaData, fixedLength, (Decimal38SparseVector) v, schemaElement));
+              return;
+            }
+          default:
+        }
+      default:
+        if (convertedType == null) {
+          nullableVarLengthColumns.add(new NullableVarBinaryColumn(parentReader, allocateSize, descriptor, columnChunkMetaData, fixedLength, (NullableVarBinaryVector) v, schemaElement));
+          return;
+        }
+        switch (convertedType) {
+          case UTF8:
+            nullableVarLengthColumns.add(new NullableVarCharColumn(parentReader, allocateSize, descriptor, columnChunkMetaData, fixedLength, (NullableVarCharVector) v, schemaElement));
+            return;
+          case DECIMAL:
+            if (v instanceof NullableDecimal28SparseVector) {
+              nullableVarLengthColumns.add(new NullableDecimal28Column(parentReader, allocateSize, descriptor, columnChunkMetaData, fixedLength, (NullableDecimal28SparseVector) v, schemaElement));
+              return;
+            } else if (v instanceof NullableDecimal38SparseVector) {
+              nullableVarLengthColumns.add(new NullableDecimal38Column(parentReader, allocateSize, descriptor, columnChunkMetaData, fixedLength, (NullableDecimal38SparseVector) v, schemaElement));
+              return;
+            }
+          default:
+        }
+    }
+    throw new UnsupportedOperationException();
+  }
+
+  private static MinorType getDecimalType(SchemaElement schemaElement) {
+    return schemaElement.getPrecision() <= 28 ? MinorType.DECIMAL28SPARSE : MinorType.DECIMAL38SPARSE;
+  }
+
   static String join(String delimiter, String... str) {
     StringBuilder builder = new StringBuilder();
     int i = 0;

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/63b03467/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetRecordWriter.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetRecordWriter.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetRecordWriter.java
new file mode 100644
index 0000000..3ba13f4
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetRecordWriter.java
@@ -0,0 +1,185 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.exec.store.parquet;
+
+import com.google.common.collect.Lists;
+import org.apache.drill.common.types.TypeProtos.MinorType;
+import org.apache.drill.exec.ExecConstants;
+import org.apache.drill.exec.record.BatchSchema;
+import org.apache.drill.exec.record.MaterializedField;
+import org.apache.drill.exec.store.ParquetOutputRecordWriter;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import parquet.column.ParquetProperties.WriterVersion;
+import parquet.column.impl.ColumnWriteStoreImpl;
+import parquet.column.page.PageWriteStore;
+import parquet.hadoop.ColumnChunkPageWriteStoreExposer;
+import parquet.hadoop.ParquetFileWriter;
+import parquet.hadoop.metadata.CompressionCodecName;
+import parquet.io.ColumnIOFactory;
+import parquet.io.MessageColumnIO;
+import parquet.io.api.RecordConsumer;
+import parquet.schema.DecimalMetadata;
+import parquet.schema.MessageType;
+import parquet.schema.OriginalType;
+import parquet.schema.PrimitiveType.PrimitiveTypeName;
+import parquet.schema.Type;
+import parquet.schema.Type.Repetition;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import static java.lang.Math.max;
+import static java.lang.Math.min;
+import static java.lang.String.format;
+
+public class ParquetRecordWriter extends ParquetOutputRecordWriter {
+
+  private static final int MINIMUM_BUFFER_SIZE = 64 * 1024;
+  private static final int MINIMUM_RECORD_COUNT_FOR_CHECK = 100;
+  private static final int MAXIMUM_RECORD_COUNT_FOR_CHECK = 10000;
+
+  private ParquetFileWriter w;
+  private MessageType schema;
+  private Map<String, String> extraMetaData = new HashMap();
+  private int blockSize;
+  private int pageSize = 1 * 1024 * 1024;
+  private int dictionaryPageSize = pageSize;
+  private boolean enableDictionary = false;
+  private boolean validating = false;
+  private CompressionCodecName codec = CompressionCodecName.SNAPPY;
+  private WriterVersion writerVersion = WriterVersion.PARQUET_1_0;
+
+  private long recordCount = 0;
+  private long recordCountForNextMemCheck = MINIMUM_RECORD_COUNT_FOR_CHECK;
+
+  private ColumnWriteStoreImpl store;
+  private PageWriteStore pageStore;
+
+  private RecordConsumer consumer;
+  private BatchSchema batchSchema;
+
+  private Configuration conf;
+  private String location;
+  private String prefix;
+  private int index = 0;
+
+  @Override
+  public void init(Map<String, String> writerOptions) throws IOException {
+    this.location = writerOptions.get("location");
+    this.prefix = writerOptions.get("prefix");
+
+    conf = new Configuration();
+    conf.set(FileSystem.FS_DEFAULT_NAME_KEY, writerOptions.get(FileSystem.FS_DEFAULT_NAME_KEY));
+    blockSize = Integer.parseInt(writerOptions.get(ExecConstants.PARQUET_BLOCK_SIZE));
+  }
+
+  @Override
+  public void updateSchema(BatchSchema batchSchema) throws IOException {
+    if (this.batchSchema == null || !this.batchSchema.equals(batchSchema)) {
+      if (this.batchSchema != null) {
+        flush();
+      }
+      this.batchSchema = batchSchema;
+      newSchema();
+    }
+  }
+
+  private void newSchema() throws IOException {
+    List<Type> types = Lists.newArrayList();
+    for (MaterializedField field : batchSchema) {
+      String name = field.getAsSchemaPath().getAsUnescapedPath();
+      MinorType minorType = field.getType().getMinorType();
+      PrimitiveTypeName primitiveTypeName = ParquetTypeHelper.getPrimitiveTypeNameForMinorType(minorType);
+      Repetition repetition = ParquetTypeHelper.getRepetitionForDataMode(field.getDataMode());
+      OriginalType originalType = ParquetTypeHelper.getOriginalTypeForMinorType(minorType);
+      DecimalMetadata decimalMetadata = ParquetTypeHelper.getDecimalMetadataForField(field);
+      int length = ParquetTypeHelper.getLengthForMinorType(minorType);
+      parquet.schema.Type type = new parquet.schema.PrimitiveType(repetition, primitiveTypeName, length, name, originalType, decimalMetadata);
+      types.add(type);
+    }
+    schema = new MessageType("root", types);
+
+    Path fileName = new Path(location, prefix + "_" + index + ".parquet");
+    w = new ParquetFileWriter(conf, schema, fileName);
+    w.start();
+
+    int initialBlockBufferSize = max(MINIMUM_BUFFER_SIZE, blockSize / this.schema.getColumns().size() / 5);
+    pageStore = ColumnChunkPageWriteStoreExposer.newColumnChunkPageWriteStore(codec, pageSize, this.schema, initialBlockBufferSize);
+    int initialPageBufferSize = max(MINIMUM_BUFFER_SIZE, min(pageSize + pageSize / 10, initialBlockBufferSize));
+    store = new ColumnWriteStoreImpl(pageStore, pageSize, initialPageBufferSize, dictionaryPageSize, enableDictionary, writerVersion);
+    MessageColumnIO columnIO = new ColumnIOFactory(validating).getColumnIO(this.schema);
+    consumer = columnIO.getRecordWriter(store);
+    setUp(schema, consumer);
+  }
+
+  private void flush() throws IOException {
+    w.startBlock(recordCount);
+    store.flush();
+    ColumnChunkPageWriteStoreExposer.flushPageStore(pageStore, w);
+    recordCount = 0;
+    w.endBlock();
+    w.end(extraMetaData);
+    store = null;
+    pageStore = null;
+    index++;
+  }
+
+  private void checkBlockSizeReached() throws IOException {
+    if (recordCount >= recordCountForNextMemCheck) { // checking the memory size is relatively expensive, so let's not do it for every record.
+      long memSize = store.memSize();
+      if (memSize > blockSize) {
+        System.out.println("Reached block size " + blockSize);
+        flush();
+        newSchema();
+        recordCountForNextMemCheck = min(max(MINIMUM_RECORD_COUNT_FOR_CHECK, recordCount / 2), MAXIMUM_RECORD_COUNT_FOR_CHECK);
+      } else {
+        float recordSize = (float) memSize / recordCount;
+        recordCountForNextMemCheck = min(
+                max(MINIMUM_RECORD_COUNT_FOR_CHECK, (recordCount + (long)(blockSize / recordSize)) / 2), // will check halfway
+                recordCount + MAXIMUM_RECORD_COUNT_FOR_CHECK // will not look more than max records ahead
+        );
+      }
+    }
+  }
+
+  @Override
+  public void startRecord() throws IOException {
+    consumer.startMessage();
+  }
+
+  @Override
+  public void endRecord() throws IOException {
+    consumer.endMessage();
+    recordCount++;
+    checkBlockSizeReached();
+  }
+
+  @Override
+  public void abort() throws IOException {
+    //To change body of implemented methods use File | Settings | File Templates.
+  }
+
+  @Override
+  public void cleanup() throws IOException {
+    flush();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/63b03467/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetWriter.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetWriter.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetWriter.java
new file mode 100644
index 0000000..ea9798a
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetWriter.java
@@ -0,0 +1,91 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.exec.store.parquet;
+
+import com.fasterxml.jackson.annotation.*;
+import com.google.common.base.Preconditions;
+import org.apache.drill.common.exceptions.ExecutionSetupException;
+import org.apache.drill.common.logical.FormatPluginConfig;
+import org.apache.drill.common.logical.StoragePluginConfig;
+import org.apache.drill.exec.physical.OperatorCost;
+import org.apache.drill.exec.physical.base.AbstractWriter;
+import org.apache.drill.exec.physical.base.PhysicalOperator;
+import org.apache.drill.exec.store.StoragePluginRegistry;
+
+import java.io.IOException;
+
+@JsonTypeName("parquet-writer")
+public class ParquetWriter extends AbstractWriter {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ParquetWriter.class);
+
+  private final String location;
+  private final ParquetFormatPlugin formatPlugin;
+
+  @JsonCreator
+  public ParquetWriter(
+          @JsonProperty("child") PhysicalOperator child,
+          @JsonProperty("location") String location,
+          @JsonProperty("storage") StoragePluginConfig storageConfig,
+          @JacksonInject StoragePluginRegistry engineRegistry) throws IOException, ExecutionSetupException {
+
+    super(child);
+    this.formatPlugin = (ParquetFormatPlugin) engineRegistry.getFormatPlugin(storageConfig, new ParquetFormatConfig());
+    Preconditions.checkNotNull(formatPlugin, "Unable to load format plugin for provided format config.");
+    this.location = location;
+  }
+
+  public ParquetWriter(PhysicalOperator child,
+                       String location,
+                       ParquetFormatPlugin formatPlugin) {
+
+    super(child);
+    this.formatPlugin = formatPlugin;
+    this.location = location;
+  }
+
+  @JsonProperty("location")
+  public String getLocation() {
+    return location;
+  }
+
+  @JsonProperty("storage")
+  public StoragePluginConfig getStorageConfig(){
+    return formatPlugin.getStorageConfig();
+  }
+
+  @JsonProperty("format")
+  public FormatPluginConfig getFormatConfig(){
+    return formatPlugin.getConfig();
+  }
+
+  @JsonIgnore
+  public ParquetFormatPlugin getFormatPlugin(){
+    return formatPlugin;
+  }
+
+  @Override
+  protected PhysicalOperator getNewWithChild(PhysicalOperator child) {
+    return new ParquetWriter(child, location, formatPlugin);
+  }
+
+  @Override
+  public OperatorCost getCost() {
+    // TODO:
+    return new OperatorCost(1,1,1,1);
+  }
+}


Re: [3/3] git commit: DRILL-694: Implement parquet writer.

Posted by Jacques Nadeau <ja...@apache.org>.
Aditya has seen intermittent failures as well.  I'm in process of trying to
merge a patch to disable until we get the intermittent failures.


On Mon, May 12, 2014 at 11:54 PM, Timothy Chen <tn...@gmail.com> wrote:

> It failed twice in two runs, not sure how intermittently are you
> seeing the failure.
>
> Tim
>
>
>
> On Mon, May 12, 2014 at 11:38 PM, Steven Phillips
> <sp...@maprtech.com> wrote:
> > Do you see the failure repeatedly, even if you run just that single test?
> >
> > I sometimes see tests that fail intermittently, but pass when run in
> > isolation.
> >
> > I will try the test again and confirm that it is passing.
> >
> >
> > On Mon, May 12, 2014 at 10:22 PM, Timothy Chen <tn...@gmail.com>
> wrote:
> >
> >> I got a unit test failure on master from the testParquetWriter?
> >>
> >> Is the test passing for you?
> >>
> >> Tim
> >>
> >> On Mon, May 12, 2014 at 8:46 PM,  <ja...@apache.org> wrote:
> >> > DRILL-694: Implement parquet writer.
> >> >
> >> > Enable "CREATE TABLE AS" with parquet as the output format.
> >> >
> >> > Add decimal metadata support to parquet reader and writer.
> >> >
> >> >
> >> > Project: http://git-wip-us.apache.org/repos/asf/incubator-drill/repo
> >> > Commit:
> >> http://git-wip-us.apache.org/repos/asf/incubator-drill/commit/63b03467
> >> > Tree:
> >> http://git-wip-us.apache.org/repos/asf/incubator-drill/tree/63b03467
> >> > Diff:
> >> http://git-wip-us.apache.org/repos/asf/incubator-drill/diff/63b03467
> >> >
> >> > Branch: refs/heads/master
> >> > Commit: 63b0346761692f0a4c851e7910791c028dc3cded
> >> > Parents: 14c628c
> >> > Author: Steven Phillips <sp...@maprtech.com>
> >> > Authored: Mon May 12 02:56:25 2014 -0700
> >> > Committer: Aditya Kishore <ad...@maprtech.com>
> >> > Committed: Mon May 12 18:18:43 2014 -0700
> >> >
> >> > ----------------------------------------------------------------------
> >> >  .../org/apache/drill/common/types/Types.java    |   4 +
> >> >  .../drill/common/util/DecimalUtility.java       |  11 +-
> >> >  exec/java-exec/pom.xml                          |   4 +-
> >> >  .../templates/EventBasedRecordWriter.java       |   6 +-
> >> >  .../templates/ParquetOutputRecordWriter.java    | 189 +++++++++++
> >> >  .../codegen/templates/ParquetTypeHelper.java    | 155 ++++++++++
> >> >  .../org/apache/drill/exec/ExecConstants.java    |   8 +
> >> >  .../server/options/SystemOptionManager.java     |   5 +-
> >> >  .../exec/store/dfs/WorkspaceSchemaFactory.java  |   9 +-
> >> >  .../drill/exec/store/parquet/BitReader.java     |   5 +-
> >> >  .../drill/exec/store/parquet/ColumnReader.java  |  14 +-
> >> >  .../store/parquet/FixedByteAlignedReader.java   |  87 +++++-
> >> >  .../exec/store/parquet/NullableBitReader.java   |   5 +-
> >> >  .../store/parquet/NullableColumnReader.java     |   5 +-
> >> >  .../parquet/NullableFixedByteAlignedReader.java |  81 ++++-
> >> >  .../exec/store/parquet/ParquetFormatPlugin.java |  48 ++-
> >> >  .../exec/store/parquet/ParquetRecordReader.java | 310
> >> ++++++++++++++-----
> >> >  .../exec/store/parquet/ParquetRecordWriter.java | 185 +++++++++++
> >> >  .../drill/exec/store/parquet/ParquetWriter.java |  91 ++++++
> >> >  .../parquet/ParquetWriterBatchCreator.java      |  36 +++
> >> >  .../store/parquet/VarLengthColumnReaders.java   | 182 ++++++++---
> >> >  .../ColumnChunkPageWriteStoreExposer.java       |  39 +++
> >> >  .../physical/impl/writer/TestParquetWriter.java | 163 ++++++++++
> >> >  .../exec/physical/impl/writer/TestWriter.java   |  19 +-
> >> >  24 files changed, 1510 insertions(+), 151 deletions(-)
> >> > ----------------------------------------------------------------------
> >> >
> >> >
> >> >
> >>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/63b03467/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 c6ac82e..a6ec29b 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
> >> > @@ -271,6 +271,10 @@ public class Types {
> >> >      return
> >> MajorType.newBuilder().setMode(mode).setMinorType(type).build();
> >> >    }
> >> >
> >> > +  public static MajorType withScaleAndPrecision(MinorType type,
> >> DataMode mode, int scale, int precision) {
> >> > +    return
> >>
> MajorType.newBuilder().setMinorType(type).setMode(mode).setScale(scale).setPrecision(precision).build();
> >> > +  }
> >> > +
> >> >    public static MajorType required(MinorType type){
> >> >      return
> >>
> MajorType.newBuilder().setMode(DataMode.REQUIRED).setMinorType(type).build();
> >> >    }
> >> >
> >> >
> >>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/63b03467/common/src/main/java/org/apache/drill/common/util/DecimalUtility.java
> >> > ----------------------------------------------------------------------
> >> > diff --git
> >> a/common/src/main/java/org/apache/drill/common/util/DecimalUtility.java
> >> b/common/src/main/java/org/apache/drill/common/util/DecimalUtility.java
> >> > index dbfd6ac..0cacc59 100644
> >> > ---
> >> a/common/src/main/java/org/apache/drill/common/util/DecimalUtility.java
> >> > +++
> >> b/common/src/main/java/org/apache/drill/common/util/DecimalUtility.java
> >> > @@ -17,8 +17,7 @@
> >> >   */
> >> >  package org.apache.drill.common.util;
> >> >
> >> > -import java.math.BigDecimal;
> >> > -import java.math.BigInteger;
> >> > +import java.math.*;
> >> >
> >> >
> >> >  import io.netty.buffer.Unpooled;
> >> > @@ -26,6 +25,7 @@ import io.netty.buffer.ByteBuf;
> >> >
> >> >  import java.math.BigDecimal;
> >> >  import java.math.BigInteger;
> >> > +import java.util.Arrays;
> >> >
> >> >  public class DecimalUtility {
> >> >
> >> > @@ -486,5 +486,12 @@ public class DecimalUtility {
> >> >        return 0;
> >> >      }
> >> >
> >> > +
> >> > +    public static BigDecimal getBigDecimalFromByteArray(byte[] bytes,
> >> int start, int length, int scale) {
> >> > +      byte[] value = Arrays.copyOfRange(bytes, start, start +
> length);
> >> > +      BigInteger unscaledValue = new BigInteger(value);
> >> > +      return new BigDecimal(unscaledValue, scale);
> >> > +    }
> >> > +
> >> >  }
> >> >
> >> >
> >> >
> >>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/63b03467/exec/java-exec/pom.xml
> >> > ----------------------------------------------------------------------
> >> > diff --git a/exec/java-exec/pom.xml b/exec/java-exec/pom.xml
> >> > index 3e26662..e282c00 100644
> >> > --- a/exec/java-exec/pom.xml
> >> > +++ b/exec/java-exec/pom.xml
> >> > @@ -95,7 +95,7 @@
> >> >      <dependency>
> >> >        <groupId>com.twitter</groupId>
> >> >        <artifactId>parquet-column</artifactId>
> >> > -      <version>1.4.0</version>
> >> > +      <version>1.5.0-SNAPSHOT</version>
> >> >        <exclusions>
> >> >          <exclusion>
> >> >            <groupId>org.apache.hadoop</groupId>
> >> > @@ -110,7 +110,7 @@
> >> >      <dependency>
> >> >        <groupId>com.twitter</groupId>
> >> >        <artifactId>parquet-hadoop</artifactId>
> >> > -      <version>1.4.0</version>
> >> > +      <version>1.5.0-SNAPSHOT</version>
> >> >        <exclusions>
> >> >          <exclusion>
> >> >            <groupId>org.apache.hadoop</groupId>
> >> >
> >> >
> >>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/63b03467/exec/java-exec/src/main/codegen/templates/EventBasedRecordWriter.java
> >> > ----------------------------------------------------------------------
> >> > diff --git
> >> a/exec/java-exec/src/main/codegen/templates/EventBasedRecordWriter.java
> >> b/exec/java-exec/src/main/codegen/templates/EventBasedRecordWriter.java
> >> > index 73b6566..b58f24c 100644
> >> > ---
> >> a/exec/java-exec/src/main/codegen/templates/EventBasedRecordWriter.java
> >> > +++
> >> b/exec/java-exec/src/main/codegen/templates/EventBasedRecordWriter.java
> >> > @@ -91,8 +91,10 @@ public class EventBasedRecordWriter {
> >> >      fieldWriters = Lists.newArrayList();
> >> >      try {
> >> >        for (int i = 0; i < schema.getFieldCount(); i++) {
> >> > -        fieldWriters.add(i,
> >> typeClassMap.get(schema.getColumn(i).getType())
> >> > -            .getConstructor(EventBasedRecordWriter.class,
> >> int.class).newInstance(this, i));
> >> > +        MajorType mt = schema.getColumn(i).getType();
> >> > +        MajorType newMt =
> >>
> MajorType.newBuilder().setMinorType(mt.getMinorType()).setMode(mt.getMode()).build();
> >> > +        fieldWriters.add(i, typeClassMap.get(newMt)
> >> > +                .getConstructor(EventBasedRecordWriter.class,
> >> int.class).newInstance(this, i));
> >> >        }
> >> >      } catch(Exception e) {
> >> >        logger.error("Failed to create FieldWriter.", e);
> >> >
> >> >
> >>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/63b03467/exec/java-exec/src/main/codegen/templates/ParquetOutputRecordWriter.java
> >> > ----------------------------------------------------------------------
> >> > diff --git
> >>
> a/exec/java-exec/src/main/codegen/templates/ParquetOutputRecordWriter.java
> >>
> b/exec/java-exec/src/main/codegen/templates/ParquetOutputRecordWriter.java
> >> > new file mode 100644
> >> > index 0000000..5f75c1c
> >> > --- /dev/null
> >> > +++
> >>
> b/exec/java-exec/src/main/codegen/templates/ParquetOutputRecordWriter.java
> >> > @@ -0,0 +1,189 @@
> >> > +/**
> >> > + * Licensed to the Apache Software Foundation (ASF) under one
> >> > + * or more contributor license agreements.  See the NOTICE file
> >> > + * distributed with this work for additional information
> >> > + * regarding copyright ownership.  The ASF licenses this file
> >> > + * to you under the Apache License, Version 2.0 (the
> >> > + * "License"); you may not use this file except in compliance
> >> > + * with the License.  You may obtain a copy of the License at
> >> > + *
> >> > + * http://www.apache.org/licenses/LICENSE-2.0
> >> > + *
> >> > + * Unless required by applicable law or agreed to in writing,
> software
> >> > + * distributed under the License is distributed on an "AS IS" BASIS,
> >> > + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
> >> implied.
> >> > + * See the License for the specific language governing permissions
> and
> >> > + * limitations under the License.
> >> > + */
> >> > +
> >> > +import parquet.io.api.Binary;
> >> > +
> >> > +import java.lang.Override;
> >> > +
> >> > +<@pp.dropOutputFile />
> >> > +<@pp.changeOutputFile
> >> name="org/apache/drill/exec/store/ParquetOutputRecordWriter.java" />
> >> > +<#include "/@includes/license.ftl" />
> >> > +
> >> > +package org.apache.drill.exec.store;
> >> > +
> >> > +import com.google.common.collect.Lists;
> >> > +import org.apache.drill.common.types.TypeProtos.MinorType;
> >> > +import org.apache.drill.exec.expr.TypeHelper;
> >> > +import org.apache.drill.exec.expr.holders.*;
> >> > +import org.apache.drill.exec.record.BatchSchema;
> >> > +import org.apache.drill.exec.store.parquet.ParquetTypeHelper;
> >> > +import org.apache.drill.exec.vector.*;
> >> > +import org.apache.drill.common.util.DecimalUtility;
> >> > +import parquet.io.api.RecordConsumer;
> >> > +import parquet.schema.MessageType;
> >> > +import parquet.io.api.Binary;
> >> > +import io.netty.buffer.ByteBuf;
> >> > +
> >> > +import java.io.IOException;
> >> > +import java.lang.UnsupportedOperationException;
> >> > +import java.util.Arrays;
> >> > +import java.util.List;
> >> > +import java.util.Map;
> >> > +
> >> > +/**
> >> > + * Abstract implementation of RecordWriter interface which exposes
> >> interface:
> >> > + *    {@link #writeHeader(List)}
> >> > + *    {@link #addField(int,String)}
> >> > + * to output the data in string format instead of implementing
> addField
> >> for each type holder.
> >> > + *
> >> > + * This is useful for text format writers such as CSV, TSV etc.
> >> > + */
> >> > +public abstract class ParquetOutputRecordWriter implements
> RecordWriter
> >> {
> >> > +
> >> > +  private RecordConsumer consumer;
> >> > +  private MessageType schema;
> >> > +
> >> > +  public void setUp(MessageType schema, RecordConsumer consumer) {
> >> > +    this.schema = schema;
> >> > +    this.consumer = consumer;
> >> > +  }
> >> > +
> >> > +<#list vv.types as type>
> >> > +  <#list type.minor as minor>
> >> > +    <#list vv.modes as mode>
> >> > +  @Override
> >> > +  public void add${mode.prefix}${minor.class}Holder(int fieldId,
> >> ${mode.prefix}${minor.class}Holder valueHolder) throws IOException {
> >> > +  <#if mode.prefix == "Nullable" >
> >> > +    if (valueHolder.isSet == 0) {
> >> > +      return;
> >> > +    }
> >> > +  <#elseif mode.prefix == "Repeated" >
> >> > +    consumer.startField(schema.getFieldName(fieldId), fieldId);
> >> > +    for (int i = valueHolder.start; i < valueHolder.end; i++) {
> >> > +  </#if>
> >> > +
> >> > +  <#if  minor.class == "TinyInt" ||
> >> > +        minor.class == "UInt1" ||
> >> > +        minor.class == "UInt2" ||
> >> > +        minor.class == "SmallInt" ||
> >> > +        minor.class == "Int" ||
> >> > +        minor.class == "Time" ||
> >> > +        minor.class == "IntervalYear" ||
> >> > +        minor.class == "Decimal9" ||
> >> > +        minor.class == "UInt4">
> >> > +    <#if mode.prefix == "Repeated" >
> >> > +
> >>  consumer.addInteger(valueHolder.vector.getAccessor().get(i));
> >> > +    <#else>
> >> > +    consumer.startField(schema.getFieldName(fieldId), fieldId);
> >> > +    consumer.addInteger(valueHolder.value);
> >> > +    consumer.endField(schema.getFieldName(fieldId), fieldId);
> >> > +    </#if>
> >> > +  <#elseif
> >> > +        minor.class == "Float4">
> >> > +      <#if mode.prefix == "Repeated" >
> >> > +
> >>  consumer.addFloat(valueHolder.vector.getAccessor().get(i));
> >> > +      <#else>
> >> > +    consumer.startField(schema.getFieldName(fieldId), fieldId);
> >> > +    consumer.addFloat(valueHolder.value);
> >> > +    consumer.endField(schema.getFieldName(fieldId), fieldId);
> >> > +      </#if>
> >> > +  <#elseif
> >> > +        minor.class == "BigInt" ||
> >> > +        minor.class == "Decimal18" ||
> >> > +        minor.class == "TimeStamp" ||
> >> > +        minor.class == "Date" ||
> >> > +        minor.class == "UInt8">
> >> > +      <#if mode.prefix == "Repeated" >
> >> > +
>  consumer.addLong(valueHolder.vector.getAccessor().get(i));
> >> > +      <#else>
> >> > +    consumer.startField(schema.getFieldName(fieldId), fieldId);
> >> > +    consumer.addLong(valueHolder.value);
> >> > +    consumer.endField(schema.getFieldName(fieldId), fieldId);
> >> > +      </#if>
> >> > +  <#elseif
> >> > +        minor.class == "Float8">
> >> > +      <#if mode.prefix == "Repeated" >
> >> > +
> >>  consumer.addDouble(valueHolder.vector.getAccessor().get(i));
> >> > +      <#else>
> >> > +    consumer.startField(schema.getFieldName(fieldId), fieldId);
> >> > +    consumer.addDouble(valueHolder.value);
> >> > +    consumer.endField(schema.getFieldName(fieldId), fieldId);
> >> > +      </#if>
> >> > +  <#elseif
> >> > +        minor.class == "Bit">
> >> > +      <#if mode.prefix == "Repeated" >
> >> > +
> >>  consumer.addBoolean(valueHolder.vector.getAccessor().get(i) == 1);
> >> > +      <#else>
> >> > +    consumer.startField(schema.getFieldName(fieldId), fieldId);
> >> > +    consumer.addBoolean(valueHolder.value == 1);
> >> > +    consumer.endField(schema.getFieldName(fieldId), fieldId);
> >> > +      </#if>
> >> > +  <#elseif
> >> > +        minor.class == "Decimal28Sparse" ||
> >> > +        minor.class == "Decimal38Sparse">
> >> > +      <#if mode.prefix == "Repeated" >
> >> > +      <#else>
> >> > +      consumer.startField(schema.getFieldName(fieldId), fieldId);
> >> > +      byte[] bytes = DecimalUtility.getBigDecimalFromSparse(
> >> > +              valueHolder.buffer, valueHolder.start,
> >> ${minor.class}Holder.nDecimalDigits,
> >> valueHolder.scale).unscaledValue().toByteArray();
> >> > +      byte[] output = new
> >>
> byte[ParquetTypeHelper.getLengthForMinorType(MinorType.${minor.class?upper_case})];
> >> > +      if (valueHolder.sign) {
> >> > +        Arrays.fill(output, 0, output.length - bytes.length,
> >> (byte)0xFF);
> >> > +      } else {
> >> > +        Arrays.fill(output, 0, output.length - bytes.length,
> (byte)0x0);
> >> > +      }
> >> > +      System.arraycopy(bytes, 0, output, output.length -
> bytes.length,
> >> bytes.length);
> >> > +      consumer.addBinary(Binary.fromByteArray(output));
> >> > +      consumer.endField(schema.getFieldName(fieldId), fieldId);
> >> > +      </#if>
> >> > +  <#elseif
> >> > +        minor.class == "TimeTZ" ||
> >> > +        minor.class == "TimeStampTZ" ||
> >> > +        minor.class == "IntervalDay" ||
> >> > +        minor.class == "Interval" ||
> >> > +        minor.class == "Decimal28Dense" ||
> >> > +        minor.class == "Decimal38Dense">
> >> > +
> >> > +      <#if mode.prefix == "Repeated" >
> >> > +      <#else>
> >> > +
> >> > +      </#if>
> >> > +  <#elseif minor.class == "VarChar" || minor.class == "Var16Char" ||
> >> minor.class == "VarBinary">
> >> > +    <#if mode.prefix == "Repeated">
> >> > +      ${minor.class}Holder singleHolder = new ${minor.class}Holder();
> >> > +      valueHolder.vector.getAccessor().get(i, singleHolder);
> >> > +      consumer.startField(schema.getFieldName(fieldId), fieldId);
> >> > +
> >>
>  consumer.addBinary(Binary.fromByteBuffer(singleHolder.buffer.nioBuffer(singleHolder.start,
> >> singleHolder.end - singleHolder.start)));
> >> > +      consumer.endField(schema.getFieldName(fieldId), fieldId);
> >> > +    <#else>
> >> > +    ByteBuf buf = valueHolder.buffer;
> >> > +    consumer.startField(schema.getFieldName(fieldId), fieldId);
> >> > +
> >>
>  consumer.addBinary(Binary.fromByteBuffer(valueHolder.buffer.nioBuffer(valueHolder.start,
> >> valueHolder.end - valueHolder.start)));
> >> > +    consumer.endField(schema.getFieldName(fieldId), fieldId);
> >> > +    </#if>
> >> > +  </#if>
> >> > +  <#if mode.prefix == "Repeated">
> >> > +    }
> >> > +    consumer.endField(schema.getFieldName(fieldId), fieldId);
> >> > +  </#if>
> >> > +  }
> >> > +    </#list>
> >> > +  </#list>
> >> > +</#list>
> >> > +
> >> > +}
> >> >
> >> >
> >>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/63b03467/exec/java-exec/src/main/codegen/templates/ParquetTypeHelper.java
> >> > ----------------------------------------------------------------------
> >> > diff --git
> >> a/exec/java-exec/src/main/codegen/templates/ParquetTypeHelper.java
> >> b/exec/java-exec/src/main/codegen/templates/ParquetTypeHelper.java
> >> > new file mode 100644
> >> > index 0000000..b268d33
> >> > --- /dev/null
> >> > +++ b/exec/java-exec/src/main/codegen/templates/ParquetTypeHelper.java
> >> > @@ -0,0 +1,155 @@
> >> > +/**
> >> > + * Licensed to the Apache Software Foundation (ASF) under one
> >> > + * or more contributor license agreements.  See the NOTICE file
> >> > + * distributed with this work for additional information
> >> > + * regarding copyright ownership.  The ASF licenses this file
> >> > + * to you under the Apache License, Version 2.0 (the
> >> > + * "License"); you may not use this file except in compliance
> >> > + * with the License.  You may obtain a copy of the License at
> >> > + *
> >> > + * http://www.apache.org/licenses/LICENSE-2.0
> >> > + *
> >> > + * Unless required by applicable law or agreed to in writing,
> software
> >> > + * distributed under the License is distributed on an "AS IS" BASIS,
> >> > + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
> >> implied.
> >> > + * See the License for the specific language governing permissions
> and
> >> > + * limitations under the License.
> >> > + */
> >> > +
> >> > +import parquet.format.ConvertedType;
> >> > +import parquet.schema.DecimalMetadata;
> >> > +import parquet.schema.OriginalType;
> >> > +import parquet.schema.PrimitiveType.PrimitiveTypeName;
> >> > +
> >> > +<@pp.dropOutputFile />
> >> > +<@pp.changeOutputFile
> >> name="org/apache/drill/exec/store/parquet/ParquetTypeHelper.java" />
> >> > +<#include "/@includes/license.ftl" />
> >> > +
> >> > +package org.apache.drill.exec.store.parquet;
> >> > +
> >> > +import org.apache.drill.common.types.TypeProtos.DataMode;
> >> > +import org.apache.drill.common.types.TypeProtos.MinorType;
> >> > +import org.apache.drill.exec.record.MaterializedField;
> >> > +import parquet.schema.OriginalType;
> >> > +import parquet.schema.DecimalMetadata;
> >> > +import parquet.schema.PrimitiveType.PrimitiveTypeName;
> >> > +import parquet.schema.Type.Repetition;
> >> > +
> >> > +import java.util.HashMap;
> >> > +import java.util.Map;
> >> > +
> >> > +public class ParquetTypeHelper {
> >> > +  private static Map<MinorType,PrimitiveTypeName> typeMap;
> >> > +  private static Map<DataMode,Repetition> modeMap;
> >> > +  private static Map<MinorType,OriginalType> originalTypeMap;
> >> > +
> >> > +  static {
> >> > +    typeMap = new HashMap();
> >> > +
> >> > +    <#list vv.types as type>
> >> > +    <#list type.minor as minor>
> >> > +    <#if    minor.class == "TinyInt" ||
> >> > +            minor.class == "UInt1" ||
> >> > +            minor.class == "UInt2" ||
> >> > +            minor.class == "SmallInt" ||
> >> > +            minor.class == "Int" ||
> >> > +            minor.class == "Time" ||
> >> > +            minor.class == "IntervalYear" ||
> >> > +            minor.class == "Decimal9" ||
> >> > +            minor.class == "UInt4">
> >> > +                    typeMap.put(MinorType.${minor.class?upper_case},
> >> PrimitiveTypeName.INT32);
> >> > +    <#elseif
> >> > +            minor.class == "Float4">
> >> > +                    typeMap.put(MinorType.${minor.class?upper_case},
> >> PrimitiveTypeName.FLOAT);
> >> > +    <#elseif
> >> > +            minor.class == "BigInt" ||
> >> > +            minor.class == "Decimal18" ||
> >> > +            minor.class == "TimeStamp" ||
> >> > +            minor.class == "Date" ||
> >> > +            minor.class == "UInt8">
> >> > +                    typeMap.put(MinorType.${minor.class?upper_case},
> >> PrimitiveTypeName.INT64);
> >> > +    <#elseif
> >> > +            minor.class == "Float8">
> >> > +                    typeMap.put(MinorType.${minor.class?upper_case},
> >> PrimitiveTypeName.DOUBLE);
> >> > +    <#elseif
> >> > +            minor.class == "Bit">
> >> > +                    typeMap.put(MinorType.${minor.class?upper_case},
> >> PrimitiveTypeName.BOOLEAN);
> >> > +    <#elseif
> >> > +            minor.class == "TimeTZ" ||
> >> > +            minor.class == "TimeStampTZ" ||
> >> > +            minor.class == "IntervalDay" ||
> >> > +            minor.class == "Interval" ||
> >> > +            minor.class == "Decimal28Dense" ||
> >> > +            minor.class == "Decimal38Dense" ||
> >> > +            minor.class == "Decimal28Sparse" ||
> >> > +            minor.class == "Decimal38Sparse">
> >> > +                    typeMap.put(MinorType.${minor.class?upper_case},
> >> PrimitiveTypeName.FIXED_LEN_BYTE_ARRAY);
> >> > +    <#elseif
> >> > +            minor.class == "VarChar" ||
> >> > +            minor.class == "Var16Char" ||
> >> > +            minor.class == "VarBinary" >
> >> > +                    typeMap.put(MinorType.${minor.class?upper_case},
> >> PrimitiveTypeName.BINARY);
> >> > +    </#if>
> >> > +    </#list>
> >> > +    </#list>
> >> > +
> >> > +    modeMap = new HashMap();
> >> > +
> >> > +    modeMap.put(DataMode.REQUIRED, Repetition.REQUIRED);
> >> > +    modeMap.put(DataMode.OPTIONAL, Repetition.OPTIONAL);
> >> > +    modeMap.put(DataMode.REPEATED, Repetition.REPEATED);
> >> > +
> >> > +    originalTypeMap = new HashMap();
> >> > +
> >> > +    <#list vv.types as type>
> >> > +    <#list type.minor as minor>
> >> > +            <#if minor.class.startsWith("Decimal")>
> >> > +
> >>
>  originalTypeMap.put(MinorType.${minor.class?upper_case},OriginalType.DECIMAL);
> >> > +            </#if>
> >> > +    </#list>
> >> > +    </#list>
> >> > +            originalTypeMap.put(MinorType.VARCHAR,
> OriginalType.UTF8);
> >> > +            originalTypeMap.put(MinorType.DATE, OriginalType.DATE);
> >> > +            originalTypeMap.put(MinorType.TIME, OriginalType.TIME);
> >> > +            originalTypeMap.put(MinorType.TIMESTAMP,
> >> OriginalType.TIMESTAMP);
> >> > +            originalTypeMap.put(MinorType.TIMESTAMPTZ,
> >> OriginalType.TIMESTAMPTZ);
> >> > +  }
> >> > +
> >> > +  public static PrimitiveTypeName
> >> getPrimitiveTypeNameForMinorType(MinorType minorType) {
> >> > +    return typeMap.get(minorType);
> >> > +  }
> >> > +
> >> > +  public static Repetition getRepetitionForDataMode(DataMode
> dataMode) {
> >> > +    return modeMap.get(dataMode);
> >> > +  }
> >> > +
> >> > +  public static OriginalType getOriginalTypeForMinorType(MinorType
> >> minorType) {
> >> > +    return originalTypeMap.get(minorType);
> >> > +  }
> >> > +
> >> > +  public static DecimalMetadata
> >> getDecimalMetadataForField(MaterializedField field) {
> >> > +    switch(field.getType().getMinorType()) {
> >> > +      case DECIMAL9:
> >> > +      case DECIMAL18:
> >> > +      case DECIMAL28SPARSE:
> >> > +      case DECIMAL28DENSE:
> >> > +      case DECIMAL38SPARSE:
> >> > +      case DECIMAL38DENSE:
> >> > +        return new DecimalMetadata(field.getPrecision(),
> >> field.getScale());
> >> > +      default:
> >> > +        return null;
> >> > +    }
> >> > +  }
> >> > +
> >> > +  public static int getLengthForMinorType(MinorType minorType) {
> >> > +    switch(minorType) {
> >> > +      case DECIMAL28SPARSE:
> >> > +        return 12;
> >> > +      case DECIMAL38SPARSE:
> >> > +        return 16;
> >> > +      default:
> >> > +        return 0;
> >> > +    }
> >> > +  }
> >> > +
> >> > +}
> >> >
> >> >
> >>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/63b03467/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 9eee08d..238fae9 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
> >> > @@ -17,6 +17,10 @@
> >> >   */
> >> >  package org.apache.drill.exec;
> >> >
> >> > +import org.apache.drill.exec.server.options.OptionValidator;
> >> > +import
> >> org.apache.drill.exec.server.options.TypeValidators.LongValidator;
> >> > +import
> >> org.apache.drill.exec.server.options.TypeValidators.StringValidator;
> >> > +
> >> >  public interface ExecConstants {
> >> >    public static final String ZK_RETRY_TIMES =
> >> "drill.exec.zk.retry.count";
> >> >    public static final String ZK_RETRY_DELAY =
> >> "drill.exec.zk.retry.delay";
> >> > @@ -63,5 +67,9 @@ public interface ExecConstants {
> >> >    public static final String FILESYSTEM_PARTITION_COLUMN_LABEL =
> >> "drill.exec.storage.file.partition.column.label";
> >> >    public static final String HAZELCAST_SUBNETS =
> >> "drill.exec.cache.hazel.subnets";
> >> >    public static final String TOP_LEVEL_MAX_ALLOC =
> >> "drill.exec.memory.top.max";
> >> > +  public static final String OUTPUT_FORMAT_OPTION = "store.format";
> >> > +  public static final OptionValidator OUTPUT_FORMAT_VALIDATOR = new
> >> StringValidator(OUTPUT_FORMAT_OPTION, "parquet");
> >> > +  public static final String PARQUET_BLOCK_SIZE =
> "parquet.block.size";
> >> > +  public static final OptionValidator PARQUET_BLOCK_SIZE_VALIDATOR =
> >> new LongValidator(PARQUET_BLOCK_SIZE, 512*1024*1024);
> >> >
> >> >  }
> >> >
> >> >
> >>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/63b03467/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 cfe8e2c..21031e5 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
> >> > @@ -22,6 +22,7 @@ import java.util.Map;
> >> >  import java.util.Map.Entry;
> >> >  import java.util.concurrent.ConcurrentMap;
> >> >
> >> > +import org.apache.drill.exec.ExecConstants;
> >> >  import org.apache.drill.exec.cache.DistributedCache;
> >> >  import org.apache.drill.exec.cache.DistributedMap;
> >> >  import org.apache.drill.exec.planner.physical.PlannerSettings;
> >> > @@ -38,7 +39,9 @@ public class SystemOptionManager implements
> >> OptionManager{
> >> >        PlannerSettings.STREAMAGG,
> >> >        PlannerSettings.HASHJOIN,
> >> >        PlannerSettings.MERGEJOIN,
> >> > -      PlannerSettings.MULTIPHASE
> >> > +      PlannerSettings.MULTIPHASE,
> >> > +      ExecConstants.OUTPUT_FORMAT_VALIDATOR,
> >> > +      ExecConstants.PARQUET_BLOCK_SIZE_VALIDATOR
> >> >    };
> >> >
> >> >    private DistributedMap<OptionValue> options;
> >> >
> >> >
> >>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/63b03467/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/WorkspaceSchemaFactory.java
> >> > ----------------------------------------------------------------------
> >> > diff --git
> >>
> a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/WorkspaceSchemaFactory.java
> >>
> b/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/WorkspaceSchemaFactory.java
> >> > index df73ea5..72b31e6 100644
> >> > ---
> >>
> a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/WorkspaceSchemaFactory.java
> >> > +++
> >>
> b/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/WorkspaceSchemaFactory.java
> >> > @@ -26,12 +26,15 @@ import com.google.common.collect.Lists;
> >> >  import com.google.common.collect.Sets;
> >> >  import net.hydromatic.optiq.Table;
> >> >  import org.apache.drill.common.exceptions.ExecutionSetupException;
> >> > +import org.apache.drill.exec.ExecConstants;
> >> >  import org.apache.drill.exec.planner.logical.CreateTableEntry;
> >> >  import org.apache.drill.exec.planner.logical.DrillTable;
> >> >  import org.apache.drill.exec.planner.logical.DynamicDrillTable;
> >> >  import
> org.apache.drill.exec.planner.logical.FileSystemCreateTableEntry;
> >> >  import org.apache.drill.exec.planner.sql.ExpandingConcurrentMap;
> >> >  import org.apache.drill.exec.rpc.user.UserSession;
> >> > +import org.apache.drill.exec.server.options.OptionValidator;
> >> > +import
> >> org.apache.drill.exec.server.options.TypeValidators.StringValidator;
> >> >  import org.apache.drill.exec.store.AbstractSchema;
> >> >  import org.apache.drill.exec.store.dfs.shim.DrillFileSystem;
> >> >  import org.apache.hadoop.fs.Path;
> >> > @@ -141,9 +144,11 @@ public class WorkspaceSchemaFactory implements
> >> ExpandingConcurrentMap.MapValueFa
> >> >        return fs;
> >> >      }
> >> >
> >> > +
> >> >      @Override
> >> >      public CreateTableEntry createNewTable(String tableName) {
> >> > -      FormatPlugin formatPlugin =
> >> plugin.getFormatPlugin(config.getStorageFormat());
> >> > +      String storage =
> >>
> session.getOptions().getOption(ExecConstants.OUTPUT_FORMAT_OPTION).string_val;
> >> > +      FormatPlugin formatPlugin = plugin.getFormatPlugin(storage);
> >> >        if (formatPlugin == null)
> >> >          throw new UnsupportedOperationException(
> >> >            String.format("Unsupported format '%s' in workspace '%s'",
> >> config.getStorageFormat(),
> >> > @@ -151,7 +156,7 @@ public class WorkspaceSchemaFactory implements
> >> ExpandingConcurrentMap.MapValueFa
> >> >
> >> >        return new FileSystemCreateTableEntry(
> >> >            (FileSystemConfig) plugin.getConfig(),
> >> > -          plugin.getFormatPlugin(config.getStorageFormat()),
> >> > +          formatPlugin,
> >> >            config.getLocation() + Path.SEPARATOR + tableName);
> >> >      }
> >> >    }
> >> >
> >> >
> >>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/63b03467/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/BitReader.java
> >> > ----------------------------------------------------------------------
> >> > diff --git
> >>
> a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/BitReader.java
> >>
> b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/BitReader.java
> >> > index c323222..7ae95cd 100644
> >> > ---
> >>
> a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/BitReader.java
> >> > +++
> >>
> b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/BitReader.java
> >> > @@ -22,6 +22,7 @@ import
> >> org.apache.drill.exec.vector.BaseDataValueVector;
> >> >  import org.apache.drill.exec.vector.ValueVector;
> >> >  import parquet.column.ColumnDescriptor;
> >> >  import parquet.format.ConvertedType;
> >> > +import parquet.format.SchemaElement;
> >> >  import parquet.hadoop.metadata.ColumnChunkMetaData;
> >> >
> >> >  final class BitReader extends ColumnReader {
> >> > @@ -31,8 +32,8 @@ final class BitReader extends ColumnReader {
> >> >    private byte[] bytes;
> >> >
> >> >    BitReader(ParquetRecordReader parentReader, int allocateSize,
> >> ColumnDescriptor descriptor, ColumnChunkMetaData columnChunkMetaData,
> >> > -            boolean fixedLength, ValueVector v, ConvertedType
> >> convertedType) throws ExecutionSetupException {
> >> > -    super(parentReader, allocateSize, descriptor,
> columnChunkMetaData,
> >> fixedLength, v, convertedType);
> >> > +            boolean fixedLength, ValueVector v, SchemaElement
> >> schemaElement) throws ExecutionSetupException {
> >> > +    super(parentReader, allocateSize, descriptor,
> columnChunkMetaData,
> >> fixedLength, v, schemaElement);
> >> >    }
> >> >
> >> >    @Override
> >> >
> >> >
> >>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/63b03467/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ColumnReader.java
> >> > ----------------------------------------------------------------------
> >> > diff --git
> >>
> a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ColumnReader.java
> >>
> b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ColumnReader.java
> >> > index 196e1fd..b9faafe 100644
> >> > ---
> >>
> a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ColumnReader.java
> >> > +++
> >>
> b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ColumnReader.java
> >> > @@ -25,8 +25,10 @@ import org.apache.drill.exec.vector.ValueVector;
> >> >  import org.apache.hadoop.fs.FSDataInputStream;
> >> >  import parquet.column.ColumnDescriptor;
> >> >  import parquet.format.ConvertedType;
> >> > +import parquet.format.SchemaElement;
> >> >  import parquet.hadoop.metadata.ColumnChunkMetaData;
> >> >  import parquet.schema.PrimitiveType;
> >> > +import parquet.schema.PrimitiveType.PrimitiveTypeName;
> >> >
> >> >  import java.io.IOException;
> >> >
> >> > @@ -48,7 +50,7 @@ abstract class ColumnReader<V extends ValueVector> {
> >> >    // status information on the current page
> >> >    final PageReadStatus pageReadStatus;
> >> >
> >> > -  final ConvertedType convertedType;
> >> > +  final SchemaElement schemaElement;
> >> >
> >> >    // quick reference to see if the field is fixed length (as this
> >> requires an instanceof)
> >> >    final boolean isFixedLength;
> >> > @@ -70,12 +72,12 @@ abstract class ColumnReader<V extends
> ValueVector> {
> >> >    long readStartInBytes = 0, readLength = 0, readLengthInBits = 0,
> >> recordsReadInThisIteration = 0;
> >> >
> >> >    protected ColumnReader(ParquetRecordReader parentReader, int
> >> allocateSize, ColumnDescriptor descriptor,
> >> > -      ColumnChunkMetaData columnChunkMetaData, boolean fixedLength, V
> >> v, ConvertedType convertedType) throws ExecutionSetupException {
> >> > +      ColumnChunkMetaData columnChunkMetaData, boolean fixedLength, V
> >> v, SchemaElement schemaElement) throws ExecutionSetupException {
> >> >      this.parentReader = parentReader;
> >> >      this.columnDescriptor = descriptor;
> >> >      this.columnChunkMetaData = columnChunkMetaData;
> >> >      this.isFixedLength = fixedLength;
> >> > -    this.convertedType = convertedType;
> >> > +    this.schemaElement = schemaElement;
> >> >
> >> >      if (allocateSize > 1) {
> >> >        valueVec =  v;
> >> > @@ -87,7 +89,11 @@ abstract class ColumnReader<V extends ValueVector>
> {
> >> >      this.pageReadStatus = new PageReadStatus(this,
> >> parentReader.fileSystem, parentReader.hadoopPath, columnChunkMetaData);
> >> >
> >> >      if (columnDescriptor.getType() !=
> >> PrimitiveType.PrimitiveTypeName.BINARY) {
> >> > -      dataTypeLengthInBits =
> >> ParquetRecordReader.getTypeLengthInBits(columnDescriptor.getType());
> >> > +      if (columnDescriptor.getType() ==
> >> PrimitiveTypeName.FIXED_LEN_BYTE_ARRAY) {
> >> > +        dataTypeLengthInBits = columnDescriptor.getTypeLength() * 8;
> >> > +      } else {
> >> > +        dataTypeLengthInBits =
> >> ParquetRecordReader.getTypeLengthInBits(columnDescriptor.getType());
> >> > +      }
> >> >      }
> >> >
> >> >    }
> >> >
> >> >
> >>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/63b03467/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/FixedByteAlignedReader.java
> >> > ----------------------------------------------------------------------
> >> > diff --git
> >>
> a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/FixedByteAlignedReader.java
> >>
> b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/FixedByteAlignedReader.java
> >> > index 0aa18cf..574b0cb 100644
> >> > ---
> >>
> a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/FixedByteAlignedReader.java
> >> > +++
> >>
> b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/FixedByteAlignedReader.java
> >> > @@ -18,20 +18,27 @@
> >> >  package org.apache.drill.exec.store.parquet;
> >> >
> >> >  import org.apache.drill.common.exceptions.ExecutionSetupException;
> >> > -import org.apache.drill.exec.vector.BaseDataValueVector;
> >> > +import org.apache.drill.common.util.DecimalUtility;
> >> > +import org.apache.drill.exec.expr.holders.Decimal28SparseHolder;
> >> > +import org.apache.drill.exec.expr.holders.Decimal38SparseHolder;
> >> > +import org.apache.drill.exec.vector.Decimal28SparseVector;
> >> > +import org.apache.drill.exec.vector.Decimal38SparseVector;
> >> >  import org.apache.drill.exec.vector.ValueVector;
> >> >  import parquet.column.ColumnDescriptor;
> >> >  import parquet.format.ConvertedType;
> >> > +import parquet.format.SchemaElement;
> >> >  import parquet.hadoop.metadata.ColumnChunkMetaData;
> >> >
> >> > +import java.math.BigDecimal;
> >> > +
> >> >  class FixedByteAlignedReader extends ColumnReader {
> >> >
> >> > -  private byte[] bytes;
> >> > +  protected byte[] bytes;
> >> >
> >> >
> >> >    FixedByteAlignedReader(ParquetRecordReader parentReader, int
> >> allocateSize, ColumnDescriptor descriptor, ColumnChunkMetaData
> >> columnChunkMetaData,
> >> > -                         boolean fixedLength, ValueVector v,
> >> ConvertedType convertedType) throws ExecutionSetupException {
> >> > -    super(parentReader, allocateSize, descriptor,
> columnChunkMetaData,
> >> fixedLength, v, convertedType);
> >> > +                         boolean fixedLength, ValueVector v,
> >> SchemaElement schemaElement) throws ExecutionSetupException {
> >> > +    super(parentReader, allocateSize, descriptor,
> columnChunkMetaData,
> >> fixedLength, v, schemaElement);
> >> >    }
> >> >
> >> >    // this method is called by its superclass during a read loop
> >> > @@ -50,4 +57,76 @@ class FixedByteAlignedReader extends ColumnReader {
> >> >      vectorData.writeBytes(bytes,
> >> >          (int) readStartInBytes, (int) readLength);
> >> >    }
> >> > +
> >> > +  public static abstract class ConvertedReader extends
> >> FixedByteAlignedReader {
> >> > +
> >> > +    protected int dataTypeLengthInBytes;
> >> > +
> >> > +    ConvertedReader(ParquetRecordReader parentReader, int
> allocateSize,
> >> ColumnDescriptor descriptor, ColumnChunkMetaData columnChunkMetaData,
> >> > +                           boolean fixedLength, ValueVector v,
> >> SchemaElement schemaElement) throws ExecutionSetupException {
> >> > +      super(parentReader, allocateSize, descriptor,
> >> columnChunkMetaData, fixedLength, v, schemaElement);
> >> > +    }
> >> > +
> >> > +    @Override
> >> > +    protected void readField(long recordsToReadInThisPass,
> ColumnReader
> >> firstColumnStatus) {
> >> > +      recordsReadInThisIteration =
> >> Math.min(pageReadStatus.currentPage.getValueCount()
> >> > +              - pageReadStatus.valuesRead, recordsToReadInThisPass -
> >> valuesReadInCurrentPass);
> >> > +
> >> > +      readStartInBytes = pageReadStatus.readPosInBytes;
> >> > +      readLengthInBits = recordsReadInThisIteration *
> >> dataTypeLengthInBits;
> >> > +      readLength = (int) Math.ceil(readLengthInBits / 8.0);
> >> > +
> >> > +      bytes = pageReadStatus.pageDataByteArray;
> >> > +
> >> > +      dataTypeLengthInBytes = (int) Math.ceil(dataTypeLengthInBits /
> >> 8.0);
> >> > +      for (int i = 0; i < recordsReadInThisIteration; i++) {
> >> > +        addNext((int)readStartInBytes + i * dataTypeLengthInBytes, i
> +
> >> valuesReadInCurrentPass);
> >> > +      }
> >> > +    }
> >> > +
> >> > +    /**
> >> > +     * Reads from bytes, converts, and writes to buffer
> >> > +     * @param start the index in bytes to start reading from
> >> > +     * @param index the index of the ValueVector
> >> > +     */
> >> > +    abstract void addNext(int start, int index);
> >> > +  }
> >> > +
> >> > +  public static class Decimal28Reader extends ConvertedReader {
> >> > +
> >> > +    Decimal28SparseVector decimal28Vector;
> >> > +
> >> > +    Decimal28Reader(ParquetRecordReader parentReader, int
> allocateSize,
> >> ColumnDescriptor descriptor, ColumnChunkMetaData columnChunkMetaData,
> >> > +                    boolean fixedLength, ValueVector v, SchemaElement
> >> schemaElement) throws ExecutionSetupException {
> >> > +      super(parentReader, allocateSize, descriptor,
> >> columnChunkMetaData, fixedLength, v, schemaElement);
> >> > +      decimal28Vector = (Decimal28SparseVector) v;
> >> > +    }
> >> > +
> >> > +    @Override
> >> > +    void addNext(int start, int index) {
> >> > +      int width = Decimal28SparseHolder.WIDTH;
> >> > +      BigDecimal intermediate =
> >> DecimalUtility.getBigDecimalFromByteArray(bytes, start,
> >> dataTypeLengthInBytes, schemaElement.getScale());
> >> > +      DecimalUtility.getSparseFromBigDecimal(intermediate,
> >> decimal28Vector.getData(), index * width, schemaElement.getScale(),
> >> > +              schemaElement.getPrecision(),
> >> Decimal28SparseHolder.nDecimalDigits);
> >> > +    }
> >> > +  }
> >> > +
> >> > +  public static class Decimal38Reader extends ConvertedReader {
> >> > +
> >> > +    Decimal38SparseVector decimal38Vector;
> >> > +
> >> > +    Decimal38Reader(ParquetRecordReader parentReader, int
> allocateSize,
> >> ColumnDescriptor descriptor, ColumnChunkMetaData columnChunkMetaData,
> >> > +                    boolean fixedLength, ValueVector v, SchemaElement
> >> schemaElement) throws ExecutionSetupException {
> >> > +      super(parentReader, allocateSize, descriptor,
> >> columnChunkMetaData, fixedLength, v, schemaElement);
> >> > +      decimal38Vector = (Decimal38SparseVector) v;
> >> > +    }
> >> > +
> >> > +    @Override
> >> > +    void addNext(int start, int index) {
> >> > +      int width = Decimal38SparseHolder.WIDTH;
> >> > +      BigDecimal intermediate =
> >> DecimalUtility.getBigDecimalFromByteArray(bytes, start,
> >> dataTypeLengthInBytes, schemaElement.getScale());
> >> > +      DecimalUtility.getSparseFromBigDecimal(intermediate,
> >> decimal38Vector.getData(), index * width, schemaElement.getScale(),
> >> > +              schemaElement.getPrecision(),
> >> Decimal38SparseHolder.nDecimalDigits);
> >> > +    }
> >> > +  }
> >> >  }
> >> > \ No newline at end of file
> >> >
> >> >
> >>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/63b03467/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/NullableBitReader.java
> >> > ----------------------------------------------------------------------
> >> > diff --git
> >>
> a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/NullableBitReader.java
> >>
> b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/NullableBitReader.java
> >> > index 22933ba..29ca30a 100644
> >> > ---
> >>
> a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/NullableBitReader.java
> >> > +++
> >>
> b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/NullableBitReader.java
> >> > @@ -24,6 +24,7 @@ import
> org.apache.drill.exec.vector.NullableBitVector;
> >> >  import org.apache.drill.exec.vector.ValueVector;
> >> >  import parquet.column.ColumnDescriptor;
> >> >  import parquet.format.ConvertedType;
> >> > +import parquet.format.SchemaElement;
> >> >  import parquet.hadoop.metadata.ColumnChunkMetaData;
> >> >
> >> >  import java.io.IOException;
> >> > @@ -39,8 +40,8 @@ import java.io.IOException;
> >> >  final class NullableBitReader extends ColumnReader {
> >> >
> >> >    NullableBitReader(ParquetRecordReader parentReader, int
> allocateSize,
> >> ColumnDescriptor descriptor, ColumnChunkMetaData columnChunkMetaData,
> >> > -                    boolean fixedLength, ValueVector v, ConvertedType
> >> convertedType) throws ExecutionSetupException {
> >> > -    super(parentReader, allocateSize, descriptor,
> columnChunkMetaData,
> >> fixedLength, v, convertedType);
> >> > +                    boolean fixedLength, ValueVector v, SchemaElement
> >> schemaElement) throws ExecutionSetupException {
> >> > +    super(parentReader, allocateSize, descriptor,
> columnChunkMetaData,
> >> fixedLength, v, schemaElement);
> >> >    }
> >> >
> >> >    @Override
> >> >
> >> >
> >>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/63b03467/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/NullableColumnReader.java
> >> > ----------------------------------------------------------------------
> >> > diff --git
> >>
> a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/NullableColumnReader.java
> >>
> b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/NullableColumnReader.java
> >> > index 66d1c5f..6040c67 100644
> >> > ---
> >>
> a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/NullableColumnReader.java
> >> > +++
> >>
> b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/NullableColumnReader.java
> >> > @@ -23,6 +23,7 @@ import
> >> org.apache.drill.exec.vector.NullableVectorDefinitionSetter;
> >> >  import org.apache.drill.exec.vector.ValueVector;
> >> >  import parquet.column.ColumnDescriptor;
> >> >  import parquet.format.ConvertedType;
> >> > +import parquet.format.SchemaElement;
> >> >  import parquet.hadoop.metadata.ColumnChunkMetaData;
> >> >
> >> >  import java.io.IOException;
> >> > @@ -36,8 +37,8 @@ abstract class NullableColumnReader extends
> >> ColumnReader{
> >> >    int bitsUsed;
> >> >
> >> >    NullableColumnReader(ParquetRecordReader parentReader, int
> >> allocateSize, ColumnDescriptor descriptor, ColumnChunkMetaData
> >> columnChunkMetaData,
> >> > -               boolean fixedLength, ValueVector v, ConvertedType
> >> convertedType) throws ExecutionSetupException {
> >> > -    super(parentReader, allocateSize, descriptor,
> columnChunkMetaData,
> >> fixedLength, v, convertedType);
> >> > +               boolean fixedLength, ValueVector v, SchemaElement
> >> schemaElement) throws ExecutionSetupException {
> >> > +    super(parentReader, allocateSize, descriptor,
> columnChunkMetaData,
> >> fixedLength, v, schemaElement);
> >> >    }
> >> >
> >> >    public void readAllFixedFields(long recordsToReadInThisPass,
> >> ColumnReader firstColumnStatus) throws IOException {
> >> >
> >> >
> >>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/63b03467/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/NullableFixedByteAlignedReader.java
> >> > ----------------------------------------------------------------------
> >> > diff --git
> >>
> a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/NullableFixedByteAlignedReader.java
> >>
> b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/NullableFixedByteAlignedReader.java
> >> > index 038f2d7..d4416c8 100644
> >> > ---
> >>
> a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/NullableFixedByteAlignedReader.java
> >> > +++
> >>
> b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/NullableFixedByteAlignedReader.java
> >> > @@ -18,19 +18,27 @@
> >> >  package org.apache.drill.exec.store.parquet;
> >> >
> >> >  import org.apache.drill.common.exceptions.ExecutionSetupException;
> >> > +import org.apache.drill.common.util.DecimalUtility;
> >> > +import
> org.apache.drill.exec.expr.holders.NullableDecimal28SparseHolder;
> >> > +import
> org.apache.drill.exec.expr.holders.NullableDecimal38SparseHolder;
> >> > +import org.apache.drill.exec.vector.NullableDecimal28SparseVector;
> >> > +import org.apache.drill.exec.vector.NullableDecimal38SparseVector;
> >> >  import org.apache.drill.exec.vector.ValueVector;
> >> >
> >> >  import parquet.column.ColumnDescriptor;
> >> >  import parquet.format.ConvertedType;
> >> > +import parquet.format.SchemaElement;
> >> >  import parquet.hadoop.metadata.ColumnChunkMetaData;
> >> >
> >> > +import java.math.BigDecimal;
> >> > +
> >> >  class NullableFixedByteAlignedReader extends NullableColumnReader {
> >> >
> >> > -  private byte[] bytes;
> >> > +  protected byte[] bytes;
> >> >
> >> >    NullableFixedByteAlignedReader(ParquetRecordReader parentReader,
> int
> >> allocateSize, ColumnDescriptor descriptor,
> >> > -      ColumnChunkMetaData columnChunkMetaData, boolean fixedLength,
> >> ValueVector v, ConvertedType convertedType) throws
> ExecutionSetupException {
> >> > -    super(parentReader, allocateSize, descriptor,
> columnChunkMetaData,
> >> fixedLength, v, convertedType);
> >> > +      ColumnChunkMetaData columnChunkMetaData, boolean fixedLength,
> >> ValueVector v, SchemaElement schemaElement) throws
> ExecutionSetupException {
> >> > +    super(parentReader, allocateSize, descriptor,
> columnChunkMetaData,
> >> fixedLength, v, schemaElement);
> >> >    }
> >> >
> >> >    // this method is called by its superclass during a read loop
> >> > @@ -48,4 +56,71 @@ class NullableFixedByteAlignedReader extends
> >> NullableColumnReader {
> >> >      // fill in data.
> >> >      vectorData.writeBytes(bytes, (int) readStartInBytes, (int)
> >> readLength);
> >> >    }
> >> > +
> >> > +  public static abstract class NullableConvertedReader extends
> >> NullableFixedByteAlignedReader {
> >> > +
> >> > +    protected int dataTypeLengthInBytes;
> >> > +
> >> > +    NullableConvertedReader(ParquetRecordReader parentReader, int
> >> allocateSize, ColumnDescriptor descriptor,
> >> > +                                   ColumnChunkMetaData
> >> columnChunkMetaData, boolean fixedLength, ValueVector v, SchemaElement
> >> schemaElement) throws ExecutionSetupException {
> >> > +      super(parentReader, allocateSize, descriptor,
> >> columnChunkMetaData, fixedLength, v, schemaElement);
> >> > +    }
> >> > +
> >> > +    @Override
> >> > +    protected void readField(long recordsToReadInThisPass,
> ColumnReader
> >> firstColumnStatus) {
> >> > +
> >> > +      this.recordsReadInThisIteration = recordsToReadInThisPass;
> >> > +
> >> > +      // set up metadata
> >> > +      this.readStartInBytes = pageReadStatus.readPosInBytes;
> >> > +      this.readLengthInBits = recordsReadInThisIteration *
> >> dataTypeLengthInBits;
> >> > +      this.readLength = (int) Math.ceil(readLengthInBits / 8.0);
> >> > +      this.bytes = pageReadStatus.pageDataByteArray;
> >> > +
> >> > +      dataTypeLengthInBytes = (int) Math.ceil(dataTypeLengthInBits /
> >> 8.0);
> >> > +      for (int i = 0; i < recordsReadInThisIteration; i++) {
> >> > +        addNext((int) readStartInBytes + i * dataTypeLengthInBytes,
> i +
> >> valuesReadInCurrentPass);
> >> > +      }
> >> > +    }
> >> > +
> >> > +    abstract void addNext(int start, int index);
> >> > +  }
> >> > +
> >> > +  public static class NullableDecimal28Reader extends
> >> NullableConvertedReader {
> >> > +
> >> > +    NullableDecimal28SparseVector decimal28Vector;
> >> > +
> >> > +    NullableDecimal28Reader(ParquetRecordReader parentReader, int
> >> allocateSize, ColumnDescriptor descriptor, ColumnChunkMetaData
> >> columnChunkMetaData,
> >> > +                            boolean fixedLength, ValueVector v,
> >> SchemaElement schemaElement) throws ExecutionSetupException {
> >> > +      super(parentReader, allocateSize, descriptor,
> >> columnChunkMetaData, fixedLength, v, schemaElement);
> >> > +      decimal28Vector = (NullableDecimal28SparseVector) v;
> >> > +    }
> >> > +
> >> > +    @Override
> >> > +    void addNext(int start, int index) {
> >> > +      int width = NullableDecimal28SparseHolder.WIDTH;
> >> > +      BigDecimal intermediate =
> >> DecimalUtility.getBigDecimalFromByteArray(bytes, start,
> >> dataTypeLengthInBytes, schemaElement.getScale());
> >> > +      DecimalUtility.getSparseFromBigDecimal(intermediate,
> >> decimal28Vector.getData(), index * width, schemaElement.getScale(),
> >> > +              schemaElement.getPrecision(),
> >> NullableDecimal28SparseHolder.nDecimalDigits);
> >> > +    }
> >> > +  }
> >> > +
> >> > +  public static class NullableDecimal38Reader extends
> >> NullableConvertedReader {
> >> > +
> >> > +    NullableDecimal38SparseVector decimal38Vector;
> >> > +
> >> > +    NullableDecimal38Reader(ParquetRecordReader parentReader, int
> >> allocateSize, ColumnDescriptor descriptor, ColumnChunkMetaData
> >> columnChunkMetaData,
> >> > +                            boolean fixedLength, ValueVector v,
> >> SchemaElement schemaElement) throws ExecutionSetupException {
> >> > +      super(parentReader, allocateSize, descriptor,
> >> columnChunkMetaData, fixedLength, v, schemaElement);
> >> > +      decimal38Vector = (NullableDecimal38SparseVector) v;
> >> > +    }
> >> > +
> >> > +    @Override
> >> > +    void addNext(int start, int index) {
> >> > +      int width = NullableDecimal38SparseHolder.WIDTH;
> >> > +      BigDecimal intermediate =
> >> DecimalUtility.getBigDecimalFromByteArray(bytes, start,
> >> dataTypeLengthInBytes, schemaElement.getScale());
> >> > +      DecimalUtility.getSparseFromBigDecimal(intermediate,
> >> decimal38Vector.getData(), index * width, schemaElement.getScale(),
> >> > +              schemaElement.getPrecision(),
> >> NullableDecimal38SparseHolder.nDecimalDigits);
> >> > +    }
> >> > +  }
> >> >  }
> >> > \ No newline at end of file
> >> >
> >> >
> >>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/63b03467/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetFormatPlugin.java
> >> > ----------------------------------------------------------------------
> >> > diff --git
> >>
> a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetFormatPlugin.java
> >>
> b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetFormatPlugin.java
> >> > index a10d30f..34287f3 100644
> >> > ---
> >>
> a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetFormatPlugin.java
> >> > +++
> >>
> b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetFormatPlugin.java
> >> > @@ -19,26 +19,30 @@ package org.apache.drill.exec.store.parquet;
> >> >
> >> >  import java.io.IOException;
> >> >  import java.util.List;
> >> > +import java.util.Map;
> >> >  import java.util.Set;
> >> >  import java.util.regex.Pattern;
> >> >
> >> > +import com.google.common.collect.Maps;
> >> > +import org.apache.drill.common.exceptions.ExecutionSetupException;
> >> >  import org.apache.drill.common.expression.SchemaPath;
> >> >  import org.apache.drill.common.logical.StoragePluginConfig;
> >> > +import org.apache.drill.exec.ExecConstants;
> >> > +import org.apache.drill.exec.ops.FragmentContext;
> >> >  import org.apache.drill.exec.physical.base.AbstractWriter;
> >> >  import org.apache.drill.exec.physical.base.PhysicalOperator;
> >> > +import org.apache.drill.exec.physical.impl.WriterRecordBatch;
> >> > +import org.apache.drill.exec.proto.ExecProtos.FragmentHandle;
> >> > +import org.apache.drill.exec.record.RecordBatch;
> >> >  import org.apache.drill.exec.server.DrillbitContext;
> >> > +import org.apache.drill.exec.store.RecordWriter;
> >> >  import org.apache.drill.exec.store.StoragePluginOptimizerRule;
> >> > -import org.apache.drill.exec.store.dfs.BasicFormatMatcher;
> >> > -import org.apache.drill.exec.store.dfs.DrillPathFilter;
> >> > -import org.apache.drill.exec.store.dfs.FileSelection;
> >> > -import org.apache.drill.exec.store.dfs.FormatMatcher;
> >> > -import org.apache.drill.exec.store.dfs.FormatPlugin;
> >> > -import org.apache.drill.exec.store.dfs.FormatSelection;
> >> > -import org.apache.drill.exec.store.dfs.MagicString;
> >> > +import org.apache.drill.exec.store.dfs.*;
> >> >  import org.apache.drill.exec.store.dfs.shim.DrillFileSystem;
> >> >  import org.apache.drill.exec.store.mock.MockStorageEngine;
> >> >  import org.apache.hadoop.conf.Configuration;
> >> >  import org.apache.hadoop.fs.FileStatus;
> >> > +import org.apache.hadoop.fs.FileSystem;
> >> >  import org.apache.hadoop.fs.Path;
> >> >  import org.apache.hadoop.fs.PathFilter;
> >> >
> >> > @@ -104,7 +108,35 @@ public class ParquetFormatPlugin implements
> >> FormatPlugin{
> >> >
> >> >    @Override
> >> >    public AbstractWriter getWriter(PhysicalOperator child, String
> >> location) throws IOException {
> >> > -    throw new UnsupportedOperationException("Parquet Writer is not
> >> supported currently.");
> >> > +    return new ParquetWriter(child, location, this);
> >> > +  }
> >> > +
> >> > +  public RecordWriter getRecordWriter(FragmentContext context,
> >> ParquetWriter writer) throws IOException {
> >> > +    Map<String, String> options = Maps.newHashMap();
> >> > +
> >> > +    options.put("location", writer.getLocation());
> >> > +
> >> > +    FragmentHandle handle = context.getHandle();
> >> > +    String fragmentId = String.format("%d_%d",
> >> handle.getMajorFragmentId(), handle.getMinorFragmentId());
> >> > +    options.put("prefix", fragmentId);
> >> > +
> >> > +    options.put(FileSystem.FS_DEFAULT_NAME_KEY,
> >> ((FileSystemConfig)writer.getStorageConfig()).connection);
> >> > +
> >> > +    options.put(ExecConstants.PARQUET_BLOCK_SIZE,
> >>
> context.getOptions().getOption(ExecConstants.PARQUET_BLOCK_SIZE).num_val.toString());
> >> > +
> >> > +    RecordWriter recordWriter = new ParquetRecordWriter();
> >> > +    recordWriter.init(options);
> >> > +
> >> > +    return recordWriter;
> >> > +  }
> >> > +
> >> > +  public RecordBatch getWriterBatch(FragmentContext context,
> >> RecordBatch incoming, ParquetWriter writer)
> >> > +          throws ExecutionSetupException {
> >> > +    try {
> >> > +      return new WriterRecordBatch(writer, incoming, context,
> >> getRecordWriter(context, writer));
> >> > +    } catch(IOException e) {
> >> > +      throw new ExecutionSetupException(String.format("Failed to
> create
> >> the WriterRecordBatch. %s", e.getMessage()), e);
> >> > +    }
> >> >    }
> >> >
> >> >    @Override
> >> >
> >> >
> >>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/63b03467/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetRecordReader.java
> >> > ----------------------------------------------------------------------
> >> > diff --git
> >>
> a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetRecordReader.java
> >>
> b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetRecordReader.java
> >> > index 5d28456..4ca13a5 100644
> >> > ---
> >>
> a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetRecordReader.java
> >> > +++
> >>
> b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetRecordReader.java
> >> > @@ -32,7 +32,9 @@ import
> >> org.apache.drill.common.expression.ExpressionPosition;
> >> >  import org.apache.drill.common.expression.FieldReference;
> >> >  import org.apache.drill.common.expression.SchemaPath;
> >> >  import org.apache.drill.common.types.TypeProtos;
> >> > +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.exception.SchemaChangeException;
> >> >  import org.apache.drill.exec.expr.TypeHelper;
> >> > @@ -41,6 +43,10 @@ import org.apache.drill.exec.ops.FragmentContext;
> >> >  import org.apache.drill.exec.physical.impl.OutputMutator;
> >> >  import org.apache.drill.exec.record.MaterializedField;
> >> >  import org.apache.drill.exec.store.RecordReader;
> >> > +import
> >>
> org.apache.drill.exec.store.parquet.FixedByteAlignedReader.Decimal28Reader;
> >> > +import
> >>
> org.apache.drill.exec.store.parquet.FixedByteAlignedReader.Decimal38Reader;
> >> > +import
> >>
> org.apache.drill.exec.store.parquet.NullableFixedByteAlignedReader.NullableDecimal28Reader;
> >> > +import
> >>
> org.apache.drill.exec.store.parquet.NullableFixedByteAlignedReader.NullableDecimal38Reader;
> >> >  import org.apache.drill.exec.vector.*;
> >> >  import org.apache.hadoop.fs.FileSystem;
> >> >  import org.apache.hadoop.fs.Path;
> >> > @@ -62,6 +68,7 @@ import parquet.schema.PrimitiveType;
> >> >  import org.apache.drill.exec.store.parquet.VarLengthColumnReaders.*;
> >> >
> >> >  import com.google.common.base.Joiner;
> >> > +import parquet.schema.PrimitiveType.PrimitiveTypeName;
> >> >
> >> >  public class ParquetRecordReader implements RecordReader {
> >> >    static final org.slf4j.Logger logger =
> >> org.slf4j.LoggerFactory.getLogger(ParquetRecordReader.class);
> >> > @@ -191,18 +198,19 @@ public class ParquetRecordReader implements
> >> RecordReader {
> >> >
> >> >      // TODO - figure out how to deal with this better once we add
> >> nested reading, note also look where this map is used below
> >> >      // store a map from column name to converted types if they are
> >> non-null
> >> > -    HashMap<String, ConvertedType> convertedTypes = new HashMap<>();
> >> > +    HashMap<String, SchemaElement> schemaElements = new HashMap<>();
> >> >      fileMetaData = new
> >>
> ParquetMetadataConverter().toParquetMetadata(ParquetFileWriter.CURRENT_VERSION,
> >> footer);
> >> >      for (SchemaElement se : fileMetaData.getSchema()) {
> >> > -      convertedTypes.put(se.getName(), se.getConverted_type());
> >> > +      schemaElements.put(se.getName(), se);
> >> >      }
> >> >
> >> >      // loop to add up the length of the fixed width columns and build
> >> the schema
> >> >      for (int i = 0; i < columns.size(); ++i) {
> >> >        column = columns.get(i);
> >> >        logger.debug("name: " + fileMetaData.getSchema().get(i).name);
> >> > -      field = MaterializedField.create(toFieldName(column.getPath()),
> >> > -          toMajorType(column.getType(), getDataMode(column),
> >> convertedTypes.get(column.getPath()[0])));
> >> > +      SchemaElement se = schemaElements.get(column.getPath()[0]);
> >> > +      MajorType mt = toMajorType(column.getType(),
> se.getType_length(),
> >> getDataMode(column), se);
> >> > +      field =
> >> MaterializedField.create(toFieldName(column.getPath()),mt);
> >> >        if ( ! fieldSelected(field)){
> >> >          continue;
> >> >        }
> >> > @@ -211,12 +219,11 @@ public class ParquetRecordReader implements
> >> RecordReader {
> >> >        if (column.getType() !=
> PrimitiveType.PrimitiveTypeName.BINARY) {
> >> >          // There is not support for the fixed binary type yet in
> >> parquet, leaving a task here as a reminder
> >> >          // TODO - implement this when the feature is added upstream
> >> > -//          if (column.getType() ==
> >> PrimitiveType.PrimitiveTypeName.FIXED_LEN_BYTE_ARRAY){
> >> > -//              byteWidthAllFixedFields +=
> column.getType().getWidth()
> >> > -//          }
> >> > -//          else { } // the code below for the rest of the fixed
> length
> >> fields
> >> > -
> >> > -        bitWidthAllFixedFields +=
> getTypeLengthInBits(column.getType());
> >> > +          if (column.getType() ==
> >> PrimitiveType.PrimitiveTypeName.FIXED_LEN_BYTE_ARRAY){
> >> > +              bitWidthAllFixedFields += se.getType_length() * 8;
> >> > +          } else {
> >> > +            bitWidthAllFixedFields +=
> >> getTypeLengthInBits(column.getType());
> >> > +          }
> >> >        } else {
> >> >          allFieldsFixedLength = false;
> >> >        }
> >> > @@ -234,16 +241,11 @@ public class ParquetRecordReader implements
> >> RecordReader {
> >> >      else {
> >> >        recordsPerBatch = DEFAULT_RECORDS_TO_READ_IF_NOT_FIXED_WIDTH;
> >> >      }
> >> > -//    for (SchemaElement se : fileMetaData.getSchema()) {
> >> > -//      if (fieldSelected())
> >> > -//      System.out.println("convertedtype :" +
> se.getConverted_type());
> >> > -//      System.out.println("name:" + se.getName());
> >> > -//      System.out.println();
> >> > -//
> >> > -//    }
> >> > +
> >> >      try {
> >> >        ValueVector v;
> >> >        ConvertedType convertedType;
> >> > +      SchemaElement schemaElement;
> >> >        ArrayList<VarLengthColumn> varLengthColumns = new
> ArrayList<>();
> >> >        ArrayList<NullableVarLengthColumn> nullableVarLengthColumns =
> new
> >> ArrayList<>();
> >> >        // initialize all of the column read status objects
> >> > @@ -251,40 +253,21 @@ public class ParquetRecordReader implements
> >> RecordReader {
> >> >        for (int i = 0; i < columns.size(); ++i) {
> >> >          column = columns.get(i);
> >> >          columnChunkMetaData =
> >> footer.getBlocks().get(0).getColumns().get(i);
> >> > -        convertedType = convertedTypes.get(column.getPath()[0]);
> >> > -        MajorType type = toMajorType(column.getType(),
> >> getDataMode(column), convertedType);
> >> > -//
> >>
>  Preconditions.checkArgument(!columnChunkMetaData.getEncodings().contains(Encoding.PLAIN_DICTIONARY),
> >> "Dictionary Encoding not currently supported");
> >> > +        schemaElement = schemaElements.get(column.getPath()[0]);
> >> > +        convertedType = schemaElement.getConverted_type();
> >> > +        MajorType type = toMajorType(column.getType(),
> >> schemaElement.getType_length(), getDataMode(column), schemaElement);
> >> >          field =
> MaterializedField.create(toFieldName(column.getPath()),
> >> type);
> >> >          // the field was not requested to be read
> >> >          if ( ! fieldSelected(field)) continue;
> >> >
> >> > -        //convertedTypes.put()
> >> >          fieldFixedLength = column.getType() !=
> >> PrimitiveType.PrimitiveTypeName.BINARY;
> >> >          v = output.addField(field, (Class<? extends ValueVector>)
> >> TypeHelper.getValueVectorClass(type.getMinorType(), type.getMode()));
> >> >          if (column.getType() !=
> PrimitiveType.PrimitiveTypeName.BINARY)
> >> {
> >> >            createFixedColumnReader(fieldFixedLength, column,
> >> columnChunkMetaData, recordsPerBatch, v,
> >> > -            convertedType);
> >> > +            schemaElement);
> >> >          } else {
> >> > -          if (column.getMaxDefinitionLevel() == 0){// column is
> required
> >> > -            if (convertedType == ConvertedType.UTF8) {
> >> > -              varLengthColumns.add(
> >> > -                new VarCharColumn(this, -1, column,
> >> columnChunkMetaData, false, (VarCharVector) v, convertedType));
> >> > -            } else {
> >> > -              varLengthColumns.add(
> >> > -                  new VarBinaryColumn(this, -1, column,
> >> columnChunkMetaData, false, (VarBinaryVector) v, convertedType));
> >> > -            }
> >> > -          }
> >> > -          else{
> >> > -            if (convertedType == ConvertedType.UTF8) {
> >> > -              nullableVarLengthColumns.add(
> >> > -                new NullableVarCharColumn(this, -1, column,
> >> columnChunkMetaData, false,
> >> > -                    (NullableVarCharVector) v, convertedType));
> >> > -            } else {
> >> > -              nullableVarLengthColumns.add(
> >> > -                new NullableVarBinaryColumn(this, -1, column,
> >> columnChunkMetaData, false,
> >> > -                  (NullableVarBinaryVector) v, convertedType));
> >> > -            }
> >> > -          }
> >> > +          // create a reader and add it to the appropriate list
> >> > +          getReader(this, -1, column, columnChunkMetaData, false, v,
> >> schemaElement, varLengthColumns, nullableVarLengthColumns);
> >> >          }
> >> >        }
> >> >        varLengthReader = new VarLenBinaryReader(this,
> varLengthColumns,
> >> nullableVarLengthColumns);
> >> > @@ -338,28 +321,41 @@ public class ParquetRecordReader implements
> >> RecordReader {
> >> >     */
> >> >    private boolean createFixedColumnReader(boolean fixedLength,
> >> ColumnDescriptor descriptor,
> >> >                                            ColumnChunkMetaData
> >> columnChunkMetaData, int allocateSize, ValueVector v,
> >> > -                                          ConvertedType
> convertedType)
> >> > +                                          SchemaElement
> schemaElement)
> >> >        throws SchemaChangeException, ExecutionSetupException {
> >> > +    ConvertedType convertedType = schemaElement.getConverted_type();
> >> >      // if the column is required
> >> >      if (descriptor.getMaxDefinitionLevel() == 0){
> >> >        if (columnChunkMetaData.getType() ==
> >> PrimitiveType.PrimitiveTypeName.BOOLEAN){
> >> >          columnStatuses.add(new BitReader(this, allocateSize,
> >> descriptor, columnChunkMetaData,
> >> > -            fixedLength, v, convertedType));
> >> > -      }
> >> > -      else{
> >> > +            fixedLength, v, schemaElement));
> >> > +      } else if (columnChunkMetaData.getType() ==
> >> PrimitiveTypeName.FIXED_LEN_BYTE_ARRAY && convertedType ==
> >> ConvertedType.DECIMAL){
> >> > +        int length = schemaElement.type_length;
> >> > +        if (length <= 12) {
> >> > +          columnStatuses.add(new Decimal28Reader(this, allocateSize,
> >> descriptor, columnChunkMetaData, fixedLength, v, schemaElement));
> >> > +        } else if (length <= 16) {
> >> > +          columnStatuses.add(new Decimal38Reader(this, allocateSize,
> >> descriptor, columnChunkMetaData, fixedLength, v, schemaElement));
> >> > +        }
> >> > +      } else {
> >> >          columnStatuses.add(new FixedByteAlignedReader(this,
> >> allocateSize, descriptor, columnChunkMetaData,
> >> > -            fixedLength, v, convertedType));
> >> > +            fixedLength, v, schemaElement));
> >> >        }
> >> >        return true;
> >> >      }
> >> >      else { // if the column is nullable
> >> >        if (columnChunkMetaData.getType() ==
> >> PrimitiveType.PrimitiveTypeName.BOOLEAN){
> >> >          columnStatuses.add(new NullableBitReader(this, allocateSize,
> >> descriptor, columnChunkMetaData,
> >> > -            fixedLength, v, convertedType));
> >> > -      }
> >> > -      else{
> >> > +            fixedLength, v, schemaElement));
> >> > +      } else if (columnChunkMetaData.getType() ==
> >> PrimitiveTypeName.FIXED_LEN_BYTE_ARRAY && convertedType ==
> >> ConvertedType.DECIMAL){
> >> > +        int length = schemaElement.type_length;
> >> > +        if (length <= 12) {
> >> > +          columnStatuses.add(new NullableDecimal28Reader(this,
> >> allocateSize, descriptor, columnChunkMetaData, fixedLength, v,
> >> schemaElement));
> >> > +        } else if (length <= 16) {
> >> > +          columnStatuses.add(new NullableDecimal38Reader(this,
> >> allocateSize, descriptor, columnChunkMetaData, fixedLength, v,
> >> schemaElement));
> >> > +        }
> >> > +      } else {
> >> >          columnStatuses.add(new NullableFixedByteAlignedReader(this,
> >> allocateSize, descriptor, columnChunkMetaData,
> >> > -            fixedLength, v, convertedType));
> >> > +            fixedLength, v, schemaElement));
> >> >        }
> >> >        return true;
> >> >      }
> >> > @@ -420,25 +416,57 @@ public class ParquetRecordReader implements
> >> RecordReader {
> >> >    }
> >> >
> >> >    static TypeProtos.MajorType
> >> toMajorType(PrimitiveType.PrimitiveTypeName primitiveTypeName,
> >> > -                                               TypeProtos.DataMode
> >> mode, ConvertedType convertedType) {
> >> > -    return toMajorType(primitiveTypeName, 0, mode, convertedType);
> >> > +                                               TypeProtos.DataMode
> >> mode, SchemaElement schemaElement) {
> >> > +    return toMajorType(primitiveTypeName, 0, mode, schemaElement);
> >> >    }
> >> >
> >> >    static TypeProtos.MajorType
> >> toMajorType(PrimitiveType.PrimitiveTypeName primitiveTypeName, int
> length,
> >> > -                                               TypeProtos.DataMode
> >> mode, ConvertedType convertedType) {
> >> > +                                               TypeProtos.DataMode
> >> mode, SchemaElement schemaElement) {
> >> > +    ConvertedType convertedType = schemaElement.getConverted_type();
> >> >      switch (mode) {
> >> >
> >> >        case OPTIONAL:
> >> >          switch (primitiveTypeName) {
> >> >            case BINARY:
> >> > -            if (convertedType == ConvertedType.UTF8)
> >> > -              return Types.optional(TypeProtos.MinorType.VARCHAR);
> >> > -            else
> >> > +            if (convertedType == null) {
> >> >                return Types.optional(TypeProtos.MinorType.VARBINARY);
> >> > +            }
> >> > +            switch (convertedType) {
> >> > +              case UTF8:
> >> > +                return Types.optional(MinorType.VARCHAR);
> >> > +              case DECIMAL:
> >> > +                return
> >> Types.withScaleAndPrecision(getDecimalType(schemaElement),
> >> DataMode.OPTIONAL, schemaElement.getScale(),
> schemaElement.getPrecision());
> >> > +              default:
> >> > +                throw new
> >> UnsupportedOperationException(String.format("unsupported type: %s %s",
> >> primitiveTypeName, convertedType));
> >> > +            }
> >> >            case INT64:
> >> > -            return Types.optional(TypeProtos.MinorType.BIGINT);
> >> > +            if (convertedType == null) {
> >> > +              return Types.optional(TypeProtos.MinorType.BIGINT);
> >> > +            }
> >> > +            switch(convertedType) {
> >> > +              case DECIMAL:
> >> > +                return
> Types.withScaleAndPrecision(MinorType.DECIMAL18,
> >> DataMode.OPTIONAL, schemaElement.getScale(),
> schemaElement.getPrecision());
> >> > +              case FINETIME:
> >> > +                throw new UnsupportedOperationException();
> >> > +              case TIMESTAMP:
> >> > +                return Types.optional(MinorType.TIMESTAMP);
> >> > +              default:
> >> > +                throw new
> >> UnsupportedOperationException(String.format("unsupported type: %s %s",
> >> primitiveTypeName, convertedType));
> >> > +            }
> >> >            case INT32:
> >> > -            return Types.optional(TypeProtos.MinorType.INT);
> >> > +            if (convertedType == null) {
> >> > +              return Types.optional(TypeProtos.MinorType.INT);
> >> > +            }
> >> > +            switch(convertedType) {
> >> > +              case DECIMAL:
> >> > +                return
> Types.withScaleAndPrecision(MinorType.DECIMAL9,
> >> DataMode.OPTIONAL, schemaElement.getScale(),
> schemaElement.getPrecision());
> >> > +              case DATE:
> >> > +                return Types.optional(MinorType.DATE);
> >> > +              case TIME:
> >> > +                return Types.optional(MinorType.TIME);
> >> > +              default:
> >> > +                throw new
> >> UnsupportedOperationException(String.format("unsupported type: %s %s",
> >> primitiveTypeName, convertedType));
> >> > +            }
> >> >            case BOOLEAN:
> >> >              return Types.optional(TypeProtos.MinorType.BIT);
> >> >            case FLOAT:
> >> > @@ -451,23 +479,58 @@ public class ParquetRecordReader implements
> >> RecordReader {
> >> >              return
> >>
> TypeProtos.MajorType.newBuilder().setMinorType(TypeProtos.MinorType.FIXEDBINARY).setWidth(12)
> >> >                  .setMode(mode).build();
> >> >            case FIXED_LEN_BYTE_ARRAY:
> >> > -            checkArgument(length > 0, "A length greater than zero
> must
> >> be provided for a FixedBinary type.");
> >> > -            return
> >>
> TypeProtos.MajorType.newBuilder().setMinorType(TypeProtos.MinorType.FIXEDBINARY)
> >> > -                .setWidth(length).setMode(mode).build();
> >> > +            if (convertedType == null) {
> >> > +              checkArgument(length > 0, "A length greater than zero
> >> must be provided for a FixedBinary type.");
> >> > +              return
> >>
> TypeProtos.MajorType.newBuilder().setMinorType(TypeProtos.MinorType.FIXEDBINARY)
> >> > +                      .setWidth(length).setMode(mode).build();
> >> > +            } else if (convertedType == ConvertedType.DECIMAL) {
> >> > +              return
> >> Types.withScaleAndPrecision(getDecimalType(schemaElement),
> >> DataMode.OPTIONAL, schemaElement.getScale(),
> schemaElement.getPrecision());
> >> > +            }
> >> >            default:
> >> >              throw new UnsupportedOperationException("Type not
> >> supported: " + primitiveTypeName);
> >> >          }
> >> >        case REQUIRED:
> >> >          switch (primitiveTypeName) {
> >> >            case BINARY:
> >> > -            if (convertedType == ConvertedType.UTF8)
> >> > -              return Types.required(TypeProtos.MinorType.VARCHAR);
> >> > -            else
> >> > +            if (convertedType == null) {
> >> >                return Types.required(TypeProtos.MinorType.VARBINARY);
> >> > +            }
> >> > +            switch (convertedType) {
> >> > +              case UTF8:
> >> > +                return Types.required(MinorType.VARCHAR);
> >> > +              case DECIMAL:
> >> > +                return
> >> Types.withScaleAndPrecision(getDecimalType(schemaElement),
> >> DataMode.REQUIRED, schemaElement.getScale(),
> schemaElement.getPrecision());
> >> > +              default:
> >> > +                throw new
> >> UnsupportedOperationException(String.format("unsupported type: %s %s",
> >> primitiveTypeName, convertedType));
> >> > +            }
> >> >            case INT64:
> >> > -            return Types.required(TypeProtos.MinorType.BIGINT);
> >> > +            if (convertedType == null) {
> >> > +              return Types.required(MinorType.BIGINT);
> >> > +            }
> >> > +            switch(convertedType) {
> >> > +              case DECIMAL:
> >> > +                return
> Types.withScaleAndPrecision(MinorType.DECIMAL18,
> >> DataMode.REQUIRED, schemaElement.getScale(),
> schemaElement.getPrecision());
> >> > +              case FINETIME:
> >> > +                throw new UnsupportedOperationException();
> >> > +              case TIMESTAMP:
> >> > +                return Types.required(MinorType.TIMESTAMP);
> >> > +              default:
> >> > +                throw new
> >> UnsupportedOperationException(String.format("unsupported type: %s %s",
> >> primitiveTypeName, convertedType));
> >> > +            }
> >> >            case INT32:
> >> > -            return Types.required(TypeProtos.MinorType.INT);
> >> > +            if (convertedType == null) {
> >> > +              return Types.required(MinorType.INT);
> >> > +            }
> >> > +            switch(convertedType) {
> >> > +              case DECIMAL:
> >> > +                return
> Types.withScaleAndPrecision(MinorType.DECIMAL9,
> >> DataMode.REQUIRED, schemaElement.getScale(),
> schemaElement.getPrecision());
> >> > +              case DATE:
> >> > +                return Types.required(MinorType.DATE);
> >> > +              case TIME:
> >> > +                return Types.required(MinorType.TIME);
> >> > +              default:
> >> > +                throw new
> >> UnsupportedOperationException(String.format("unsupported type: %s %s",
> >> primitiveTypeName, convertedType));
> >> > +            }
> >> >            case BOOLEAN:
> >> >              return Types.required(TypeProtos.MinorType.BIT);
> >> >            case FLOAT:
> >> > @@ -480,23 +543,58 @@ public class ParquetRecordReader implements
> >> RecordReader {
> >> >              return
> >>
> TypeProtos.MajorType.newBuilder().setMinorType(TypeProtos.MinorType.FIXEDBINARY).setWidth(12)
> >> >                  .setMode(mode).build();
> >> >            case FIXED_LEN_BYTE_ARRAY:
> >> > -            checkArgument(length > 0, "A length greater than zero
> must
> >> be provided for a FixedBinary type.");
> >> > -            return
> >>
> TypeProtos.MajorType.newBuilder().setMinorType(TypeProtos.MinorType.FIXEDBINARY)
> >> > -                .setWidth(length).setMode(mode).build();
> >> > +            if (convertedType == null) {
> >> > +              checkArgument(length > 0, "A length greater than zero
> >> must be provided for a FixedBinary type.");
> >> > +              return
> >>
> TypeProtos.MajorType.newBuilder().setMinorType(TypeProtos.MinorType.FIXEDBINARY)
> >> > +                      .setWidth(length).setMode(mode).build();
> >> > +            } else if (convertedType == ConvertedType.DECIMAL) {
> >> > +              return
> >> Types.withScaleAndPrecision(getDecimalType(schemaElement),
> >> DataMode.REQUIRED, schemaElement.getScale(),
> schemaElement.getPrecision());
> >> > +            }
> >> >            default:
> >> >              throw new UnsupportedOperationException("Type not
> >> supported: " + primitiveTypeName);
> >> >          }
> >> >        case REPEATED:
> >> >          switch (primitiveTypeName) {
> >> >            case BINARY:
> >> > -            if (convertedType == ConvertedType.UTF8)
> >> > -              return Types.required(TypeProtos.MinorType.VARCHAR);
> >> > -            else
> >> > +            if (convertedType == null) {
> >> >                return Types.repeated(TypeProtos.MinorType.VARBINARY);
> >> > +            }
> >> > +            switch (schemaElement.getConverted_type()) {
> >> > +              case UTF8:
> >> > +                return Types.repeated(MinorType.VARCHAR);
> >> > +              case DECIMAL:
> >> > +                return
> >> Types.withScaleAndPrecision(getDecimalType(schemaElement),
> >> DataMode.REPEATED, schemaElement.getScale(),
> schemaElement.getPrecision());
> >> > +              default:
> >> > +                throw new
> >> UnsupportedOperationException(String.format("unsupported type: %s %s",
> >> primitiveTypeName, convertedType));
> >> > +            }
> >> >            case INT64:
> >> > -            return Types.repeated(TypeProtos.MinorType.BIGINT);
> >> > +            if (convertedType == null) {
> >> > +              return Types.repeated(MinorType.BIGINT);
> >> > +            }
> >> > +            switch(convertedType) {
> >> > +              case DECIMAL:
> >> > +                return
> Types.withScaleAndPrecision(MinorType.DECIMAL18,
> >> DataMode.REPEATED, schemaElement.getScale(),
> schemaElement.getPrecision());
> >> > +              case FINETIME:
> >> > +                throw new UnsupportedOperationException();
> >> > +              case TIMESTAMP:
> >> > +                return Types.repeated(MinorType.TIMESTAMP);
> >> > +              default:
> >> > +                throw new
> >> UnsupportedOperationException(String.format("unsupported type: %s %s",
> >> primitiveTypeName, convertedType));
> >> > +            }
> >> >            case INT32:
> >> > -            return Types.repeated(TypeProtos.MinorType.INT);
> >> > +            if (convertedType == null) {
> >> > +              return Types.repeated(MinorType.INT);
> >> > +            }
> >> > +            switch(convertedType) {
> >> > +              case DECIMAL:
> >> > +                return
> Types.withScaleAndPrecision(MinorType.DECIMAL9,
> >> DataMode.REPEATED, schemaElement.getScale(),
> schemaElement.getPrecision());
> >> > +              case DATE:
> >> > +                return Types.repeated(MinorType.DATE);
> >> > +              case TIME:
> >> > +                return Types.repeated(MinorType.TIME);
> >> > +              default:
> >> > +                throw new
> >> UnsupportedOperationException(String.format("unsupported type: %s %s",
> >> primitiveTypeName, convertedType));
> >> > +            }
> >> >            case BOOLEAN:
> >> >              return Types.repeated(TypeProtos.MinorType.BIT);
> >> >            case FLOAT:
> >> > @@ -509,9 +607,13 @@ public class ParquetRecordReader implements
> >> RecordReader {
> >> >              return
> >>
> TypeProtos.MajorType.newBuilder().setMinorType(TypeProtos.MinorType.FIXEDBINARY).setWidth(12)
> >> >                  .setMode(mode).build();
> >> >            case FIXED_LEN_BYTE_ARRAY:
> >> > -            checkArgument(length > 0, "A length greater than zero
> must
> >> be provided for a FixedBinary type.");
> >> > -            return
> >>
> TypeProtos.MajorType.newBuilder().setMinorType(TypeProtos.MinorType.FIXEDBINARY)
> >> > -                .setWidth(length).setMode(mode).build();
> >> > +            if (convertedType == null) {
> >> > +              checkArgument(length > 0, "A length greater than zero
> >> must be provided for a FixedBinary type.");
> >> > +              return
> >>
> TypeProtos.MajorType.newBuilder().setMinorType(TypeProtos.MinorType.FIXEDBINARY)
> >> > +                      .setWidth(length).setMode(mode).build();
> >> > +            } else if (convertedType == ConvertedType.DECIMAL) {
> >> > +              return
> >> Types.withScaleAndPrecision(getDecimalType(schemaElement),
> >> DataMode.REPEATED, schemaElement.getScale(),
> schemaElement.getPrecision());
> >> > +            }
> >> >            default:
> >> >              throw new UnsupportedOperationException("Type not
> >> supported: " + primitiveTypeName);
> >> >          }
> >> > @@ -519,6 +621,58 @@ public class ParquetRecordReader implements
> >> RecordReader {
> >> >      throw new UnsupportedOperationException("Type not supported: " +
> >> primitiveTypeName + " Mode: " + mode);
> >> >    }
> >> >
> >> > +  private static void getReader(ParquetRecordReader parentReader, int
> >> allocateSize, ColumnDescriptor descriptor,
> >> > +                                        ColumnChunkMetaData
> >> columnChunkMetaData, boolean fixedLength, ValueVector v,
> >> > +                                        SchemaElement schemaElement,
> >> List<VarLengthColumn> varLengthColumns,
> >> > +                                        List<NullableVarLengthColumn>
> >> nullableVarLengthColumns) throws ExecutionSetupException {
> >> > +    ConvertedType convertedType = schemaElement.getConverted_type();
> >> > +    switch (descriptor.getMaxDefinitionLevel()) {
> >> > +      case 0:
> >> > +        if (convertedType == null) {
> >> > +          varLengthColumns.add(new VarBinaryColumn(parentReader,
> >> allocateSize, descriptor, columnChunkMetaData, fixedLength,
> >> (VarBinaryVector) v, schemaElement));
> >> > +          return;
> >> > +        }
> >> > +        switch (convertedType) {
> >> > +          case UTF8:
> >> > +            varLengthColumns.add(new VarCharColumn(parentReader,
> >> allocateSize, descriptor, columnChunkMetaData, fixedLength,
> (VarCharVector)
> >> v, schemaElement));
> >> > +            return;
> >> > +          case DECIMAL:
> >> > +            if (v instanceof Decimal28SparseVector) {
> >> > +              varLengthColumns.add(new Decimal28Column(parentReader,
> >> allocateSize, descriptor, columnChunkMetaData, fixedLength,
> >> (Decimal28SparseVector) v, schemaElement));
> >> > +              return;
> >> > +            } else if (v instanceof Decimal38SparseVector) {
> >> > +              varLengthColumns.add(new Decimal38Column(parentReader,
> >> allocateSize, descriptor, columnChunkMetaData, fixedLength,
> >> (Decimal38SparseVector) v, schemaElement));
> >> > +              return;
> >> > +            }
> >> > +          default:
> >> > +        }
> >> > +      default:
> >> > +        if (convertedType == null) {
> >> > +          nullableVarLengthColumns.add(new
> >> NullableVarBinaryColumn(parentReader, allocateSize, descriptor,
> >> columnChunkMetaData, fixedLength, (NullableVarBinaryVector) v,
> >> schemaElement));
> >> > +          return;
> >> > +        }
> >> > +        switch (convertedType) {
> >> > +          case UTF8:
> >> > +            nullableVarLengthColumns.add(new
> >> NullableVarCharColumn(parentReader, allocateSize, descriptor,
> >> columnChunkMetaData, fixedLength, (NullableVarCharVector) v,
> >> schemaElement));
> >> > +            return;
> >> > +          case DECIMAL:
> >> > +            if (v instanceof NullableDecimal28SparseVector) {
> >> > +              nullableVarLengthColumns.add(new
> >> NullableDecimal28Column(parentReader, allocateSize, descriptor,
> >> columnChunkMetaData, fixedLength, (NullableDecimal28SparseVector) v,
> >> schemaElement));
> >> > +              return;
> >> > +            } else if (v instanceof NullableDecimal38SparseVector) {
> >> > +              nullableVarLengthColumns.add(new
> >> NullableDecimal38Column(parentReader, allocateSize, descriptor,
> >> columnChunkMetaData, fixedLength, (NullableDecimal38SparseVector) v,
> >> schemaElement));
> >> > +              return;
> >> > +            }
> >> > +          default:
> >> > +        }
> >> > +    }
> >> > +    throw new UnsupportedOperationException();
> >> > +  }
> >> > +
> >> > +  private static MinorType getDecimalType(SchemaElement
> schemaElement) {
> >> > +    return schemaElement.getPrecision() <= 28 ?
> >> MinorType.DECIMAL28SPARSE : MinorType.DECIMAL38SPARSE;
> >> > +  }
> >> > +
> >> >    static String join(String delimiter, String... str) {
> >> >      StringBuilder builder = new StringBuilder();
> >> >      int i = 0;
> >> >
> >> >
> >>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/63b03467/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetRecordWriter.java
> >> > ----------------------------------------------------------------------
> >> > diff --git
> >>
> a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetRecordWriter.java
> >>
> b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetRecordWriter.java
> >> > new file mode 100644
> >> > index 0000000..3ba13f4
> >> > --- /dev/null
> >> > +++
> >>
> b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetRecordWriter.java
> >> > @@ -0,0 +1,185 @@
> >> > +/**
> >> > + * Licensed to the Apache Software Foundation (ASF) under one
> >> > + * or more contributor license agreements.  See the NOTICE file
> >> > + * distributed with this work for additional information
> >> > + * regarding copyright ownership.  The ASF licenses this file
> >> > + * to you under the Apache License, Version 2.0 (the
> >> > + * "License"); you may not use this file except in compliance
> >> > + * with the License.  You may obtain a copy of the License at
> >> > + *
> >> > + * http://www.apache.org/licenses/LICENSE-2.0
> >> > + *
> >> > + * Unless required by applicable law or agreed to in writing,
> software
> >> > + * distributed under the License is distributed on an "AS IS" BASIS,
> >> > + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
> >> implied.
> >> > + * See the License for the specific language governing permissions
> and
> >> > + * limitations under the License.
> >> > + */
> >> > +package org.apache.drill.exec.store.parquet;
> >> > +
> >> > +import com.google.common.collect.Lists;
> >> > +import org.apache.drill.common.types.TypeProtos.MinorType;
> >> > +import org.apache.drill.exec.ExecConstants;
> >> > +import org.apache.drill.exec.record.BatchSchema;
> >> > +import org.apache.drill.exec.record.MaterializedField;
> >> > +import org.apache.drill.exec.store.ParquetOutputRecordWriter;
> >> > +import org.apache.hadoop.conf.Configuration;
> >> > +import org.apache.hadoop.fs.FileSystem;
> >> > +import org.apache.hadoop.fs.Path;
> >> > +import parquet.column.ParquetProperties.WriterVersion;
> >> > +import parquet.column.impl.ColumnWriteStoreImpl;
> >> > +import parquet.column.page.PageWriteStore;
> >> > +import parquet.hadoop.ColumnChunkPageWriteStoreExposer;
> >> > +import parquet.hadoop.ParquetFileWriter;
> >> > +import parquet.hadoop.metadata.CompressionCodecName;
> >> > +import parquet.io.ColumnIOFactory;
> >> > +import parquet.io.MessageColumnIO;
> >> > +import parquet.io.api.RecordConsumer;
> >> > +import parquet.schema.DecimalMetadata;
> >> > +import parquet.schema.MessageType;
> >> > +import parquet.schema.OriginalType;
> >> > +import parquet.schema.PrimitiveType.PrimitiveTypeName;
> >> > +import parquet.schema.Type;
> >> > +import parquet.schema.Type.Repetition;
> >> > +
> >> > +import java.io.IOException;
> >> > +import java.util.HashMap;
> >> > +import java.util.List;
> >> > +import java.util.Map;
> >> > +
> >> > +import static java.lang.Math.max;
> >> > +import static java.lang.Math.min;
> >> > +import static java.lang.String.format;
> >> > +
> >> > +public class ParquetRecordWriter extends ParquetOutputRecordWriter {
> >> > +
> >> > +  private static final int MINIMUM_BUFFER_SIZE = 64 * 1024;
> >> > +  private static final int MINIMUM_RECORD_COUNT_FOR_CHECK = 100;
> >> > +  private static final int MAXIMUM_RECORD_COUNT_FOR_CHECK = 10000;
> >> > +
> >> > +  private ParquetFileWriter w;
> >> > +  private MessageType schema;
> >> > +  private Map<String, String> extraMetaData = new HashMap();
> >> > +  private int blockSize;
> >> > +  private int pageSize = 1 * 1024 * 1024;
> >> > +  private int dictionaryPageSize = pageSize;
> >> > +  private boolean enableDictionary = false;
> >> > +  private boolean validating = false;
> >> > +  private CompressionCodecName codec = CompressionCodecName.SNAPPY;
> >> > +  private WriterVersion writerVersion = WriterVersion.PARQUET_1_0;
> >> > +
> >> > +  private long recordCount = 0;
> >> > +  private long recordCountForNextMemCheck =
> >> MINIMUM_RECORD_COUNT_FOR_CHECK;
> >> > +
> >> > +  private ColumnWriteStoreImpl store;
> >> > +  private PageWriteStore pageStore;
> >> > +
> >> > +  private RecordConsumer consumer;
> >> > +  private BatchSchema batchSchema;
> >> > +
> >> > +  private Configuration conf;
> >> > +  private String location;
> >> > +  private String prefix;
> >> > +  private int index = 0;
> >> > +
> >> > +  @Override
> >> > +  public void init(Map<String, String> writerOptions) throws
> >> IOException {
> >> > +    this.location = writerOptions.get("location");
> >> > +    this.prefix = writerOptions.get("prefix");
> >> > +
> >> > +    conf = new Configuration();
> >> > +    conf.set(FileSystem.FS_DEFAULT_NAME_KEY,
> >> writerOptions.get(FileSystem.FS_DEFAULT_NAME_KEY));
> >> > +    blockSize =
> >> Integer.parseInt(writerOptions.get(ExecConstants.PARQUET_BLOCK_SIZE));
> >> > +  }
> >> > +
> >> > +  @Override
> >> > +  public void updateSchema(BatchSchema batchSchema) throws
> IOException {
> >> > +    if (this.batchSchema == null ||
> >> !this.batchSchema.equals(batchSchema)) {
> >> > +      if (this.batchSchema != null) {
> >> > +        flush();
> >> > +      }
> >> > +      this.batchSchema = batchSchema;
> >> > +      newSchema();
> >> > +    }
> >> > +  }
> >> > +
> >> > +  private void newSchema() throws IOException {
> >> > +    List<Type> types = Lists.newArrayList();
> >> > +    for (MaterializedField field : batchSchema) {
> >> > +      String name = field.getAsSchemaPath().getAsUnescapedPath();
> >> > +      MinorType minorType = field.getType().getMinorType();
> >> > +      PrimitiveTypeName primitiveTypeName =
> >> ParquetTypeHelper.getPrimitiveTypeNameForMinorType(minorType);
> >> > +      Repetition repetition =
> >> ParquetTypeHelper.getRepetitionForDataMode(field.getDataMode());
> >> > +      OriginalType originalType =
> >> ParquetTypeHelper.getOriginalTypeForMinorType(minorType);
> >> > +      DecimalMetadata decimalMetadata =
> >> ParquetTypeHelper.getDecimalMetadataForField(field);
> >> > +      int length =
> ParquetTypeHelper.getLengthForMinorType(minorType);
> >> > +      parquet.schema.Type type = new
> >> parquet.schema.PrimitiveType(repetition, primitiveTypeName, length,
> name,
> >> originalType, decimalMetadata);
> >> > +      types.add(type);
> >> > +    }
> >> > +    schema = new MessageType("root", types);
> >> > +
> >> > +    Path fileName = new Path(location, prefix + "_" + index +
> >> ".parquet");
> >> > +    w = new ParquetFileWriter(conf, schema, fileName);
> >> > +    w.start();
> >> > +
> >> > +    int initialBlockBufferSize = max(MINIMUM_BUFFER_SIZE, blockSize /
> >> this.schema.getColumns().size() / 5);
> >> > +    pageStore =
> >> ColumnChunkPageWriteStoreExposer.newColumnChunkPageWriteStore(codec,
> >> pageSize, this.schema, initialBlockBufferSize);
> >> > +    int initialPageBufferSize = max(MINIMUM_BUFFER_SIZE,
> min(pageSize +
> >> pageSize / 10, initialBlockBufferSize));
> >> > +    store = new ColumnWriteStoreImpl(pageStore, pageSize,
> >> initialPageBufferSize, dictionaryPageSize, enableDictionary,
> writerVersion);
> >> > +    MessageColumnIO columnIO = new
> >> ColumnIOFactory(validating).getColumnIO(this.schema);
> >> > +    consumer = columnIO.getRecordWriter(store);
> >> > +    setUp(schema, consumer);
> >> > +  }
> >> > +
> >> > +  private void flush() throws IOException {
> >> > +    w.startBlock(recordCount);
> >> > +    store.flush();
> >> > +    ColumnChunkPageWriteStoreExposer.flushPageStore(pageStore, w);
> >> > +    recordCount = 0;
> >> > +    w.endBlock();
> >> > +    w.end(extraMetaData);
> >> > +    store = null;
> >> > +    pageStore = null;
> >> > +    index++;
> >> > +  }
> >> > +
> >> > +  private void checkBlockSizeReached() throws IOException {
> >> > +    if (recordCount >= recordCountForNextMemCheck) { // checking the
> >> memory size is relatively expensive, so let's not do it for every
> record.
> >> > +      long memSize = store.memSize();
> >> > +      if (memSize > blockSize) {
> >> > +        System.out.println("Reached block size " + blockSize);
> >> > +        flush();
> >> > +        newSchema();
> >> > +        recordCountForNextMemCheck =
> >> min(max(MINIMUM_RECORD_COUNT_FOR_CHECK, recordCount / 2),
> >> MAXIMUM_RECORD_COUNT_FOR_CHECK);
> >> > +      } else {
> >> > +        float recordSize = (float) memSize / recordCount;
> >> > +        recordCountForNextMemCheck = min(
> >> > +                max(MINIMUM_RECORD_COUNT_FOR_CHECK, (recordCount +
> >> (long)(blockSize / recordSize)) / 2), // will check halfway
> >> > +                recordCount + MAXIMUM_RECORD_COUNT_FOR_CHECK // will
> >> not look more than max records ahead
> >> > +        );
> >> > +      }
> >> > +    }
> >> > +  }
> >> > +
> >> > +  @Override
> >> > +  public void startRecord() throws IOException {
> >> > +    consumer.startMessage();
> >> > +  }
> >> > +
> >> > +  @Override
> >> > +  public void endRecord() throws IOException {
> >> > +    consumer.endMessage();
> >> > +    recordCount++;
> >> > +    checkBlockSizeReached();
> >> > +  }
> >> > +
> >> > +  @Override
> >> > +  public void abort() throws IOException {
> >> > +    //To change body of implemented methods use File | Settings |
> File
> >> Templates.
> >> > +  }
> >> > +
> >> > +  @Override
> >> > +  public void cleanup() throws IOException {
> >> > +    flush();
> >> > +  }
> >> > +}
> >> >
> >> >
> >>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/63b03467/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetWriter.java
> >> > ----------------------------------------------------------------------
> >> > diff --git
> >>
> a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetWriter.java
> >>
> b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetWriter.java
> >> > new file mode 100644
> >> > index 0000000..ea9798a
> >> > --- /dev/null
> >> > +++
> >>
> b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetWriter.java
> >> > @@ -0,0 +1,91 @@
> >> > +/**
> >> > + * Licensed to the Apache Software Foundation (ASF) under one
> >> > + * or more contributor license agreements.  See the NOTICE file
> >> > + * distributed with this work for additional information
> >> > + * regarding copyright ownership.  The ASF licenses this file
> >> > + * to you under the Apache License, Version 2.0 (the
> >> > + * "License"); you may not use this file except in compliance
> >> > + * with the License.  You may obtain a copy of the License at
> >> > + *
> >> > + * http://www.apache.org/licenses/LICENSE-2.0
> >> > + *
> >> > + * Unless required by applicable law or agreed to in writing,
> software
> >> > + * distributed under the License is distributed on an "AS IS" BASIS,
> >> > + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
> >> implied.
> >> > + * See the License for the specific language governing permissions
> and
> >> > + * limitations under the License.
> >> > + */
> >> > +package org.apache.drill.exec.store.parquet;
> >> > +
> >> > +import com.fasterxml.jackson.annotation.*;
> >> > +import com.google.common.base.Preconditions;
> >> > +import org.apache.drill.common.exceptions.ExecutionSetupException;
> >> > +import org.apache.drill.common.logical.FormatPluginConfig;
> >> > +import org.apache.drill.common.logical.StoragePluginConfig;
> >> > +import org.apache.drill.exec.physical.OperatorCost;
> >> > +import org.apache.drill.exec.physical.base.AbstractWriter;
> >> > +import org.apache.drill.exec.physical.base.PhysicalOperator;
> >> > +import org.apache.drill.exec.store.StoragePluginRegistry;
> >> > +
> >> > +import java.io.IOException;
> >> > +
> >> > +@JsonTypeName("parquet-writer")
> >> > +public class ParquetWriter extends AbstractWriter {
> >> > +  static final org.slf4j.Logger logger =
> >> org.slf4j.LoggerFactory.getLogger(ParquetWriter.class);
> >> > +
> >> > +  private final String location;
> >> > +  private final ParquetFormatPlugin formatPlugin;
> >> > +
> >> > +  @JsonCreator
> >> > +  public ParquetWriter(
> >> > +          @JsonProperty("child") PhysicalOperator child,
> >> > +          @JsonProperty("location") String location,
> >> > +          @JsonProperty("storage") StoragePluginConfig storageConfig,
> >> > +          @JacksonInject StoragePluginRegistry engineRegistry) throws
> >> IOException, ExecutionSetupException {
> >> > +
> >> > +    super(child);
> >> > +    this.formatPlugin = (ParquetFormatPlugin)
> >> engineRegistry.getFormatPlugin(storageConfig, new
> ParquetFormatConfig());
> >> > +    Preconditions.checkNotNull(formatPlugin, "Unable to load format
> >> plugin for provided format config.");
> >> > +    this.location = location;
> >> > +  }
> >> > +
> >> > +  public ParquetWriter(PhysicalOperator child,
> >> > +                       String location,
> >> > +                       ParquetFormatPlugin formatPlugin) {
> >> > +
> >> > +    super(child);
> >> > +    this.formatPlugin = formatPlugin;
> >> > +    this.location = location;
> >> > +  }
> >> > +
> >> > +  @JsonProperty("location")
> >> > +  public String getLocation() {
> >> > +    return location;
> >> > +  }
> >> > +
> >> > +  @JsonProperty("storage")
> >> > +  public StoragePluginConfig getStorageConfig(){
> >> > +    return formatPlugin.getStorageConfig();
> >> > +  }
> >> > +
> >> > +  @JsonProperty("format")
> >> > +  public FormatPluginConfig getFormatConfig(){
> >> > +    return formatPlugin.getConfig();
> >> > +  }
> >> > +
> >> > +  @JsonIgnore
> >> > +  public ParquetFormatPlugin getFormatPlugin(){
> >> > +    return formatPlugin;
> >> > +  }
> >> > +
> >> > +  @Override
> >> > +  protected PhysicalOperator getNewWithChild(PhysicalOperator child)
> {
> >> > +    return new ParquetWriter(child, location, formatPlugin);
> >> > +  }
> >> > +
> >> > +  @Override
> >> > +  public OperatorCost getCost() {
> >> > +    // TODO:
> >> > +    return new OperatorCost(1,1,1,1);
> >> > +  }
> >> > +}
> >> >
> >>
> >
> >
> >
> > --
> >  Steven Phillips
> >  Software Engineer
> >
> >  mapr.com
>

Re: [3/3] git commit: DRILL-694: Implement parquet writer.

Posted by Timothy Chen <tn...@gmail.com>.
It failed twice in two runs, not sure how intermittently are you
seeing the failure.

Tim



On Mon, May 12, 2014 at 11:38 PM, Steven Phillips
<sp...@maprtech.com> wrote:
> Do you see the failure repeatedly, even if you run just that single test?
>
> I sometimes see tests that fail intermittently, but pass when run in
> isolation.
>
> I will try the test again and confirm that it is passing.
>
>
> On Mon, May 12, 2014 at 10:22 PM, Timothy Chen <tn...@gmail.com> wrote:
>
>> I got a unit test failure on master from the testParquetWriter?
>>
>> Is the test passing for you?
>>
>> Tim
>>
>> On Mon, May 12, 2014 at 8:46 PM,  <ja...@apache.org> wrote:
>> > DRILL-694: Implement parquet writer.
>> >
>> > Enable "CREATE TABLE AS" with parquet as the output format.
>> >
>> > Add decimal metadata support to parquet reader and writer.
>> >
>> >
>> > Project: http://git-wip-us.apache.org/repos/asf/incubator-drill/repo
>> > Commit:
>> http://git-wip-us.apache.org/repos/asf/incubator-drill/commit/63b03467
>> > Tree:
>> http://git-wip-us.apache.org/repos/asf/incubator-drill/tree/63b03467
>> > Diff:
>> http://git-wip-us.apache.org/repos/asf/incubator-drill/diff/63b03467
>> >
>> > Branch: refs/heads/master
>> > Commit: 63b0346761692f0a4c851e7910791c028dc3cded
>> > Parents: 14c628c
>> > Author: Steven Phillips <sp...@maprtech.com>
>> > Authored: Mon May 12 02:56:25 2014 -0700
>> > Committer: Aditya Kishore <ad...@maprtech.com>
>> > Committed: Mon May 12 18:18:43 2014 -0700
>> >
>> > ----------------------------------------------------------------------
>> >  .../org/apache/drill/common/types/Types.java    |   4 +
>> >  .../drill/common/util/DecimalUtility.java       |  11 +-
>> >  exec/java-exec/pom.xml                          |   4 +-
>> >  .../templates/EventBasedRecordWriter.java       |   6 +-
>> >  .../templates/ParquetOutputRecordWriter.java    | 189 +++++++++++
>> >  .../codegen/templates/ParquetTypeHelper.java    | 155 ++++++++++
>> >  .../org/apache/drill/exec/ExecConstants.java    |   8 +
>> >  .../server/options/SystemOptionManager.java     |   5 +-
>> >  .../exec/store/dfs/WorkspaceSchemaFactory.java  |   9 +-
>> >  .../drill/exec/store/parquet/BitReader.java     |   5 +-
>> >  .../drill/exec/store/parquet/ColumnReader.java  |  14 +-
>> >  .../store/parquet/FixedByteAlignedReader.java   |  87 +++++-
>> >  .../exec/store/parquet/NullableBitReader.java   |   5 +-
>> >  .../store/parquet/NullableColumnReader.java     |   5 +-
>> >  .../parquet/NullableFixedByteAlignedReader.java |  81 ++++-
>> >  .../exec/store/parquet/ParquetFormatPlugin.java |  48 ++-
>> >  .../exec/store/parquet/ParquetRecordReader.java | 310
>> ++++++++++++++-----
>> >  .../exec/store/parquet/ParquetRecordWriter.java | 185 +++++++++++
>> >  .../drill/exec/store/parquet/ParquetWriter.java |  91 ++++++
>> >  .../parquet/ParquetWriterBatchCreator.java      |  36 +++
>> >  .../store/parquet/VarLengthColumnReaders.java   | 182 ++++++++---
>> >  .../ColumnChunkPageWriteStoreExposer.java       |  39 +++
>> >  .../physical/impl/writer/TestParquetWriter.java | 163 ++++++++++
>> >  .../exec/physical/impl/writer/TestWriter.java   |  19 +-
>> >  24 files changed, 1510 insertions(+), 151 deletions(-)
>> > ----------------------------------------------------------------------
>> >
>> >
>> >
>> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/63b03467/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 c6ac82e..a6ec29b 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
>> > @@ -271,6 +271,10 @@ public class Types {
>> >      return
>> MajorType.newBuilder().setMode(mode).setMinorType(type).build();
>> >    }
>> >
>> > +  public static MajorType withScaleAndPrecision(MinorType type,
>> DataMode mode, int scale, int precision) {
>> > +    return
>> MajorType.newBuilder().setMinorType(type).setMode(mode).setScale(scale).setPrecision(precision).build();
>> > +  }
>> > +
>> >    public static MajorType required(MinorType type){
>> >      return
>> MajorType.newBuilder().setMode(DataMode.REQUIRED).setMinorType(type).build();
>> >    }
>> >
>> >
>> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/63b03467/common/src/main/java/org/apache/drill/common/util/DecimalUtility.java
>> > ----------------------------------------------------------------------
>> > diff --git
>> a/common/src/main/java/org/apache/drill/common/util/DecimalUtility.java
>> b/common/src/main/java/org/apache/drill/common/util/DecimalUtility.java
>> > index dbfd6ac..0cacc59 100644
>> > ---
>> a/common/src/main/java/org/apache/drill/common/util/DecimalUtility.java
>> > +++
>> b/common/src/main/java/org/apache/drill/common/util/DecimalUtility.java
>> > @@ -17,8 +17,7 @@
>> >   */
>> >  package org.apache.drill.common.util;
>> >
>> > -import java.math.BigDecimal;
>> > -import java.math.BigInteger;
>> > +import java.math.*;
>> >
>> >
>> >  import io.netty.buffer.Unpooled;
>> > @@ -26,6 +25,7 @@ import io.netty.buffer.ByteBuf;
>> >
>> >  import java.math.BigDecimal;
>> >  import java.math.BigInteger;
>> > +import java.util.Arrays;
>> >
>> >  public class DecimalUtility {
>> >
>> > @@ -486,5 +486,12 @@ public class DecimalUtility {
>> >        return 0;
>> >      }
>> >
>> > +
>> > +    public static BigDecimal getBigDecimalFromByteArray(byte[] bytes,
>> int start, int length, int scale) {
>> > +      byte[] value = Arrays.copyOfRange(bytes, start, start + length);
>> > +      BigInteger unscaledValue = new BigInteger(value);
>> > +      return new BigDecimal(unscaledValue, scale);
>> > +    }
>> > +
>> >  }
>> >
>> >
>> >
>> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/63b03467/exec/java-exec/pom.xml
>> > ----------------------------------------------------------------------
>> > diff --git a/exec/java-exec/pom.xml b/exec/java-exec/pom.xml
>> > index 3e26662..e282c00 100644
>> > --- a/exec/java-exec/pom.xml
>> > +++ b/exec/java-exec/pom.xml
>> > @@ -95,7 +95,7 @@
>> >      <dependency>
>> >        <groupId>com.twitter</groupId>
>> >        <artifactId>parquet-column</artifactId>
>> > -      <version>1.4.0</version>
>> > +      <version>1.5.0-SNAPSHOT</version>
>> >        <exclusions>
>> >          <exclusion>
>> >            <groupId>org.apache.hadoop</groupId>
>> > @@ -110,7 +110,7 @@
>> >      <dependency>
>> >        <groupId>com.twitter</groupId>
>> >        <artifactId>parquet-hadoop</artifactId>
>> > -      <version>1.4.0</version>
>> > +      <version>1.5.0-SNAPSHOT</version>
>> >        <exclusions>
>> >          <exclusion>
>> >            <groupId>org.apache.hadoop</groupId>
>> >
>> >
>> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/63b03467/exec/java-exec/src/main/codegen/templates/EventBasedRecordWriter.java
>> > ----------------------------------------------------------------------
>> > diff --git
>> a/exec/java-exec/src/main/codegen/templates/EventBasedRecordWriter.java
>> b/exec/java-exec/src/main/codegen/templates/EventBasedRecordWriter.java
>> > index 73b6566..b58f24c 100644
>> > ---
>> a/exec/java-exec/src/main/codegen/templates/EventBasedRecordWriter.java
>> > +++
>> b/exec/java-exec/src/main/codegen/templates/EventBasedRecordWriter.java
>> > @@ -91,8 +91,10 @@ public class EventBasedRecordWriter {
>> >      fieldWriters = Lists.newArrayList();
>> >      try {
>> >        for (int i = 0; i < schema.getFieldCount(); i++) {
>> > -        fieldWriters.add(i,
>> typeClassMap.get(schema.getColumn(i).getType())
>> > -            .getConstructor(EventBasedRecordWriter.class,
>> int.class).newInstance(this, i));
>> > +        MajorType mt = schema.getColumn(i).getType();
>> > +        MajorType newMt =
>> MajorType.newBuilder().setMinorType(mt.getMinorType()).setMode(mt.getMode()).build();
>> > +        fieldWriters.add(i, typeClassMap.get(newMt)
>> > +                .getConstructor(EventBasedRecordWriter.class,
>> int.class).newInstance(this, i));
>> >        }
>> >      } catch(Exception e) {
>> >        logger.error("Failed to create FieldWriter.", e);
>> >
>> >
>> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/63b03467/exec/java-exec/src/main/codegen/templates/ParquetOutputRecordWriter.java
>> > ----------------------------------------------------------------------
>> > diff --git
>> a/exec/java-exec/src/main/codegen/templates/ParquetOutputRecordWriter.java
>> b/exec/java-exec/src/main/codegen/templates/ParquetOutputRecordWriter.java
>> > new file mode 100644
>> > index 0000000..5f75c1c
>> > --- /dev/null
>> > +++
>> b/exec/java-exec/src/main/codegen/templates/ParquetOutputRecordWriter.java
>> > @@ -0,0 +1,189 @@
>> > +/**
>> > + * Licensed to the Apache Software Foundation (ASF) under one
>> > + * or more contributor license agreements.  See the NOTICE file
>> > + * distributed with this work for additional information
>> > + * regarding copyright ownership.  The ASF licenses this file
>> > + * to you under the Apache License, Version 2.0 (the
>> > + * "License"); you may not use this file except in compliance
>> > + * with the License.  You may obtain a copy of the License at
>> > + *
>> > + * http://www.apache.org/licenses/LICENSE-2.0
>> > + *
>> > + * Unless required by applicable law or agreed to in writing, software
>> > + * distributed under the License is distributed on an "AS IS" BASIS,
>> > + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
>> implied.
>> > + * See the License for the specific language governing permissions and
>> > + * limitations under the License.
>> > + */
>> > +
>> > +import parquet.io.api.Binary;
>> > +
>> > +import java.lang.Override;
>> > +
>> > +<@pp.dropOutputFile />
>> > +<@pp.changeOutputFile
>> name="org/apache/drill/exec/store/ParquetOutputRecordWriter.java" />
>> > +<#include "/@includes/license.ftl" />
>> > +
>> > +package org.apache.drill.exec.store;
>> > +
>> > +import com.google.common.collect.Lists;
>> > +import org.apache.drill.common.types.TypeProtos.MinorType;
>> > +import org.apache.drill.exec.expr.TypeHelper;
>> > +import org.apache.drill.exec.expr.holders.*;
>> > +import org.apache.drill.exec.record.BatchSchema;
>> > +import org.apache.drill.exec.store.parquet.ParquetTypeHelper;
>> > +import org.apache.drill.exec.vector.*;
>> > +import org.apache.drill.common.util.DecimalUtility;
>> > +import parquet.io.api.RecordConsumer;
>> > +import parquet.schema.MessageType;
>> > +import parquet.io.api.Binary;
>> > +import io.netty.buffer.ByteBuf;
>> > +
>> > +import java.io.IOException;
>> > +import java.lang.UnsupportedOperationException;
>> > +import java.util.Arrays;
>> > +import java.util.List;
>> > +import java.util.Map;
>> > +
>> > +/**
>> > + * Abstract implementation of RecordWriter interface which exposes
>> interface:
>> > + *    {@link #writeHeader(List)}
>> > + *    {@link #addField(int,String)}
>> > + * to output the data in string format instead of implementing addField
>> for each type holder.
>> > + *
>> > + * This is useful for text format writers such as CSV, TSV etc.
>> > + */
>> > +public abstract class ParquetOutputRecordWriter implements RecordWriter
>> {
>> > +
>> > +  private RecordConsumer consumer;
>> > +  private MessageType schema;
>> > +
>> > +  public void setUp(MessageType schema, RecordConsumer consumer) {
>> > +    this.schema = schema;
>> > +    this.consumer = consumer;
>> > +  }
>> > +
>> > +<#list vv.types as type>
>> > +  <#list type.minor as minor>
>> > +    <#list vv.modes as mode>
>> > +  @Override
>> > +  public void add${mode.prefix}${minor.class}Holder(int fieldId,
>> ${mode.prefix}${minor.class}Holder valueHolder) throws IOException {
>> > +  <#if mode.prefix == "Nullable" >
>> > +    if (valueHolder.isSet == 0) {
>> > +      return;
>> > +    }
>> > +  <#elseif mode.prefix == "Repeated" >
>> > +    consumer.startField(schema.getFieldName(fieldId), fieldId);
>> > +    for (int i = valueHolder.start; i < valueHolder.end; i++) {
>> > +  </#if>
>> > +
>> > +  <#if  minor.class == "TinyInt" ||
>> > +        minor.class == "UInt1" ||
>> > +        minor.class == "UInt2" ||
>> > +        minor.class == "SmallInt" ||
>> > +        minor.class == "Int" ||
>> > +        minor.class == "Time" ||
>> > +        minor.class == "IntervalYear" ||
>> > +        minor.class == "Decimal9" ||
>> > +        minor.class == "UInt4">
>> > +    <#if mode.prefix == "Repeated" >
>> > +
>>  consumer.addInteger(valueHolder.vector.getAccessor().get(i));
>> > +    <#else>
>> > +    consumer.startField(schema.getFieldName(fieldId), fieldId);
>> > +    consumer.addInteger(valueHolder.value);
>> > +    consumer.endField(schema.getFieldName(fieldId), fieldId);
>> > +    </#if>
>> > +  <#elseif
>> > +        minor.class == "Float4">
>> > +      <#if mode.prefix == "Repeated" >
>> > +
>>  consumer.addFloat(valueHolder.vector.getAccessor().get(i));
>> > +      <#else>
>> > +    consumer.startField(schema.getFieldName(fieldId), fieldId);
>> > +    consumer.addFloat(valueHolder.value);
>> > +    consumer.endField(schema.getFieldName(fieldId), fieldId);
>> > +      </#if>
>> > +  <#elseif
>> > +        minor.class == "BigInt" ||
>> > +        minor.class == "Decimal18" ||
>> > +        minor.class == "TimeStamp" ||
>> > +        minor.class == "Date" ||
>> > +        minor.class == "UInt8">
>> > +      <#if mode.prefix == "Repeated" >
>> > +              consumer.addLong(valueHolder.vector.getAccessor().get(i));
>> > +      <#else>
>> > +    consumer.startField(schema.getFieldName(fieldId), fieldId);
>> > +    consumer.addLong(valueHolder.value);
>> > +    consumer.endField(schema.getFieldName(fieldId), fieldId);
>> > +      </#if>
>> > +  <#elseif
>> > +        minor.class == "Float8">
>> > +      <#if mode.prefix == "Repeated" >
>> > +
>>  consumer.addDouble(valueHolder.vector.getAccessor().get(i));
>> > +      <#else>
>> > +    consumer.startField(schema.getFieldName(fieldId), fieldId);
>> > +    consumer.addDouble(valueHolder.value);
>> > +    consumer.endField(schema.getFieldName(fieldId), fieldId);
>> > +      </#if>
>> > +  <#elseif
>> > +        minor.class == "Bit">
>> > +      <#if mode.prefix == "Repeated" >
>> > +
>>  consumer.addBoolean(valueHolder.vector.getAccessor().get(i) == 1);
>> > +      <#else>
>> > +    consumer.startField(schema.getFieldName(fieldId), fieldId);
>> > +    consumer.addBoolean(valueHolder.value == 1);
>> > +    consumer.endField(schema.getFieldName(fieldId), fieldId);
>> > +      </#if>
>> > +  <#elseif
>> > +        minor.class == "Decimal28Sparse" ||
>> > +        minor.class == "Decimal38Sparse">
>> > +      <#if mode.prefix == "Repeated" >
>> > +      <#else>
>> > +      consumer.startField(schema.getFieldName(fieldId), fieldId);
>> > +      byte[] bytes = DecimalUtility.getBigDecimalFromSparse(
>> > +              valueHolder.buffer, valueHolder.start,
>> ${minor.class}Holder.nDecimalDigits,
>> valueHolder.scale).unscaledValue().toByteArray();
>> > +      byte[] output = new
>> byte[ParquetTypeHelper.getLengthForMinorType(MinorType.${minor.class?upper_case})];
>> > +      if (valueHolder.sign) {
>> > +        Arrays.fill(output, 0, output.length - bytes.length,
>> (byte)0xFF);
>> > +      } else {
>> > +        Arrays.fill(output, 0, output.length - bytes.length, (byte)0x0);
>> > +      }
>> > +      System.arraycopy(bytes, 0, output, output.length - bytes.length,
>> bytes.length);
>> > +      consumer.addBinary(Binary.fromByteArray(output));
>> > +      consumer.endField(schema.getFieldName(fieldId), fieldId);
>> > +      </#if>
>> > +  <#elseif
>> > +        minor.class == "TimeTZ" ||
>> > +        minor.class == "TimeStampTZ" ||
>> > +        minor.class == "IntervalDay" ||
>> > +        minor.class == "Interval" ||
>> > +        minor.class == "Decimal28Dense" ||
>> > +        minor.class == "Decimal38Dense">
>> > +
>> > +      <#if mode.prefix == "Repeated" >
>> > +      <#else>
>> > +
>> > +      </#if>
>> > +  <#elseif minor.class == "VarChar" || minor.class == "Var16Char" ||
>> minor.class == "VarBinary">
>> > +    <#if mode.prefix == "Repeated">
>> > +      ${minor.class}Holder singleHolder = new ${minor.class}Holder();
>> > +      valueHolder.vector.getAccessor().get(i, singleHolder);
>> > +      consumer.startField(schema.getFieldName(fieldId), fieldId);
>> > +
>>  consumer.addBinary(Binary.fromByteBuffer(singleHolder.buffer.nioBuffer(singleHolder.start,
>> singleHolder.end - singleHolder.start)));
>> > +      consumer.endField(schema.getFieldName(fieldId), fieldId);
>> > +    <#else>
>> > +    ByteBuf buf = valueHolder.buffer;
>> > +    consumer.startField(schema.getFieldName(fieldId), fieldId);
>> > +
>>  consumer.addBinary(Binary.fromByteBuffer(valueHolder.buffer.nioBuffer(valueHolder.start,
>> valueHolder.end - valueHolder.start)));
>> > +    consumer.endField(schema.getFieldName(fieldId), fieldId);
>> > +    </#if>
>> > +  </#if>
>> > +  <#if mode.prefix == "Repeated">
>> > +    }
>> > +    consumer.endField(schema.getFieldName(fieldId), fieldId);
>> > +  </#if>
>> > +  }
>> > +    </#list>
>> > +  </#list>
>> > +</#list>
>> > +
>> > +}
>> >
>> >
>> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/63b03467/exec/java-exec/src/main/codegen/templates/ParquetTypeHelper.java
>> > ----------------------------------------------------------------------
>> > diff --git
>> a/exec/java-exec/src/main/codegen/templates/ParquetTypeHelper.java
>> b/exec/java-exec/src/main/codegen/templates/ParquetTypeHelper.java
>> > new file mode 100644
>> > index 0000000..b268d33
>> > --- /dev/null
>> > +++ b/exec/java-exec/src/main/codegen/templates/ParquetTypeHelper.java
>> > @@ -0,0 +1,155 @@
>> > +/**
>> > + * Licensed to the Apache Software Foundation (ASF) under one
>> > + * or more contributor license agreements.  See the NOTICE file
>> > + * distributed with this work for additional information
>> > + * regarding copyright ownership.  The ASF licenses this file
>> > + * to you under the Apache License, Version 2.0 (the
>> > + * "License"); you may not use this file except in compliance
>> > + * with the License.  You may obtain a copy of the License at
>> > + *
>> > + * http://www.apache.org/licenses/LICENSE-2.0
>> > + *
>> > + * Unless required by applicable law or agreed to in writing, software
>> > + * distributed under the License is distributed on an "AS IS" BASIS,
>> > + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
>> implied.
>> > + * See the License for the specific language governing permissions and
>> > + * limitations under the License.
>> > + */
>> > +
>> > +import parquet.format.ConvertedType;
>> > +import parquet.schema.DecimalMetadata;
>> > +import parquet.schema.OriginalType;
>> > +import parquet.schema.PrimitiveType.PrimitiveTypeName;
>> > +
>> > +<@pp.dropOutputFile />
>> > +<@pp.changeOutputFile
>> name="org/apache/drill/exec/store/parquet/ParquetTypeHelper.java" />
>> > +<#include "/@includes/license.ftl" />
>> > +
>> > +package org.apache.drill.exec.store.parquet;
>> > +
>> > +import org.apache.drill.common.types.TypeProtos.DataMode;
>> > +import org.apache.drill.common.types.TypeProtos.MinorType;
>> > +import org.apache.drill.exec.record.MaterializedField;
>> > +import parquet.schema.OriginalType;
>> > +import parquet.schema.DecimalMetadata;
>> > +import parquet.schema.PrimitiveType.PrimitiveTypeName;
>> > +import parquet.schema.Type.Repetition;
>> > +
>> > +import java.util.HashMap;
>> > +import java.util.Map;
>> > +
>> > +public class ParquetTypeHelper {
>> > +  private static Map<MinorType,PrimitiveTypeName> typeMap;
>> > +  private static Map<DataMode,Repetition> modeMap;
>> > +  private static Map<MinorType,OriginalType> originalTypeMap;
>> > +
>> > +  static {
>> > +    typeMap = new HashMap();
>> > +
>> > +    <#list vv.types as type>
>> > +    <#list type.minor as minor>
>> > +    <#if    minor.class == "TinyInt" ||
>> > +            minor.class == "UInt1" ||
>> > +            minor.class == "UInt2" ||
>> > +            minor.class == "SmallInt" ||
>> > +            minor.class == "Int" ||
>> > +            minor.class == "Time" ||
>> > +            minor.class == "IntervalYear" ||
>> > +            minor.class == "Decimal9" ||
>> > +            minor.class == "UInt4">
>> > +                    typeMap.put(MinorType.${minor.class?upper_case},
>> PrimitiveTypeName.INT32);
>> > +    <#elseif
>> > +            minor.class == "Float4">
>> > +                    typeMap.put(MinorType.${minor.class?upper_case},
>> PrimitiveTypeName.FLOAT);
>> > +    <#elseif
>> > +            minor.class == "BigInt" ||
>> > +            minor.class == "Decimal18" ||
>> > +            minor.class == "TimeStamp" ||
>> > +            minor.class == "Date" ||
>> > +            minor.class == "UInt8">
>> > +                    typeMap.put(MinorType.${minor.class?upper_case},
>> PrimitiveTypeName.INT64);
>> > +    <#elseif
>> > +            minor.class == "Float8">
>> > +                    typeMap.put(MinorType.${minor.class?upper_case},
>> PrimitiveTypeName.DOUBLE);
>> > +    <#elseif
>> > +            minor.class == "Bit">
>> > +                    typeMap.put(MinorType.${minor.class?upper_case},
>> PrimitiveTypeName.BOOLEAN);
>> > +    <#elseif
>> > +            minor.class == "TimeTZ" ||
>> > +            minor.class == "TimeStampTZ" ||
>> > +            minor.class == "IntervalDay" ||
>> > +            minor.class == "Interval" ||
>> > +            minor.class == "Decimal28Dense" ||
>> > +            minor.class == "Decimal38Dense" ||
>> > +            minor.class == "Decimal28Sparse" ||
>> > +            minor.class == "Decimal38Sparse">
>> > +                    typeMap.put(MinorType.${minor.class?upper_case},
>> PrimitiveTypeName.FIXED_LEN_BYTE_ARRAY);
>> > +    <#elseif
>> > +            minor.class == "VarChar" ||
>> > +            minor.class == "Var16Char" ||
>> > +            minor.class == "VarBinary" >
>> > +                    typeMap.put(MinorType.${minor.class?upper_case},
>> PrimitiveTypeName.BINARY);
>> > +    </#if>
>> > +    </#list>
>> > +    </#list>
>> > +
>> > +    modeMap = new HashMap();
>> > +
>> > +    modeMap.put(DataMode.REQUIRED, Repetition.REQUIRED);
>> > +    modeMap.put(DataMode.OPTIONAL, Repetition.OPTIONAL);
>> > +    modeMap.put(DataMode.REPEATED, Repetition.REPEATED);
>> > +
>> > +    originalTypeMap = new HashMap();
>> > +
>> > +    <#list vv.types as type>
>> > +    <#list type.minor as minor>
>> > +            <#if minor.class.startsWith("Decimal")>
>> > +
>>  originalTypeMap.put(MinorType.${minor.class?upper_case},OriginalType.DECIMAL);
>> > +            </#if>
>> > +    </#list>
>> > +    </#list>
>> > +            originalTypeMap.put(MinorType.VARCHAR, OriginalType.UTF8);
>> > +            originalTypeMap.put(MinorType.DATE, OriginalType.DATE);
>> > +            originalTypeMap.put(MinorType.TIME, OriginalType.TIME);
>> > +            originalTypeMap.put(MinorType.TIMESTAMP,
>> OriginalType.TIMESTAMP);
>> > +            originalTypeMap.put(MinorType.TIMESTAMPTZ,
>> OriginalType.TIMESTAMPTZ);
>> > +  }
>> > +
>> > +  public static PrimitiveTypeName
>> getPrimitiveTypeNameForMinorType(MinorType minorType) {
>> > +    return typeMap.get(minorType);
>> > +  }
>> > +
>> > +  public static Repetition getRepetitionForDataMode(DataMode dataMode) {
>> > +    return modeMap.get(dataMode);
>> > +  }
>> > +
>> > +  public static OriginalType getOriginalTypeForMinorType(MinorType
>> minorType) {
>> > +    return originalTypeMap.get(minorType);
>> > +  }
>> > +
>> > +  public static DecimalMetadata
>> getDecimalMetadataForField(MaterializedField field) {
>> > +    switch(field.getType().getMinorType()) {
>> > +      case DECIMAL9:
>> > +      case DECIMAL18:
>> > +      case DECIMAL28SPARSE:
>> > +      case DECIMAL28DENSE:
>> > +      case DECIMAL38SPARSE:
>> > +      case DECIMAL38DENSE:
>> > +        return new DecimalMetadata(field.getPrecision(),
>> field.getScale());
>> > +      default:
>> > +        return null;
>> > +    }
>> > +  }
>> > +
>> > +  public static int getLengthForMinorType(MinorType minorType) {
>> > +    switch(minorType) {
>> > +      case DECIMAL28SPARSE:
>> > +        return 12;
>> > +      case DECIMAL38SPARSE:
>> > +        return 16;
>> > +      default:
>> > +        return 0;
>> > +    }
>> > +  }
>> > +
>> > +}
>> >
>> >
>> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/63b03467/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 9eee08d..238fae9 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
>> > @@ -17,6 +17,10 @@
>> >   */
>> >  package org.apache.drill.exec;
>> >
>> > +import org.apache.drill.exec.server.options.OptionValidator;
>> > +import
>> org.apache.drill.exec.server.options.TypeValidators.LongValidator;
>> > +import
>> org.apache.drill.exec.server.options.TypeValidators.StringValidator;
>> > +
>> >  public interface ExecConstants {
>> >    public static final String ZK_RETRY_TIMES =
>> "drill.exec.zk.retry.count";
>> >    public static final String ZK_RETRY_DELAY =
>> "drill.exec.zk.retry.delay";
>> > @@ -63,5 +67,9 @@ public interface ExecConstants {
>> >    public static final String FILESYSTEM_PARTITION_COLUMN_LABEL =
>> "drill.exec.storage.file.partition.column.label";
>> >    public static final String HAZELCAST_SUBNETS =
>> "drill.exec.cache.hazel.subnets";
>> >    public static final String TOP_LEVEL_MAX_ALLOC =
>> "drill.exec.memory.top.max";
>> > +  public static final String OUTPUT_FORMAT_OPTION = "store.format";
>> > +  public static final OptionValidator OUTPUT_FORMAT_VALIDATOR = new
>> StringValidator(OUTPUT_FORMAT_OPTION, "parquet");
>> > +  public static final String PARQUET_BLOCK_SIZE = "parquet.block.size";
>> > +  public static final OptionValidator PARQUET_BLOCK_SIZE_VALIDATOR =
>> new LongValidator(PARQUET_BLOCK_SIZE, 512*1024*1024);
>> >
>> >  }
>> >
>> >
>> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/63b03467/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 cfe8e2c..21031e5 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
>> > @@ -22,6 +22,7 @@ import java.util.Map;
>> >  import java.util.Map.Entry;
>> >  import java.util.concurrent.ConcurrentMap;
>> >
>> > +import org.apache.drill.exec.ExecConstants;
>> >  import org.apache.drill.exec.cache.DistributedCache;
>> >  import org.apache.drill.exec.cache.DistributedMap;
>> >  import org.apache.drill.exec.planner.physical.PlannerSettings;
>> > @@ -38,7 +39,9 @@ public class SystemOptionManager implements
>> OptionManager{
>> >        PlannerSettings.STREAMAGG,
>> >        PlannerSettings.HASHJOIN,
>> >        PlannerSettings.MERGEJOIN,
>> > -      PlannerSettings.MULTIPHASE
>> > +      PlannerSettings.MULTIPHASE,
>> > +      ExecConstants.OUTPUT_FORMAT_VALIDATOR,
>> > +      ExecConstants.PARQUET_BLOCK_SIZE_VALIDATOR
>> >    };
>> >
>> >    private DistributedMap<OptionValue> options;
>> >
>> >
>> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/63b03467/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/WorkspaceSchemaFactory.java
>> > ----------------------------------------------------------------------
>> > diff --git
>> a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/WorkspaceSchemaFactory.java
>> b/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/WorkspaceSchemaFactory.java
>> > index df73ea5..72b31e6 100644
>> > ---
>> a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/WorkspaceSchemaFactory.java
>> > +++
>> b/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/WorkspaceSchemaFactory.java
>> > @@ -26,12 +26,15 @@ import com.google.common.collect.Lists;
>> >  import com.google.common.collect.Sets;
>> >  import net.hydromatic.optiq.Table;
>> >  import org.apache.drill.common.exceptions.ExecutionSetupException;
>> > +import org.apache.drill.exec.ExecConstants;
>> >  import org.apache.drill.exec.planner.logical.CreateTableEntry;
>> >  import org.apache.drill.exec.planner.logical.DrillTable;
>> >  import org.apache.drill.exec.planner.logical.DynamicDrillTable;
>> >  import org.apache.drill.exec.planner.logical.FileSystemCreateTableEntry;
>> >  import org.apache.drill.exec.planner.sql.ExpandingConcurrentMap;
>> >  import org.apache.drill.exec.rpc.user.UserSession;
>> > +import org.apache.drill.exec.server.options.OptionValidator;
>> > +import
>> org.apache.drill.exec.server.options.TypeValidators.StringValidator;
>> >  import org.apache.drill.exec.store.AbstractSchema;
>> >  import org.apache.drill.exec.store.dfs.shim.DrillFileSystem;
>> >  import org.apache.hadoop.fs.Path;
>> > @@ -141,9 +144,11 @@ public class WorkspaceSchemaFactory implements
>> ExpandingConcurrentMap.MapValueFa
>> >        return fs;
>> >      }
>> >
>> > +
>> >      @Override
>> >      public CreateTableEntry createNewTable(String tableName) {
>> > -      FormatPlugin formatPlugin =
>> plugin.getFormatPlugin(config.getStorageFormat());
>> > +      String storage =
>> session.getOptions().getOption(ExecConstants.OUTPUT_FORMAT_OPTION).string_val;
>> > +      FormatPlugin formatPlugin = plugin.getFormatPlugin(storage);
>> >        if (formatPlugin == null)
>> >          throw new UnsupportedOperationException(
>> >            String.format("Unsupported format '%s' in workspace '%s'",
>> config.getStorageFormat(),
>> > @@ -151,7 +156,7 @@ public class WorkspaceSchemaFactory implements
>> ExpandingConcurrentMap.MapValueFa
>> >
>> >        return new FileSystemCreateTableEntry(
>> >            (FileSystemConfig) plugin.getConfig(),
>> > -          plugin.getFormatPlugin(config.getStorageFormat()),
>> > +          formatPlugin,
>> >            config.getLocation() + Path.SEPARATOR + tableName);
>> >      }
>> >    }
>> >
>> >
>> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/63b03467/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/BitReader.java
>> > ----------------------------------------------------------------------
>> > diff --git
>> a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/BitReader.java
>> b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/BitReader.java
>> > index c323222..7ae95cd 100644
>> > ---
>> a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/BitReader.java
>> > +++
>> b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/BitReader.java
>> > @@ -22,6 +22,7 @@ import
>> org.apache.drill.exec.vector.BaseDataValueVector;
>> >  import org.apache.drill.exec.vector.ValueVector;
>> >  import parquet.column.ColumnDescriptor;
>> >  import parquet.format.ConvertedType;
>> > +import parquet.format.SchemaElement;
>> >  import parquet.hadoop.metadata.ColumnChunkMetaData;
>> >
>> >  final class BitReader extends ColumnReader {
>> > @@ -31,8 +32,8 @@ final class BitReader extends ColumnReader {
>> >    private byte[] bytes;
>> >
>> >    BitReader(ParquetRecordReader parentReader, int allocateSize,
>> ColumnDescriptor descriptor, ColumnChunkMetaData columnChunkMetaData,
>> > -            boolean fixedLength, ValueVector v, ConvertedType
>> convertedType) throws ExecutionSetupException {
>> > -    super(parentReader, allocateSize, descriptor, columnChunkMetaData,
>> fixedLength, v, convertedType);
>> > +            boolean fixedLength, ValueVector v, SchemaElement
>> schemaElement) throws ExecutionSetupException {
>> > +    super(parentReader, allocateSize, descriptor, columnChunkMetaData,
>> fixedLength, v, schemaElement);
>> >    }
>> >
>> >    @Override
>> >
>> >
>> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/63b03467/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ColumnReader.java
>> > ----------------------------------------------------------------------
>> > diff --git
>> a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ColumnReader.java
>> b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ColumnReader.java
>> > index 196e1fd..b9faafe 100644
>> > ---
>> a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ColumnReader.java
>> > +++
>> b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ColumnReader.java
>> > @@ -25,8 +25,10 @@ import org.apache.drill.exec.vector.ValueVector;
>> >  import org.apache.hadoop.fs.FSDataInputStream;
>> >  import parquet.column.ColumnDescriptor;
>> >  import parquet.format.ConvertedType;
>> > +import parquet.format.SchemaElement;
>> >  import parquet.hadoop.metadata.ColumnChunkMetaData;
>> >  import parquet.schema.PrimitiveType;
>> > +import parquet.schema.PrimitiveType.PrimitiveTypeName;
>> >
>> >  import java.io.IOException;
>> >
>> > @@ -48,7 +50,7 @@ abstract class ColumnReader<V extends ValueVector> {
>> >    // status information on the current page
>> >    final PageReadStatus pageReadStatus;
>> >
>> > -  final ConvertedType convertedType;
>> > +  final SchemaElement schemaElement;
>> >
>> >    // quick reference to see if the field is fixed length (as this
>> requires an instanceof)
>> >    final boolean isFixedLength;
>> > @@ -70,12 +72,12 @@ abstract class ColumnReader<V extends ValueVector> {
>> >    long readStartInBytes = 0, readLength = 0, readLengthInBits = 0,
>> recordsReadInThisIteration = 0;
>> >
>> >    protected ColumnReader(ParquetRecordReader parentReader, int
>> allocateSize, ColumnDescriptor descriptor,
>> > -      ColumnChunkMetaData columnChunkMetaData, boolean fixedLength, V
>> v, ConvertedType convertedType) throws ExecutionSetupException {
>> > +      ColumnChunkMetaData columnChunkMetaData, boolean fixedLength, V
>> v, SchemaElement schemaElement) throws ExecutionSetupException {
>> >      this.parentReader = parentReader;
>> >      this.columnDescriptor = descriptor;
>> >      this.columnChunkMetaData = columnChunkMetaData;
>> >      this.isFixedLength = fixedLength;
>> > -    this.convertedType = convertedType;
>> > +    this.schemaElement = schemaElement;
>> >
>> >      if (allocateSize > 1) {
>> >        valueVec =  v;
>> > @@ -87,7 +89,11 @@ abstract class ColumnReader<V extends ValueVector> {
>> >      this.pageReadStatus = new PageReadStatus(this,
>> parentReader.fileSystem, parentReader.hadoopPath, columnChunkMetaData);
>> >
>> >      if (columnDescriptor.getType() !=
>> PrimitiveType.PrimitiveTypeName.BINARY) {
>> > -      dataTypeLengthInBits =
>> ParquetRecordReader.getTypeLengthInBits(columnDescriptor.getType());
>> > +      if (columnDescriptor.getType() ==
>> PrimitiveTypeName.FIXED_LEN_BYTE_ARRAY) {
>> > +        dataTypeLengthInBits = columnDescriptor.getTypeLength() * 8;
>> > +      } else {
>> > +        dataTypeLengthInBits =
>> ParquetRecordReader.getTypeLengthInBits(columnDescriptor.getType());
>> > +      }
>> >      }
>> >
>> >    }
>> >
>> >
>> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/63b03467/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/FixedByteAlignedReader.java
>> > ----------------------------------------------------------------------
>> > diff --git
>> a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/FixedByteAlignedReader.java
>> b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/FixedByteAlignedReader.java
>> > index 0aa18cf..574b0cb 100644
>> > ---
>> a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/FixedByteAlignedReader.java
>> > +++
>> b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/FixedByteAlignedReader.java
>> > @@ -18,20 +18,27 @@
>> >  package org.apache.drill.exec.store.parquet;
>> >
>> >  import org.apache.drill.common.exceptions.ExecutionSetupException;
>> > -import org.apache.drill.exec.vector.BaseDataValueVector;
>> > +import org.apache.drill.common.util.DecimalUtility;
>> > +import org.apache.drill.exec.expr.holders.Decimal28SparseHolder;
>> > +import org.apache.drill.exec.expr.holders.Decimal38SparseHolder;
>> > +import org.apache.drill.exec.vector.Decimal28SparseVector;
>> > +import org.apache.drill.exec.vector.Decimal38SparseVector;
>> >  import org.apache.drill.exec.vector.ValueVector;
>> >  import parquet.column.ColumnDescriptor;
>> >  import parquet.format.ConvertedType;
>> > +import parquet.format.SchemaElement;
>> >  import parquet.hadoop.metadata.ColumnChunkMetaData;
>> >
>> > +import java.math.BigDecimal;
>> > +
>> >  class FixedByteAlignedReader extends ColumnReader {
>> >
>> > -  private byte[] bytes;
>> > +  protected byte[] bytes;
>> >
>> >
>> >    FixedByteAlignedReader(ParquetRecordReader parentReader, int
>> allocateSize, ColumnDescriptor descriptor, ColumnChunkMetaData
>> columnChunkMetaData,
>> > -                         boolean fixedLength, ValueVector v,
>> ConvertedType convertedType) throws ExecutionSetupException {
>> > -    super(parentReader, allocateSize, descriptor, columnChunkMetaData,
>> fixedLength, v, convertedType);
>> > +                         boolean fixedLength, ValueVector v,
>> SchemaElement schemaElement) throws ExecutionSetupException {
>> > +    super(parentReader, allocateSize, descriptor, columnChunkMetaData,
>> fixedLength, v, schemaElement);
>> >    }
>> >
>> >    // this method is called by its superclass during a read loop
>> > @@ -50,4 +57,76 @@ class FixedByteAlignedReader extends ColumnReader {
>> >      vectorData.writeBytes(bytes,
>> >          (int) readStartInBytes, (int) readLength);
>> >    }
>> > +
>> > +  public static abstract class ConvertedReader extends
>> FixedByteAlignedReader {
>> > +
>> > +    protected int dataTypeLengthInBytes;
>> > +
>> > +    ConvertedReader(ParquetRecordReader parentReader, int allocateSize,
>> ColumnDescriptor descriptor, ColumnChunkMetaData columnChunkMetaData,
>> > +                           boolean fixedLength, ValueVector v,
>> SchemaElement schemaElement) throws ExecutionSetupException {
>> > +      super(parentReader, allocateSize, descriptor,
>> columnChunkMetaData, fixedLength, v, schemaElement);
>> > +    }
>> > +
>> > +    @Override
>> > +    protected void readField(long recordsToReadInThisPass, ColumnReader
>> firstColumnStatus) {
>> > +      recordsReadInThisIteration =
>> Math.min(pageReadStatus.currentPage.getValueCount()
>> > +              - pageReadStatus.valuesRead, recordsToReadInThisPass -
>> valuesReadInCurrentPass);
>> > +
>> > +      readStartInBytes = pageReadStatus.readPosInBytes;
>> > +      readLengthInBits = recordsReadInThisIteration *
>> dataTypeLengthInBits;
>> > +      readLength = (int) Math.ceil(readLengthInBits / 8.0);
>> > +
>> > +      bytes = pageReadStatus.pageDataByteArray;
>> > +
>> > +      dataTypeLengthInBytes = (int) Math.ceil(dataTypeLengthInBits /
>> 8.0);
>> > +      for (int i = 0; i < recordsReadInThisIteration; i++) {
>> > +        addNext((int)readStartInBytes + i * dataTypeLengthInBytes, i +
>> valuesReadInCurrentPass);
>> > +      }
>> > +    }
>> > +
>> > +    /**
>> > +     * Reads from bytes, converts, and writes to buffer
>> > +     * @param start the index in bytes to start reading from
>> > +     * @param index the index of the ValueVector
>> > +     */
>> > +    abstract void addNext(int start, int index);
>> > +  }
>> > +
>> > +  public static class Decimal28Reader extends ConvertedReader {
>> > +
>> > +    Decimal28SparseVector decimal28Vector;
>> > +
>> > +    Decimal28Reader(ParquetRecordReader parentReader, int allocateSize,
>> ColumnDescriptor descriptor, ColumnChunkMetaData columnChunkMetaData,
>> > +                    boolean fixedLength, ValueVector v, SchemaElement
>> schemaElement) throws ExecutionSetupException {
>> > +      super(parentReader, allocateSize, descriptor,
>> columnChunkMetaData, fixedLength, v, schemaElement);
>> > +      decimal28Vector = (Decimal28SparseVector) v;
>> > +    }
>> > +
>> > +    @Override
>> > +    void addNext(int start, int index) {
>> > +      int width = Decimal28SparseHolder.WIDTH;
>> > +      BigDecimal intermediate =
>> DecimalUtility.getBigDecimalFromByteArray(bytes, start,
>> dataTypeLengthInBytes, schemaElement.getScale());
>> > +      DecimalUtility.getSparseFromBigDecimal(intermediate,
>> decimal28Vector.getData(), index * width, schemaElement.getScale(),
>> > +              schemaElement.getPrecision(),
>> Decimal28SparseHolder.nDecimalDigits);
>> > +    }
>> > +  }
>> > +
>> > +  public static class Decimal38Reader extends ConvertedReader {
>> > +
>> > +    Decimal38SparseVector decimal38Vector;
>> > +
>> > +    Decimal38Reader(ParquetRecordReader parentReader, int allocateSize,
>> ColumnDescriptor descriptor, ColumnChunkMetaData columnChunkMetaData,
>> > +                    boolean fixedLength, ValueVector v, SchemaElement
>> schemaElement) throws ExecutionSetupException {
>> > +      super(parentReader, allocateSize, descriptor,
>> columnChunkMetaData, fixedLength, v, schemaElement);
>> > +      decimal38Vector = (Decimal38SparseVector) v;
>> > +    }
>> > +
>> > +    @Override
>> > +    void addNext(int start, int index) {
>> > +      int width = Decimal38SparseHolder.WIDTH;
>> > +      BigDecimal intermediate =
>> DecimalUtility.getBigDecimalFromByteArray(bytes, start,
>> dataTypeLengthInBytes, schemaElement.getScale());
>> > +      DecimalUtility.getSparseFromBigDecimal(intermediate,
>> decimal38Vector.getData(), index * width, schemaElement.getScale(),
>> > +              schemaElement.getPrecision(),
>> Decimal38SparseHolder.nDecimalDigits);
>> > +    }
>> > +  }
>> >  }
>> > \ No newline at end of file
>> >
>> >
>> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/63b03467/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/NullableBitReader.java
>> > ----------------------------------------------------------------------
>> > diff --git
>> a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/NullableBitReader.java
>> b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/NullableBitReader.java
>> > index 22933ba..29ca30a 100644
>> > ---
>> a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/NullableBitReader.java
>> > +++
>> b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/NullableBitReader.java
>> > @@ -24,6 +24,7 @@ import org.apache.drill.exec.vector.NullableBitVector;
>> >  import org.apache.drill.exec.vector.ValueVector;
>> >  import parquet.column.ColumnDescriptor;
>> >  import parquet.format.ConvertedType;
>> > +import parquet.format.SchemaElement;
>> >  import parquet.hadoop.metadata.ColumnChunkMetaData;
>> >
>> >  import java.io.IOException;
>> > @@ -39,8 +40,8 @@ import java.io.IOException;
>> >  final class NullableBitReader extends ColumnReader {
>> >
>> >    NullableBitReader(ParquetRecordReader parentReader, int allocateSize,
>> ColumnDescriptor descriptor, ColumnChunkMetaData columnChunkMetaData,
>> > -                    boolean fixedLength, ValueVector v, ConvertedType
>> convertedType) throws ExecutionSetupException {
>> > -    super(parentReader, allocateSize, descriptor, columnChunkMetaData,
>> fixedLength, v, convertedType);
>> > +                    boolean fixedLength, ValueVector v, SchemaElement
>> schemaElement) throws ExecutionSetupException {
>> > +    super(parentReader, allocateSize, descriptor, columnChunkMetaData,
>> fixedLength, v, schemaElement);
>> >    }
>> >
>> >    @Override
>> >
>> >
>> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/63b03467/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/NullableColumnReader.java
>> > ----------------------------------------------------------------------
>> > diff --git
>> a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/NullableColumnReader.java
>> b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/NullableColumnReader.java
>> > index 66d1c5f..6040c67 100644
>> > ---
>> a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/NullableColumnReader.java
>> > +++
>> b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/NullableColumnReader.java
>> > @@ -23,6 +23,7 @@ import
>> org.apache.drill.exec.vector.NullableVectorDefinitionSetter;
>> >  import org.apache.drill.exec.vector.ValueVector;
>> >  import parquet.column.ColumnDescriptor;
>> >  import parquet.format.ConvertedType;
>> > +import parquet.format.SchemaElement;
>> >  import parquet.hadoop.metadata.ColumnChunkMetaData;
>> >
>> >  import java.io.IOException;
>> > @@ -36,8 +37,8 @@ abstract class NullableColumnReader extends
>> ColumnReader{
>> >    int bitsUsed;
>> >
>> >    NullableColumnReader(ParquetRecordReader parentReader, int
>> allocateSize, ColumnDescriptor descriptor, ColumnChunkMetaData
>> columnChunkMetaData,
>> > -               boolean fixedLength, ValueVector v, ConvertedType
>> convertedType) throws ExecutionSetupException {
>> > -    super(parentReader, allocateSize, descriptor, columnChunkMetaData,
>> fixedLength, v, convertedType);
>> > +               boolean fixedLength, ValueVector v, SchemaElement
>> schemaElement) throws ExecutionSetupException {
>> > +    super(parentReader, allocateSize, descriptor, columnChunkMetaData,
>> fixedLength, v, schemaElement);
>> >    }
>> >
>> >    public void readAllFixedFields(long recordsToReadInThisPass,
>> ColumnReader firstColumnStatus) throws IOException {
>> >
>> >
>> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/63b03467/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/NullableFixedByteAlignedReader.java
>> > ----------------------------------------------------------------------
>> > diff --git
>> a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/NullableFixedByteAlignedReader.java
>> b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/NullableFixedByteAlignedReader.java
>> > index 038f2d7..d4416c8 100644
>> > ---
>> a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/NullableFixedByteAlignedReader.java
>> > +++
>> b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/NullableFixedByteAlignedReader.java
>> > @@ -18,19 +18,27 @@
>> >  package org.apache.drill.exec.store.parquet;
>> >
>> >  import org.apache.drill.common.exceptions.ExecutionSetupException;
>> > +import org.apache.drill.common.util.DecimalUtility;
>> > +import org.apache.drill.exec.expr.holders.NullableDecimal28SparseHolder;
>> > +import org.apache.drill.exec.expr.holders.NullableDecimal38SparseHolder;
>> > +import org.apache.drill.exec.vector.NullableDecimal28SparseVector;
>> > +import org.apache.drill.exec.vector.NullableDecimal38SparseVector;
>> >  import org.apache.drill.exec.vector.ValueVector;
>> >
>> >  import parquet.column.ColumnDescriptor;
>> >  import parquet.format.ConvertedType;
>> > +import parquet.format.SchemaElement;
>> >  import parquet.hadoop.metadata.ColumnChunkMetaData;
>> >
>> > +import java.math.BigDecimal;
>> > +
>> >  class NullableFixedByteAlignedReader extends NullableColumnReader {
>> >
>> > -  private byte[] bytes;
>> > +  protected byte[] bytes;
>> >
>> >    NullableFixedByteAlignedReader(ParquetRecordReader parentReader, int
>> allocateSize, ColumnDescriptor descriptor,
>> > -      ColumnChunkMetaData columnChunkMetaData, boolean fixedLength,
>> ValueVector v, ConvertedType convertedType) throws ExecutionSetupException {
>> > -    super(parentReader, allocateSize, descriptor, columnChunkMetaData,
>> fixedLength, v, convertedType);
>> > +      ColumnChunkMetaData columnChunkMetaData, boolean fixedLength,
>> ValueVector v, SchemaElement schemaElement) throws ExecutionSetupException {
>> > +    super(parentReader, allocateSize, descriptor, columnChunkMetaData,
>> fixedLength, v, schemaElement);
>> >    }
>> >
>> >    // this method is called by its superclass during a read loop
>> > @@ -48,4 +56,71 @@ class NullableFixedByteAlignedReader extends
>> NullableColumnReader {
>> >      // fill in data.
>> >      vectorData.writeBytes(bytes, (int) readStartInBytes, (int)
>> readLength);
>> >    }
>> > +
>> > +  public static abstract class NullableConvertedReader extends
>> NullableFixedByteAlignedReader {
>> > +
>> > +    protected int dataTypeLengthInBytes;
>> > +
>> > +    NullableConvertedReader(ParquetRecordReader parentReader, int
>> allocateSize, ColumnDescriptor descriptor,
>> > +                                   ColumnChunkMetaData
>> columnChunkMetaData, boolean fixedLength, ValueVector v, SchemaElement
>> schemaElement) throws ExecutionSetupException {
>> > +      super(parentReader, allocateSize, descriptor,
>> columnChunkMetaData, fixedLength, v, schemaElement);
>> > +    }
>> > +
>> > +    @Override
>> > +    protected void readField(long recordsToReadInThisPass, ColumnReader
>> firstColumnStatus) {
>> > +
>> > +      this.recordsReadInThisIteration = recordsToReadInThisPass;
>> > +
>> > +      // set up metadata
>> > +      this.readStartInBytes = pageReadStatus.readPosInBytes;
>> > +      this.readLengthInBits = recordsReadInThisIteration *
>> dataTypeLengthInBits;
>> > +      this.readLength = (int) Math.ceil(readLengthInBits / 8.0);
>> > +      this.bytes = pageReadStatus.pageDataByteArray;
>> > +
>> > +      dataTypeLengthInBytes = (int) Math.ceil(dataTypeLengthInBits /
>> 8.0);
>> > +      for (int i = 0; i < recordsReadInThisIteration; i++) {
>> > +        addNext((int) readStartInBytes + i * dataTypeLengthInBytes, i +
>> valuesReadInCurrentPass);
>> > +      }
>> > +    }
>> > +
>> > +    abstract void addNext(int start, int index);
>> > +  }
>> > +
>> > +  public static class NullableDecimal28Reader extends
>> NullableConvertedReader {
>> > +
>> > +    NullableDecimal28SparseVector decimal28Vector;
>> > +
>> > +    NullableDecimal28Reader(ParquetRecordReader parentReader, int
>> allocateSize, ColumnDescriptor descriptor, ColumnChunkMetaData
>> columnChunkMetaData,
>> > +                            boolean fixedLength, ValueVector v,
>> SchemaElement schemaElement) throws ExecutionSetupException {
>> > +      super(parentReader, allocateSize, descriptor,
>> columnChunkMetaData, fixedLength, v, schemaElement);
>> > +      decimal28Vector = (NullableDecimal28SparseVector) v;
>> > +    }
>> > +
>> > +    @Override
>> > +    void addNext(int start, int index) {
>> > +      int width = NullableDecimal28SparseHolder.WIDTH;
>> > +      BigDecimal intermediate =
>> DecimalUtility.getBigDecimalFromByteArray(bytes, start,
>> dataTypeLengthInBytes, schemaElement.getScale());
>> > +      DecimalUtility.getSparseFromBigDecimal(intermediate,
>> decimal28Vector.getData(), index * width, schemaElement.getScale(),
>> > +              schemaElement.getPrecision(),
>> NullableDecimal28SparseHolder.nDecimalDigits);
>> > +    }
>> > +  }
>> > +
>> > +  public static class NullableDecimal38Reader extends
>> NullableConvertedReader {
>> > +
>> > +    NullableDecimal38SparseVector decimal38Vector;
>> > +
>> > +    NullableDecimal38Reader(ParquetRecordReader parentReader, int
>> allocateSize, ColumnDescriptor descriptor, ColumnChunkMetaData
>> columnChunkMetaData,
>> > +                            boolean fixedLength, ValueVector v,
>> SchemaElement schemaElement) throws ExecutionSetupException {
>> > +      super(parentReader, allocateSize, descriptor,
>> columnChunkMetaData, fixedLength, v, schemaElement);
>> > +      decimal38Vector = (NullableDecimal38SparseVector) v;
>> > +    }
>> > +
>> > +    @Override
>> > +    void addNext(int start, int index) {
>> > +      int width = NullableDecimal38SparseHolder.WIDTH;
>> > +      BigDecimal intermediate =
>> DecimalUtility.getBigDecimalFromByteArray(bytes, start,
>> dataTypeLengthInBytes, schemaElement.getScale());
>> > +      DecimalUtility.getSparseFromBigDecimal(intermediate,
>> decimal38Vector.getData(), index * width, schemaElement.getScale(),
>> > +              schemaElement.getPrecision(),
>> NullableDecimal38SparseHolder.nDecimalDigits);
>> > +    }
>> > +  }
>> >  }
>> > \ No newline at end of file
>> >
>> >
>> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/63b03467/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetFormatPlugin.java
>> > ----------------------------------------------------------------------
>> > diff --git
>> a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetFormatPlugin.java
>> b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetFormatPlugin.java
>> > index a10d30f..34287f3 100644
>> > ---
>> a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetFormatPlugin.java
>> > +++
>> b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetFormatPlugin.java
>> > @@ -19,26 +19,30 @@ package org.apache.drill.exec.store.parquet;
>> >
>> >  import java.io.IOException;
>> >  import java.util.List;
>> > +import java.util.Map;
>> >  import java.util.Set;
>> >  import java.util.regex.Pattern;
>> >
>> > +import com.google.common.collect.Maps;
>> > +import org.apache.drill.common.exceptions.ExecutionSetupException;
>> >  import org.apache.drill.common.expression.SchemaPath;
>> >  import org.apache.drill.common.logical.StoragePluginConfig;
>> > +import org.apache.drill.exec.ExecConstants;
>> > +import org.apache.drill.exec.ops.FragmentContext;
>> >  import org.apache.drill.exec.physical.base.AbstractWriter;
>> >  import org.apache.drill.exec.physical.base.PhysicalOperator;
>> > +import org.apache.drill.exec.physical.impl.WriterRecordBatch;
>> > +import org.apache.drill.exec.proto.ExecProtos.FragmentHandle;
>> > +import org.apache.drill.exec.record.RecordBatch;
>> >  import org.apache.drill.exec.server.DrillbitContext;
>> > +import org.apache.drill.exec.store.RecordWriter;
>> >  import org.apache.drill.exec.store.StoragePluginOptimizerRule;
>> > -import org.apache.drill.exec.store.dfs.BasicFormatMatcher;
>> > -import org.apache.drill.exec.store.dfs.DrillPathFilter;
>> > -import org.apache.drill.exec.store.dfs.FileSelection;
>> > -import org.apache.drill.exec.store.dfs.FormatMatcher;
>> > -import org.apache.drill.exec.store.dfs.FormatPlugin;
>> > -import org.apache.drill.exec.store.dfs.FormatSelection;
>> > -import org.apache.drill.exec.store.dfs.MagicString;
>> > +import org.apache.drill.exec.store.dfs.*;
>> >  import org.apache.drill.exec.store.dfs.shim.DrillFileSystem;
>> >  import org.apache.drill.exec.store.mock.MockStorageEngine;
>> >  import org.apache.hadoop.conf.Configuration;
>> >  import org.apache.hadoop.fs.FileStatus;
>> > +import org.apache.hadoop.fs.FileSystem;
>> >  import org.apache.hadoop.fs.Path;
>> >  import org.apache.hadoop.fs.PathFilter;
>> >
>> > @@ -104,7 +108,35 @@ public class ParquetFormatPlugin implements
>> FormatPlugin{
>> >
>> >    @Override
>> >    public AbstractWriter getWriter(PhysicalOperator child, String
>> location) throws IOException {
>> > -    throw new UnsupportedOperationException("Parquet Writer is not
>> supported currently.");
>> > +    return new ParquetWriter(child, location, this);
>> > +  }
>> > +
>> > +  public RecordWriter getRecordWriter(FragmentContext context,
>> ParquetWriter writer) throws IOException {
>> > +    Map<String, String> options = Maps.newHashMap();
>> > +
>> > +    options.put("location", writer.getLocation());
>> > +
>> > +    FragmentHandle handle = context.getHandle();
>> > +    String fragmentId = String.format("%d_%d",
>> handle.getMajorFragmentId(), handle.getMinorFragmentId());
>> > +    options.put("prefix", fragmentId);
>> > +
>> > +    options.put(FileSystem.FS_DEFAULT_NAME_KEY,
>> ((FileSystemConfig)writer.getStorageConfig()).connection);
>> > +
>> > +    options.put(ExecConstants.PARQUET_BLOCK_SIZE,
>> context.getOptions().getOption(ExecConstants.PARQUET_BLOCK_SIZE).num_val.toString());
>> > +
>> > +    RecordWriter recordWriter = new ParquetRecordWriter();
>> > +    recordWriter.init(options);
>> > +
>> > +    return recordWriter;
>> > +  }
>> > +
>> > +  public RecordBatch getWriterBatch(FragmentContext context,
>> RecordBatch incoming, ParquetWriter writer)
>> > +          throws ExecutionSetupException {
>> > +    try {
>> > +      return new WriterRecordBatch(writer, incoming, context,
>> getRecordWriter(context, writer));
>> > +    } catch(IOException e) {
>> > +      throw new ExecutionSetupException(String.format("Failed to create
>> the WriterRecordBatch. %s", e.getMessage()), e);
>> > +    }
>> >    }
>> >
>> >    @Override
>> >
>> >
>> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/63b03467/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetRecordReader.java
>> > ----------------------------------------------------------------------
>> > diff --git
>> a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetRecordReader.java
>> b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetRecordReader.java
>> > index 5d28456..4ca13a5 100644
>> > ---
>> a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetRecordReader.java
>> > +++
>> b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetRecordReader.java
>> > @@ -32,7 +32,9 @@ import
>> org.apache.drill.common.expression.ExpressionPosition;
>> >  import org.apache.drill.common.expression.FieldReference;
>> >  import org.apache.drill.common.expression.SchemaPath;
>> >  import org.apache.drill.common.types.TypeProtos;
>> > +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.exception.SchemaChangeException;
>> >  import org.apache.drill.exec.expr.TypeHelper;
>> > @@ -41,6 +43,10 @@ import org.apache.drill.exec.ops.FragmentContext;
>> >  import org.apache.drill.exec.physical.impl.OutputMutator;
>> >  import org.apache.drill.exec.record.MaterializedField;
>> >  import org.apache.drill.exec.store.RecordReader;
>> > +import
>> org.apache.drill.exec.store.parquet.FixedByteAlignedReader.Decimal28Reader;
>> > +import
>> org.apache.drill.exec.store.parquet.FixedByteAlignedReader.Decimal38Reader;
>> > +import
>> org.apache.drill.exec.store.parquet.NullableFixedByteAlignedReader.NullableDecimal28Reader;
>> > +import
>> org.apache.drill.exec.store.parquet.NullableFixedByteAlignedReader.NullableDecimal38Reader;
>> >  import org.apache.drill.exec.vector.*;
>> >  import org.apache.hadoop.fs.FileSystem;
>> >  import org.apache.hadoop.fs.Path;
>> > @@ -62,6 +68,7 @@ import parquet.schema.PrimitiveType;
>> >  import org.apache.drill.exec.store.parquet.VarLengthColumnReaders.*;
>> >
>> >  import com.google.common.base.Joiner;
>> > +import parquet.schema.PrimitiveType.PrimitiveTypeName;
>> >
>> >  public class ParquetRecordReader implements RecordReader {
>> >    static final org.slf4j.Logger logger =
>> org.slf4j.LoggerFactory.getLogger(ParquetRecordReader.class);
>> > @@ -191,18 +198,19 @@ public class ParquetRecordReader implements
>> RecordReader {
>> >
>> >      // TODO - figure out how to deal with this better once we add
>> nested reading, note also look where this map is used below
>> >      // store a map from column name to converted types if they are
>> non-null
>> > -    HashMap<String, ConvertedType> convertedTypes = new HashMap<>();
>> > +    HashMap<String, SchemaElement> schemaElements = new HashMap<>();
>> >      fileMetaData = new
>> ParquetMetadataConverter().toParquetMetadata(ParquetFileWriter.CURRENT_VERSION,
>> footer);
>> >      for (SchemaElement se : fileMetaData.getSchema()) {
>> > -      convertedTypes.put(se.getName(), se.getConverted_type());
>> > +      schemaElements.put(se.getName(), se);
>> >      }
>> >
>> >      // loop to add up the length of the fixed width columns and build
>> the schema
>> >      for (int i = 0; i < columns.size(); ++i) {
>> >        column = columns.get(i);
>> >        logger.debug("name: " + fileMetaData.getSchema().get(i).name);
>> > -      field = MaterializedField.create(toFieldName(column.getPath()),
>> > -          toMajorType(column.getType(), getDataMode(column),
>> convertedTypes.get(column.getPath()[0])));
>> > +      SchemaElement se = schemaElements.get(column.getPath()[0]);
>> > +      MajorType mt = toMajorType(column.getType(), se.getType_length(),
>> getDataMode(column), se);
>> > +      field =
>> MaterializedField.create(toFieldName(column.getPath()),mt);
>> >        if ( ! fieldSelected(field)){
>> >          continue;
>> >        }
>> > @@ -211,12 +219,11 @@ public class ParquetRecordReader implements
>> RecordReader {
>> >        if (column.getType() != PrimitiveType.PrimitiveTypeName.BINARY) {
>> >          // There is not support for the fixed binary type yet in
>> parquet, leaving a task here as a reminder
>> >          // TODO - implement this when the feature is added upstream
>> > -//          if (column.getType() ==
>> PrimitiveType.PrimitiveTypeName.FIXED_LEN_BYTE_ARRAY){
>> > -//              byteWidthAllFixedFields += column.getType().getWidth()
>> > -//          }
>> > -//          else { } // the code below for the rest of the fixed length
>> fields
>> > -
>> > -        bitWidthAllFixedFields += getTypeLengthInBits(column.getType());
>> > +          if (column.getType() ==
>> PrimitiveType.PrimitiveTypeName.FIXED_LEN_BYTE_ARRAY){
>> > +              bitWidthAllFixedFields += se.getType_length() * 8;
>> > +          } else {
>> > +            bitWidthAllFixedFields +=
>> getTypeLengthInBits(column.getType());
>> > +          }
>> >        } else {
>> >          allFieldsFixedLength = false;
>> >        }
>> > @@ -234,16 +241,11 @@ public class ParquetRecordReader implements
>> RecordReader {
>> >      else {
>> >        recordsPerBatch = DEFAULT_RECORDS_TO_READ_IF_NOT_FIXED_WIDTH;
>> >      }
>> > -//    for (SchemaElement se : fileMetaData.getSchema()) {
>> > -//      if (fieldSelected())
>> > -//      System.out.println("convertedtype :" + se.getConverted_type());
>> > -//      System.out.println("name:" + se.getName());
>> > -//      System.out.println();
>> > -//
>> > -//    }
>> > +
>> >      try {
>> >        ValueVector v;
>> >        ConvertedType convertedType;
>> > +      SchemaElement schemaElement;
>> >        ArrayList<VarLengthColumn> varLengthColumns = new ArrayList<>();
>> >        ArrayList<NullableVarLengthColumn> nullableVarLengthColumns = new
>> ArrayList<>();
>> >        // initialize all of the column read status objects
>> > @@ -251,40 +253,21 @@ public class ParquetRecordReader implements
>> RecordReader {
>> >        for (int i = 0; i < columns.size(); ++i) {
>> >          column = columns.get(i);
>> >          columnChunkMetaData =
>> footer.getBlocks().get(0).getColumns().get(i);
>> > -        convertedType = convertedTypes.get(column.getPath()[0]);
>> > -        MajorType type = toMajorType(column.getType(),
>> getDataMode(column), convertedType);
>> > -//
>>  Preconditions.checkArgument(!columnChunkMetaData.getEncodings().contains(Encoding.PLAIN_DICTIONARY),
>> "Dictionary Encoding not currently supported");
>> > +        schemaElement = schemaElements.get(column.getPath()[0]);
>> > +        convertedType = schemaElement.getConverted_type();
>> > +        MajorType type = toMajorType(column.getType(),
>> schemaElement.getType_length(), getDataMode(column), schemaElement);
>> >          field = MaterializedField.create(toFieldName(column.getPath()),
>> type);
>> >          // the field was not requested to be read
>> >          if ( ! fieldSelected(field)) continue;
>> >
>> > -        //convertedTypes.put()
>> >          fieldFixedLength = column.getType() !=
>> PrimitiveType.PrimitiveTypeName.BINARY;
>> >          v = output.addField(field, (Class<? extends ValueVector>)
>> TypeHelper.getValueVectorClass(type.getMinorType(), type.getMode()));
>> >          if (column.getType() != PrimitiveType.PrimitiveTypeName.BINARY)
>> {
>> >            createFixedColumnReader(fieldFixedLength, column,
>> columnChunkMetaData, recordsPerBatch, v,
>> > -            convertedType);
>> > +            schemaElement);
>> >          } else {
>> > -          if (column.getMaxDefinitionLevel() == 0){// column is required
>> > -            if (convertedType == ConvertedType.UTF8) {
>> > -              varLengthColumns.add(
>> > -                new VarCharColumn(this, -1, column,
>> columnChunkMetaData, false, (VarCharVector) v, convertedType));
>> > -            } else {
>> > -              varLengthColumns.add(
>> > -                  new VarBinaryColumn(this, -1, column,
>> columnChunkMetaData, false, (VarBinaryVector) v, convertedType));
>> > -            }
>> > -          }
>> > -          else{
>> > -            if (convertedType == ConvertedType.UTF8) {
>> > -              nullableVarLengthColumns.add(
>> > -                new NullableVarCharColumn(this, -1, column,
>> columnChunkMetaData, false,
>> > -                    (NullableVarCharVector) v, convertedType));
>> > -            } else {
>> > -              nullableVarLengthColumns.add(
>> > -                new NullableVarBinaryColumn(this, -1, column,
>> columnChunkMetaData, false,
>> > -                  (NullableVarBinaryVector) v, convertedType));
>> > -            }
>> > -          }
>> > +          // create a reader and add it to the appropriate list
>> > +          getReader(this, -1, column, columnChunkMetaData, false, v,
>> schemaElement, varLengthColumns, nullableVarLengthColumns);
>> >          }
>> >        }
>> >        varLengthReader = new VarLenBinaryReader(this, varLengthColumns,
>> nullableVarLengthColumns);
>> > @@ -338,28 +321,41 @@ public class ParquetRecordReader implements
>> RecordReader {
>> >     */
>> >    private boolean createFixedColumnReader(boolean fixedLength,
>> ColumnDescriptor descriptor,
>> >                                            ColumnChunkMetaData
>> columnChunkMetaData, int allocateSize, ValueVector v,
>> > -                                          ConvertedType convertedType)
>> > +                                          SchemaElement schemaElement)
>> >        throws SchemaChangeException, ExecutionSetupException {
>> > +    ConvertedType convertedType = schemaElement.getConverted_type();
>> >      // if the column is required
>> >      if (descriptor.getMaxDefinitionLevel() == 0){
>> >        if (columnChunkMetaData.getType() ==
>> PrimitiveType.PrimitiveTypeName.BOOLEAN){
>> >          columnStatuses.add(new BitReader(this, allocateSize,
>> descriptor, columnChunkMetaData,
>> > -            fixedLength, v, convertedType));
>> > -      }
>> > -      else{
>> > +            fixedLength, v, schemaElement));
>> > +      } else if (columnChunkMetaData.getType() ==
>> PrimitiveTypeName.FIXED_LEN_BYTE_ARRAY && convertedType ==
>> ConvertedType.DECIMAL){
>> > +        int length = schemaElement.type_length;
>> > +        if (length <= 12) {
>> > +          columnStatuses.add(new Decimal28Reader(this, allocateSize,
>> descriptor, columnChunkMetaData, fixedLength, v, schemaElement));
>> > +        } else if (length <= 16) {
>> > +          columnStatuses.add(new Decimal38Reader(this, allocateSize,
>> descriptor, columnChunkMetaData, fixedLength, v, schemaElement));
>> > +        }
>> > +      } else {
>> >          columnStatuses.add(new FixedByteAlignedReader(this,
>> allocateSize, descriptor, columnChunkMetaData,
>> > -            fixedLength, v, convertedType));
>> > +            fixedLength, v, schemaElement));
>> >        }
>> >        return true;
>> >      }
>> >      else { // if the column is nullable
>> >        if (columnChunkMetaData.getType() ==
>> PrimitiveType.PrimitiveTypeName.BOOLEAN){
>> >          columnStatuses.add(new NullableBitReader(this, allocateSize,
>> descriptor, columnChunkMetaData,
>> > -            fixedLength, v, convertedType));
>> > -      }
>> > -      else{
>> > +            fixedLength, v, schemaElement));
>> > +      } else if (columnChunkMetaData.getType() ==
>> PrimitiveTypeName.FIXED_LEN_BYTE_ARRAY && convertedType ==
>> ConvertedType.DECIMAL){
>> > +        int length = schemaElement.type_length;
>> > +        if (length <= 12) {
>> > +          columnStatuses.add(new NullableDecimal28Reader(this,
>> allocateSize, descriptor, columnChunkMetaData, fixedLength, v,
>> schemaElement));
>> > +        } else if (length <= 16) {
>> > +          columnStatuses.add(new NullableDecimal38Reader(this,
>> allocateSize, descriptor, columnChunkMetaData, fixedLength, v,
>> schemaElement));
>> > +        }
>> > +      } else {
>> >          columnStatuses.add(new NullableFixedByteAlignedReader(this,
>> allocateSize, descriptor, columnChunkMetaData,
>> > -            fixedLength, v, convertedType));
>> > +            fixedLength, v, schemaElement));
>> >        }
>> >        return true;
>> >      }
>> > @@ -420,25 +416,57 @@ public class ParquetRecordReader implements
>> RecordReader {
>> >    }
>> >
>> >    static TypeProtos.MajorType
>> toMajorType(PrimitiveType.PrimitiveTypeName primitiveTypeName,
>> > -                                               TypeProtos.DataMode
>> mode, ConvertedType convertedType) {
>> > -    return toMajorType(primitiveTypeName, 0, mode, convertedType);
>> > +                                               TypeProtos.DataMode
>> mode, SchemaElement schemaElement) {
>> > +    return toMajorType(primitiveTypeName, 0, mode, schemaElement);
>> >    }
>> >
>> >    static TypeProtos.MajorType
>> toMajorType(PrimitiveType.PrimitiveTypeName primitiveTypeName, int length,
>> > -                                               TypeProtos.DataMode
>> mode, ConvertedType convertedType) {
>> > +                                               TypeProtos.DataMode
>> mode, SchemaElement schemaElement) {
>> > +    ConvertedType convertedType = schemaElement.getConverted_type();
>> >      switch (mode) {
>> >
>> >        case OPTIONAL:
>> >          switch (primitiveTypeName) {
>> >            case BINARY:
>> > -            if (convertedType == ConvertedType.UTF8)
>> > -              return Types.optional(TypeProtos.MinorType.VARCHAR);
>> > -            else
>> > +            if (convertedType == null) {
>> >                return Types.optional(TypeProtos.MinorType.VARBINARY);
>> > +            }
>> > +            switch (convertedType) {
>> > +              case UTF8:
>> > +                return Types.optional(MinorType.VARCHAR);
>> > +              case DECIMAL:
>> > +                return
>> Types.withScaleAndPrecision(getDecimalType(schemaElement),
>> DataMode.OPTIONAL, schemaElement.getScale(), schemaElement.getPrecision());
>> > +              default:
>> > +                throw new
>> UnsupportedOperationException(String.format("unsupported type: %s %s",
>> primitiveTypeName, convertedType));
>> > +            }
>> >            case INT64:
>> > -            return Types.optional(TypeProtos.MinorType.BIGINT);
>> > +            if (convertedType == null) {
>> > +              return Types.optional(TypeProtos.MinorType.BIGINT);
>> > +            }
>> > +            switch(convertedType) {
>> > +              case DECIMAL:
>> > +                return Types.withScaleAndPrecision(MinorType.DECIMAL18,
>> DataMode.OPTIONAL, schemaElement.getScale(), schemaElement.getPrecision());
>> > +              case FINETIME:
>> > +                throw new UnsupportedOperationException();
>> > +              case TIMESTAMP:
>> > +                return Types.optional(MinorType.TIMESTAMP);
>> > +              default:
>> > +                throw new
>> UnsupportedOperationException(String.format("unsupported type: %s %s",
>> primitiveTypeName, convertedType));
>> > +            }
>> >            case INT32:
>> > -            return Types.optional(TypeProtos.MinorType.INT);
>> > +            if (convertedType == null) {
>> > +              return Types.optional(TypeProtos.MinorType.INT);
>> > +            }
>> > +            switch(convertedType) {
>> > +              case DECIMAL:
>> > +                return Types.withScaleAndPrecision(MinorType.DECIMAL9,
>> DataMode.OPTIONAL, schemaElement.getScale(), schemaElement.getPrecision());
>> > +              case DATE:
>> > +                return Types.optional(MinorType.DATE);
>> > +              case TIME:
>> > +                return Types.optional(MinorType.TIME);
>> > +              default:
>> > +                throw new
>> UnsupportedOperationException(String.format("unsupported type: %s %s",
>> primitiveTypeName, convertedType));
>> > +            }
>> >            case BOOLEAN:
>> >              return Types.optional(TypeProtos.MinorType.BIT);
>> >            case FLOAT:
>> > @@ -451,23 +479,58 @@ public class ParquetRecordReader implements
>> RecordReader {
>> >              return
>> TypeProtos.MajorType.newBuilder().setMinorType(TypeProtos.MinorType.FIXEDBINARY).setWidth(12)
>> >                  .setMode(mode).build();
>> >            case FIXED_LEN_BYTE_ARRAY:
>> > -            checkArgument(length > 0, "A length greater than zero must
>> be provided for a FixedBinary type.");
>> > -            return
>> TypeProtos.MajorType.newBuilder().setMinorType(TypeProtos.MinorType.FIXEDBINARY)
>> > -                .setWidth(length).setMode(mode).build();
>> > +            if (convertedType == null) {
>> > +              checkArgument(length > 0, "A length greater than zero
>> must be provided for a FixedBinary type.");
>> > +              return
>> TypeProtos.MajorType.newBuilder().setMinorType(TypeProtos.MinorType.FIXEDBINARY)
>> > +                      .setWidth(length).setMode(mode).build();
>> > +            } else if (convertedType == ConvertedType.DECIMAL) {
>> > +              return
>> Types.withScaleAndPrecision(getDecimalType(schemaElement),
>> DataMode.OPTIONAL, schemaElement.getScale(), schemaElement.getPrecision());
>> > +            }
>> >            default:
>> >              throw new UnsupportedOperationException("Type not
>> supported: " + primitiveTypeName);
>> >          }
>> >        case REQUIRED:
>> >          switch (primitiveTypeName) {
>> >            case BINARY:
>> > -            if (convertedType == ConvertedType.UTF8)
>> > -              return Types.required(TypeProtos.MinorType.VARCHAR);
>> > -            else
>> > +            if (convertedType == null) {
>> >                return Types.required(TypeProtos.MinorType.VARBINARY);
>> > +            }
>> > +            switch (convertedType) {
>> > +              case UTF8:
>> > +                return Types.required(MinorType.VARCHAR);
>> > +              case DECIMAL:
>> > +                return
>> Types.withScaleAndPrecision(getDecimalType(schemaElement),
>> DataMode.REQUIRED, schemaElement.getScale(), schemaElement.getPrecision());
>> > +              default:
>> > +                throw new
>> UnsupportedOperationException(String.format("unsupported type: %s %s",
>> primitiveTypeName, convertedType));
>> > +            }
>> >            case INT64:
>> > -            return Types.required(TypeProtos.MinorType.BIGINT);
>> > +            if (convertedType == null) {
>> > +              return Types.required(MinorType.BIGINT);
>> > +            }
>> > +            switch(convertedType) {
>> > +              case DECIMAL:
>> > +                return Types.withScaleAndPrecision(MinorType.DECIMAL18,
>> DataMode.REQUIRED, schemaElement.getScale(), schemaElement.getPrecision());
>> > +              case FINETIME:
>> > +                throw new UnsupportedOperationException();
>> > +              case TIMESTAMP:
>> > +                return Types.required(MinorType.TIMESTAMP);
>> > +              default:
>> > +                throw new
>> UnsupportedOperationException(String.format("unsupported type: %s %s",
>> primitiveTypeName, convertedType));
>> > +            }
>> >            case INT32:
>> > -            return Types.required(TypeProtos.MinorType.INT);
>> > +            if (convertedType == null) {
>> > +              return Types.required(MinorType.INT);
>> > +            }
>> > +            switch(convertedType) {
>> > +              case DECIMAL:
>> > +                return Types.withScaleAndPrecision(MinorType.DECIMAL9,
>> DataMode.REQUIRED, schemaElement.getScale(), schemaElement.getPrecision());
>> > +              case DATE:
>> > +                return Types.required(MinorType.DATE);
>> > +              case TIME:
>> > +                return Types.required(MinorType.TIME);
>> > +              default:
>> > +                throw new
>> UnsupportedOperationException(String.format("unsupported type: %s %s",
>> primitiveTypeName, convertedType));
>> > +            }
>> >            case BOOLEAN:
>> >              return Types.required(TypeProtos.MinorType.BIT);
>> >            case FLOAT:
>> > @@ -480,23 +543,58 @@ public class ParquetRecordReader implements
>> RecordReader {
>> >              return
>> TypeProtos.MajorType.newBuilder().setMinorType(TypeProtos.MinorType.FIXEDBINARY).setWidth(12)
>> >                  .setMode(mode).build();
>> >            case FIXED_LEN_BYTE_ARRAY:
>> > -            checkArgument(length > 0, "A length greater than zero must
>> be provided for a FixedBinary type.");
>> > -            return
>> TypeProtos.MajorType.newBuilder().setMinorType(TypeProtos.MinorType.FIXEDBINARY)
>> > -                .setWidth(length).setMode(mode).build();
>> > +            if (convertedType == null) {
>> > +              checkArgument(length > 0, "A length greater than zero
>> must be provided for a FixedBinary type.");
>> > +              return
>> TypeProtos.MajorType.newBuilder().setMinorType(TypeProtos.MinorType.FIXEDBINARY)
>> > +                      .setWidth(length).setMode(mode).build();
>> > +            } else if (convertedType == ConvertedType.DECIMAL) {
>> > +              return
>> Types.withScaleAndPrecision(getDecimalType(schemaElement),
>> DataMode.REQUIRED, schemaElement.getScale(), schemaElement.getPrecision());
>> > +            }
>> >            default:
>> >              throw new UnsupportedOperationException("Type not
>> supported: " + primitiveTypeName);
>> >          }
>> >        case REPEATED:
>> >          switch (primitiveTypeName) {
>> >            case BINARY:
>> > -            if (convertedType == ConvertedType.UTF8)
>> > -              return Types.required(TypeProtos.MinorType.VARCHAR);
>> > -            else
>> > +            if (convertedType == null) {
>> >                return Types.repeated(TypeProtos.MinorType.VARBINARY);
>> > +            }
>> > +            switch (schemaElement.getConverted_type()) {
>> > +              case UTF8:
>> > +                return Types.repeated(MinorType.VARCHAR);
>> > +              case DECIMAL:
>> > +                return
>> Types.withScaleAndPrecision(getDecimalType(schemaElement),
>> DataMode.REPEATED, schemaElement.getScale(), schemaElement.getPrecision());
>> > +              default:
>> > +                throw new
>> UnsupportedOperationException(String.format("unsupported type: %s %s",
>> primitiveTypeName, convertedType));
>> > +            }
>> >            case INT64:
>> > -            return Types.repeated(TypeProtos.MinorType.BIGINT);
>> > +            if (convertedType == null) {
>> > +              return Types.repeated(MinorType.BIGINT);
>> > +            }
>> > +            switch(convertedType) {
>> > +              case DECIMAL:
>> > +                return Types.withScaleAndPrecision(MinorType.DECIMAL18,
>> DataMode.REPEATED, schemaElement.getScale(), schemaElement.getPrecision());
>> > +              case FINETIME:
>> > +                throw new UnsupportedOperationException();
>> > +              case TIMESTAMP:
>> > +                return Types.repeated(MinorType.TIMESTAMP);
>> > +              default:
>> > +                throw new
>> UnsupportedOperationException(String.format("unsupported type: %s %s",
>> primitiveTypeName, convertedType));
>> > +            }
>> >            case INT32:
>> > -            return Types.repeated(TypeProtos.MinorType.INT);
>> > +            if (convertedType == null) {
>> > +              return Types.repeated(MinorType.INT);
>> > +            }
>> > +            switch(convertedType) {
>> > +              case DECIMAL:
>> > +                return Types.withScaleAndPrecision(MinorType.DECIMAL9,
>> DataMode.REPEATED, schemaElement.getScale(), schemaElement.getPrecision());
>> > +              case DATE:
>> > +                return Types.repeated(MinorType.DATE);
>> > +              case TIME:
>> > +                return Types.repeated(MinorType.TIME);
>> > +              default:
>> > +                throw new
>> UnsupportedOperationException(String.format("unsupported type: %s %s",
>> primitiveTypeName, convertedType));
>> > +            }
>> >            case BOOLEAN:
>> >              return Types.repeated(TypeProtos.MinorType.BIT);
>> >            case FLOAT:
>> > @@ -509,9 +607,13 @@ public class ParquetRecordReader implements
>> RecordReader {
>> >              return
>> TypeProtos.MajorType.newBuilder().setMinorType(TypeProtos.MinorType.FIXEDBINARY).setWidth(12)
>> >                  .setMode(mode).build();
>> >            case FIXED_LEN_BYTE_ARRAY:
>> > -            checkArgument(length > 0, "A length greater than zero must
>> be provided for a FixedBinary type.");
>> > -            return
>> TypeProtos.MajorType.newBuilder().setMinorType(TypeProtos.MinorType.FIXEDBINARY)
>> > -                .setWidth(length).setMode(mode).build();
>> > +            if (convertedType == null) {
>> > +              checkArgument(length > 0, "A length greater than zero
>> must be provided for a FixedBinary type.");
>> > +              return
>> TypeProtos.MajorType.newBuilder().setMinorType(TypeProtos.MinorType.FIXEDBINARY)
>> > +                      .setWidth(length).setMode(mode).build();
>> > +            } else if (convertedType == ConvertedType.DECIMAL) {
>> > +              return
>> Types.withScaleAndPrecision(getDecimalType(schemaElement),
>> DataMode.REPEATED, schemaElement.getScale(), schemaElement.getPrecision());
>> > +            }
>> >            default:
>> >              throw new UnsupportedOperationException("Type not
>> supported: " + primitiveTypeName);
>> >          }
>> > @@ -519,6 +621,58 @@ public class ParquetRecordReader implements
>> RecordReader {
>> >      throw new UnsupportedOperationException("Type not supported: " +
>> primitiveTypeName + " Mode: " + mode);
>> >    }
>> >
>> > +  private static void getReader(ParquetRecordReader parentReader, int
>> allocateSize, ColumnDescriptor descriptor,
>> > +                                        ColumnChunkMetaData
>> columnChunkMetaData, boolean fixedLength, ValueVector v,
>> > +                                        SchemaElement schemaElement,
>> List<VarLengthColumn> varLengthColumns,
>> > +                                        List<NullableVarLengthColumn>
>> nullableVarLengthColumns) throws ExecutionSetupException {
>> > +    ConvertedType convertedType = schemaElement.getConverted_type();
>> > +    switch (descriptor.getMaxDefinitionLevel()) {
>> > +      case 0:
>> > +        if (convertedType == null) {
>> > +          varLengthColumns.add(new VarBinaryColumn(parentReader,
>> allocateSize, descriptor, columnChunkMetaData, fixedLength,
>> (VarBinaryVector) v, schemaElement));
>> > +          return;
>> > +        }
>> > +        switch (convertedType) {
>> > +          case UTF8:
>> > +            varLengthColumns.add(new VarCharColumn(parentReader,
>> allocateSize, descriptor, columnChunkMetaData, fixedLength, (VarCharVector)
>> v, schemaElement));
>> > +            return;
>> > +          case DECIMAL:
>> > +            if (v instanceof Decimal28SparseVector) {
>> > +              varLengthColumns.add(new Decimal28Column(parentReader,
>> allocateSize, descriptor, columnChunkMetaData, fixedLength,
>> (Decimal28SparseVector) v, schemaElement));
>> > +              return;
>> > +            } else if (v instanceof Decimal38SparseVector) {
>> > +              varLengthColumns.add(new Decimal38Column(parentReader,
>> allocateSize, descriptor, columnChunkMetaData, fixedLength,
>> (Decimal38SparseVector) v, schemaElement));
>> > +              return;
>> > +            }
>> > +          default:
>> > +        }
>> > +      default:
>> > +        if (convertedType == null) {
>> > +          nullableVarLengthColumns.add(new
>> NullableVarBinaryColumn(parentReader, allocateSize, descriptor,
>> columnChunkMetaData, fixedLength, (NullableVarBinaryVector) v,
>> schemaElement));
>> > +          return;
>> > +        }
>> > +        switch (convertedType) {
>> > +          case UTF8:
>> > +            nullableVarLengthColumns.add(new
>> NullableVarCharColumn(parentReader, allocateSize, descriptor,
>> columnChunkMetaData, fixedLength, (NullableVarCharVector) v,
>> schemaElement));
>> > +            return;
>> > +          case DECIMAL:
>> > +            if (v instanceof NullableDecimal28SparseVector) {
>> > +              nullableVarLengthColumns.add(new
>> NullableDecimal28Column(parentReader, allocateSize, descriptor,
>> columnChunkMetaData, fixedLength, (NullableDecimal28SparseVector) v,
>> schemaElement));
>> > +              return;
>> > +            } else if (v instanceof NullableDecimal38SparseVector) {
>> > +              nullableVarLengthColumns.add(new
>> NullableDecimal38Column(parentReader, allocateSize, descriptor,
>> columnChunkMetaData, fixedLength, (NullableDecimal38SparseVector) v,
>> schemaElement));
>> > +              return;
>> > +            }
>> > +          default:
>> > +        }
>> > +    }
>> > +    throw new UnsupportedOperationException();
>> > +  }
>> > +
>> > +  private static MinorType getDecimalType(SchemaElement schemaElement) {
>> > +    return schemaElement.getPrecision() <= 28 ?
>> MinorType.DECIMAL28SPARSE : MinorType.DECIMAL38SPARSE;
>> > +  }
>> > +
>> >    static String join(String delimiter, String... str) {
>> >      StringBuilder builder = new StringBuilder();
>> >      int i = 0;
>> >
>> >
>> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/63b03467/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetRecordWriter.java
>> > ----------------------------------------------------------------------
>> > diff --git
>> a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetRecordWriter.java
>> b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetRecordWriter.java
>> > new file mode 100644
>> > index 0000000..3ba13f4
>> > --- /dev/null
>> > +++
>> b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetRecordWriter.java
>> > @@ -0,0 +1,185 @@
>> > +/**
>> > + * Licensed to the Apache Software Foundation (ASF) under one
>> > + * or more contributor license agreements.  See the NOTICE file
>> > + * distributed with this work for additional information
>> > + * regarding copyright ownership.  The ASF licenses this file
>> > + * to you under the Apache License, Version 2.0 (the
>> > + * "License"); you may not use this file except in compliance
>> > + * with the License.  You may obtain a copy of the License at
>> > + *
>> > + * http://www.apache.org/licenses/LICENSE-2.0
>> > + *
>> > + * Unless required by applicable law or agreed to in writing, software
>> > + * distributed under the License is distributed on an "AS IS" BASIS,
>> > + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
>> implied.
>> > + * See the License for the specific language governing permissions and
>> > + * limitations under the License.
>> > + */
>> > +package org.apache.drill.exec.store.parquet;
>> > +
>> > +import com.google.common.collect.Lists;
>> > +import org.apache.drill.common.types.TypeProtos.MinorType;
>> > +import org.apache.drill.exec.ExecConstants;
>> > +import org.apache.drill.exec.record.BatchSchema;
>> > +import org.apache.drill.exec.record.MaterializedField;
>> > +import org.apache.drill.exec.store.ParquetOutputRecordWriter;
>> > +import org.apache.hadoop.conf.Configuration;
>> > +import org.apache.hadoop.fs.FileSystem;
>> > +import org.apache.hadoop.fs.Path;
>> > +import parquet.column.ParquetProperties.WriterVersion;
>> > +import parquet.column.impl.ColumnWriteStoreImpl;
>> > +import parquet.column.page.PageWriteStore;
>> > +import parquet.hadoop.ColumnChunkPageWriteStoreExposer;
>> > +import parquet.hadoop.ParquetFileWriter;
>> > +import parquet.hadoop.metadata.CompressionCodecName;
>> > +import parquet.io.ColumnIOFactory;
>> > +import parquet.io.MessageColumnIO;
>> > +import parquet.io.api.RecordConsumer;
>> > +import parquet.schema.DecimalMetadata;
>> > +import parquet.schema.MessageType;
>> > +import parquet.schema.OriginalType;
>> > +import parquet.schema.PrimitiveType.PrimitiveTypeName;
>> > +import parquet.schema.Type;
>> > +import parquet.schema.Type.Repetition;
>> > +
>> > +import java.io.IOException;
>> > +import java.util.HashMap;
>> > +import java.util.List;
>> > +import java.util.Map;
>> > +
>> > +import static java.lang.Math.max;
>> > +import static java.lang.Math.min;
>> > +import static java.lang.String.format;
>> > +
>> > +public class ParquetRecordWriter extends ParquetOutputRecordWriter {
>> > +
>> > +  private static final int MINIMUM_BUFFER_SIZE = 64 * 1024;
>> > +  private static final int MINIMUM_RECORD_COUNT_FOR_CHECK = 100;
>> > +  private static final int MAXIMUM_RECORD_COUNT_FOR_CHECK = 10000;
>> > +
>> > +  private ParquetFileWriter w;
>> > +  private MessageType schema;
>> > +  private Map<String, String> extraMetaData = new HashMap();
>> > +  private int blockSize;
>> > +  private int pageSize = 1 * 1024 * 1024;
>> > +  private int dictionaryPageSize = pageSize;
>> > +  private boolean enableDictionary = false;
>> > +  private boolean validating = false;
>> > +  private CompressionCodecName codec = CompressionCodecName.SNAPPY;
>> > +  private WriterVersion writerVersion = WriterVersion.PARQUET_1_0;
>> > +
>> > +  private long recordCount = 0;
>> > +  private long recordCountForNextMemCheck =
>> MINIMUM_RECORD_COUNT_FOR_CHECK;
>> > +
>> > +  private ColumnWriteStoreImpl store;
>> > +  private PageWriteStore pageStore;
>> > +
>> > +  private RecordConsumer consumer;
>> > +  private BatchSchema batchSchema;
>> > +
>> > +  private Configuration conf;
>> > +  private String location;
>> > +  private String prefix;
>> > +  private int index = 0;
>> > +
>> > +  @Override
>> > +  public void init(Map<String, String> writerOptions) throws
>> IOException {
>> > +    this.location = writerOptions.get("location");
>> > +    this.prefix = writerOptions.get("prefix");
>> > +
>> > +    conf = new Configuration();
>> > +    conf.set(FileSystem.FS_DEFAULT_NAME_KEY,
>> writerOptions.get(FileSystem.FS_DEFAULT_NAME_KEY));
>> > +    blockSize =
>> Integer.parseInt(writerOptions.get(ExecConstants.PARQUET_BLOCK_SIZE));
>> > +  }
>> > +
>> > +  @Override
>> > +  public void updateSchema(BatchSchema batchSchema) throws IOException {
>> > +    if (this.batchSchema == null ||
>> !this.batchSchema.equals(batchSchema)) {
>> > +      if (this.batchSchema != null) {
>> > +        flush();
>> > +      }
>> > +      this.batchSchema = batchSchema;
>> > +      newSchema();
>> > +    }
>> > +  }
>> > +
>> > +  private void newSchema() throws IOException {
>> > +    List<Type> types = Lists.newArrayList();
>> > +    for (MaterializedField field : batchSchema) {
>> > +      String name = field.getAsSchemaPath().getAsUnescapedPath();
>> > +      MinorType minorType = field.getType().getMinorType();
>> > +      PrimitiveTypeName primitiveTypeName =
>> ParquetTypeHelper.getPrimitiveTypeNameForMinorType(minorType);
>> > +      Repetition repetition =
>> ParquetTypeHelper.getRepetitionForDataMode(field.getDataMode());
>> > +      OriginalType originalType =
>> ParquetTypeHelper.getOriginalTypeForMinorType(minorType);
>> > +      DecimalMetadata decimalMetadata =
>> ParquetTypeHelper.getDecimalMetadataForField(field);
>> > +      int length = ParquetTypeHelper.getLengthForMinorType(minorType);
>> > +      parquet.schema.Type type = new
>> parquet.schema.PrimitiveType(repetition, primitiveTypeName, length, name,
>> originalType, decimalMetadata);
>> > +      types.add(type);
>> > +    }
>> > +    schema = new MessageType("root", types);
>> > +
>> > +    Path fileName = new Path(location, prefix + "_" + index +
>> ".parquet");
>> > +    w = new ParquetFileWriter(conf, schema, fileName);
>> > +    w.start();
>> > +
>> > +    int initialBlockBufferSize = max(MINIMUM_BUFFER_SIZE, blockSize /
>> this.schema.getColumns().size() / 5);
>> > +    pageStore =
>> ColumnChunkPageWriteStoreExposer.newColumnChunkPageWriteStore(codec,
>> pageSize, this.schema, initialBlockBufferSize);
>> > +    int initialPageBufferSize = max(MINIMUM_BUFFER_SIZE, min(pageSize +
>> pageSize / 10, initialBlockBufferSize));
>> > +    store = new ColumnWriteStoreImpl(pageStore, pageSize,
>> initialPageBufferSize, dictionaryPageSize, enableDictionary, writerVersion);
>> > +    MessageColumnIO columnIO = new
>> ColumnIOFactory(validating).getColumnIO(this.schema);
>> > +    consumer = columnIO.getRecordWriter(store);
>> > +    setUp(schema, consumer);
>> > +  }
>> > +
>> > +  private void flush() throws IOException {
>> > +    w.startBlock(recordCount);
>> > +    store.flush();
>> > +    ColumnChunkPageWriteStoreExposer.flushPageStore(pageStore, w);
>> > +    recordCount = 0;
>> > +    w.endBlock();
>> > +    w.end(extraMetaData);
>> > +    store = null;
>> > +    pageStore = null;
>> > +    index++;
>> > +  }
>> > +
>> > +  private void checkBlockSizeReached() throws IOException {
>> > +    if (recordCount >= recordCountForNextMemCheck) { // checking the
>> memory size is relatively expensive, so let's not do it for every record.
>> > +      long memSize = store.memSize();
>> > +      if (memSize > blockSize) {
>> > +        System.out.println("Reached block size " + blockSize);
>> > +        flush();
>> > +        newSchema();
>> > +        recordCountForNextMemCheck =
>> min(max(MINIMUM_RECORD_COUNT_FOR_CHECK, recordCount / 2),
>> MAXIMUM_RECORD_COUNT_FOR_CHECK);
>> > +      } else {
>> > +        float recordSize = (float) memSize / recordCount;
>> > +        recordCountForNextMemCheck = min(
>> > +                max(MINIMUM_RECORD_COUNT_FOR_CHECK, (recordCount +
>> (long)(blockSize / recordSize)) / 2), // will check halfway
>> > +                recordCount + MAXIMUM_RECORD_COUNT_FOR_CHECK // will
>> not look more than max records ahead
>> > +        );
>> > +      }
>> > +    }
>> > +  }
>> > +
>> > +  @Override
>> > +  public void startRecord() throws IOException {
>> > +    consumer.startMessage();
>> > +  }
>> > +
>> > +  @Override
>> > +  public void endRecord() throws IOException {
>> > +    consumer.endMessage();
>> > +    recordCount++;
>> > +    checkBlockSizeReached();
>> > +  }
>> > +
>> > +  @Override
>> > +  public void abort() throws IOException {
>> > +    //To change body of implemented methods use File | Settings | File
>> Templates.
>> > +  }
>> > +
>> > +  @Override
>> > +  public void cleanup() throws IOException {
>> > +    flush();
>> > +  }
>> > +}
>> >
>> >
>> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/63b03467/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetWriter.java
>> > ----------------------------------------------------------------------
>> > diff --git
>> a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetWriter.java
>> b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetWriter.java
>> > new file mode 100644
>> > index 0000000..ea9798a
>> > --- /dev/null
>> > +++
>> b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetWriter.java
>> > @@ -0,0 +1,91 @@
>> > +/**
>> > + * Licensed to the Apache Software Foundation (ASF) under one
>> > + * or more contributor license agreements.  See the NOTICE file
>> > + * distributed with this work for additional information
>> > + * regarding copyright ownership.  The ASF licenses this file
>> > + * to you under the Apache License, Version 2.0 (the
>> > + * "License"); you may not use this file except in compliance
>> > + * with the License.  You may obtain a copy of the License at
>> > + *
>> > + * http://www.apache.org/licenses/LICENSE-2.0
>> > + *
>> > + * Unless required by applicable law or agreed to in writing, software
>> > + * distributed under the License is distributed on an "AS IS" BASIS,
>> > + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
>> implied.
>> > + * See the License for the specific language governing permissions and
>> > + * limitations under the License.
>> > + */
>> > +package org.apache.drill.exec.store.parquet;
>> > +
>> > +import com.fasterxml.jackson.annotation.*;
>> > +import com.google.common.base.Preconditions;
>> > +import org.apache.drill.common.exceptions.ExecutionSetupException;
>> > +import org.apache.drill.common.logical.FormatPluginConfig;
>> > +import org.apache.drill.common.logical.StoragePluginConfig;
>> > +import org.apache.drill.exec.physical.OperatorCost;
>> > +import org.apache.drill.exec.physical.base.AbstractWriter;
>> > +import org.apache.drill.exec.physical.base.PhysicalOperator;
>> > +import org.apache.drill.exec.store.StoragePluginRegistry;
>> > +
>> > +import java.io.IOException;
>> > +
>> > +@JsonTypeName("parquet-writer")
>> > +public class ParquetWriter extends AbstractWriter {
>> > +  static final org.slf4j.Logger logger =
>> org.slf4j.LoggerFactory.getLogger(ParquetWriter.class);
>> > +
>> > +  private final String location;
>> > +  private final ParquetFormatPlugin formatPlugin;
>> > +
>> > +  @JsonCreator
>> > +  public ParquetWriter(
>> > +          @JsonProperty("child") PhysicalOperator child,
>> > +          @JsonProperty("location") String location,
>> > +          @JsonProperty("storage") StoragePluginConfig storageConfig,
>> > +          @JacksonInject StoragePluginRegistry engineRegistry) throws
>> IOException, ExecutionSetupException {
>> > +
>> > +    super(child);
>> > +    this.formatPlugin = (ParquetFormatPlugin)
>> engineRegistry.getFormatPlugin(storageConfig, new ParquetFormatConfig());
>> > +    Preconditions.checkNotNull(formatPlugin, "Unable to load format
>> plugin for provided format config.");
>> > +    this.location = location;
>> > +  }
>> > +
>> > +  public ParquetWriter(PhysicalOperator child,
>> > +                       String location,
>> > +                       ParquetFormatPlugin formatPlugin) {
>> > +
>> > +    super(child);
>> > +    this.formatPlugin = formatPlugin;
>> > +    this.location = location;
>> > +  }
>> > +
>> > +  @JsonProperty("location")
>> > +  public String getLocation() {
>> > +    return location;
>> > +  }
>> > +
>> > +  @JsonProperty("storage")
>> > +  public StoragePluginConfig getStorageConfig(){
>> > +    return formatPlugin.getStorageConfig();
>> > +  }
>> > +
>> > +  @JsonProperty("format")
>> > +  public FormatPluginConfig getFormatConfig(){
>> > +    return formatPlugin.getConfig();
>> > +  }
>> > +
>> > +  @JsonIgnore
>> > +  public ParquetFormatPlugin getFormatPlugin(){
>> > +    return formatPlugin;
>> > +  }
>> > +
>> > +  @Override
>> > +  protected PhysicalOperator getNewWithChild(PhysicalOperator child) {
>> > +    return new ParquetWriter(child, location, formatPlugin);
>> > +  }
>> > +
>> > +  @Override
>> > +  public OperatorCost getCost() {
>> > +    // TODO:
>> > +    return new OperatorCost(1,1,1,1);
>> > +  }
>> > +}
>> >
>>
>
>
>
> --
>  Steven Phillips
>  Software Engineer
>
>  mapr.com

Re: [3/3] git commit: DRILL-694: Implement parquet writer.

Posted by Steven Phillips <sp...@maprtech.com>.
Do you see the failure repeatedly, even if you run just that single test?

I sometimes see tests that fail intermittently, but pass when run in
isolation.

I will try the test again and confirm that it is passing.


On Mon, May 12, 2014 at 10:22 PM, Timothy Chen <tn...@gmail.com> wrote:

> I got a unit test failure on master from the testParquetWriter?
>
> Is the test passing for you?
>
> Tim
>
> On Mon, May 12, 2014 at 8:46 PM,  <ja...@apache.org> wrote:
> > DRILL-694: Implement parquet writer.
> >
> > Enable "CREATE TABLE AS" with parquet as the output format.
> >
> > Add decimal metadata support to parquet reader and writer.
> >
> >
> > Project: http://git-wip-us.apache.org/repos/asf/incubator-drill/repo
> > Commit:
> http://git-wip-us.apache.org/repos/asf/incubator-drill/commit/63b03467
> > Tree:
> http://git-wip-us.apache.org/repos/asf/incubator-drill/tree/63b03467
> > Diff:
> http://git-wip-us.apache.org/repos/asf/incubator-drill/diff/63b03467
> >
> > Branch: refs/heads/master
> > Commit: 63b0346761692f0a4c851e7910791c028dc3cded
> > Parents: 14c628c
> > Author: Steven Phillips <sp...@maprtech.com>
> > Authored: Mon May 12 02:56:25 2014 -0700
> > Committer: Aditya Kishore <ad...@maprtech.com>
> > Committed: Mon May 12 18:18:43 2014 -0700
> >
> > ----------------------------------------------------------------------
> >  .../org/apache/drill/common/types/Types.java    |   4 +
> >  .../drill/common/util/DecimalUtility.java       |  11 +-
> >  exec/java-exec/pom.xml                          |   4 +-
> >  .../templates/EventBasedRecordWriter.java       |   6 +-
> >  .../templates/ParquetOutputRecordWriter.java    | 189 +++++++++++
> >  .../codegen/templates/ParquetTypeHelper.java    | 155 ++++++++++
> >  .../org/apache/drill/exec/ExecConstants.java    |   8 +
> >  .../server/options/SystemOptionManager.java     |   5 +-
> >  .../exec/store/dfs/WorkspaceSchemaFactory.java  |   9 +-
> >  .../drill/exec/store/parquet/BitReader.java     |   5 +-
> >  .../drill/exec/store/parquet/ColumnReader.java  |  14 +-
> >  .../store/parquet/FixedByteAlignedReader.java   |  87 +++++-
> >  .../exec/store/parquet/NullableBitReader.java   |   5 +-
> >  .../store/parquet/NullableColumnReader.java     |   5 +-
> >  .../parquet/NullableFixedByteAlignedReader.java |  81 ++++-
> >  .../exec/store/parquet/ParquetFormatPlugin.java |  48 ++-
> >  .../exec/store/parquet/ParquetRecordReader.java | 310
> ++++++++++++++-----
> >  .../exec/store/parquet/ParquetRecordWriter.java | 185 +++++++++++
> >  .../drill/exec/store/parquet/ParquetWriter.java |  91 ++++++
> >  .../parquet/ParquetWriterBatchCreator.java      |  36 +++
> >  .../store/parquet/VarLengthColumnReaders.java   | 182 ++++++++---
> >  .../ColumnChunkPageWriteStoreExposer.java       |  39 +++
> >  .../physical/impl/writer/TestParquetWriter.java | 163 ++++++++++
> >  .../exec/physical/impl/writer/TestWriter.java   |  19 +-
> >  24 files changed, 1510 insertions(+), 151 deletions(-)
> > ----------------------------------------------------------------------
> >
> >
> >
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/63b03467/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 c6ac82e..a6ec29b 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
> > @@ -271,6 +271,10 @@ public class Types {
> >      return
> MajorType.newBuilder().setMode(mode).setMinorType(type).build();
> >    }
> >
> > +  public static MajorType withScaleAndPrecision(MinorType type,
> DataMode mode, int scale, int precision) {
> > +    return
> MajorType.newBuilder().setMinorType(type).setMode(mode).setScale(scale).setPrecision(precision).build();
> > +  }
> > +
> >    public static MajorType required(MinorType type){
> >      return
> MajorType.newBuilder().setMode(DataMode.REQUIRED).setMinorType(type).build();
> >    }
> >
> >
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/63b03467/common/src/main/java/org/apache/drill/common/util/DecimalUtility.java
> > ----------------------------------------------------------------------
> > diff --git
> a/common/src/main/java/org/apache/drill/common/util/DecimalUtility.java
> b/common/src/main/java/org/apache/drill/common/util/DecimalUtility.java
> > index dbfd6ac..0cacc59 100644
> > ---
> a/common/src/main/java/org/apache/drill/common/util/DecimalUtility.java
> > +++
> b/common/src/main/java/org/apache/drill/common/util/DecimalUtility.java
> > @@ -17,8 +17,7 @@
> >   */
> >  package org.apache.drill.common.util;
> >
> > -import java.math.BigDecimal;
> > -import java.math.BigInteger;
> > +import java.math.*;
> >
> >
> >  import io.netty.buffer.Unpooled;
> > @@ -26,6 +25,7 @@ import io.netty.buffer.ByteBuf;
> >
> >  import java.math.BigDecimal;
> >  import java.math.BigInteger;
> > +import java.util.Arrays;
> >
> >  public class DecimalUtility {
> >
> > @@ -486,5 +486,12 @@ public class DecimalUtility {
> >        return 0;
> >      }
> >
> > +
> > +    public static BigDecimal getBigDecimalFromByteArray(byte[] bytes,
> int start, int length, int scale) {
> > +      byte[] value = Arrays.copyOfRange(bytes, start, start + length);
> > +      BigInteger unscaledValue = new BigInteger(value);
> > +      return new BigDecimal(unscaledValue, scale);
> > +    }
> > +
> >  }
> >
> >
> >
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/63b03467/exec/java-exec/pom.xml
> > ----------------------------------------------------------------------
> > diff --git a/exec/java-exec/pom.xml b/exec/java-exec/pom.xml
> > index 3e26662..e282c00 100644
> > --- a/exec/java-exec/pom.xml
> > +++ b/exec/java-exec/pom.xml
> > @@ -95,7 +95,7 @@
> >      <dependency>
> >        <groupId>com.twitter</groupId>
> >        <artifactId>parquet-column</artifactId>
> > -      <version>1.4.0</version>
> > +      <version>1.5.0-SNAPSHOT</version>
> >        <exclusions>
> >          <exclusion>
> >            <groupId>org.apache.hadoop</groupId>
> > @@ -110,7 +110,7 @@
> >      <dependency>
> >        <groupId>com.twitter</groupId>
> >        <artifactId>parquet-hadoop</artifactId>
> > -      <version>1.4.0</version>
> > +      <version>1.5.0-SNAPSHOT</version>
> >        <exclusions>
> >          <exclusion>
> >            <groupId>org.apache.hadoop</groupId>
> >
> >
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/63b03467/exec/java-exec/src/main/codegen/templates/EventBasedRecordWriter.java
> > ----------------------------------------------------------------------
> > diff --git
> a/exec/java-exec/src/main/codegen/templates/EventBasedRecordWriter.java
> b/exec/java-exec/src/main/codegen/templates/EventBasedRecordWriter.java
> > index 73b6566..b58f24c 100644
> > ---
> a/exec/java-exec/src/main/codegen/templates/EventBasedRecordWriter.java
> > +++
> b/exec/java-exec/src/main/codegen/templates/EventBasedRecordWriter.java
> > @@ -91,8 +91,10 @@ public class EventBasedRecordWriter {
> >      fieldWriters = Lists.newArrayList();
> >      try {
> >        for (int i = 0; i < schema.getFieldCount(); i++) {
> > -        fieldWriters.add(i,
> typeClassMap.get(schema.getColumn(i).getType())
> > -            .getConstructor(EventBasedRecordWriter.class,
> int.class).newInstance(this, i));
> > +        MajorType mt = schema.getColumn(i).getType();
> > +        MajorType newMt =
> MajorType.newBuilder().setMinorType(mt.getMinorType()).setMode(mt.getMode()).build();
> > +        fieldWriters.add(i, typeClassMap.get(newMt)
> > +                .getConstructor(EventBasedRecordWriter.class,
> int.class).newInstance(this, i));
> >        }
> >      } catch(Exception e) {
> >        logger.error("Failed to create FieldWriter.", e);
> >
> >
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/63b03467/exec/java-exec/src/main/codegen/templates/ParquetOutputRecordWriter.java
> > ----------------------------------------------------------------------
> > diff --git
> a/exec/java-exec/src/main/codegen/templates/ParquetOutputRecordWriter.java
> b/exec/java-exec/src/main/codegen/templates/ParquetOutputRecordWriter.java
> > new file mode 100644
> > index 0000000..5f75c1c
> > --- /dev/null
> > +++
> b/exec/java-exec/src/main/codegen/templates/ParquetOutputRecordWriter.java
> > @@ -0,0 +1,189 @@
> > +/**
> > + * Licensed to the Apache Software Foundation (ASF) under one
> > + * or more contributor license agreements.  See the NOTICE file
> > + * distributed with this work for additional information
> > + * regarding copyright ownership.  The ASF licenses this file
> > + * to you under the Apache License, Version 2.0 (the
> > + * "License"); you may not use this file except in compliance
> > + * with the License.  You may obtain a copy of the License at
> > + *
> > + * http://www.apache.org/licenses/LICENSE-2.0
> > + *
> > + * Unless required by applicable law or agreed to in writing, software
> > + * distributed under the License is distributed on an "AS IS" BASIS,
> > + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
> implied.
> > + * See the License for the specific language governing permissions and
> > + * limitations under the License.
> > + */
> > +
> > +import parquet.io.api.Binary;
> > +
> > +import java.lang.Override;
> > +
> > +<@pp.dropOutputFile />
> > +<@pp.changeOutputFile
> name="org/apache/drill/exec/store/ParquetOutputRecordWriter.java" />
> > +<#include "/@includes/license.ftl" />
> > +
> > +package org.apache.drill.exec.store;
> > +
> > +import com.google.common.collect.Lists;
> > +import org.apache.drill.common.types.TypeProtos.MinorType;
> > +import org.apache.drill.exec.expr.TypeHelper;
> > +import org.apache.drill.exec.expr.holders.*;
> > +import org.apache.drill.exec.record.BatchSchema;
> > +import org.apache.drill.exec.store.parquet.ParquetTypeHelper;
> > +import org.apache.drill.exec.vector.*;
> > +import org.apache.drill.common.util.DecimalUtility;
> > +import parquet.io.api.RecordConsumer;
> > +import parquet.schema.MessageType;
> > +import parquet.io.api.Binary;
> > +import io.netty.buffer.ByteBuf;
> > +
> > +import java.io.IOException;
> > +import java.lang.UnsupportedOperationException;
> > +import java.util.Arrays;
> > +import java.util.List;
> > +import java.util.Map;
> > +
> > +/**
> > + * Abstract implementation of RecordWriter interface which exposes
> interface:
> > + *    {@link #writeHeader(List)}
> > + *    {@link #addField(int,String)}
> > + * to output the data in string format instead of implementing addField
> for each type holder.
> > + *
> > + * This is useful for text format writers such as CSV, TSV etc.
> > + */
> > +public abstract class ParquetOutputRecordWriter implements RecordWriter
> {
> > +
> > +  private RecordConsumer consumer;
> > +  private MessageType schema;
> > +
> > +  public void setUp(MessageType schema, RecordConsumer consumer) {
> > +    this.schema = schema;
> > +    this.consumer = consumer;
> > +  }
> > +
> > +<#list vv.types as type>
> > +  <#list type.minor as minor>
> > +    <#list vv.modes as mode>
> > +  @Override
> > +  public void add${mode.prefix}${minor.class}Holder(int fieldId,
> ${mode.prefix}${minor.class}Holder valueHolder) throws IOException {
> > +  <#if mode.prefix == "Nullable" >
> > +    if (valueHolder.isSet == 0) {
> > +      return;
> > +    }
> > +  <#elseif mode.prefix == "Repeated" >
> > +    consumer.startField(schema.getFieldName(fieldId), fieldId);
> > +    for (int i = valueHolder.start; i < valueHolder.end; i++) {
> > +  </#if>
> > +
> > +  <#if  minor.class == "TinyInt" ||
> > +        minor.class == "UInt1" ||
> > +        minor.class == "UInt2" ||
> > +        minor.class == "SmallInt" ||
> > +        minor.class == "Int" ||
> > +        minor.class == "Time" ||
> > +        minor.class == "IntervalYear" ||
> > +        minor.class == "Decimal9" ||
> > +        minor.class == "UInt4">
> > +    <#if mode.prefix == "Repeated" >
> > +
>  consumer.addInteger(valueHolder.vector.getAccessor().get(i));
> > +    <#else>
> > +    consumer.startField(schema.getFieldName(fieldId), fieldId);
> > +    consumer.addInteger(valueHolder.value);
> > +    consumer.endField(schema.getFieldName(fieldId), fieldId);
> > +    </#if>
> > +  <#elseif
> > +        minor.class == "Float4">
> > +      <#if mode.prefix == "Repeated" >
> > +
>  consumer.addFloat(valueHolder.vector.getAccessor().get(i));
> > +      <#else>
> > +    consumer.startField(schema.getFieldName(fieldId), fieldId);
> > +    consumer.addFloat(valueHolder.value);
> > +    consumer.endField(schema.getFieldName(fieldId), fieldId);
> > +      </#if>
> > +  <#elseif
> > +        minor.class == "BigInt" ||
> > +        minor.class == "Decimal18" ||
> > +        minor.class == "TimeStamp" ||
> > +        minor.class == "Date" ||
> > +        minor.class == "UInt8">
> > +      <#if mode.prefix == "Repeated" >
> > +              consumer.addLong(valueHolder.vector.getAccessor().get(i));
> > +      <#else>
> > +    consumer.startField(schema.getFieldName(fieldId), fieldId);
> > +    consumer.addLong(valueHolder.value);
> > +    consumer.endField(schema.getFieldName(fieldId), fieldId);
> > +      </#if>
> > +  <#elseif
> > +        minor.class == "Float8">
> > +      <#if mode.prefix == "Repeated" >
> > +
>  consumer.addDouble(valueHolder.vector.getAccessor().get(i));
> > +      <#else>
> > +    consumer.startField(schema.getFieldName(fieldId), fieldId);
> > +    consumer.addDouble(valueHolder.value);
> > +    consumer.endField(schema.getFieldName(fieldId), fieldId);
> > +      </#if>
> > +  <#elseif
> > +        minor.class == "Bit">
> > +      <#if mode.prefix == "Repeated" >
> > +
>  consumer.addBoolean(valueHolder.vector.getAccessor().get(i) == 1);
> > +      <#else>
> > +    consumer.startField(schema.getFieldName(fieldId), fieldId);
> > +    consumer.addBoolean(valueHolder.value == 1);
> > +    consumer.endField(schema.getFieldName(fieldId), fieldId);
> > +      </#if>
> > +  <#elseif
> > +        minor.class == "Decimal28Sparse" ||
> > +        minor.class == "Decimal38Sparse">
> > +      <#if mode.prefix == "Repeated" >
> > +      <#else>
> > +      consumer.startField(schema.getFieldName(fieldId), fieldId);
> > +      byte[] bytes = DecimalUtility.getBigDecimalFromSparse(
> > +              valueHolder.buffer, valueHolder.start,
> ${minor.class}Holder.nDecimalDigits,
> valueHolder.scale).unscaledValue().toByteArray();
> > +      byte[] output = new
> byte[ParquetTypeHelper.getLengthForMinorType(MinorType.${minor.class?upper_case})];
> > +      if (valueHolder.sign) {
> > +        Arrays.fill(output, 0, output.length - bytes.length,
> (byte)0xFF);
> > +      } else {
> > +        Arrays.fill(output, 0, output.length - bytes.length, (byte)0x0);
> > +      }
> > +      System.arraycopy(bytes, 0, output, output.length - bytes.length,
> bytes.length);
> > +      consumer.addBinary(Binary.fromByteArray(output));
> > +      consumer.endField(schema.getFieldName(fieldId), fieldId);
> > +      </#if>
> > +  <#elseif
> > +        minor.class == "TimeTZ" ||
> > +        minor.class == "TimeStampTZ" ||
> > +        minor.class == "IntervalDay" ||
> > +        minor.class == "Interval" ||
> > +        minor.class == "Decimal28Dense" ||
> > +        minor.class == "Decimal38Dense">
> > +
> > +      <#if mode.prefix == "Repeated" >
> > +      <#else>
> > +
> > +      </#if>
> > +  <#elseif minor.class == "VarChar" || minor.class == "Var16Char" ||
> minor.class == "VarBinary">
> > +    <#if mode.prefix == "Repeated">
> > +      ${minor.class}Holder singleHolder = new ${minor.class}Holder();
> > +      valueHolder.vector.getAccessor().get(i, singleHolder);
> > +      consumer.startField(schema.getFieldName(fieldId), fieldId);
> > +
>  consumer.addBinary(Binary.fromByteBuffer(singleHolder.buffer.nioBuffer(singleHolder.start,
> singleHolder.end - singleHolder.start)));
> > +      consumer.endField(schema.getFieldName(fieldId), fieldId);
> > +    <#else>
> > +    ByteBuf buf = valueHolder.buffer;
> > +    consumer.startField(schema.getFieldName(fieldId), fieldId);
> > +
>  consumer.addBinary(Binary.fromByteBuffer(valueHolder.buffer.nioBuffer(valueHolder.start,
> valueHolder.end - valueHolder.start)));
> > +    consumer.endField(schema.getFieldName(fieldId), fieldId);
> > +    </#if>
> > +  </#if>
> > +  <#if mode.prefix == "Repeated">
> > +    }
> > +    consumer.endField(schema.getFieldName(fieldId), fieldId);
> > +  </#if>
> > +  }
> > +    </#list>
> > +  </#list>
> > +</#list>
> > +
> > +}
> >
> >
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/63b03467/exec/java-exec/src/main/codegen/templates/ParquetTypeHelper.java
> > ----------------------------------------------------------------------
> > diff --git
> a/exec/java-exec/src/main/codegen/templates/ParquetTypeHelper.java
> b/exec/java-exec/src/main/codegen/templates/ParquetTypeHelper.java
> > new file mode 100644
> > index 0000000..b268d33
> > --- /dev/null
> > +++ b/exec/java-exec/src/main/codegen/templates/ParquetTypeHelper.java
> > @@ -0,0 +1,155 @@
> > +/**
> > + * Licensed to the Apache Software Foundation (ASF) under one
> > + * or more contributor license agreements.  See the NOTICE file
> > + * distributed with this work for additional information
> > + * regarding copyright ownership.  The ASF licenses this file
> > + * to you under the Apache License, Version 2.0 (the
> > + * "License"); you may not use this file except in compliance
> > + * with the License.  You may obtain a copy of the License at
> > + *
> > + * http://www.apache.org/licenses/LICENSE-2.0
> > + *
> > + * Unless required by applicable law or agreed to in writing, software
> > + * distributed under the License is distributed on an "AS IS" BASIS,
> > + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
> implied.
> > + * See the License for the specific language governing permissions and
> > + * limitations under the License.
> > + */
> > +
> > +import parquet.format.ConvertedType;
> > +import parquet.schema.DecimalMetadata;
> > +import parquet.schema.OriginalType;
> > +import parquet.schema.PrimitiveType.PrimitiveTypeName;
> > +
> > +<@pp.dropOutputFile />
> > +<@pp.changeOutputFile
> name="org/apache/drill/exec/store/parquet/ParquetTypeHelper.java" />
> > +<#include "/@includes/license.ftl" />
> > +
> > +package org.apache.drill.exec.store.parquet;
> > +
> > +import org.apache.drill.common.types.TypeProtos.DataMode;
> > +import org.apache.drill.common.types.TypeProtos.MinorType;
> > +import org.apache.drill.exec.record.MaterializedField;
> > +import parquet.schema.OriginalType;
> > +import parquet.schema.DecimalMetadata;
> > +import parquet.schema.PrimitiveType.PrimitiveTypeName;
> > +import parquet.schema.Type.Repetition;
> > +
> > +import java.util.HashMap;
> > +import java.util.Map;
> > +
> > +public class ParquetTypeHelper {
> > +  private static Map<MinorType,PrimitiveTypeName> typeMap;
> > +  private static Map<DataMode,Repetition> modeMap;
> > +  private static Map<MinorType,OriginalType> originalTypeMap;
> > +
> > +  static {
> > +    typeMap = new HashMap();
> > +
> > +    <#list vv.types as type>
> > +    <#list type.minor as minor>
> > +    <#if    minor.class == "TinyInt" ||
> > +            minor.class == "UInt1" ||
> > +            minor.class == "UInt2" ||
> > +            minor.class == "SmallInt" ||
> > +            minor.class == "Int" ||
> > +            minor.class == "Time" ||
> > +            minor.class == "IntervalYear" ||
> > +            minor.class == "Decimal9" ||
> > +            minor.class == "UInt4">
> > +                    typeMap.put(MinorType.${minor.class?upper_case},
> PrimitiveTypeName.INT32);
> > +    <#elseif
> > +            minor.class == "Float4">
> > +                    typeMap.put(MinorType.${minor.class?upper_case},
> PrimitiveTypeName.FLOAT);
> > +    <#elseif
> > +            minor.class == "BigInt" ||
> > +            minor.class == "Decimal18" ||
> > +            minor.class == "TimeStamp" ||
> > +            minor.class == "Date" ||
> > +            minor.class == "UInt8">
> > +                    typeMap.put(MinorType.${minor.class?upper_case},
> PrimitiveTypeName.INT64);
> > +    <#elseif
> > +            minor.class == "Float8">
> > +                    typeMap.put(MinorType.${minor.class?upper_case},
> PrimitiveTypeName.DOUBLE);
> > +    <#elseif
> > +            minor.class == "Bit">
> > +                    typeMap.put(MinorType.${minor.class?upper_case},
> PrimitiveTypeName.BOOLEAN);
> > +    <#elseif
> > +            minor.class == "TimeTZ" ||
> > +            minor.class == "TimeStampTZ" ||
> > +            minor.class == "IntervalDay" ||
> > +            minor.class == "Interval" ||
> > +            minor.class == "Decimal28Dense" ||
> > +            minor.class == "Decimal38Dense" ||
> > +            minor.class == "Decimal28Sparse" ||
> > +            minor.class == "Decimal38Sparse">
> > +                    typeMap.put(MinorType.${minor.class?upper_case},
> PrimitiveTypeName.FIXED_LEN_BYTE_ARRAY);
> > +    <#elseif
> > +            minor.class == "VarChar" ||
> > +            minor.class == "Var16Char" ||
> > +            minor.class == "VarBinary" >
> > +                    typeMap.put(MinorType.${minor.class?upper_case},
> PrimitiveTypeName.BINARY);
> > +    </#if>
> > +    </#list>
> > +    </#list>
> > +
> > +    modeMap = new HashMap();
> > +
> > +    modeMap.put(DataMode.REQUIRED, Repetition.REQUIRED);
> > +    modeMap.put(DataMode.OPTIONAL, Repetition.OPTIONAL);
> > +    modeMap.put(DataMode.REPEATED, Repetition.REPEATED);
> > +
> > +    originalTypeMap = new HashMap();
> > +
> > +    <#list vv.types as type>
> > +    <#list type.minor as minor>
> > +            <#if minor.class.startsWith("Decimal")>
> > +
>  originalTypeMap.put(MinorType.${minor.class?upper_case},OriginalType.DECIMAL);
> > +            </#if>
> > +    </#list>
> > +    </#list>
> > +            originalTypeMap.put(MinorType.VARCHAR, OriginalType.UTF8);
> > +            originalTypeMap.put(MinorType.DATE, OriginalType.DATE);
> > +            originalTypeMap.put(MinorType.TIME, OriginalType.TIME);
> > +            originalTypeMap.put(MinorType.TIMESTAMP,
> OriginalType.TIMESTAMP);
> > +            originalTypeMap.put(MinorType.TIMESTAMPTZ,
> OriginalType.TIMESTAMPTZ);
> > +  }
> > +
> > +  public static PrimitiveTypeName
> getPrimitiveTypeNameForMinorType(MinorType minorType) {
> > +    return typeMap.get(minorType);
> > +  }
> > +
> > +  public static Repetition getRepetitionForDataMode(DataMode dataMode) {
> > +    return modeMap.get(dataMode);
> > +  }
> > +
> > +  public static OriginalType getOriginalTypeForMinorType(MinorType
> minorType) {
> > +    return originalTypeMap.get(minorType);
> > +  }
> > +
> > +  public static DecimalMetadata
> getDecimalMetadataForField(MaterializedField field) {
> > +    switch(field.getType().getMinorType()) {
> > +      case DECIMAL9:
> > +      case DECIMAL18:
> > +      case DECIMAL28SPARSE:
> > +      case DECIMAL28DENSE:
> > +      case DECIMAL38SPARSE:
> > +      case DECIMAL38DENSE:
> > +        return new DecimalMetadata(field.getPrecision(),
> field.getScale());
> > +      default:
> > +        return null;
> > +    }
> > +  }
> > +
> > +  public static int getLengthForMinorType(MinorType minorType) {
> > +    switch(minorType) {
> > +      case DECIMAL28SPARSE:
> > +        return 12;
> > +      case DECIMAL38SPARSE:
> > +        return 16;
> > +      default:
> > +        return 0;
> > +    }
> > +  }
> > +
> > +}
> >
> >
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/63b03467/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 9eee08d..238fae9 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
> > @@ -17,6 +17,10 @@
> >   */
> >  package org.apache.drill.exec;
> >
> > +import org.apache.drill.exec.server.options.OptionValidator;
> > +import
> org.apache.drill.exec.server.options.TypeValidators.LongValidator;
> > +import
> org.apache.drill.exec.server.options.TypeValidators.StringValidator;
> > +
> >  public interface ExecConstants {
> >    public static final String ZK_RETRY_TIMES =
> "drill.exec.zk.retry.count";
> >    public static final String ZK_RETRY_DELAY =
> "drill.exec.zk.retry.delay";
> > @@ -63,5 +67,9 @@ public interface ExecConstants {
> >    public static final String FILESYSTEM_PARTITION_COLUMN_LABEL =
> "drill.exec.storage.file.partition.column.label";
> >    public static final String HAZELCAST_SUBNETS =
> "drill.exec.cache.hazel.subnets";
> >    public static final String TOP_LEVEL_MAX_ALLOC =
> "drill.exec.memory.top.max";
> > +  public static final String OUTPUT_FORMAT_OPTION = "store.format";
> > +  public static final OptionValidator OUTPUT_FORMAT_VALIDATOR = new
> StringValidator(OUTPUT_FORMAT_OPTION, "parquet");
> > +  public static final String PARQUET_BLOCK_SIZE = "parquet.block.size";
> > +  public static final OptionValidator PARQUET_BLOCK_SIZE_VALIDATOR =
> new LongValidator(PARQUET_BLOCK_SIZE, 512*1024*1024);
> >
> >  }
> >
> >
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/63b03467/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 cfe8e2c..21031e5 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
> > @@ -22,6 +22,7 @@ import java.util.Map;
> >  import java.util.Map.Entry;
> >  import java.util.concurrent.ConcurrentMap;
> >
> > +import org.apache.drill.exec.ExecConstants;
> >  import org.apache.drill.exec.cache.DistributedCache;
> >  import org.apache.drill.exec.cache.DistributedMap;
> >  import org.apache.drill.exec.planner.physical.PlannerSettings;
> > @@ -38,7 +39,9 @@ public class SystemOptionManager implements
> OptionManager{
> >        PlannerSettings.STREAMAGG,
> >        PlannerSettings.HASHJOIN,
> >        PlannerSettings.MERGEJOIN,
> > -      PlannerSettings.MULTIPHASE
> > +      PlannerSettings.MULTIPHASE,
> > +      ExecConstants.OUTPUT_FORMAT_VALIDATOR,
> > +      ExecConstants.PARQUET_BLOCK_SIZE_VALIDATOR
> >    };
> >
> >    private DistributedMap<OptionValue> options;
> >
> >
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/63b03467/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/WorkspaceSchemaFactory.java
> > ----------------------------------------------------------------------
> > diff --git
> a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/WorkspaceSchemaFactory.java
> b/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/WorkspaceSchemaFactory.java
> > index df73ea5..72b31e6 100644
> > ---
> a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/WorkspaceSchemaFactory.java
> > +++
> b/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/WorkspaceSchemaFactory.java
> > @@ -26,12 +26,15 @@ import com.google.common.collect.Lists;
> >  import com.google.common.collect.Sets;
> >  import net.hydromatic.optiq.Table;
> >  import org.apache.drill.common.exceptions.ExecutionSetupException;
> > +import org.apache.drill.exec.ExecConstants;
> >  import org.apache.drill.exec.planner.logical.CreateTableEntry;
> >  import org.apache.drill.exec.planner.logical.DrillTable;
> >  import org.apache.drill.exec.planner.logical.DynamicDrillTable;
> >  import org.apache.drill.exec.planner.logical.FileSystemCreateTableEntry;
> >  import org.apache.drill.exec.planner.sql.ExpandingConcurrentMap;
> >  import org.apache.drill.exec.rpc.user.UserSession;
> > +import org.apache.drill.exec.server.options.OptionValidator;
> > +import
> org.apache.drill.exec.server.options.TypeValidators.StringValidator;
> >  import org.apache.drill.exec.store.AbstractSchema;
> >  import org.apache.drill.exec.store.dfs.shim.DrillFileSystem;
> >  import org.apache.hadoop.fs.Path;
> > @@ -141,9 +144,11 @@ public class WorkspaceSchemaFactory implements
> ExpandingConcurrentMap.MapValueFa
> >        return fs;
> >      }
> >
> > +
> >      @Override
> >      public CreateTableEntry createNewTable(String tableName) {
> > -      FormatPlugin formatPlugin =
> plugin.getFormatPlugin(config.getStorageFormat());
> > +      String storage =
> session.getOptions().getOption(ExecConstants.OUTPUT_FORMAT_OPTION).string_val;
> > +      FormatPlugin formatPlugin = plugin.getFormatPlugin(storage);
> >        if (formatPlugin == null)
> >          throw new UnsupportedOperationException(
> >            String.format("Unsupported format '%s' in workspace '%s'",
> config.getStorageFormat(),
> > @@ -151,7 +156,7 @@ public class WorkspaceSchemaFactory implements
> ExpandingConcurrentMap.MapValueFa
> >
> >        return new FileSystemCreateTableEntry(
> >            (FileSystemConfig) plugin.getConfig(),
> > -          plugin.getFormatPlugin(config.getStorageFormat()),
> > +          formatPlugin,
> >            config.getLocation() + Path.SEPARATOR + tableName);
> >      }
> >    }
> >
> >
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/63b03467/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/BitReader.java
> > ----------------------------------------------------------------------
> > diff --git
> a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/BitReader.java
> b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/BitReader.java
> > index c323222..7ae95cd 100644
> > ---
> a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/BitReader.java
> > +++
> b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/BitReader.java
> > @@ -22,6 +22,7 @@ import
> org.apache.drill.exec.vector.BaseDataValueVector;
> >  import org.apache.drill.exec.vector.ValueVector;
> >  import parquet.column.ColumnDescriptor;
> >  import parquet.format.ConvertedType;
> > +import parquet.format.SchemaElement;
> >  import parquet.hadoop.metadata.ColumnChunkMetaData;
> >
> >  final class BitReader extends ColumnReader {
> > @@ -31,8 +32,8 @@ final class BitReader extends ColumnReader {
> >    private byte[] bytes;
> >
> >    BitReader(ParquetRecordReader parentReader, int allocateSize,
> ColumnDescriptor descriptor, ColumnChunkMetaData columnChunkMetaData,
> > -            boolean fixedLength, ValueVector v, ConvertedType
> convertedType) throws ExecutionSetupException {
> > -    super(parentReader, allocateSize, descriptor, columnChunkMetaData,
> fixedLength, v, convertedType);
> > +            boolean fixedLength, ValueVector v, SchemaElement
> schemaElement) throws ExecutionSetupException {
> > +    super(parentReader, allocateSize, descriptor, columnChunkMetaData,
> fixedLength, v, schemaElement);
> >    }
> >
> >    @Override
> >
> >
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/63b03467/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ColumnReader.java
> > ----------------------------------------------------------------------
> > diff --git
> a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ColumnReader.java
> b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ColumnReader.java
> > index 196e1fd..b9faafe 100644
> > ---
> a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ColumnReader.java
> > +++
> b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ColumnReader.java
> > @@ -25,8 +25,10 @@ import org.apache.drill.exec.vector.ValueVector;
> >  import org.apache.hadoop.fs.FSDataInputStream;
> >  import parquet.column.ColumnDescriptor;
> >  import parquet.format.ConvertedType;
> > +import parquet.format.SchemaElement;
> >  import parquet.hadoop.metadata.ColumnChunkMetaData;
> >  import parquet.schema.PrimitiveType;
> > +import parquet.schema.PrimitiveType.PrimitiveTypeName;
> >
> >  import java.io.IOException;
> >
> > @@ -48,7 +50,7 @@ abstract class ColumnReader<V extends ValueVector> {
> >    // status information on the current page
> >    final PageReadStatus pageReadStatus;
> >
> > -  final ConvertedType convertedType;
> > +  final SchemaElement schemaElement;
> >
> >    // quick reference to see if the field is fixed length (as this
> requires an instanceof)
> >    final boolean isFixedLength;
> > @@ -70,12 +72,12 @@ abstract class ColumnReader<V extends ValueVector> {
> >    long readStartInBytes = 0, readLength = 0, readLengthInBits = 0,
> recordsReadInThisIteration = 0;
> >
> >    protected ColumnReader(ParquetRecordReader parentReader, int
> allocateSize, ColumnDescriptor descriptor,
> > -      ColumnChunkMetaData columnChunkMetaData, boolean fixedLength, V
> v, ConvertedType convertedType) throws ExecutionSetupException {
> > +      ColumnChunkMetaData columnChunkMetaData, boolean fixedLength, V
> v, SchemaElement schemaElement) throws ExecutionSetupException {
> >      this.parentReader = parentReader;
> >      this.columnDescriptor = descriptor;
> >      this.columnChunkMetaData = columnChunkMetaData;
> >      this.isFixedLength = fixedLength;
> > -    this.convertedType = convertedType;
> > +    this.schemaElement = schemaElement;
> >
> >      if (allocateSize > 1) {
> >        valueVec =  v;
> > @@ -87,7 +89,11 @@ abstract class ColumnReader<V extends ValueVector> {
> >      this.pageReadStatus = new PageReadStatus(this,
> parentReader.fileSystem, parentReader.hadoopPath, columnChunkMetaData);
> >
> >      if (columnDescriptor.getType() !=
> PrimitiveType.PrimitiveTypeName.BINARY) {
> > -      dataTypeLengthInBits =
> ParquetRecordReader.getTypeLengthInBits(columnDescriptor.getType());
> > +      if (columnDescriptor.getType() ==
> PrimitiveTypeName.FIXED_LEN_BYTE_ARRAY) {
> > +        dataTypeLengthInBits = columnDescriptor.getTypeLength() * 8;
> > +      } else {
> > +        dataTypeLengthInBits =
> ParquetRecordReader.getTypeLengthInBits(columnDescriptor.getType());
> > +      }
> >      }
> >
> >    }
> >
> >
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/63b03467/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/FixedByteAlignedReader.java
> > ----------------------------------------------------------------------
> > diff --git
> a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/FixedByteAlignedReader.java
> b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/FixedByteAlignedReader.java
> > index 0aa18cf..574b0cb 100644
> > ---
> a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/FixedByteAlignedReader.java
> > +++
> b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/FixedByteAlignedReader.java
> > @@ -18,20 +18,27 @@
> >  package org.apache.drill.exec.store.parquet;
> >
> >  import org.apache.drill.common.exceptions.ExecutionSetupException;
> > -import org.apache.drill.exec.vector.BaseDataValueVector;
> > +import org.apache.drill.common.util.DecimalUtility;
> > +import org.apache.drill.exec.expr.holders.Decimal28SparseHolder;
> > +import org.apache.drill.exec.expr.holders.Decimal38SparseHolder;
> > +import org.apache.drill.exec.vector.Decimal28SparseVector;
> > +import org.apache.drill.exec.vector.Decimal38SparseVector;
> >  import org.apache.drill.exec.vector.ValueVector;
> >  import parquet.column.ColumnDescriptor;
> >  import parquet.format.ConvertedType;
> > +import parquet.format.SchemaElement;
> >  import parquet.hadoop.metadata.ColumnChunkMetaData;
> >
> > +import java.math.BigDecimal;
> > +
> >  class FixedByteAlignedReader extends ColumnReader {
> >
> > -  private byte[] bytes;
> > +  protected byte[] bytes;
> >
> >
> >    FixedByteAlignedReader(ParquetRecordReader parentReader, int
> allocateSize, ColumnDescriptor descriptor, ColumnChunkMetaData
> columnChunkMetaData,
> > -                         boolean fixedLength, ValueVector v,
> ConvertedType convertedType) throws ExecutionSetupException {
> > -    super(parentReader, allocateSize, descriptor, columnChunkMetaData,
> fixedLength, v, convertedType);
> > +                         boolean fixedLength, ValueVector v,
> SchemaElement schemaElement) throws ExecutionSetupException {
> > +    super(parentReader, allocateSize, descriptor, columnChunkMetaData,
> fixedLength, v, schemaElement);
> >    }
> >
> >    // this method is called by its superclass during a read loop
> > @@ -50,4 +57,76 @@ class FixedByteAlignedReader extends ColumnReader {
> >      vectorData.writeBytes(bytes,
> >          (int) readStartInBytes, (int) readLength);
> >    }
> > +
> > +  public static abstract class ConvertedReader extends
> FixedByteAlignedReader {
> > +
> > +    protected int dataTypeLengthInBytes;
> > +
> > +    ConvertedReader(ParquetRecordReader parentReader, int allocateSize,
> ColumnDescriptor descriptor, ColumnChunkMetaData columnChunkMetaData,
> > +                           boolean fixedLength, ValueVector v,
> SchemaElement schemaElement) throws ExecutionSetupException {
> > +      super(parentReader, allocateSize, descriptor,
> columnChunkMetaData, fixedLength, v, schemaElement);
> > +    }
> > +
> > +    @Override
> > +    protected void readField(long recordsToReadInThisPass, ColumnReader
> firstColumnStatus) {
> > +      recordsReadInThisIteration =
> Math.min(pageReadStatus.currentPage.getValueCount()
> > +              - pageReadStatus.valuesRead, recordsToReadInThisPass -
> valuesReadInCurrentPass);
> > +
> > +      readStartInBytes = pageReadStatus.readPosInBytes;
> > +      readLengthInBits = recordsReadInThisIteration *
> dataTypeLengthInBits;
> > +      readLength = (int) Math.ceil(readLengthInBits / 8.0);
> > +
> > +      bytes = pageReadStatus.pageDataByteArray;
> > +
> > +      dataTypeLengthInBytes = (int) Math.ceil(dataTypeLengthInBits /
> 8.0);
> > +      for (int i = 0; i < recordsReadInThisIteration; i++) {
> > +        addNext((int)readStartInBytes + i * dataTypeLengthInBytes, i +
> valuesReadInCurrentPass);
> > +      }
> > +    }
> > +
> > +    /**
> > +     * Reads from bytes, converts, and writes to buffer
> > +     * @param start the index in bytes to start reading from
> > +     * @param index the index of the ValueVector
> > +     */
> > +    abstract void addNext(int start, int index);
> > +  }
> > +
> > +  public static class Decimal28Reader extends ConvertedReader {
> > +
> > +    Decimal28SparseVector decimal28Vector;
> > +
> > +    Decimal28Reader(ParquetRecordReader parentReader, int allocateSize,
> ColumnDescriptor descriptor, ColumnChunkMetaData columnChunkMetaData,
> > +                    boolean fixedLength, ValueVector v, SchemaElement
> schemaElement) throws ExecutionSetupException {
> > +      super(parentReader, allocateSize, descriptor,
> columnChunkMetaData, fixedLength, v, schemaElement);
> > +      decimal28Vector = (Decimal28SparseVector) v;
> > +    }
> > +
> > +    @Override
> > +    void addNext(int start, int index) {
> > +      int width = Decimal28SparseHolder.WIDTH;
> > +      BigDecimal intermediate =
> DecimalUtility.getBigDecimalFromByteArray(bytes, start,
> dataTypeLengthInBytes, schemaElement.getScale());
> > +      DecimalUtility.getSparseFromBigDecimal(intermediate,
> decimal28Vector.getData(), index * width, schemaElement.getScale(),
> > +              schemaElement.getPrecision(),
> Decimal28SparseHolder.nDecimalDigits);
> > +    }
> > +  }
> > +
> > +  public static class Decimal38Reader extends ConvertedReader {
> > +
> > +    Decimal38SparseVector decimal38Vector;
> > +
> > +    Decimal38Reader(ParquetRecordReader parentReader, int allocateSize,
> ColumnDescriptor descriptor, ColumnChunkMetaData columnChunkMetaData,
> > +                    boolean fixedLength, ValueVector v, SchemaElement
> schemaElement) throws ExecutionSetupException {
> > +      super(parentReader, allocateSize, descriptor,
> columnChunkMetaData, fixedLength, v, schemaElement);
> > +      decimal38Vector = (Decimal38SparseVector) v;
> > +    }
> > +
> > +    @Override
> > +    void addNext(int start, int index) {
> > +      int width = Decimal38SparseHolder.WIDTH;
> > +      BigDecimal intermediate =
> DecimalUtility.getBigDecimalFromByteArray(bytes, start,
> dataTypeLengthInBytes, schemaElement.getScale());
> > +      DecimalUtility.getSparseFromBigDecimal(intermediate,
> decimal38Vector.getData(), index * width, schemaElement.getScale(),
> > +              schemaElement.getPrecision(),
> Decimal38SparseHolder.nDecimalDigits);
> > +    }
> > +  }
> >  }
> > \ No newline at end of file
> >
> >
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/63b03467/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/NullableBitReader.java
> > ----------------------------------------------------------------------
> > diff --git
> a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/NullableBitReader.java
> b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/NullableBitReader.java
> > index 22933ba..29ca30a 100644
> > ---
> a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/NullableBitReader.java
> > +++
> b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/NullableBitReader.java
> > @@ -24,6 +24,7 @@ import org.apache.drill.exec.vector.NullableBitVector;
> >  import org.apache.drill.exec.vector.ValueVector;
> >  import parquet.column.ColumnDescriptor;
> >  import parquet.format.ConvertedType;
> > +import parquet.format.SchemaElement;
> >  import parquet.hadoop.metadata.ColumnChunkMetaData;
> >
> >  import java.io.IOException;
> > @@ -39,8 +40,8 @@ import java.io.IOException;
> >  final class NullableBitReader extends ColumnReader {
> >
> >    NullableBitReader(ParquetRecordReader parentReader, int allocateSize,
> ColumnDescriptor descriptor, ColumnChunkMetaData columnChunkMetaData,
> > -                    boolean fixedLength, ValueVector v, ConvertedType
> convertedType) throws ExecutionSetupException {
> > -    super(parentReader, allocateSize, descriptor, columnChunkMetaData,
> fixedLength, v, convertedType);
> > +                    boolean fixedLength, ValueVector v, SchemaElement
> schemaElement) throws ExecutionSetupException {
> > +    super(parentReader, allocateSize, descriptor, columnChunkMetaData,
> fixedLength, v, schemaElement);
> >    }
> >
> >    @Override
> >
> >
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/63b03467/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/NullableColumnReader.java
> > ----------------------------------------------------------------------
> > diff --git
> a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/NullableColumnReader.java
> b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/NullableColumnReader.java
> > index 66d1c5f..6040c67 100644
> > ---
> a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/NullableColumnReader.java
> > +++
> b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/NullableColumnReader.java
> > @@ -23,6 +23,7 @@ import
> org.apache.drill.exec.vector.NullableVectorDefinitionSetter;
> >  import org.apache.drill.exec.vector.ValueVector;
> >  import parquet.column.ColumnDescriptor;
> >  import parquet.format.ConvertedType;
> > +import parquet.format.SchemaElement;
> >  import parquet.hadoop.metadata.ColumnChunkMetaData;
> >
> >  import java.io.IOException;
> > @@ -36,8 +37,8 @@ abstract class NullableColumnReader extends
> ColumnReader{
> >    int bitsUsed;
> >
> >    NullableColumnReader(ParquetRecordReader parentReader, int
> allocateSize, ColumnDescriptor descriptor, ColumnChunkMetaData
> columnChunkMetaData,
> > -               boolean fixedLength, ValueVector v, ConvertedType
> convertedType) throws ExecutionSetupException {
> > -    super(parentReader, allocateSize, descriptor, columnChunkMetaData,
> fixedLength, v, convertedType);
> > +               boolean fixedLength, ValueVector v, SchemaElement
> schemaElement) throws ExecutionSetupException {
> > +    super(parentReader, allocateSize, descriptor, columnChunkMetaData,
> fixedLength, v, schemaElement);
> >    }
> >
> >    public void readAllFixedFields(long recordsToReadInThisPass,
> ColumnReader firstColumnStatus) throws IOException {
> >
> >
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/63b03467/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/NullableFixedByteAlignedReader.java
> > ----------------------------------------------------------------------
> > diff --git
> a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/NullableFixedByteAlignedReader.java
> b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/NullableFixedByteAlignedReader.java
> > index 038f2d7..d4416c8 100644
> > ---
> a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/NullableFixedByteAlignedReader.java
> > +++
> b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/NullableFixedByteAlignedReader.java
> > @@ -18,19 +18,27 @@
> >  package org.apache.drill.exec.store.parquet;
> >
> >  import org.apache.drill.common.exceptions.ExecutionSetupException;
> > +import org.apache.drill.common.util.DecimalUtility;
> > +import org.apache.drill.exec.expr.holders.NullableDecimal28SparseHolder;
> > +import org.apache.drill.exec.expr.holders.NullableDecimal38SparseHolder;
> > +import org.apache.drill.exec.vector.NullableDecimal28SparseVector;
> > +import org.apache.drill.exec.vector.NullableDecimal38SparseVector;
> >  import org.apache.drill.exec.vector.ValueVector;
> >
> >  import parquet.column.ColumnDescriptor;
> >  import parquet.format.ConvertedType;
> > +import parquet.format.SchemaElement;
> >  import parquet.hadoop.metadata.ColumnChunkMetaData;
> >
> > +import java.math.BigDecimal;
> > +
> >  class NullableFixedByteAlignedReader extends NullableColumnReader {
> >
> > -  private byte[] bytes;
> > +  protected byte[] bytes;
> >
> >    NullableFixedByteAlignedReader(ParquetRecordReader parentReader, int
> allocateSize, ColumnDescriptor descriptor,
> > -      ColumnChunkMetaData columnChunkMetaData, boolean fixedLength,
> ValueVector v, ConvertedType convertedType) throws ExecutionSetupException {
> > -    super(parentReader, allocateSize, descriptor, columnChunkMetaData,
> fixedLength, v, convertedType);
> > +      ColumnChunkMetaData columnChunkMetaData, boolean fixedLength,
> ValueVector v, SchemaElement schemaElement) throws ExecutionSetupException {
> > +    super(parentReader, allocateSize, descriptor, columnChunkMetaData,
> fixedLength, v, schemaElement);
> >    }
> >
> >    // this method is called by its superclass during a read loop
> > @@ -48,4 +56,71 @@ class NullableFixedByteAlignedReader extends
> NullableColumnReader {
> >      // fill in data.
> >      vectorData.writeBytes(bytes, (int) readStartInBytes, (int)
> readLength);
> >    }
> > +
> > +  public static abstract class NullableConvertedReader extends
> NullableFixedByteAlignedReader {
> > +
> > +    protected int dataTypeLengthInBytes;
> > +
> > +    NullableConvertedReader(ParquetRecordReader parentReader, int
> allocateSize, ColumnDescriptor descriptor,
> > +                                   ColumnChunkMetaData
> columnChunkMetaData, boolean fixedLength, ValueVector v, SchemaElement
> schemaElement) throws ExecutionSetupException {
> > +      super(parentReader, allocateSize, descriptor,
> columnChunkMetaData, fixedLength, v, schemaElement);
> > +    }
> > +
> > +    @Override
> > +    protected void readField(long recordsToReadInThisPass, ColumnReader
> firstColumnStatus) {
> > +
> > +      this.recordsReadInThisIteration = recordsToReadInThisPass;
> > +
> > +      // set up metadata
> > +      this.readStartInBytes = pageReadStatus.readPosInBytes;
> > +      this.readLengthInBits = recordsReadInThisIteration *
> dataTypeLengthInBits;
> > +      this.readLength = (int) Math.ceil(readLengthInBits / 8.0);
> > +      this.bytes = pageReadStatus.pageDataByteArray;
> > +
> > +      dataTypeLengthInBytes = (int) Math.ceil(dataTypeLengthInBits /
> 8.0);
> > +      for (int i = 0; i < recordsReadInThisIteration; i++) {
> > +        addNext((int) readStartInBytes + i * dataTypeLengthInBytes, i +
> valuesReadInCurrentPass);
> > +      }
> > +    }
> > +
> > +    abstract void addNext(int start, int index);
> > +  }
> > +
> > +  public static class NullableDecimal28Reader extends
> NullableConvertedReader {
> > +
> > +    NullableDecimal28SparseVector decimal28Vector;
> > +
> > +    NullableDecimal28Reader(ParquetRecordReader parentReader, int
> allocateSize, ColumnDescriptor descriptor, ColumnChunkMetaData
> columnChunkMetaData,
> > +                            boolean fixedLength, ValueVector v,
> SchemaElement schemaElement) throws ExecutionSetupException {
> > +      super(parentReader, allocateSize, descriptor,
> columnChunkMetaData, fixedLength, v, schemaElement);
> > +      decimal28Vector = (NullableDecimal28SparseVector) v;
> > +    }
> > +
> > +    @Override
> > +    void addNext(int start, int index) {
> > +      int width = NullableDecimal28SparseHolder.WIDTH;
> > +      BigDecimal intermediate =
> DecimalUtility.getBigDecimalFromByteArray(bytes, start,
> dataTypeLengthInBytes, schemaElement.getScale());
> > +      DecimalUtility.getSparseFromBigDecimal(intermediate,
> decimal28Vector.getData(), index * width, schemaElement.getScale(),
> > +              schemaElement.getPrecision(),
> NullableDecimal28SparseHolder.nDecimalDigits);
> > +    }
> > +  }
> > +
> > +  public static class NullableDecimal38Reader extends
> NullableConvertedReader {
> > +
> > +    NullableDecimal38SparseVector decimal38Vector;
> > +
> > +    NullableDecimal38Reader(ParquetRecordReader parentReader, int
> allocateSize, ColumnDescriptor descriptor, ColumnChunkMetaData
> columnChunkMetaData,
> > +                            boolean fixedLength, ValueVector v,
> SchemaElement schemaElement) throws ExecutionSetupException {
> > +      super(parentReader, allocateSize, descriptor,
> columnChunkMetaData, fixedLength, v, schemaElement);
> > +      decimal38Vector = (NullableDecimal38SparseVector) v;
> > +    }
> > +
> > +    @Override
> > +    void addNext(int start, int index) {
> > +      int width = NullableDecimal38SparseHolder.WIDTH;
> > +      BigDecimal intermediate =
> DecimalUtility.getBigDecimalFromByteArray(bytes, start,
> dataTypeLengthInBytes, schemaElement.getScale());
> > +      DecimalUtility.getSparseFromBigDecimal(intermediate,
> decimal38Vector.getData(), index * width, schemaElement.getScale(),
> > +              schemaElement.getPrecision(),
> NullableDecimal38SparseHolder.nDecimalDigits);
> > +    }
> > +  }
> >  }
> > \ No newline at end of file
> >
> >
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/63b03467/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetFormatPlugin.java
> > ----------------------------------------------------------------------
> > diff --git
> a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetFormatPlugin.java
> b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetFormatPlugin.java
> > index a10d30f..34287f3 100644
> > ---
> a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetFormatPlugin.java
> > +++
> b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetFormatPlugin.java
> > @@ -19,26 +19,30 @@ package org.apache.drill.exec.store.parquet;
> >
> >  import java.io.IOException;
> >  import java.util.List;
> > +import java.util.Map;
> >  import java.util.Set;
> >  import java.util.regex.Pattern;
> >
> > +import com.google.common.collect.Maps;
> > +import org.apache.drill.common.exceptions.ExecutionSetupException;
> >  import org.apache.drill.common.expression.SchemaPath;
> >  import org.apache.drill.common.logical.StoragePluginConfig;
> > +import org.apache.drill.exec.ExecConstants;
> > +import org.apache.drill.exec.ops.FragmentContext;
> >  import org.apache.drill.exec.physical.base.AbstractWriter;
> >  import org.apache.drill.exec.physical.base.PhysicalOperator;
> > +import org.apache.drill.exec.physical.impl.WriterRecordBatch;
> > +import org.apache.drill.exec.proto.ExecProtos.FragmentHandle;
> > +import org.apache.drill.exec.record.RecordBatch;
> >  import org.apache.drill.exec.server.DrillbitContext;
> > +import org.apache.drill.exec.store.RecordWriter;
> >  import org.apache.drill.exec.store.StoragePluginOptimizerRule;
> > -import org.apache.drill.exec.store.dfs.BasicFormatMatcher;
> > -import org.apache.drill.exec.store.dfs.DrillPathFilter;
> > -import org.apache.drill.exec.store.dfs.FileSelection;
> > -import org.apache.drill.exec.store.dfs.FormatMatcher;
> > -import org.apache.drill.exec.store.dfs.FormatPlugin;
> > -import org.apache.drill.exec.store.dfs.FormatSelection;
> > -import org.apache.drill.exec.store.dfs.MagicString;
> > +import org.apache.drill.exec.store.dfs.*;
> >  import org.apache.drill.exec.store.dfs.shim.DrillFileSystem;
> >  import org.apache.drill.exec.store.mock.MockStorageEngine;
> >  import org.apache.hadoop.conf.Configuration;
> >  import org.apache.hadoop.fs.FileStatus;
> > +import org.apache.hadoop.fs.FileSystem;
> >  import org.apache.hadoop.fs.Path;
> >  import org.apache.hadoop.fs.PathFilter;
> >
> > @@ -104,7 +108,35 @@ public class ParquetFormatPlugin implements
> FormatPlugin{
> >
> >    @Override
> >    public AbstractWriter getWriter(PhysicalOperator child, String
> location) throws IOException {
> > -    throw new UnsupportedOperationException("Parquet Writer is not
> supported currently.");
> > +    return new ParquetWriter(child, location, this);
> > +  }
> > +
> > +  public RecordWriter getRecordWriter(FragmentContext context,
> ParquetWriter writer) throws IOException {
> > +    Map<String, String> options = Maps.newHashMap();
> > +
> > +    options.put("location", writer.getLocation());
> > +
> > +    FragmentHandle handle = context.getHandle();
> > +    String fragmentId = String.format("%d_%d",
> handle.getMajorFragmentId(), handle.getMinorFragmentId());
> > +    options.put("prefix", fragmentId);
> > +
> > +    options.put(FileSystem.FS_DEFAULT_NAME_KEY,
> ((FileSystemConfig)writer.getStorageConfig()).connection);
> > +
> > +    options.put(ExecConstants.PARQUET_BLOCK_SIZE,
> context.getOptions().getOption(ExecConstants.PARQUET_BLOCK_SIZE).num_val.toString());
> > +
> > +    RecordWriter recordWriter = new ParquetRecordWriter();
> > +    recordWriter.init(options);
> > +
> > +    return recordWriter;
> > +  }
> > +
> > +  public RecordBatch getWriterBatch(FragmentContext context,
> RecordBatch incoming, ParquetWriter writer)
> > +          throws ExecutionSetupException {
> > +    try {
> > +      return new WriterRecordBatch(writer, incoming, context,
> getRecordWriter(context, writer));
> > +    } catch(IOException e) {
> > +      throw new ExecutionSetupException(String.format("Failed to create
> the WriterRecordBatch. %s", e.getMessage()), e);
> > +    }
> >    }
> >
> >    @Override
> >
> >
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/63b03467/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetRecordReader.java
> > ----------------------------------------------------------------------
> > diff --git
> a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetRecordReader.java
> b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetRecordReader.java
> > index 5d28456..4ca13a5 100644
> > ---
> a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetRecordReader.java
> > +++
> b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetRecordReader.java
> > @@ -32,7 +32,9 @@ import
> org.apache.drill.common.expression.ExpressionPosition;
> >  import org.apache.drill.common.expression.FieldReference;
> >  import org.apache.drill.common.expression.SchemaPath;
> >  import org.apache.drill.common.types.TypeProtos;
> > +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.exception.SchemaChangeException;
> >  import org.apache.drill.exec.expr.TypeHelper;
> > @@ -41,6 +43,10 @@ import org.apache.drill.exec.ops.FragmentContext;
> >  import org.apache.drill.exec.physical.impl.OutputMutator;
> >  import org.apache.drill.exec.record.MaterializedField;
> >  import org.apache.drill.exec.store.RecordReader;
> > +import
> org.apache.drill.exec.store.parquet.FixedByteAlignedReader.Decimal28Reader;
> > +import
> org.apache.drill.exec.store.parquet.FixedByteAlignedReader.Decimal38Reader;
> > +import
> org.apache.drill.exec.store.parquet.NullableFixedByteAlignedReader.NullableDecimal28Reader;
> > +import
> org.apache.drill.exec.store.parquet.NullableFixedByteAlignedReader.NullableDecimal38Reader;
> >  import org.apache.drill.exec.vector.*;
> >  import org.apache.hadoop.fs.FileSystem;
> >  import org.apache.hadoop.fs.Path;
> > @@ -62,6 +68,7 @@ import parquet.schema.PrimitiveType;
> >  import org.apache.drill.exec.store.parquet.VarLengthColumnReaders.*;
> >
> >  import com.google.common.base.Joiner;
> > +import parquet.schema.PrimitiveType.PrimitiveTypeName;
> >
> >  public class ParquetRecordReader implements RecordReader {
> >    static final org.slf4j.Logger logger =
> org.slf4j.LoggerFactory.getLogger(ParquetRecordReader.class);
> > @@ -191,18 +198,19 @@ public class ParquetRecordReader implements
> RecordReader {
> >
> >      // TODO - figure out how to deal with this better once we add
> nested reading, note also look where this map is used below
> >      // store a map from column name to converted types if they are
> non-null
> > -    HashMap<String, ConvertedType> convertedTypes = new HashMap<>();
> > +    HashMap<String, SchemaElement> schemaElements = new HashMap<>();
> >      fileMetaData = new
> ParquetMetadataConverter().toParquetMetadata(ParquetFileWriter.CURRENT_VERSION,
> footer);
> >      for (SchemaElement se : fileMetaData.getSchema()) {
> > -      convertedTypes.put(se.getName(), se.getConverted_type());
> > +      schemaElements.put(se.getName(), se);
> >      }
> >
> >      // loop to add up the length of the fixed width columns and build
> the schema
> >      for (int i = 0; i < columns.size(); ++i) {
> >        column = columns.get(i);
> >        logger.debug("name: " + fileMetaData.getSchema().get(i).name);
> > -      field = MaterializedField.create(toFieldName(column.getPath()),
> > -          toMajorType(column.getType(), getDataMode(column),
> convertedTypes.get(column.getPath()[0])));
> > +      SchemaElement se = schemaElements.get(column.getPath()[0]);
> > +      MajorType mt = toMajorType(column.getType(), se.getType_length(),
> getDataMode(column), se);
> > +      field =
> MaterializedField.create(toFieldName(column.getPath()),mt);
> >        if ( ! fieldSelected(field)){
> >          continue;
> >        }
> > @@ -211,12 +219,11 @@ public class ParquetRecordReader implements
> RecordReader {
> >        if (column.getType() != PrimitiveType.PrimitiveTypeName.BINARY) {
> >          // There is not support for the fixed binary type yet in
> parquet, leaving a task here as a reminder
> >          // TODO - implement this when the feature is added upstream
> > -//          if (column.getType() ==
> PrimitiveType.PrimitiveTypeName.FIXED_LEN_BYTE_ARRAY){
> > -//              byteWidthAllFixedFields += column.getType().getWidth()
> > -//          }
> > -//          else { } // the code below for the rest of the fixed length
> fields
> > -
> > -        bitWidthAllFixedFields += getTypeLengthInBits(column.getType());
> > +          if (column.getType() ==
> PrimitiveType.PrimitiveTypeName.FIXED_LEN_BYTE_ARRAY){
> > +              bitWidthAllFixedFields += se.getType_length() * 8;
> > +          } else {
> > +            bitWidthAllFixedFields +=
> getTypeLengthInBits(column.getType());
> > +          }
> >        } else {
> >          allFieldsFixedLength = false;
> >        }
> > @@ -234,16 +241,11 @@ public class ParquetRecordReader implements
> RecordReader {
> >      else {
> >        recordsPerBatch = DEFAULT_RECORDS_TO_READ_IF_NOT_FIXED_WIDTH;
> >      }
> > -//    for (SchemaElement se : fileMetaData.getSchema()) {
> > -//      if (fieldSelected())
> > -//      System.out.println("convertedtype :" + se.getConverted_type());
> > -//      System.out.println("name:" + se.getName());
> > -//      System.out.println();
> > -//
> > -//    }
> > +
> >      try {
> >        ValueVector v;
> >        ConvertedType convertedType;
> > +      SchemaElement schemaElement;
> >        ArrayList<VarLengthColumn> varLengthColumns = new ArrayList<>();
> >        ArrayList<NullableVarLengthColumn> nullableVarLengthColumns = new
> ArrayList<>();
> >        // initialize all of the column read status objects
> > @@ -251,40 +253,21 @@ public class ParquetRecordReader implements
> RecordReader {
> >        for (int i = 0; i < columns.size(); ++i) {
> >          column = columns.get(i);
> >          columnChunkMetaData =
> footer.getBlocks().get(0).getColumns().get(i);
> > -        convertedType = convertedTypes.get(column.getPath()[0]);
> > -        MajorType type = toMajorType(column.getType(),
> getDataMode(column), convertedType);
> > -//
>  Preconditions.checkArgument(!columnChunkMetaData.getEncodings().contains(Encoding.PLAIN_DICTIONARY),
> "Dictionary Encoding not currently supported");
> > +        schemaElement = schemaElements.get(column.getPath()[0]);
> > +        convertedType = schemaElement.getConverted_type();
> > +        MajorType type = toMajorType(column.getType(),
> schemaElement.getType_length(), getDataMode(column), schemaElement);
> >          field = MaterializedField.create(toFieldName(column.getPath()),
> type);
> >          // the field was not requested to be read
> >          if ( ! fieldSelected(field)) continue;
> >
> > -        //convertedTypes.put()
> >          fieldFixedLength = column.getType() !=
> PrimitiveType.PrimitiveTypeName.BINARY;
> >          v = output.addField(field, (Class<? extends ValueVector>)
> TypeHelper.getValueVectorClass(type.getMinorType(), type.getMode()));
> >          if (column.getType() != PrimitiveType.PrimitiveTypeName.BINARY)
> {
> >            createFixedColumnReader(fieldFixedLength, column,
> columnChunkMetaData, recordsPerBatch, v,
> > -            convertedType);
> > +            schemaElement);
> >          } else {
> > -          if (column.getMaxDefinitionLevel() == 0){// column is required
> > -            if (convertedType == ConvertedType.UTF8) {
> > -              varLengthColumns.add(
> > -                new VarCharColumn(this, -1, column,
> columnChunkMetaData, false, (VarCharVector) v, convertedType));
> > -            } else {
> > -              varLengthColumns.add(
> > -                  new VarBinaryColumn(this, -1, column,
> columnChunkMetaData, false, (VarBinaryVector) v, convertedType));
> > -            }
> > -          }
> > -          else{
> > -            if (convertedType == ConvertedType.UTF8) {
> > -              nullableVarLengthColumns.add(
> > -                new NullableVarCharColumn(this, -1, column,
> columnChunkMetaData, false,
> > -                    (NullableVarCharVector) v, convertedType));
> > -            } else {
> > -              nullableVarLengthColumns.add(
> > -                new NullableVarBinaryColumn(this, -1, column,
> columnChunkMetaData, false,
> > -                  (NullableVarBinaryVector) v, convertedType));
> > -            }
> > -          }
> > +          // create a reader and add it to the appropriate list
> > +          getReader(this, -1, column, columnChunkMetaData, false, v,
> schemaElement, varLengthColumns, nullableVarLengthColumns);
> >          }
> >        }
> >        varLengthReader = new VarLenBinaryReader(this, varLengthColumns,
> nullableVarLengthColumns);
> > @@ -338,28 +321,41 @@ public class ParquetRecordReader implements
> RecordReader {
> >     */
> >    private boolean createFixedColumnReader(boolean fixedLength,
> ColumnDescriptor descriptor,
> >                                            ColumnChunkMetaData
> columnChunkMetaData, int allocateSize, ValueVector v,
> > -                                          ConvertedType convertedType)
> > +                                          SchemaElement schemaElement)
> >        throws SchemaChangeException, ExecutionSetupException {
> > +    ConvertedType convertedType = schemaElement.getConverted_type();
> >      // if the column is required
> >      if (descriptor.getMaxDefinitionLevel() == 0){
> >        if (columnChunkMetaData.getType() ==
> PrimitiveType.PrimitiveTypeName.BOOLEAN){
> >          columnStatuses.add(new BitReader(this, allocateSize,
> descriptor, columnChunkMetaData,
> > -            fixedLength, v, convertedType));
> > -      }
> > -      else{
> > +            fixedLength, v, schemaElement));
> > +      } else if (columnChunkMetaData.getType() ==
> PrimitiveTypeName.FIXED_LEN_BYTE_ARRAY && convertedType ==
> ConvertedType.DECIMAL){
> > +        int length = schemaElement.type_length;
> > +        if (length <= 12) {
> > +          columnStatuses.add(new Decimal28Reader(this, allocateSize,
> descriptor, columnChunkMetaData, fixedLength, v, schemaElement));
> > +        } else if (length <= 16) {
> > +          columnStatuses.add(new Decimal38Reader(this, allocateSize,
> descriptor, columnChunkMetaData, fixedLength, v, schemaElement));
> > +        }
> > +      } else {
> >          columnStatuses.add(new FixedByteAlignedReader(this,
> allocateSize, descriptor, columnChunkMetaData,
> > -            fixedLength, v, convertedType));
> > +            fixedLength, v, schemaElement));
> >        }
> >        return true;
> >      }
> >      else { // if the column is nullable
> >        if (columnChunkMetaData.getType() ==
> PrimitiveType.PrimitiveTypeName.BOOLEAN){
> >          columnStatuses.add(new NullableBitReader(this, allocateSize,
> descriptor, columnChunkMetaData,
> > -            fixedLength, v, convertedType));
> > -      }
> > -      else{
> > +            fixedLength, v, schemaElement));
> > +      } else if (columnChunkMetaData.getType() ==
> PrimitiveTypeName.FIXED_LEN_BYTE_ARRAY && convertedType ==
> ConvertedType.DECIMAL){
> > +        int length = schemaElement.type_length;
> > +        if (length <= 12) {
> > +          columnStatuses.add(new NullableDecimal28Reader(this,
> allocateSize, descriptor, columnChunkMetaData, fixedLength, v,
> schemaElement));
> > +        } else if (length <= 16) {
> > +          columnStatuses.add(new NullableDecimal38Reader(this,
> allocateSize, descriptor, columnChunkMetaData, fixedLength, v,
> schemaElement));
> > +        }
> > +      } else {
> >          columnStatuses.add(new NullableFixedByteAlignedReader(this,
> allocateSize, descriptor, columnChunkMetaData,
> > -            fixedLength, v, convertedType));
> > +            fixedLength, v, schemaElement));
> >        }
> >        return true;
> >      }
> > @@ -420,25 +416,57 @@ public class ParquetRecordReader implements
> RecordReader {
> >    }
> >
> >    static TypeProtos.MajorType
> toMajorType(PrimitiveType.PrimitiveTypeName primitiveTypeName,
> > -                                               TypeProtos.DataMode
> mode, ConvertedType convertedType) {
> > -    return toMajorType(primitiveTypeName, 0, mode, convertedType);
> > +                                               TypeProtos.DataMode
> mode, SchemaElement schemaElement) {
> > +    return toMajorType(primitiveTypeName, 0, mode, schemaElement);
> >    }
> >
> >    static TypeProtos.MajorType
> toMajorType(PrimitiveType.PrimitiveTypeName primitiveTypeName, int length,
> > -                                               TypeProtos.DataMode
> mode, ConvertedType convertedType) {
> > +                                               TypeProtos.DataMode
> mode, SchemaElement schemaElement) {
> > +    ConvertedType convertedType = schemaElement.getConverted_type();
> >      switch (mode) {
> >
> >        case OPTIONAL:
> >          switch (primitiveTypeName) {
> >            case BINARY:
> > -            if (convertedType == ConvertedType.UTF8)
> > -              return Types.optional(TypeProtos.MinorType.VARCHAR);
> > -            else
> > +            if (convertedType == null) {
> >                return Types.optional(TypeProtos.MinorType.VARBINARY);
> > +            }
> > +            switch (convertedType) {
> > +              case UTF8:
> > +                return Types.optional(MinorType.VARCHAR);
> > +              case DECIMAL:
> > +                return
> Types.withScaleAndPrecision(getDecimalType(schemaElement),
> DataMode.OPTIONAL, schemaElement.getScale(), schemaElement.getPrecision());
> > +              default:
> > +                throw new
> UnsupportedOperationException(String.format("unsupported type: %s %s",
> primitiveTypeName, convertedType));
> > +            }
> >            case INT64:
> > -            return Types.optional(TypeProtos.MinorType.BIGINT);
> > +            if (convertedType == null) {
> > +              return Types.optional(TypeProtos.MinorType.BIGINT);
> > +            }
> > +            switch(convertedType) {
> > +              case DECIMAL:
> > +                return Types.withScaleAndPrecision(MinorType.DECIMAL18,
> DataMode.OPTIONAL, schemaElement.getScale(), schemaElement.getPrecision());
> > +              case FINETIME:
> > +                throw new UnsupportedOperationException();
> > +              case TIMESTAMP:
> > +                return Types.optional(MinorType.TIMESTAMP);
> > +              default:
> > +                throw new
> UnsupportedOperationException(String.format("unsupported type: %s %s",
> primitiveTypeName, convertedType));
> > +            }
> >            case INT32:
> > -            return Types.optional(TypeProtos.MinorType.INT);
> > +            if (convertedType == null) {
> > +              return Types.optional(TypeProtos.MinorType.INT);
> > +            }
> > +            switch(convertedType) {
> > +              case DECIMAL:
> > +                return Types.withScaleAndPrecision(MinorType.DECIMAL9,
> DataMode.OPTIONAL, schemaElement.getScale(), schemaElement.getPrecision());
> > +              case DATE:
> > +                return Types.optional(MinorType.DATE);
> > +              case TIME:
> > +                return Types.optional(MinorType.TIME);
> > +              default:
> > +                throw new
> UnsupportedOperationException(String.format("unsupported type: %s %s",
> primitiveTypeName, convertedType));
> > +            }
> >            case BOOLEAN:
> >              return Types.optional(TypeProtos.MinorType.BIT);
> >            case FLOAT:
> > @@ -451,23 +479,58 @@ public class ParquetRecordReader implements
> RecordReader {
> >              return
> TypeProtos.MajorType.newBuilder().setMinorType(TypeProtos.MinorType.FIXEDBINARY).setWidth(12)
> >                  .setMode(mode).build();
> >            case FIXED_LEN_BYTE_ARRAY:
> > -            checkArgument(length > 0, "A length greater than zero must
> be provided for a FixedBinary type.");
> > -            return
> TypeProtos.MajorType.newBuilder().setMinorType(TypeProtos.MinorType.FIXEDBINARY)
> > -                .setWidth(length).setMode(mode).build();
> > +            if (convertedType == null) {
> > +              checkArgument(length > 0, "A length greater than zero
> must be provided for a FixedBinary type.");
> > +              return
> TypeProtos.MajorType.newBuilder().setMinorType(TypeProtos.MinorType.FIXEDBINARY)
> > +                      .setWidth(length).setMode(mode).build();
> > +            } else if (convertedType == ConvertedType.DECIMAL) {
> > +              return
> Types.withScaleAndPrecision(getDecimalType(schemaElement),
> DataMode.OPTIONAL, schemaElement.getScale(), schemaElement.getPrecision());
> > +            }
> >            default:
> >              throw new UnsupportedOperationException("Type not
> supported: " + primitiveTypeName);
> >          }
> >        case REQUIRED:
> >          switch (primitiveTypeName) {
> >            case BINARY:
> > -            if (convertedType == ConvertedType.UTF8)
> > -              return Types.required(TypeProtos.MinorType.VARCHAR);
> > -            else
> > +            if (convertedType == null) {
> >                return Types.required(TypeProtos.MinorType.VARBINARY);
> > +            }
> > +            switch (convertedType) {
> > +              case UTF8:
> > +                return Types.required(MinorType.VARCHAR);
> > +              case DECIMAL:
> > +                return
> Types.withScaleAndPrecision(getDecimalType(schemaElement),
> DataMode.REQUIRED, schemaElement.getScale(), schemaElement.getPrecision());
> > +              default:
> > +                throw new
> UnsupportedOperationException(String.format("unsupported type: %s %s",
> primitiveTypeName, convertedType));
> > +            }
> >            case INT64:
> > -            return Types.required(TypeProtos.MinorType.BIGINT);
> > +            if (convertedType == null) {
> > +              return Types.required(MinorType.BIGINT);
> > +            }
> > +            switch(convertedType) {
> > +              case DECIMAL:
> > +                return Types.withScaleAndPrecision(MinorType.DECIMAL18,
> DataMode.REQUIRED, schemaElement.getScale(), schemaElement.getPrecision());
> > +              case FINETIME:
> > +                throw new UnsupportedOperationException();
> > +              case TIMESTAMP:
> > +                return Types.required(MinorType.TIMESTAMP);
> > +              default:
> > +                throw new
> UnsupportedOperationException(String.format("unsupported type: %s %s",
> primitiveTypeName, convertedType));
> > +            }
> >            case INT32:
> > -            return Types.required(TypeProtos.MinorType.INT);
> > +            if (convertedType == null) {
> > +              return Types.required(MinorType.INT);
> > +            }
> > +            switch(convertedType) {
> > +              case DECIMAL:
> > +                return Types.withScaleAndPrecision(MinorType.DECIMAL9,
> DataMode.REQUIRED, schemaElement.getScale(), schemaElement.getPrecision());
> > +              case DATE:
> > +                return Types.required(MinorType.DATE);
> > +              case TIME:
> > +                return Types.required(MinorType.TIME);
> > +              default:
> > +                throw new
> UnsupportedOperationException(String.format("unsupported type: %s %s",
> primitiveTypeName, convertedType));
> > +            }
> >            case BOOLEAN:
> >              return Types.required(TypeProtos.MinorType.BIT);
> >            case FLOAT:
> > @@ -480,23 +543,58 @@ public class ParquetRecordReader implements
> RecordReader {
> >              return
> TypeProtos.MajorType.newBuilder().setMinorType(TypeProtos.MinorType.FIXEDBINARY).setWidth(12)
> >                  .setMode(mode).build();
> >            case FIXED_LEN_BYTE_ARRAY:
> > -            checkArgument(length > 0, "A length greater than zero must
> be provided for a FixedBinary type.");
> > -            return
> TypeProtos.MajorType.newBuilder().setMinorType(TypeProtos.MinorType.FIXEDBINARY)
> > -                .setWidth(length).setMode(mode).build();
> > +            if (convertedType == null) {
> > +              checkArgument(length > 0, "A length greater than zero
> must be provided for a FixedBinary type.");
> > +              return
> TypeProtos.MajorType.newBuilder().setMinorType(TypeProtos.MinorType.FIXEDBINARY)
> > +                      .setWidth(length).setMode(mode).build();
> > +            } else if (convertedType == ConvertedType.DECIMAL) {
> > +              return
> Types.withScaleAndPrecision(getDecimalType(schemaElement),
> DataMode.REQUIRED, schemaElement.getScale(), schemaElement.getPrecision());
> > +            }
> >            default:
> >              throw new UnsupportedOperationException("Type not
> supported: " + primitiveTypeName);
> >          }
> >        case REPEATED:
> >          switch (primitiveTypeName) {
> >            case BINARY:
> > -            if (convertedType == ConvertedType.UTF8)
> > -              return Types.required(TypeProtos.MinorType.VARCHAR);
> > -            else
> > +            if (convertedType == null) {
> >                return Types.repeated(TypeProtos.MinorType.VARBINARY);
> > +            }
> > +            switch (schemaElement.getConverted_type()) {
> > +              case UTF8:
> > +                return Types.repeated(MinorType.VARCHAR);
> > +              case DECIMAL:
> > +                return
> Types.withScaleAndPrecision(getDecimalType(schemaElement),
> DataMode.REPEATED, schemaElement.getScale(), schemaElement.getPrecision());
> > +              default:
> > +                throw new
> UnsupportedOperationException(String.format("unsupported type: %s %s",
> primitiveTypeName, convertedType));
> > +            }
> >            case INT64:
> > -            return Types.repeated(TypeProtos.MinorType.BIGINT);
> > +            if (convertedType == null) {
> > +              return Types.repeated(MinorType.BIGINT);
> > +            }
> > +            switch(convertedType) {
> > +              case DECIMAL:
> > +                return Types.withScaleAndPrecision(MinorType.DECIMAL18,
> DataMode.REPEATED, schemaElement.getScale(), schemaElement.getPrecision());
> > +              case FINETIME:
> > +                throw new UnsupportedOperationException();
> > +              case TIMESTAMP:
> > +                return Types.repeated(MinorType.TIMESTAMP);
> > +              default:
> > +                throw new
> UnsupportedOperationException(String.format("unsupported type: %s %s",
> primitiveTypeName, convertedType));
> > +            }
> >            case INT32:
> > -            return Types.repeated(TypeProtos.MinorType.INT);
> > +            if (convertedType == null) {
> > +              return Types.repeated(MinorType.INT);
> > +            }
> > +            switch(convertedType) {
> > +              case DECIMAL:
> > +                return Types.withScaleAndPrecision(MinorType.DECIMAL9,
> DataMode.REPEATED, schemaElement.getScale(), schemaElement.getPrecision());
> > +              case DATE:
> > +                return Types.repeated(MinorType.DATE);
> > +              case TIME:
> > +                return Types.repeated(MinorType.TIME);
> > +              default:
> > +                throw new
> UnsupportedOperationException(String.format("unsupported type: %s %s",
> primitiveTypeName, convertedType));
> > +            }
> >            case BOOLEAN:
> >              return Types.repeated(TypeProtos.MinorType.BIT);
> >            case FLOAT:
> > @@ -509,9 +607,13 @@ public class ParquetRecordReader implements
> RecordReader {
> >              return
> TypeProtos.MajorType.newBuilder().setMinorType(TypeProtos.MinorType.FIXEDBINARY).setWidth(12)
> >                  .setMode(mode).build();
> >            case FIXED_LEN_BYTE_ARRAY:
> > -            checkArgument(length > 0, "A length greater than zero must
> be provided for a FixedBinary type.");
> > -            return
> TypeProtos.MajorType.newBuilder().setMinorType(TypeProtos.MinorType.FIXEDBINARY)
> > -                .setWidth(length).setMode(mode).build();
> > +            if (convertedType == null) {
> > +              checkArgument(length > 0, "A length greater than zero
> must be provided for a FixedBinary type.");
> > +              return
> TypeProtos.MajorType.newBuilder().setMinorType(TypeProtos.MinorType.FIXEDBINARY)
> > +                      .setWidth(length).setMode(mode).build();
> > +            } else if (convertedType == ConvertedType.DECIMAL) {
> > +              return
> Types.withScaleAndPrecision(getDecimalType(schemaElement),
> DataMode.REPEATED, schemaElement.getScale(), schemaElement.getPrecision());
> > +            }
> >            default:
> >              throw new UnsupportedOperationException("Type not
> supported: " + primitiveTypeName);
> >          }
> > @@ -519,6 +621,58 @@ public class ParquetRecordReader implements
> RecordReader {
> >      throw new UnsupportedOperationException("Type not supported: " +
> primitiveTypeName + " Mode: " + mode);
> >    }
> >
> > +  private static void getReader(ParquetRecordReader parentReader, int
> allocateSize, ColumnDescriptor descriptor,
> > +                                        ColumnChunkMetaData
> columnChunkMetaData, boolean fixedLength, ValueVector v,
> > +                                        SchemaElement schemaElement,
> List<VarLengthColumn> varLengthColumns,
> > +                                        List<NullableVarLengthColumn>
> nullableVarLengthColumns) throws ExecutionSetupException {
> > +    ConvertedType convertedType = schemaElement.getConverted_type();
> > +    switch (descriptor.getMaxDefinitionLevel()) {
> > +      case 0:
> > +        if (convertedType == null) {
> > +          varLengthColumns.add(new VarBinaryColumn(parentReader,
> allocateSize, descriptor, columnChunkMetaData, fixedLength,
> (VarBinaryVector) v, schemaElement));
> > +          return;
> > +        }
> > +        switch (convertedType) {
> > +          case UTF8:
> > +            varLengthColumns.add(new VarCharColumn(parentReader,
> allocateSize, descriptor, columnChunkMetaData, fixedLength, (VarCharVector)
> v, schemaElement));
> > +            return;
> > +          case DECIMAL:
> > +            if (v instanceof Decimal28SparseVector) {
> > +              varLengthColumns.add(new Decimal28Column(parentReader,
> allocateSize, descriptor, columnChunkMetaData, fixedLength,
> (Decimal28SparseVector) v, schemaElement));
> > +              return;
> > +            } else if (v instanceof Decimal38SparseVector) {
> > +              varLengthColumns.add(new Decimal38Column(parentReader,
> allocateSize, descriptor, columnChunkMetaData, fixedLength,
> (Decimal38SparseVector) v, schemaElement));
> > +              return;
> > +            }
> > +          default:
> > +        }
> > +      default:
> > +        if (convertedType == null) {
> > +          nullableVarLengthColumns.add(new
> NullableVarBinaryColumn(parentReader, allocateSize, descriptor,
> columnChunkMetaData, fixedLength, (NullableVarBinaryVector) v,
> schemaElement));
> > +          return;
> > +        }
> > +        switch (convertedType) {
> > +          case UTF8:
> > +            nullableVarLengthColumns.add(new
> NullableVarCharColumn(parentReader, allocateSize, descriptor,
> columnChunkMetaData, fixedLength, (NullableVarCharVector) v,
> schemaElement));
> > +            return;
> > +          case DECIMAL:
> > +            if (v instanceof NullableDecimal28SparseVector) {
> > +              nullableVarLengthColumns.add(new
> NullableDecimal28Column(parentReader, allocateSize, descriptor,
> columnChunkMetaData, fixedLength, (NullableDecimal28SparseVector) v,
> schemaElement));
> > +              return;
> > +            } else if (v instanceof NullableDecimal38SparseVector) {
> > +              nullableVarLengthColumns.add(new
> NullableDecimal38Column(parentReader, allocateSize, descriptor,
> columnChunkMetaData, fixedLength, (NullableDecimal38SparseVector) v,
> schemaElement));
> > +              return;
> > +            }
> > +          default:
> > +        }
> > +    }
> > +    throw new UnsupportedOperationException();
> > +  }
> > +
> > +  private static MinorType getDecimalType(SchemaElement schemaElement) {
> > +    return schemaElement.getPrecision() <= 28 ?
> MinorType.DECIMAL28SPARSE : MinorType.DECIMAL38SPARSE;
> > +  }
> > +
> >    static String join(String delimiter, String... str) {
> >      StringBuilder builder = new StringBuilder();
> >      int i = 0;
> >
> >
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/63b03467/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetRecordWriter.java
> > ----------------------------------------------------------------------
> > diff --git
> a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetRecordWriter.java
> b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetRecordWriter.java
> > new file mode 100644
> > index 0000000..3ba13f4
> > --- /dev/null
> > +++
> b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetRecordWriter.java
> > @@ -0,0 +1,185 @@
> > +/**
> > + * Licensed to the Apache Software Foundation (ASF) under one
> > + * or more contributor license agreements.  See the NOTICE file
> > + * distributed with this work for additional information
> > + * regarding copyright ownership.  The ASF licenses this file
> > + * to you under the Apache License, Version 2.0 (the
> > + * "License"); you may not use this file except in compliance
> > + * with the License.  You may obtain a copy of the License at
> > + *
> > + * http://www.apache.org/licenses/LICENSE-2.0
> > + *
> > + * Unless required by applicable law or agreed to in writing, software
> > + * distributed under the License is distributed on an "AS IS" BASIS,
> > + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
> implied.
> > + * See the License for the specific language governing permissions and
> > + * limitations under the License.
> > + */
> > +package org.apache.drill.exec.store.parquet;
> > +
> > +import com.google.common.collect.Lists;
> > +import org.apache.drill.common.types.TypeProtos.MinorType;
> > +import org.apache.drill.exec.ExecConstants;
> > +import org.apache.drill.exec.record.BatchSchema;
> > +import org.apache.drill.exec.record.MaterializedField;
> > +import org.apache.drill.exec.store.ParquetOutputRecordWriter;
> > +import org.apache.hadoop.conf.Configuration;
> > +import org.apache.hadoop.fs.FileSystem;
> > +import org.apache.hadoop.fs.Path;
> > +import parquet.column.ParquetProperties.WriterVersion;
> > +import parquet.column.impl.ColumnWriteStoreImpl;
> > +import parquet.column.page.PageWriteStore;
> > +import parquet.hadoop.ColumnChunkPageWriteStoreExposer;
> > +import parquet.hadoop.ParquetFileWriter;
> > +import parquet.hadoop.metadata.CompressionCodecName;
> > +import parquet.io.ColumnIOFactory;
> > +import parquet.io.MessageColumnIO;
> > +import parquet.io.api.RecordConsumer;
> > +import parquet.schema.DecimalMetadata;
> > +import parquet.schema.MessageType;
> > +import parquet.schema.OriginalType;
> > +import parquet.schema.PrimitiveType.PrimitiveTypeName;
> > +import parquet.schema.Type;
> > +import parquet.schema.Type.Repetition;
> > +
> > +import java.io.IOException;
> > +import java.util.HashMap;
> > +import java.util.List;
> > +import java.util.Map;
> > +
> > +import static java.lang.Math.max;
> > +import static java.lang.Math.min;
> > +import static java.lang.String.format;
> > +
> > +public class ParquetRecordWriter extends ParquetOutputRecordWriter {
> > +
> > +  private static final int MINIMUM_BUFFER_SIZE = 64 * 1024;
> > +  private static final int MINIMUM_RECORD_COUNT_FOR_CHECK = 100;
> > +  private static final int MAXIMUM_RECORD_COUNT_FOR_CHECK = 10000;
> > +
> > +  private ParquetFileWriter w;
> > +  private MessageType schema;
> > +  private Map<String, String> extraMetaData = new HashMap();
> > +  private int blockSize;
> > +  private int pageSize = 1 * 1024 * 1024;
> > +  private int dictionaryPageSize = pageSize;
> > +  private boolean enableDictionary = false;
> > +  private boolean validating = false;
> > +  private CompressionCodecName codec = CompressionCodecName.SNAPPY;
> > +  private WriterVersion writerVersion = WriterVersion.PARQUET_1_0;
> > +
> > +  private long recordCount = 0;
> > +  private long recordCountForNextMemCheck =
> MINIMUM_RECORD_COUNT_FOR_CHECK;
> > +
> > +  private ColumnWriteStoreImpl store;
> > +  private PageWriteStore pageStore;
> > +
> > +  private RecordConsumer consumer;
> > +  private BatchSchema batchSchema;
> > +
> > +  private Configuration conf;
> > +  private String location;
> > +  private String prefix;
> > +  private int index = 0;
> > +
> > +  @Override
> > +  public void init(Map<String, String> writerOptions) throws
> IOException {
> > +    this.location = writerOptions.get("location");
> > +    this.prefix = writerOptions.get("prefix");
> > +
> > +    conf = new Configuration();
> > +    conf.set(FileSystem.FS_DEFAULT_NAME_KEY,
> writerOptions.get(FileSystem.FS_DEFAULT_NAME_KEY));
> > +    blockSize =
> Integer.parseInt(writerOptions.get(ExecConstants.PARQUET_BLOCK_SIZE));
> > +  }
> > +
> > +  @Override
> > +  public void updateSchema(BatchSchema batchSchema) throws IOException {
> > +    if (this.batchSchema == null ||
> !this.batchSchema.equals(batchSchema)) {
> > +      if (this.batchSchema != null) {
> > +        flush();
> > +      }
> > +      this.batchSchema = batchSchema;
> > +      newSchema();
> > +    }
> > +  }
> > +
> > +  private void newSchema() throws IOException {
> > +    List<Type> types = Lists.newArrayList();
> > +    for (MaterializedField field : batchSchema) {
> > +      String name = field.getAsSchemaPath().getAsUnescapedPath();
> > +      MinorType minorType = field.getType().getMinorType();
> > +      PrimitiveTypeName primitiveTypeName =
> ParquetTypeHelper.getPrimitiveTypeNameForMinorType(minorType);
> > +      Repetition repetition =
> ParquetTypeHelper.getRepetitionForDataMode(field.getDataMode());
> > +      OriginalType originalType =
> ParquetTypeHelper.getOriginalTypeForMinorType(minorType);
> > +      DecimalMetadata decimalMetadata =
> ParquetTypeHelper.getDecimalMetadataForField(field);
> > +      int length = ParquetTypeHelper.getLengthForMinorType(minorType);
> > +      parquet.schema.Type type = new
> parquet.schema.PrimitiveType(repetition, primitiveTypeName, length, name,
> originalType, decimalMetadata);
> > +      types.add(type);
> > +    }
> > +    schema = new MessageType("root", types);
> > +
> > +    Path fileName = new Path(location, prefix + "_" + index +
> ".parquet");
> > +    w = new ParquetFileWriter(conf, schema, fileName);
> > +    w.start();
> > +
> > +    int initialBlockBufferSize = max(MINIMUM_BUFFER_SIZE, blockSize /
> this.schema.getColumns().size() / 5);
> > +    pageStore =
> ColumnChunkPageWriteStoreExposer.newColumnChunkPageWriteStore(codec,
> pageSize, this.schema, initialBlockBufferSize);
> > +    int initialPageBufferSize = max(MINIMUM_BUFFER_SIZE, min(pageSize +
> pageSize / 10, initialBlockBufferSize));
> > +    store = new ColumnWriteStoreImpl(pageStore, pageSize,
> initialPageBufferSize, dictionaryPageSize, enableDictionary, writerVersion);
> > +    MessageColumnIO columnIO = new
> ColumnIOFactory(validating).getColumnIO(this.schema);
> > +    consumer = columnIO.getRecordWriter(store);
> > +    setUp(schema, consumer);
> > +  }
> > +
> > +  private void flush() throws IOException {
> > +    w.startBlock(recordCount);
> > +    store.flush();
> > +    ColumnChunkPageWriteStoreExposer.flushPageStore(pageStore, w);
> > +    recordCount = 0;
> > +    w.endBlock();
> > +    w.end(extraMetaData);
> > +    store = null;
> > +    pageStore = null;
> > +    index++;
> > +  }
> > +
> > +  private void checkBlockSizeReached() throws IOException {
> > +    if (recordCount >= recordCountForNextMemCheck) { // checking the
> memory size is relatively expensive, so let's not do it for every record.
> > +      long memSize = store.memSize();
> > +      if (memSize > blockSize) {
> > +        System.out.println("Reached block size " + blockSize);
> > +        flush();
> > +        newSchema();
> > +        recordCountForNextMemCheck =
> min(max(MINIMUM_RECORD_COUNT_FOR_CHECK, recordCount / 2),
> MAXIMUM_RECORD_COUNT_FOR_CHECK);
> > +      } else {
> > +        float recordSize = (float) memSize / recordCount;
> > +        recordCountForNextMemCheck = min(
> > +                max(MINIMUM_RECORD_COUNT_FOR_CHECK, (recordCount +
> (long)(blockSize / recordSize)) / 2), // will check halfway
> > +                recordCount + MAXIMUM_RECORD_COUNT_FOR_CHECK // will
> not look more than max records ahead
> > +        );
> > +      }
> > +    }
> > +  }
> > +
> > +  @Override
> > +  public void startRecord() throws IOException {
> > +    consumer.startMessage();
> > +  }
> > +
> > +  @Override
> > +  public void endRecord() throws IOException {
> > +    consumer.endMessage();
> > +    recordCount++;
> > +    checkBlockSizeReached();
> > +  }
> > +
> > +  @Override
> > +  public void abort() throws IOException {
> > +    //To change body of implemented methods use File | Settings | File
> Templates.
> > +  }
> > +
> > +  @Override
> > +  public void cleanup() throws IOException {
> > +    flush();
> > +  }
> > +}
> >
> >
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/63b03467/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetWriter.java
> > ----------------------------------------------------------------------
> > diff --git
> a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetWriter.java
> b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetWriter.java
> > new file mode 100644
> > index 0000000..ea9798a
> > --- /dev/null
> > +++
> b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetWriter.java
> > @@ -0,0 +1,91 @@
> > +/**
> > + * Licensed to the Apache Software Foundation (ASF) under one
> > + * or more contributor license agreements.  See the NOTICE file
> > + * distributed with this work for additional information
> > + * regarding copyright ownership.  The ASF licenses this file
> > + * to you under the Apache License, Version 2.0 (the
> > + * "License"); you may not use this file except in compliance
> > + * with the License.  You may obtain a copy of the License at
> > + *
> > + * http://www.apache.org/licenses/LICENSE-2.0
> > + *
> > + * Unless required by applicable law or agreed to in writing, software
> > + * distributed under the License is distributed on an "AS IS" BASIS,
> > + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
> implied.
> > + * See the License for the specific language governing permissions and
> > + * limitations under the License.
> > + */
> > +package org.apache.drill.exec.store.parquet;
> > +
> > +import com.fasterxml.jackson.annotation.*;
> > +import com.google.common.base.Preconditions;
> > +import org.apache.drill.common.exceptions.ExecutionSetupException;
> > +import org.apache.drill.common.logical.FormatPluginConfig;
> > +import org.apache.drill.common.logical.StoragePluginConfig;
> > +import org.apache.drill.exec.physical.OperatorCost;
> > +import org.apache.drill.exec.physical.base.AbstractWriter;
> > +import org.apache.drill.exec.physical.base.PhysicalOperator;
> > +import org.apache.drill.exec.store.StoragePluginRegistry;
> > +
> > +import java.io.IOException;
> > +
> > +@JsonTypeName("parquet-writer")
> > +public class ParquetWriter extends AbstractWriter {
> > +  static final org.slf4j.Logger logger =
> org.slf4j.LoggerFactory.getLogger(ParquetWriter.class);
> > +
> > +  private final String location;
> > +  private final ParquetFormatPlugin formatPlugin;
> > +
> > +  @JsonCreator
> > +  public ParquetWriter(
> > +          @JsonProperty("child") PhysicalOperator child,
> > +          @JsonProperty("location") String location,
> > +          @JsonProperty("storage") StoragePluginConfig storageConfig,
> > +          @JacksonInject StoragePluginRegistry engineRegistry) throws
> IOException, ExecutionSetupException {
> > +
> > +    super(child);
> > +    this.formatPlugin = (ParquetFormatPlugin)
> engineRegistry.getFormatPlugin(storageConfig, new ParquetFormatConfig());
> > +    Preconditions.checkNotNull(formatPlugin, "Unable to load format
> plugin for provided format config.");
> > +    this.location = location;
> > +  }
> > +
> > +  public ParquetWriter(PhysicalOperator child,
> > +                       String location,
> > +                       ParquetFormatPlugin formatPlugin) {
> > +
> > +    super(child);
> > +    this.formatPlugin = formatPlugin;
> > +    this.location = location;
> > +  }
> > +
> > +  @JsonProperty("location")
> > +  public String getLocation() {
> > +    return location;
> > +  }
> > +
> > +  @JsonProperty("storage")
> > +  public StoragePluginConfig getStorageConfig(){
> > +    return formatPlugin.getStorageConfig();
> > +  }
> > +
> > +  @JsonProperty("format")
> > +  public FormatPluginConfig getFormatConfig(){
> > +    return formatPlugin.getConfig();
> > +  }
> > +
> > +  @JsonIgnore
> > +  public ParquetFormatPlugin getFormatPlugin(){
> > +    return formatPlugin;
> > +  }
> > +
> > +  @Override
> > +  protected PhysicalOperator getNewWithChild(PhysicalOperator child) {
> > +    return new ParquetWriter(child, location, formatPlugin);
> > +  }
> > +
> > +  @Override
> > +  public OperatorCost getCost() {
> > +    // TODO:
> > +    return new OperatorCost(1,1,1,1);
> > +  }
> > +}
> >
>



-- 
 Steven Phillips
 Software Engineer

 mapr.com

Re: [3/3] git commit: DRILL-694: Implement parquet writer.

Posted by Timothy Chen <tn...@gmail.com>.
I got a unit test failure on master from the testParquetWriter?

Is the test passing for you?

Tim

On Mon, May 12, 2014 at 8:46 PM,  <ja...@apache.org> wrote:
> DRILL-694: Implement parquet writer.
>
> Enable "CREATE TABLE AS" with parquet as the output format.
>
> Add decimal metadata support to parquet reader and writer.
>
>
> Project: http://git-wip-us.apache.org/repos/asf/incubator-drill/repo
> Commit: http://git-wip-us.apache.org/repos/asf/incubator-drill/commit/63b03467
> Tree: http://git-wip-us.apache.org/repos/asf/incubator-drill/tree/63b03467
> Diff: http://git-wip-us.apache.org/repos/asf/incubator-drill/diff/63b03467
>
> Branch: refs/heads/master
> Commit: 63b0346761692f0a4c851e7910791c028dc3cded
> Parents: 14c628c
> Author: Steven Phillips <sp...@maprtech.com>
> Authored: Mon May 12 02:56:25 2014 -0700
> Committer: Aditya Kishore <ad...@maprtech.com>
> Committed: Mon May 12 18:18:43 2014 -0700
>
> ----------------------------------------------------------------------
>  .../org/apache/drill/common/types/Types.java    |   4 +
>  .../drill/common/util/DecimalUtility.java       |  11 +-
>  exec/java-exec/pom.xml                          |   4 +-
>  .../templates/EventBasedRecordWriter.java       |   6 +-
>  .../templates/ParquetOutputRecordWriter.java    | 189 +++++++++++
>  .../codegen/templates/ParquetTypeHelper.java    | 155 ++++++++++
>  .../org/apache/drill/exec/ExecConstants.java    |   8 +
>  .../server/options/SystemOptionManager.java     |   5 +-
>  .../exec/store/dfs/WorkspaceSchemaFactory.java  |   9 +-
>  .../drill/exec/store/parquet/BitReader.java     |   5 +-
>  .../drill/exec/store/parquet/ColumnReader.java  |  14 +-
>  .../store/parquet/FixedByteAlignedReader.java   |  87 +++++-
>  .../exec/store/parquet/NullableBitReader.java   |   5 +-
>  .../store/parquet/NullableColumnReader.java     |   5 +-
>  .../parquet/NullableFixedByteAlignedReader.java |  81 ++++-
>  .../exec/store/parquet/ParquetFormatPlugin.java |  48 ++-
>  .../exec/store/parquet/ParquetRecordReader.java | 310 ++++++++++++++-----
>  .../exec/store/parquet/ParquetRecordWriter.java | 185 +++++++++++
>  .../drill/exec/store/parquet/ParquetWriter.java |  91 ++++++
>  .../parquet/ParquetWriterBatchCreator.java      |  36 +++
>  .../store/parquet/VarLengthColumnReaders.java   | 182 ++++++++---
>  .../ColumnChunkPageWriteStoreExposer.java       |  39 +++
>  .../physical/impl/writer/TestParquetWriter.java | 163 ++++++++++
>  .../exec/physical/impl/writer/TestWriter.java   |  19 +-
>  24 files changed, 1510 insertions(+), 151 deletions(-)
> ----------------------------------------------------------------------
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/63b03467/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 c6ac82e..a6ec29b 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
> @@ -271,6 +271,10 @@ public class Types {
>      return MajorType.newBuilder().setMode(mode).setMinorType(type).build();
>    }
>
> +  public static MajorType withScaleAndPrecision(MinorType type, DataMode mode, int scale, int precision) {
> +    return MajorType.newBuilder().setMinorType(type).setMode(mode).setScale(scale).setPrecision(precision).build();
> +  }
> +
>    public static MajorType required(MinorType type){
>      return MajorType.newBuilder().setMode(DataMode.REQUIRED).setMinorType(type).build();
>    }
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/63b03467/common/src/main/java/org/apache/drill/common/util/DecimalUtility.java
> ----------------------------------------------------------------------
> diff --git a/common/src/main/java/org/apache/drill/common/util/DecimalUtility.java b/common/src/main/java/org/apache/drill/common/util/DecimalUtility.java
> index dbfd6ac..0cacc59 100644
> --- a/common/src/main/java/org/apache/drill/common/util/DecimalUtility.java
> +++ b/common/src/main/java/org/apache/drill/common/util/DecimalUtility.java
> @@ -17,8 +17,7 @@
>   */
>  package org.apache.drill.common.util;
>
> -import java.math.BigDecimal;
> -import java.math.BigInteger;
> +import java.math.*;
>
>
>  import io.netty.buffer.Unpooled;
> @@ -26,6 +25,7 @@ import io.netty.buffer.ByteBuf;
>
>  import java.math.BigDecimal;
>  import java.math.BigInteger;
> +import java.util.Arrays;
>
>  public class DecimalUtility {
>
> @@ -486,5 +486,12 @@ public class DecimalUtility {
>        return 0;
>      }
>
> +
> +    public static BigDecimal getBigDecimalFromByteArray(byte[] bytes, int start, int length, int scale) {
> +      byte[] value = Arrays.copyOfRange(bytes, start, start + length);
> +      BigInteger unscaledValue = new BigInteger(value);
> +      return new BigDecimal(unscaledValue, scale);
> +    }
> +
>  }
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/63b03467/exec/java-exec/pom.xml
> ----------------------------------------------------------------------
> diff --git a/exec/java-exec/pom.xml b/exec/java-exec/pom.xml
> index 3e26662..e282c00 100644
> --- a/exec/java-exec/pom.xml
> +++ b/exec/java-exec/pom.xml
> @@ -95,7 +95,7 @@
>      <dependency>
>        <groupId>com.twitter</groupId>
>        <artifactId>parquet-column</artifactId>
> -      <version>1.4.0</version>
> +      <version>1.5.0-SNAPSHOT</version>
>        <exclusions>
>          <exclusion>
>            <groupId>org.apache.hadoop</groupId>
> @@ -110,7 +110,7 @@
>      <dependency>
>        <groupId>com.twitter</groupId>
>        <artifactId>parquet-hadoop</artifactId>
> -      <version>1.4.0</version>
> +      <version>1.5.0-SNAPSHOT</version>
>        <exclusions>
>          <exclusion>
>            <groupId>org.apache.hadoop</groupId>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/63b03467/exec/java-exec/src/main/codegen/templates/EventBasedRecordWriter.java
> ----------------------------------------------------------------------
> diff --git a/exec/java-exec/src/main/codegen/templates/EventBasedRecordWriter.java b/exec/java-exec/src/main/codegen/templates/EventBasedRecordWriter.java
> index 73b6566..b58f24c 100644
> --- a/exec/java-exec/src/main/codegen/templates/EventBasedRecordWriter.java
> +++ b/exec/java-exec/src/main/codegen/templates/EventBasedRecordWriter.java
> @@ -91,8 +91,10 @@ public class EventBasedRecordWriter {
>      fieldWriters = Lists.newArrayList();
>      try {
>        for (int i = 0; i < schema.getFieldCount(); i++) {
> -        fieldWriters.add(i, typeClassMap.get(schema.getColumn(i).getType())
> -            .getConstructor(EventBasedRecordWriter.class, int.class).newInstance(this, i));
> +        MajorType mt = schema.getColumn(i).getType();
> +        MajorType newMt = MajorType.newBuilder().setMinorType(mt.getMinorType()).setMode(mt.getMode()).build();
> +        fieldWriters.add(i, typeClassMap.get(newMt)
> +                .getConstructor(EventBasedRecordWriter.class, int.class).newInstance(this, i));
>        }
>      } catch(Exception e) {
>        logger.error("Failed to create FieldWriter.", e);
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/63b03467/exec/java-exec/src/main/codegen/templates/ParquetOutputRecordWriter.java
> ----------------------------------------------------------------------
> diff --git a/exec/java-exec/src/main/codegen/templates/ParquetOutputRecordWriter.java b/exec/java-exec/src/main/codegen/templates/ParquetOutputRecordWriter.java
> new file mode 100644
> index 0000000..5f75c1c
> --- /dev/null
> +++ b/exec/java-exec/src/main/codegen/templates/ParquetOutputRecordWriter.java
> @@ -0,0 +1,189 @@
> +/**
> + * Licensed to the Apache Software Foundation (ASF) under one
> + * or more contributor license agreements.  See the NOTICE file
> + * distributed with this work for additional information
> + * regarding copyright ownership.  The ASF licenses this file
> + * to you under the Apache License, Version 2.0 (the
> + * "License"); you may not use this file except in compliance
> + * with the License.  You may obtain a copy of the License at
> + *
> + * http://www.apache.org/licenses/LICENSE-2.0
> + *
> + * Unless required by applicable law or agreed to in writing, software
> + * distributed under the License is distributed on an "AS IS" BASIS,
> + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
> + * See the License for the specific language governing permissions and
> + * limitations under the License.
> + */
> +
> +import parquet.io.api.Binary;
> +
> +import java.lang.Override;
> +
> +<@pp.dropOutputFile />
> +<@pp.changeOutputFile name="org/apache/drill/exec/store/ParquetOutputRecordWriter.java" />
> +<#include "/@includes/license.ftl" />
> +
> +package org.apache.drill.exec.store;
> +
> +import com.google.common.collect.Lists;
> +import org.apache.drill.common.types.TypeProtos.MinorType;
> +import org.apache.drill.exec.expr.TypeHelper;
> +import org.apache.drill.exec.expr.holders.*;
> +import org.apache.drill.exec.record.BatchSchema;
> +import org.apache.drill.exec.store.parquet.ParquetTypeHelper;
> +import org.apache.drill.exec.vector.*;
> +import org.apache.drill.common.util.DecimalUtility;
> +import parquet.io.api.RecordConsumer;
> +import parquet.schema.MessageType;
> +import parquet.io.api.Binary;
> +import io.netty.buffer.ByteBuf;
> +
> +import java.io.IOException;
> +import java.lang.UnsupportedOperationException;
> +import java.util.Arrays;
> +import java.util.List;
> +import java.util.Map;
> +
> +/**
> + * Abstract implementation of RecordWriter interface which exposes interface:
> + *    {@link #writeHeader(List)}
> + *    {@link #addField(int,String)}
> + * to output the data in string format instead of implementing addField for each type holder.
> + *
> + * This is useful for text format writers such as CSV, TSV etc.
> + */
> +public abstract class ParquetOutputRecordWriter implements RecordWriter {
> +
> +  private RecordConsumer consumer;
> +  private MessageType schema;
> +
> +  public void setUp(MessageType schema, RecordConsumer consumer) {
> +    this.schema = schema;
> +    this.consumer = consumer;
> +  }
> +
> +<#list vv.types as type>
> +  <#list type.minor as minor>
> +    <#list vv.modes as mode>
> +  @Override
> +  public void add${mode.prefix}${minor.class}Holder(int fieldId, ${mode.prefix}${minor.class}Holder valueHolder) throws IOException {
> +  <#if mode.prefix == "Nullable" >
> +    if (valueHolder.isSet == 0) {
> +      return;
> +    }
> +  <#elseif mode.prefix == "Repeated" >
> +    consumer.startField(schema.getFieldName(fieldId), fieldId);
> +    for (int i = valueHolder.start; i < valueHolder.end; i++) {
> +  </#if>
> +
> +  <#if  minor.class == "TinyInt" ||
> +        minor.class == "UInt1" ||
> +        minor.class == "UInt2" ||
> +        minor.class == "SmallInt" ||
> +        minor.class == "Int" ||
> +        minor.class == "Time" ||
> +        minor.class == "IntervalYear" ||
> +        minor.class == "Decimal9" ||
> +        minor.class == "UInt4">
> +    <#if mode.prefix == "Repeated" >
> +            consumer.addInteger(valueHolder.vector.getAccessor().get(i));
> +    <#else>
> +    consumer.startField(schema.getFieldName(fieldId), fieldId);
> +    consumer.addInteger(valueHolder.value);
> +    consumer.endField(schema.getFieldName(fieldId), fieldId);
> +    </#if>
> +  <#elseif
> +        minor.class == "Float4">
> +      <#if mode.prefix == "Repeated" >
> +              consumer.addFloat(valueHolder.vector.getAccessor().get(i));
> +      <#else>
> +    consumer.startField(schema.getFieldName(fieldId), fieldId);
> +    consumer.addFloat(valueHolder.value);
> +    consumer.endField(schema.getFieldName(fieldId), fieldId);
> +      </#if>
> +  <#elseif
> +        minor.class == "BigInt" ||
> +        minor.class == "Decimal18" ||
> +        minor.class == "TimeStamp" ||
> +        minor.class == "Date" ||
> +        minor.class == "UInt8">
> +      <#if mode.prefix == "Repeated" >
> +              consumer.addLong(valueHolder.vector.getAccessor().get(i));
> +      <#else>
> +    consumer.startField(schema.getFieldName(fieldId), fieldId);
> +    consumer.addLong(valueHolder.value);
> +    consumer.endField(schema.getFieldName(fieldId), fieldId);
> +      </#if>
> +  <#elseif
> +        minor.class == "Float8">
> +      <#if mode.prefix == "Repeated" >
> +              consumer.addDouble(valueHolder.vector.getAccessor().get(i));
> +      <#else>
> +    consumer.startField(schema.getFieldName(fieldId), fieldId);
> +    consumer.addDouble(valueHolder.value);
> +    consumer.endField(schema.getFieldName(fieldId), fieldId);
> +      </#if>
> +  <#elseif
> +        minor.class == "Bit">
> +      <#if mode.prefix == "Repeated" >
> +              consumer.addBoolean(valueHolder.vector.getAccessor().get(i) == 1);
> +      <#else>
> +    consumer.startField(schema.getFieldName(fieldId), fieldId);
> +    consumer.addBoolean(valueHolder.value == 1);
> +    consumer.endField(schema.getFieldName(fieldId), fieldId);
> +      </#if>
> +  <#elseif
> +        minor.class == "Decimal28Sparse" ||
> +        minor.class == "Decimal38Sparse">
> +      <#if mode.prefix == "Repeated" >
> +      <#else>
> +      consumer.startField(schema.getFieldName(fieldId), fieldId);
> +      byte[] bytes = DecimalUtility.getBigDecimalFromSparse(
> +              valueHolder.buffer, valueHolder.start, ${minor.class}Holder.nDecimalDigits, valueHolder.scale).unscaledValue().toByteArray();
> +      byte[] output = new byte[ParquetTypeHelper.getLengthForMinorType(MinorType.${minor.class?upper_case})];
> +      if (valueHolder.sign) {
> +        Arrays.fill(output, 0, output.length - bytes.length, (byte)0xFF);
> +      } else {
> +        Arrays.fill(output, 0, output.length - bytes.length, (byte)0x0);
> +      }
> +      System.arraycopy(bytes, 0, output, output.length - bytes.length, bytes.length);
> +      consumer.addBinary(Binary.fromByteArray(output));
> +      consumer.endField(schema.getFieldName(fieldId), fieldId);
> +      </#if>
> +  <#elseif
> +        minor.class == "TimeTZ" ||
> +        minor.class == "TimeStampTZ" ||
> +        minor.class == "IntervalDay" ||
> +        minor.class == "Interval" ||
> +        minor.class == "Decimal28Dense" ||
> +        minor.class == "Decimal38Dense">
> +
> +      <#if mode.prefix == "Repeated" >
> +      <#else>
> +
> +      </#if>
> +  <#elseif minor.class == "VarChar" || minor.class == "Var16Char" || minor.class == "VarBinary">
> +    <#if mode.prefix == "Repeated">
> +      ${minor.class}Holder singleHolder = new ${minor.class}Holder();
> +      valueHolder.vector.getAccessor().get(i, singleHolder);
> +      consumer.startField(schema.getFieldName(fieldId), fieldId);
> +      consumer.addBinary(Binary.fromByteBuffer(singleHolder.buffer.nioBuffer(singleHolder.start, singleHolder.end - singleHolder.start)));
> +      consumer.endField(schema.getFieldName(fieldId), fieldId);
> +    <#else>
> +    ByteBuf buf = valueHolder.buffer;
> +    consumer.startField(schema.getFieldName(fieldId), fieldId);
> +    consumer.addBinary(Binary.fromByteBuffer(valueHolder.buffer.nioBuffer(valueHolder.start, valueHolder.end - valueHolder.start)));
> +    consumer.endField(schema.getFieldName(fieldId), fieldId);
> +    </#if>
> +  </#if>
> +  <#if mode.prefix == "Repeated">
> +    }
> +    consumer.endField(schema.getFieldName(fieldId), fieldId);
> +  </#if>
> +  }
> +    </#list>
> +  </#list>
> +</#list>
> +
> +}
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/63b03467/exec/java-exec/src/main/codegen/templates/ParquetTypeHelper.java
> ----------------------------------------------------------------------
> diff --git a/exec/java-exec/src/main/codegen/templates/ParquetTypeHelper.java b/exec/java-exec/src/main/codegen/templates/ParquetTypeHelper.java
> new file mode 100644
> index 0000000..b268d33
> --- /dev/null
> +++ b/exec/java-exec/src/main/codegen/templates/ParquetTypeHelper.java
> @@ -0,0 +1,155 @@
> +/**
> + * Licensed to the Apache Software Foundation (ASF) under one
> + * or more contributor license agreements.  See the NOTICE file
> + * distributed with this work for additional information
> + * regarding copyright ownership.  The ASF licenses this file
> + * to you under the Apache License, Version 2.0 (the
> + * "License"); you may not use this file except in compliance
> + * with the License.  You may obtain a copy of the License at
> + *
> + * http://www.apache.org/licenses/LICENSE-2.0
> + *
> + * Unless required by applicable law or agreed to in writing, software
> + * distributed under the License is distributed on an "AS IS" BASIS,
> + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
> + * See the License for the specific language governing permissions and
> + * limitations under the License.
> + */
> +
> +import parquet.format.ConvertedType;
> +import parquet.schema.DecimalMetadata;
> +import parquet.schema.OriginalType;
> +import parquet.schema.PrimitiveType.PrimitiveTypeName;
> +
> +<@pp.dropOutputFile />
> +<@pp.changeOutputFile name="org/apache/drill/exec/store/parquet/ParquetTypeHelper.java" />
> +<#include "/@includes/license.ftl" />
> +
> +package org.apache.drill.exec.store.parquet;
> +
> +import org.apache.drill.common.types.TypeProtos.DataMode;
> +import org.apache.drill.common.types.TypeProtos.MinorType;
> +import org.apache.drill.exec.record.MaterializedField;
> +import parquet.schema.OriginalType;
> +import parquet.schema.DecimalMetadata;
> +import parquet.schema.PrimitiveType.PrimitiveTypeName;
> +import parquet.schema.Type.Repetition;
> +
> +import java.util.HashMap;
> +import java.util.Map;
> +
> +public class ParquetTypeHelper {
> +  private static Map<MinorType,PrimitiveTypeName> typeMap;
> +  private static Map<DataMode,Repetition> modeMap;
> +  private static Map<MinorType,OriginalType> originalTypeMap;
> +
> +  static {
> +    typeMap = new HashMap();
> +
> +    <#list vv.types as type>
> +    <#list type.minor as minor>
> +    <#if    minor.class == "TinyInt" ||
> +            minor.class == "UInt1" ||
> +            minor.class == "UInt2" ||
> +            minor.class == "SmallInt" ||
> +            minor.class == "Int" ||
> +            minor.class == "Time" ||
> +            minor.class == "IntervalYear" ||
> +            minor.class == "Decimal9" ||
> +            minor.class == "UInt4">
> +                    typeMap.put(MinorType.${minor.class?upper_case}, PrimitiveTypeName.INT32);
> +    <#elseif
> +            minor.class == "Float4">
> +                    typeMap.put(MinorType.${minor.class?upper_case}, PrimitiveTypeName.FLOAT);
> +    <#elseif
> +            minor.class == "BigInt" ||
> +            minor.class == "Decimal18" ||
> +            minor.class == "TimeStamp" ||
> +            minor.class == "Date" ||
> +            minor.class == "UInt8">
> +                    typeMap.put(MinorType.${minor.class?upper_case}, PrimitiveTypeName.INT64);
> +    <#elseif
> +            minor.class == "Float8">
> +                    typeMap.put(MinorType.${minor.class?upper_case}, PrimitiveTypeName.DOUBLE);
> +    <#elseif
> +            minor.class == "Bit">
> +                    typeMap.put(MinorType.${minor.class?upper_case}, PrimitiveTypeName.BOOLEAN);
> +    <#elseif
> +            minor.class == "TimeTZ" ||
> +            minor.class == "TimeStampTZ" ||
> +            minor.class == "IntervalDay" ||
> +            minor.class == "Interval" ||
> +            minor.class == "Decimal28Dense" ||
> +            minor.class == "Decimal38Dense" ||
> +            minor.class == "Decimal28Sparse" ||
> +            minor.class == "Decimal38Sparse">
> +                    typeMap.put(MinorType.${minor.class?upper_case}, PrimitiveTypeName.FIXED_LEN_BYTE_ARRAY);
> +    <#elseif
> +            minor.class == "VarChar" ||
> +            minor.class == "Var16Char" ||
> +            minor.class == "VarBinary" >
> +                    typeMap.put(MinorType.${minor.class?upper_case}, PrimitiveTypeName.BINARY);
> +    </#if>
> +    </#list>
> +    </#list>
> +
> +    modeMap = new HashMap();
> +
> +    modeMap.put(DataMode.REQUIRED, Repetition.REQUIRED);
> +    modeMap.put(DataMode.OPTIONAL, Repetition.OPTIONAL);
> +    modeMap.put(DataMode.REPEATED, Repetition.REPEATED);
> +
> +    originalTypeMap = new HashMap();
> +
> +    <#list vv.types as type>
> +    <#list type.minor as minor>
> +            <#if minor.class.startsWith("Decimal")>
> +            originalTypeMap.put(MinorType.${minor.class?upper_case},OriginalType.DECIMAL);
> +            </#if>
> +    </#list>
> +    </#list>
> +            originalTypeMap.put(MinorType.VARCHAR, OriginalType.UTF8);
> +            originalTypeMap.put(MinorType.DATE, OriginalType.DATE);
> +            originalTypeMap.put(MinorType.TIME, OriginalType.TIME);
> +            originalTypeMap.put(MinorType.TIMESTAMP, OriginalType.TIMESTAMP);
> +            originalTypeMap.put(MinorType.TIMESTAMPTZ, OriginalType.TIMESTAMPTZ);
> +  }
> +
> +  public static PrimitiveTypeName getPrimitiveTypeNameForMinorType(MinorType minorType) {
> +    return typeMap.get(minorType);
> +  }
> +
> +  public static Repetition getRepetitionForDataMode(DataMode dataMode) {
> +    return modeMap.get(dataMode);
> +  }
> +
> +  public static OriginalType getOriginalTypeForMinorType(MinorType minorType) {
> +    return originalTypeMap.get(minorType);
> +  }
> +
> +  public static DecimalMetadata getDecimalMetadataForField(MaterializedField field) {
> +    switch(field.getType().getMinorType()) {
> +      case DECIMAL9:
> +      case DECIMAL18:
> +      case DECIMAL28SPARSE:
> +      case DECIMAL28DENSE:
> +      case DECIMAL38SPARSE:
> +      case DECIMAL38DENSE:
> +        return new DecimalMetadata(field.getPrecision(), field.getScale());
> +      default:
> +        return null;
> +    }
> +  }
> +
> +  public static int getLengthForMinorType(MinorType minorType) {
> +    switch(minorType) {
> +      case DECIMAL28SPARSE:
> +        return 12;
> +      case DECIMAL38SPARSE:
> +        return 16;
> +      default:
> +        return 0;
> +    }
> +  }
> +
> +}
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/63b03467/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 9eee08d..238fae9 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
> @@ -17,6 +17,10 @@
>   */
>  package org.apache.drill.exec;
>
> +import org.apache.drill.exec.server.options.OptionValidator;
> +import org.apache.drill.exec.server.options.TypeValidators.LongValidator;
> +import org.apache.drill.exec.server.options.TypeValidators.StringValidator;
> +
>  public interface ExecConstants {
>    public static final String ZK_RETRY_TIMES = "drill.exec.zk.retry.count";
>    public static final String ZK_RETRY_DELAY = "drill.exec.zk.retry.delay";
> @@ -63,5 +67,9 @@ public interface ExecConstants {
>    public static final String FILESYSTEM_PARTITION_COLUMN_LABEL = "drill.exec.storage.file.partition.column.label";
>    public static final String HAZELCAST_SUBNETS = "drill.exec.cache.hazel.subnets";
>    public static final String TOP_LEVEL_MAX_ALLOC = "drill.exec.memory.top.max";
> +  public static final String OUTPUT_FORMAT_OPTION = "store.format";
> +  public static final OptionValidator OUTPUT_FORMAT_VALIDATOR = new StringValidator(OUTPUT_FORMAT_OPTION, "parquet");
> +  public static final String PARQUET_BLOCK_SIZE = "parquet.block.size";
> +  public static final OptionValidator PARQUET_BLOCK_SIZE_VALIDATOR = new LongValidator(PARQUET_BLOCK_SIZE, 512*1024*1024);
>
>  }
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/63b03467/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 cfe8e2c..21031e5 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
> @@ -22,6 +22,7 @@ import java.util.Map;
>  import java.util.Map.Entry;
>  import java.util.concurrent.ConcurrentMap;
>
> +import org.apache.drill.exec.ExecConstants;
>  import org.apache.drill.exec.cache.DistributedCache;
>  import org.apache.drill.exec.cache.DistributedMap;
>  import org.apache.drill.exec.planner.physical.PlannerSettings;
> @@ -38,7 +39,9 @@ public class SystemOptionManager implements OptionManager{
>        PlannerSettings.STREAMAGG,
>        PlannerSettings.HASHJOIN,
>        PlannerSettings.MERGEJOIN,
> -      PlannerSettings.MULTIPHASE
> +      PlannerSettings.MULTIPHASE,
> +      ExecConstants.OUTPUT_FORMAT_VALIDATOR,
> +      ExecConstants.PARQUET_BLOCK_SIZE_VALIDATOR
>    };
>
>    private DistributedMap<OptionValue> options;
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/63b03467/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/WorkspaceSchemaFactory.java
> ----------------------------------------------------------------------
> diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/WorkspaceSchemaFactory.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/WorkspaceSchemaFactory.java
> index df73ea5..72b31e6 100644
> --- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/WorkspaceSchemaFactory.java
> +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/WorkspaceSchemaFactory.java
> @@ -26,12 +26,15 @@ import com.google.common.collect.Lists;
>  import com.google.common.collect.Sets;
>  import net.hydromatic.optiq.Table;
>  import org.apache.drill.common.exceptions.ExecutionSetupException;
> +import org.apache.drill.exec.ExecConstants;
>  import org.apache.drill.exec.planner.logical.CreateTableEntry;
>  import org.apache.drill.exec.planner.logical.DrillTable;
>  import org.apache.drill.exec.planner.logical.DynamicDrillTable;
>  import org.apache.drill.exec.planner.logical.FileSystemCreateTableEntry;
>  import org.apache.drill.exec.planner.sql.ExpandingConcurrentMap;
>  import org.apache.drill.exec.rpc.user.UserSession;
> +import org.apache.drill.exec.server.options.OptionValidator;
> +import org.apache.drill.exec.server.options.TypeValidators.StringValidator;
>  import org.apache.drill.exec.store.AbstractSchema;
>  import org.apache.drill.exec.store.dfs.shim.DrillFileSystem;
>  import org.apache.hadoop.fs.Path;
> @@ -141,9 +144,11 @@ public class WorkspaceSchemaFactory implements ExpandingConcurrentMap.MapValueFa
>        return fs;
>      }
>
> +
>      @Override
>      public CreateTableEntry createNewTable(String tableName) {
> -      FormatPlugin formatPlugin = plugin.getFormatPlugin(config.getStorageFormat());
> +      String storage = session.getOptions().getOption(ExecConstants.OUTPUT_FORMAT_OPTION).string_val;
> +      FormatPlugin formatPlugin = plugin.getFormatPlugin(storage);
>        if (formatPlugin == null)
>          throw new UnsupportedOperationException(
>            String.format("Unsupported format '%s' in workspace '%s'", config.getStorageFormat(),
> @@ -151,7 +156,7 @@ public class WorkspaceSchemaFactory implements ExpandingConcurrentMap.MapValueFa
>
>        return new FileSystemCreateTableEntry(
>            (FileSystemConfig) plugin.getConfig(),
> -          plugin.getFormatPlugin(config.getStorageFormat()),
> +          formatPlugin,
>            config.getLocation() + Path.SEPARATOR + tableName);
>      }
>    }
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/63b03467/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/BitReader.java
> ----------------------------------------------------------------------
> diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/BitReader.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/BitReader.java
> index c323222..7ae95cd 100644
> --- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/BitReader.java
> +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/BitReader.java
> @@ -22,6 +22,7 @@ import org.apache.drill.exec.vector.BaseDataValueVector;
>  import org.apache.drill.exec.vector.ValueVector;
>  import parquet.column.ColumnDescriptor;
>  import parquet.format.ConvertedType;
> +import parquet.format.SchemaElement;
>  import parquet.hadoop.metadata.ColumnChunkMetaData;
>
>  final class BitReader extends ColumnReader {
> @@ -31,8 +32,8 @@ final class BitReader extends ColumnReader {
>    private byte[] bytes;
>
>    BitReader(ParquetRecordReader parentReader, int allocateSize, ColumnDescriptor descriptor, ColumnChunkMetaData columnChunkMetaData,
> -            boolean fixedLength, ValueVector v, ConvertedType convertedType) throws ExecutionSetupException {
> -    super(parentReader, allocateSize, descriptor, columnChunkMetaData, fixedLength, v, convertedType);
> +            boolean fixedLength, ValueVector v, SchemaElement schemaElement) throws ExecutionSetupException {
> +    super(parentReader, allocateSize, descriptor, columnChunkMetaData, fixedLength, v, schemaElement);
>    }
>
>    @Override
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/63b03467/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ColumnReader.java
> ----------------------------------------------------------------------
> diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ColumnReader.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ColumnReader.java
> index 196e1fd..b9faafe 100644
> --- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ColumnReader.java
> +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ColumnReader.java
> @@ -25,8 +25,10 @@ import org.apache.drill.exec.vector.ValueVector;
>  import org.apache.hadoop.fs.FSDataInputStream;
>  import parquet.column.ColumnDescriptor;
>  import parquet.format.ConvertedType;
> +import parquet.format.SchemaElement;
>  import parquet.hadoop.metadata.ColumnChunkMetaData;
>  import parquet.schema.PrimitiveType;
> +import parquet.schema.PrimitiveType.PrimitiveTypeName;
>
>  import java.io.IOException;
>
> @@ -48,7 +50,7 @@ abstract class ColumnReader<V extends ValueVector> {
>    // status information on the current page
>    final PageReadStatus pageReadStatus;
>
> -  final ConvertedType convertedType;
> +  final SchemaElement schemaElement;
>
>    // quick reference to see if the field is fixed length (as this requires an instanceof)
>    final boolean isFixedLength;
> @@ -70,12 +72,12 @@ abstract class ColumnReader<V extends ValueVector> {
>    long readStartInBytes = 0, readLength = 0, readLengthInBits = 0, recordsReadInThisIteration = 0;
>
>    protected ColumnReader(ParquetRecordReader parentReader, int allocateSize, ColumnDescriptor descriptor,
> -      ColumnChunkMetaData columnChunkMetaData, boolean fixedLength, V v, ConvertedType convertedType) throws ExecutionSetupException {
> +      ColumnChunkMetaData columnChunkMetaData, boolean fixedLength, V v, SchemaElement schemaElement) throws ExecutionSetupException {
>      this.parentReader = parentReader;
>      this.columnDescriptor = descriptor;
>      this.columnChunkMetaData = columnChunkMetaData;
>      this.isFixedLength = fixedLength;
> -    this.convertedType = convertedType;
> +    this.schemaElement = schemaElement;
>
>      if (allocateSize > 1) {
>        valueVec =  v;
> @@ -87,7 +89,11 @@ abstract class ColumnReader<V extends ValueVector> {
>      this.pageReadStatus = new PageReadStatus(this, parentReader.fileSystem, parentReader.hadoopPath, columnChunkMetaData);
>
>      if (columnDescriptor.getType() != PrimitiveType.PrimitiveTypeName.BINARY) {
> -      dataTypeLengthInBits = ParquetRecordReader.getTypeLengthInBits(columnDescriptor.getType());
> +      if (columnDescriptor.getType() == PrimitiveTypeName.FIXED_LEN_BYTE_ARRAY) {
> +        dataTypeLengthInBits = columnDescriptor.getTypeLength() * 8;
> +      } else {
> +        dataTypeLengthInBits = ParquetRecordReader.getTypeLengthInBits(columnDescriptor.getType());
> +      }
>      }
>
>    }
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/63b03467/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/FixedByteAlignedReader.java
> ----------------------------------------------------------------------
> diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/FixedByteAlignedReader.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/FixedByteAlignedReader.java
> index 0aa18cf..574b0cb 100644
> --- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/FixedByteAlignedReader.java
> +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/FixedByteAlignedReader.java
> @@ -18,20 +18,27 @@
>  package org.apache.drill.exec.store.parquet;
>
>  import org.apache.drill.common.exceptions.ExecutionSetupException;
> -import org.apache.drill.exec.vector.BaseDataValueVector;
> +import org.apache.drill.common.util.DecimalUtility;
> +import org.apache.drill.exec.expr.holders.Decimal28SparseHolder;
> +import org.apache.drill.exec.expr.holders.Decimal38SparseHolder;
> +import org.apache.drill.exec.vector.Decimal28SparseVector;
> +import org.apache.drill.exec.vector.Decimal38SparseVector;
>  import org.apache.drill.exec.vector.ValueVector;
>  import parquet.column.ColumnDescriptor;
>  import parquet.format.ConvertedType;
> +import parquet.format.SchemaElement;
>  import parquet.hadoop.metadata.ColumnChunkMetaData;
>
> +import java.math.BigDecimal;
> +
>  class FixedByteAlignedReader extends ColumnReader {
>
> -  private byte[] bytes;
> +  protected byte[] bytes;
>
>
>    FixedByteAlignedReader(ParquetRecordReader parentReader, int allocateSize, ColumnDescriptor descriptor, ColumnChunkMetaData columnChunkMetaData,
> -                         boolean fixedLength, ValueVector v, ConvertedType convertedType) throws ExecutionSetupException {
> -    super(parentReader, allocateSize, descriptor, columnChunkMetaData, fixedLength, v, convertedType);
> +                         boolean fixedLength, ValueVector v, SchemaElement schemaElement) throws ExecutionSetupException {
> +    super(parentReader, allocateSize, descriptor, columnChunkMetaData, fixedLength, v, schemaElement);
>    }
>
>    // this method is called by its superclass during a read loop
> @@ -50,4 +57,76 @@ class FixedByteAlignedReader extends ColumnReader {
>      vectorData.writeBytes(bytes,
>          (int) readStartInBytes, (int) readLength);
>    }
> +
> +  public static abstract class ConvertedReader extends FixedByteAlignedReader {
> +
> +    protected int dataTypeLengthInBytes;
> +
> +    ConvertedReader(ParquetRecordReader parentReader, int allocateSize, ColumnDescriptor descriptor, ColumnChunkMetaData columnChunkMetaData,
> +                           boolean fixedLength, ValueVector v, SchemaElement schemaElement) throws ExecutionSetupException {
> +      super(parentReader, allocateSize, descriptor, columnChunkMetaData, fixedLength, v, schemaElement);
> +    }
> +
> +    @Override
> +    protected void readField(long recordsToReadInThisPass, ColumnReader firstColumnStatus) {
> +      recordsReadInThisIteration = Math.min(pageReadStatus.currentPage.getValueCount()
> +              - pageReadStatus.valuesRead, recordsToReadInThisPass - valuesReadInCurrentPass);
> +
> +      readStartInBytes = pageReadStatus.readPosInBytes;
> +      readLengthInBits = recordsReadInThisIteration * dataTypeLengthInBits;
> +      readLength = (int) Math.ceil(readLengthInBits / 8.0);
> +
> +      bytes = pageReadStatus.pageDataByteArray;
> +
> +      dataTypeLengthInBytes = (int) Math.ceil(dataTypeLengthInBits / 8.0);
> +      for (int i = 0; i < recordsReadInThisIteration; i++) {
> +        addNext((int)readStartInBytes + i * dataTypeLengthInBytes, i + valuesReadInCurrentPass);
> +      }
> +    }
> +
> +    /**
> +     * Reads from bytes, converts, and writes to buffer
> +     * @param start the index in bytes to start reading from
> +     * @param index the index of the ValueVector
> +     */
> +    abstract void addNext(int start, int index);
> +  }
> +
> +  public static class Decimal28Reader extends ConvertedReader {
> +
> +    Decimal28SparseVector decimal28Vector;
> +
> +    Decimal28Reader(ParquetRecordReader parentReader, int allocateSize, ColumnDescriptor descriptor, ColumnChunkMetaData columnChunkMetaData,
> +                    boolean fixedLength, ValueVector v, SchemaElement schemaElement) throws ExecutionSetupException {
> +      super(parentReader, allocateSize, descriptor, columnChunkMetaData, fixedLength, v, schemaElement);
> +      decimal28Vector = (Decimal28SparseVector) v;
> +    }
> +
> +    @Override
> +    void addNext(int start, int index) {
> +      int width = Decimal28SparseHolder.WIDTH;
> +      BigDecimal intermediate = DecimalUtility.getBigDecimalFromByteArray(bytes, start, dataTypeLengthInBytes, schemaElement.getScale());
> +      DecimalUtility.getSparseFromBigDecimal(intermediate, decimal28Vector.getData(), index * width, schemaElement.getScale(),
> +              schemaElement.getPrecision(), Decimal28SparseHolder.nDecimalDigits);
> +    }
> +  }
> +
> +  public static class Decimal38Reader extends ConvertedReader {
> +
> +    Decimal38SparseVector decimal38Vector;
> +
> +    Decimal38Reader(ParquetRecordReader parentReader, int allocateSize, ColumnDescriptor descriptor, ColumnChunkMetaData columnChunkMetaData,
> +                    boolean fixedLength, ValueVector v, SchemaElement schemaElement) throws ExecutionSetupException {
> +      super(parentReader, allocateSize, descriptor, columnChunkMetaData, fixedLength, v, schemaElement);
> +      decimal38Vector = (Decimal38SparseVector) v;
> +    }
> +
> +    @Override
> +    void addNext(int start, int index) {
> +      int width = Decimal38SparseHolder.WIDTH;
> +      BigDecimal intermediate = DecimalUtility.getBigDecimalFromByteArray(bytes, start, dataTypeLengthInBytes, schemaElement.getScale());
> +      DecimalUtility.getSparseFromBigDecimal(intermediate, decimal38Vector.getData(), index * width, schemaElement.getScale(),
> +              schemaElement.getPrecision(), Decimal38SparseHolder.nDecimalDigits);
> +    }
> +  }
>  }
> \ No newline at end of file
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/63b03467/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/NullableBitReader.java
> ----------------------------------------------------------------------
> diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/NullableBitReader.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/NullableBitReader.java
> index 22933ba..29ca30a 100644
> --- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/NullableBitReader.java
> +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/NullableBitReader.java
> @@ -24,6 +24,7 @@ import org.apache.drill.exec.vector.NullableBitVector;
>  import org.apache.drill.exec.vector.ValueVector;
>  import parquet.column.ColumnDescriptor;
>  import parquet.format.ConvertedType;
> +import parquet.format.SchemaElement;
>  import parquet.hadoop.metadata.ColumnChunkMetaData;
>
>  import java.io.IOException;
> @@ -39,8 +40,8 @@ import java.io.IOException;
>  final class NullableBitReader extends ColumnReader {
>
>    NullableBitReader(ParquetRecordReader parentReader, int allocateSize, ColumnDescriptor descriptor, ColumnChunkMetaData columnChunkMetaData,
> -                    boolean fixedLength, ValueVector v, ConvertedType convertedType) throws ExecutionSetupException {
> -    super(parentReader, allocateSize, descriptor, columnChunkMetaData, fixedLength, v, convertedType);
> +                    boolean fixedLength, ValueVector v, SchemaElement schemaElement) throws ExecutionSetupException {
> +    super(parentReader, allocateSize, descriptor, columnChunkMetaData, fixedLength, v, schemaElement);
>    }
>
>    @Override
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/63b03467/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/NullableColumnReader.java
> ----------------------------------------------------------------------
> diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/NullableColumnReader.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/NullableColumnReader.java
> index 66d1c5f..6040c67 100644
> --- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/NullableColumnReader.java
> +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/NullableColumnReader.java
> @@ -23,6 +23,7 @@ import org.apache.drill.exec.vector.NullableVectorDefinitionSetter;
>  import org.apache.drill.exec.vector.ValueVector;
>  import parquet.column.ColumnDescriptor;
>  import parquet.format.ConvertedType;
> +import parquet.format.SchemaElement;
>  import parquet.hadoop.metadata.ColumnChunkMetaData;
>
>  import java.io.IOException;
> @@ -36,8 +37,8 @@ abstract class NullableColumnReader extends ColumnReader{
>    int bitsUsed;
>
>    NullableColumnReader(ParquetRecordReader parentReader, int allocateSize, ColumnDescriptor descriptor, ColumnChunkMetaData columnChunkMetaData,
> -               boolean fixedLength, ValueVector v, ConvertedType convertedType) throws ExecutionSetupException {
> -    super(parentReader, allocateSize, descriptor, columnChunkMetaData, fixedLength, v, convertedType);
> +               boolean fixedLength, ValueVector v, SchemaElement schemaElement) throws ExecutionSetupException {
> +    super(parentReader, allocateSize, descriptor, columnChunkMetaData, fixedLength, v, schemaElement);
>    }
>
>    public void readAllFixedFields(long recordsToReadInThisPass, ColumnReader firstColumnStatus) throws IOException {
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/63b03467/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/NullableFixedByteAlignedReader.java
> ----------------------------------------------------------------------
> diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/NullableFixedByteAlignedReader.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/NullableFixedByteAlignedReader.java
> index 038f2d7..d4416c8 100644
> --- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/NullableFixedByteAlignedReader.java
> +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/NullableFixedByteAlignedReader.java
> @@ -18,19 +18,27 @@
>  package org.apache.drill.exec.store.parquet;
>
>  import org.apache.drill.common.exceptions.ExecutionSetupException;
> +import org.apache.drill.common.util.DecimalUtility;
> +import org.apache.drill.exec.expr.holders.NullableDecimal28SparseHolder;
> +import org.apache.drill.exec.expr.holders.NullableDecimal38SparseHolder;
> +import org.apache.drill.exec.vector.NullableDecimal28SparseVector;
> +import org.apache.drill.exec.vector.NullableDecimal38SparseVector;
>  import org.apache.drill.exec.vector.ValueVector;
>
>  import parquet.column.ColumnDescriptor;
>  import parquet.format.ConvertedType;
> +import parquet.format.SchemaElement;
>  import parquet.hadoop.metadata.ColumnChunkMetaData;
>
> +import java.math.BigDecimal;
> +
>  class NullableFixedByteAlignedReader extends NullableColumnReader {
>
> -  private byte[] bytes;
> +  protected byte[] bytes;
>
>    NullableFixedByteAlignedReader(ParquetRecordReader parentReader, int allocateSize, ColumnDescriptor descriptor,
> -      ColumnChunkMetaData columnChunkMetaData, boolean fixedLength, ValueVector v, ConvertedType convertedType) throws ExecutionSetupException {
> -    super(parentReader, allocateSize, descriptor, columnChunkMetaData, fixedLength, v, convertedType);
> +      ColumnChunkMetaData columnChunkMetaData, boolean fixedLength, ValueVector v, SchemaElement schemaElement) throws ExecutionSetupException {
> +    super(parentReader, allocateSize, descriptor, columnChunkMetaData, fixedLength, v, schemaElement);
>    }
>
>    // this method is called by its superclass during a read loop
> @@ -48,4 +56,71 @@ class NullableFixedByteAlignedReader extends NullableColumnReader {
>      // fill in data.
>      vectorData.writeBytes(bytes, (int) readStartInBytes, (int) readLength);
>    }
> +
> +  public static abstract class NullableConvertedReader extends NullableFixedByteAlignedReader {
> +
> +    protected int dataTypeLengthInBytes;
> +
> +    NullableConvertedReader(ParquetRecordReader parentReader, int allocateSize, ColumnDescriptor descriptor,
> +                                   ColumnChunkMetaData columnChunkMetaData, boolean fixedLength, ValueVector v, SchemaElement schemaElement) throws ExecutionSetupException {
> +      super(parentReader, allocateSize, descriptor, columnChunkMetaData, fixedLength, v, schemaElement);
> +    }
> +
> +    @Override
> +    protected void readField(long recordsToReadInThisPass, ColumnReader firstColumnStatus) {
> +
> +      this.recordsReadInThisIteration = recordsToReadInThisPass;
> +
> +      // set up metadata
> +      this.readStartInBytes = pageReadStatus.readPosInBytes;
> +      this.readLengthInBits = recordsReadInThisIteration * dataTypeLengthInBits;
> +      this.readLength = (int) Math.ceil(readLengthInBits / 8.0);
> +      this.bytes = pageReadStatus.pageDataByteArray;
> +
> +      dataTypeLengthInBytes = (int) Math.ceil(dataTypeLengthInBits / 8.0);
> +      for (int i = 0; i < recordsReadInThisIteration; i++) {
> +        addNext((int) readStartInBytes + i * dataTypeLengthInBytes, i + valuesReadInCurrentPass);
> +      }
> +    }
> +
> +    abstract void addNext(int start, int index);
> +  }
> +
> +  public static class NullableDecimal28Reader extends NullableConvertedReader {
> +
> +    NullableDecimal28SparseVector decimal28Vector;
> +
> +    NullableDecimal28Reader(ParquetRecordReader parentReader, int allocateSize, ColumnDescriptor descriptor, ColumnChunkMetaData columnChunkMetaData,
> +                            boolean fixedLength, ValueVector v, SchemaElement schemaElement) throws ExecutionSetupException {
> +      super(parentReader, allocateSize, descriptor, columnChunkMetaData, fixedLength, v, schemaElement);
> +      decimal28Vector = (NullableDecimal28SparseVector) v;
> +    }
> +
> +    @Override
> +    void addNext(int start, int index) {
> +      int width = NullableDecimal28SparseHolder.WIDTH;
> +      BigDecimal intermediate = DecimalUtility.getBigDecimalFromByteArray(bytes, start, dataTypeLengthInBytes, schemaElement.getScale());
> +      DecimalUtility.getSparseFromBigDecimal(intermediate, decimal28Vector.getData(), index * width, schemaElement.getScale(),
> +              schemaElement.getPrecision(), NullableDecimal28SparseHolder.nDecimalDigits);
> +    }
> +  }
> +
> +  public static class NullableDecimal38Reader extends NullableConvertedReader {
> +
> +    NullableDecimal38SparseVector decimal38Vector;
> +
> +    NullableDecimal38Reader(ParquetRecordReader parentReader, int allocateSize, ColumnDescriptor descriptor, ColumnChunkMetaData columnChunkMetaData,
> +                            boolean fixedLength, ValueVector v, SchemaElement schemaElement) throws ExecutionSetupException {
> +      super(parentReader, allocateSize, descriptor, columnChunkMetaData, fixedLength, v, schemaElement);
> +      decimal38Vector = (NullableDecimal38SparseVector) v;
> +    }
> +
> +    @Override
> +    void addNext(int start, int index) {
> +      int width = NullableDecimal38SparseHolder.WIDTH;
> +      BigDecimal intermediate = DecimalUtility.getBigDecimalFromByteArray(bytes, start, dataTypeLengthInBytes, schemaElement.getScale());
> +      DecimalUtility.getSparseFromBigDecimal(intermediate, decimal38Vector.getData(), index * width, schemaElement.getScale(),
> +              schemaElement.getPrecision(), NullableDecimal38SparseHolder.nDecimalDigits);
> +    }
> +  }
>  }
> \ No newline at end of file
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/63b03467/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetFormatPlugin.java
> ----------------------------------------------------------------------
> diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetFormatPlugin.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetFormatPlugin.java
> index a10d30f..34287f3 100644
> --- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetFormatPlugin.java
> +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetFormatPlugin.java
> @@ -19,26 +19,30 @@ package org.apache.drill.exec.store.parquet;
>
>  import java.io.IOException;
>  import java.util.List;
> +import java.util.Map;
>  import java.util.Set;
>  import java.util.regex.Pattern;
>
> +import com.google.common.collect.Maps;
> +import org.apache.drill.common.exceptions.ExecutionSetupException;
>  import org.apache.drill.common.expression.SchemaPath;
>  import org.apache.drill.common.logical.StoragePluginConfig;
> +import org.apache.drill.exec.ExecConstants;
> +import org.apache.drill.exec.ops.FragmentContext;
>  import org.apache.drill.exec.physical.base.AbstractWriter;
>  import org.apache.drill.exec.physical.base.PhysicalOperator;
> +import org.apache.drill.exec.physical.impl.WriterRecordBatch;
> +import org.apache.drill.exec.proto.ExecProtos.FragmentHandle;
> +import org.apache.drill.exec.record.RecordBatch;
>  import org.apache.drill.exec.server.DrillbitContext;
> +import org.apache.drill.exec.store.RecordWriter;
>  import org.apache.drill.exec.store.StoragePluginOptimizerRule;
> -import org.apache.drill.exec.store.dfs.BasicFormatMatcher;
> -import org.apache.drill.exec.store.dfs.DrillPathFilter;
> -import org.apache.drill.exec.store.dfs.FileSelection;
> -import org.apache.drill.exec.store.dfs.FormatMatcher;
> -import org.apache.drill.exec.store.dfs.FormatPlugin;
> -import org.apache.drill.exec.store.dfs.FormatSelection;
> -import org.apache.drill.exec.store.dfs.MagicString;
> +import org.apache.drill.exec.store.dfs.*;
>  import org.apache.drill.exec.store.dfs.shim.DrillFileSystem;
>  import org.apache.drill.exec.store.mock.MockStorageEngine;
>  import org.apache.hadoop.conf.Configuration;
>  import org.apache.hadoop.fs.FileStatus;
> +import org.apache.hadoop.fs.FileSystem;
>  import org.apache.hadoop.fs.Path;
>  import org.apache.hadoop.fs.PathFilter;
>
> @@ -104,7 +108,35 @@ public class ParquetFormatPlugin implements FormatPlugin{
>
>    @Override
>    public AbstractWriter getWriter(PhysicalOperator child, String location) throws IOException {
> -    throw new UnsupportedOperationException("Parquet Writer is not supported currently.");
> +    return new ParquetWriter(child, location, this);
> +  }
> +
> +  public RecordWriter getRecordWriter(FragmentContext context, ParquetWriter writer) throws IOException {
> +    Map<String, String> options = Maps.newHashMap();
> +
> +    options.put("location", writer.getLocation());
> +
> +    FragmentHandle handle = context.getHandle();
> +    String fragmentId = String.format("%d_%d", handle.getMajorFragmentId(), handle.getMinorFragmentId());
> +    options.put("prefix", fragmentId);
> +
> +    options.put(FileSystem.FS_DEFAULT_NAME_KEY, ((FileSystemConfig)writer.getStorageConfig()).connection);
> +
> +    options.put(ExecConstants.PARQUET_BLOCK_SIZE, context.getOptions().getOption(ExecConstants.PARQUET_BLOCK_SIZE).num_val.toString());
> +
> +    RecordWriter recordWriter = new ParquetRecordWriter();
> +    recordWriter.init(options);
> +
> +    return recordWriter;
> +  }
> +
> +  public RecordBatch getWriterBatch(FragmentContext context, RecordBatch incoming, ParquetWriter writer)
> +          throws ExecutionSetupException {
> +    try {
> +      return new WriterRecordBatch(writer, incoming, context, getRecordWriter(context, writer));
> +    } catch(IOException e) {
> +      throw new ExecutionSetupException(String.format("Failed to create the WriterRecordBatch. %s", e.getMessage()), e);
> +    }
>    }
>
>    @Override
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/63b03467/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetRecordReader.java
> ----------------------------------------------------------------------
> diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetRecordReader.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetRecordReader.java
> index 5d28456..4ca13a5 100644
> --- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetRecordReader.java
> +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetRecordReader.java
> @@ -32,7 +32,9 @@ import org.apache.drill.common.expression.ExpressionPosition;
>  import org.apache.drill.common.expression.FieldReference;
>  import org.apache.drill.common.expression.SchemaPath;
>  import org.apache.drill.common.types.TypeProtos;
> +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.exception.SchemaChangeException;
>  import org.apache.drill.exec.expr.TypeHelper;
> @@ -41,6 +43,10 @@ import org.apache.drill.exec.ops.FragmentContext;
>  import org.apache.drill.exec.physical.impl.OutputMutator;
>  import org.apache.drill.exec.record.MaterializedField;
>  import org.apache.drill.exec.store.RecordReader;
> +import org.apache.drill.exec.store.parquet.FixedByteAlignedReader.Decimal28Reader;
> +import org.apache.drill.exec.store.parquet.FixedByteAlignedReader.Decimal38Reader;
> +import org.apache.drill.exec.store.parquet.NullableFixedByteAlignedReader.NullableDecimal28Reader;
> +import org.apache.drill.exec.store.parquet.NullableFixedByteAlignedReader.NullableDecimal38Reader;
>  import org.apache.drill.exec.vector.*;
>  import org.apache.hadoop.fs.FileSystem;
>  import org.apache.hadoop.fs.Path;
> @@ -62,6 +68,7 @@ import parquet.schema.PrimitiveType;
>  import org.apache.drill.exec.store.parquet.VarLengthColumnReaders.*;
>
>  import com.google.common.base.Joiner;
> +import parquet.schema.PrimitiveType.PrimitiveTypeName;
>
>  public class ParquetRecordReader implements RecordReader {
>    static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ParquetRecordReader.class);
> @@ -191,18 +198,19 @@ public class ParquetRecordReader implements RecordReader {
>
>      // TODO - figure out how to deal with this better once we add nested reading, note also look where this map is used below
>      // store a map from column name to converted types if they are non-null
> -    HashMap<String, ConvertedType> convertedTypes = new HashMap<>();
> +    HashMap<String, SchemaElement> schemaElements = new HashMap<>();
>      fileMetaData = new ParquetMetadataConverter().toParquetMetadata(ParquetFileWriter.CURRENT_VERSION, footer);
>      for (SchemaElement se : fileMetaData.getSchema()) {
> -      convertedTypes.put(se.getName(), se.getConverted_type());
> +      schemaElements.put(se.getName(), se);
>      }
>
>      // loop to add up the length of the fixed width columns and build the schema
>      for (int i = 0; i < columns.size(); ++i) {
>        column = columns.get(i);
>        logger.debug("name: " + fileMetaData.getSchema().get(i).name);
> -      field = MaterializedField.create(toFieldName(column.getPath()),
> -          toMajorType(column.getType(), getDataMode(column), convertedTypes.get(column.getPath()[0])));
> +      SchemaElement se = schemaElements.get(column.getPath()[0]);
> +      MajorType mt = toMajorType(column.getType(), se.getType_length(), getDataMode(column), se);
> +      field = MaterializedField.create(toFieldName(column.getPath()),mt);
>        if ( ! fieldSelected(field)){
>          continue;
>        }
> @@ -211,12 +219,11 @@ public class ParquetRecordReader implements RecordReader {
>        if (column.getType() != PrimitiveType.PrimitiveTypeName.BINARY) {
>          // There is not support for the fixed binary type yet in parquet, leaving a task here as a reminder
>          // TODO - implement this when the feature is added upstream
> -//          if (column.getType() == PrimitiveType.PrimitiveTypeName.FIXED_LEN_BYTE_ARRAY){
> -//              byteWidthAllFixedFields += column.getType().getWidth()
> -//          }
> -//          else { } // the code below for the rest of the fixed length fields
> -
> -        bitWidthAllFixedFields += getTypeLengthInBits(column.getType());
> +          if (column.getType() == PrimitiveType.PrimitiveTypeName.FIXED_LEN_BYTE_ARRAY){
> +              bitWidthAllFixedFields += se.getType_length() * 8;
> +          } else {
> +            bitWidthAllFixedFields += getTypeLengthInBits(column.getType());
> +          }
>        } else {
>          allFieldsFixedLength = false;
>        }
> @@ -234,16 +241,11 @@ public class ParquetRecordReader implements RecordReader {
>      else {
>        recordsPerBatch = DEFAULT_RECORDS_TO_READ_IF_NOT_FIXED_WIDTH;
>      }
> -//    for (SchemaElement se : fileMetaData.getSchema()) {
> -//      if (fieldSelected())
> -//      System.out.println("convertedtype :" + se.getConverted_type());
> -//      System.out.println("name:" + se.getName());
> -//      System.out.println();
> -//
> -//    }
> +
>      try {
>        ValueVector v;
>        ConvertedType convertedType;
> +      SchemaElement schemaElement;
>        ArrayList<VarLengthColumn> varLengthColumns = new ArrayList<>();
>        ArrayList<NullableVarLengthColumn> nullableVarLengthColumns = new ArrayList<>();
>        // initialize all of the column read status objects
> @@ -251,40 +253,21 @@ public class ParquetRecordReader implements RecordReader {
>        for (int i = 0; i < columns.size(); ++i) {
>          column = columns.get(i);
>          columnChunkMetaData = footer.getBlocks().get(0).getColumns().get(i);
> -        convertedType = convertedTypes.get(column.getPath()[0]);
> -        MajorType type = toMajorType(column.getType(), getDataMode(column), convertedType);
> -//        Preconditions.checkArgument(!columnChunkMetaData.getEncodings().contains(Encoding.PLAIN_DICTIONARY), "Dictionary Encoding not currently supported");
> +        schemaElement = schemaElements.get(column.getPath()[0]);
> +        convertedType = schemaElement.getConverted_type();
> +        MajorType type = toMajorType(column.getType(), schemaElement.getType_length(), getDataMode(column), schemaElement);
>          field = MaterializedField.create(toFieldName(column.getPath()), type);
>          // the field was not requested to be read
>          if ( ! fieldSelected(field)) continue;
>
> -        //convertedTypes.put()
>          fieldFixedLength = column.getType() != PrimitiveType.PrimitiveTypeName.BINARY;
>          v = output.addField(field, (Class<? extends ValueVector>) TypeHelper.getValueVectorClass(type.getMinorType(), type.getMode()));
>          if (column.getType() != PrimitiveType.PrimitiveTypeName.BINARY) {
>            createFixedColumnReader(fieldFixedLength, column, columnChunkMetaData, recordsPerBatch, v,
> -            convertedType);
> +            schemaElement);
>          } else {
> -          if (column.getMaxDefinitionLevel() == 0){// column is required
> -            if (convertedType == ConvertedType.UTF8) {
> -              varLengthColumns.add(
> -                new VarCharColumn(this, -1, column, columnChunkMetaData, false, (VarCharVector) v, convertedType));
> -            } else {
> -              varLengthColumns.add(
> -                  new VarBinaryColumn(this, -1, column, columnChunkMetaData, false, (VarBinaryVector) v, convertedType));
> -            }
> -          }
> -          else{
> -            if (convertedType == ConvertedType.UTF8) {
> -              nullableVarLengthColumns.add(
> -                new NullableVarCharColumn(this, -1, column, columnChunkMetaData, false,
> -                    (NullableVarCharVector) v, convertedType));
> -            } else {
> -              nullableVarLengthColumns.add(
> -                new NullableVarBinaryColumn(this, -1, column, columnChunkMetaData, false,
> -                  (NullableVarBinaryVector) v, convertedType));
> -            }
> -          }
> +          // create a reader and add it to the appropriate list
> +          getReader(this, -1, column, columnChunkMetaData, false, v, schemaElement, varLengthColumns, nullableVarLengthColumns);
>          }
>        }
>        varLengthReader = new VarLenBinaryReader(this, varLengthColumns, nullableVarLengthColumns);
> @@ -338,28 +321,41 @@ public class ParquetRecordReader implements RecordReader {
>     */
>    private boolean createFixedColumnReader(boolean fixedLength, ColumnDescriptor descriptor,
>                                            ColumnChunkMetaData columnChunkMetaData, int allocateSize, ValueVector v,
> -                                          ConvertedType convertedType)
> +                                          SchemaElement schemaElement)
>        throws SchemaChangeException, ExecutionSetupException {
> +    ConvertedType convertedType = schemaElement.getConverted_type();
>      // if the column is required
>      if (descriptor.getMaxDefinitionLevel() == 0){
>        if (columnChunkMetaData.getType() == PrimitiveType.PrimitiveTypeName.BOOLEAN){
>          columnStatuses.add(new BitReader(this, allocateSize, descriptor, columnChunkMetaData,
> -            fixedLength, v, convertedType));
> -      }
> -      else{
> +            fixedLength, v, schemaElement));
> +      } else if (columnChunkMetaData.getType() == PrimitiveTypeName.FIXED_LEN_BYTE_ARRAY && convertedType == ConvertedType.DECIMAL){
> +        int length = schemaElement.type_length;
> +        if (length <= 12) {
> +          columnStatuses.add(new Decimal28Reader(this, allocateSize, descriptor, columnChunkMetaData, fixedLength, v, schemaElement));
> +        } else if (length <= 16) {
> +          columnStatuses.add(new Decimal38Reader(this, allocateSize, descriptor, columnChunkMetaData, fixedLength, v, schemaElement));
> +        }
> +      } else {
>          columnStatuses.add(new FixedByteAlignedReader(this, allocateSize, descriptor, columnChunkMetaData,
> -            fixedLength, v, convertedType));
> +            fixedLength, v, schemaElement));
>        }
>        return true;
>      }
>      else { // if the column is nullable
>        if (columnChunkMetaData.getType() == PrimitiveType.PrimitiveTypeName.BOOLEAN){
>          columnStatuses.add(new NullableBitReader(this, allocateSize, descriptor, columnChunkMetaData,
> -            fixedLength, v, convertedType));
> -      }
> -      else{
> +            fixedLength, v, schemaElement));
> +      } else if (columnChunkMetaData.getType() == PrimitiveTypeName.FIXED_LEN_BYTE_ARRAY && convertedType == ConvertedType.DECIMAL){
> +        int length = schemaElement.type_length;
> +        if (length <= 12) {
> +          columnStatuses.add(new NullableDecimal28Reader(this, allocateSize, descriptor, columnChunkMetaData, fixedLength, v, schemaElement));
> +        } else if (length <= 16) {
> +          columnStatuses.add(new NullableDecimal38Reader(this, allocateSize, descriptor, columnChunkMetaData, fixedLength, v, schemaElement));
> +        }
> +      } else {
>          columnStatuses.add(new NullableFixedByteAlignedReader(this, allocateSize, descriptor, columnChunkMetaData,
> -            fixedLength, v, convertedType));
> +            fixedLength, v, schemaElement));
>        }
>        return true;
>      }
> @@ -420,25 +416,57 @@ public class ParquetRecordReader implements RecordReader {
>    }
>
>    static TypeProtos.MajorType toMajorType(PrimitiveType.PrimitiveTypeName primitiveTypeName,
> -                                               TypeProtos.DataMode mode, ConvertedType convertedType) {
> -    return toMajorType(primitiveTypeName, 0, mode, convertedType);
> +                                               TypeProtos.DataMode mode, SchemaElement schemaElement) {
> +    return toMajorType(primitiveTypeName, 0, mode, schemaElement);
>    }
>
>    static TypeProtos.MajorType toMajorType(PrimitiveType.PrimitiveTypeName primitiveTypeName, int length,
> -                                               TypeProtos.DataMode mode, ConvertedType convertedType) {
> +                                               TypeProtos.DataMode mode, SchemaElement schemaElement) {
> +    ConvertedType convertedType = schemaElement.getConverted_type();
>      switch (mode) {
>
>        case OPTIONAL:
>          switch (primitiveTypeName) {
>            case BINARY:
> -            if (convertedType == ConvertedType.UTF8)
> -              return Types.optional(TypeProtos.MinorType.VARCHAR);
> -            else
> +            if (convertedType == null) {
>                return Types.optional(TypeProtos.MinorType.VARBINARY);
> +            }
> +            switch (convertedType) {
> +              case UTF8:
> +                return Types.optional(MinorType.VARCHAR);
> +              case DECIMAL:
> +                return Types.withScaleAndPrecision(getDecimalType(schemaElement), DataMode.OPTIONAL, schemaElement.getScale(), schemaElement.getPrecision());
> +              default:
> +                throw new UnsupportedOperationException(String.format("unsupported type: %s %s", primitiveTypeName, convertedType));
> +            }
>            case INT64:
> -            return Types.optional(TypeProtos.MinorType.BIGINT);
> +            if (convertedType == null) {
> +              return Types.optional(TypeProtos.MinorType.BIGINT);
> +            }
> +            switch(convertedType) {
> +              case DECIMAL:
> +                return Types.withScaleAndPrecision(MinorType.DECIMAL18, DataMode.OPTIONAL, schemaElement.getScale(), schemaElement.getPrecision());
> +              case FINETIME:
> +                throw new UnsupportedOperationException();
> +              case TIMESTAMP:
> +                return Types.optional(MinorType.TIMESTAMP);
> +              default:
> +                throw new UnsupportedOperationException(String.format("unsupported type: %s %s", primitiveTypeName, convertedType));
> +            }
>            case INT32:
> -            return Types.optional(TypeProtos.MinorType.INT);
> +            if (convertedType == null) {
> +              return Types.optional(TypeProtos.MinorType.INT);
> +            }
> +            switch(convertedType) {
> +              case DECIMAL:
> +                return Types.withScaleAndPrecision(MinorType.DECIMAL9, DataMode.OPTIONAL, schemaElement.getScale(), schemaElement.getPrecision());
> +              case DATE:
> +                return Types.optional(MinorType.DATE);
> +              case TIME:
> +                return Types.optional(MinorType.TIME);
> +              default:
> +                throw new UnsupportedOperationException(String.format("unsupported type: %s %s", primitiveTypeName, convertedType));
> +            }
>            case BOOLEAN:
>              return Types.optional(TypeProtos.MinorType.BIT);
>            case FLOAT:
> @@ -451,23 +479,58 @@ public class ParquetRecordReader implements RecordReader {
>              return TypeProtos.MajorType.newBuilder().setMinorType(TypeProtos.MinorType.FIXEDBINARY).setWidth(12)
>                  .setMode(mode).build();
>            case FIXED_LEN_BYTE_ARRAY:
> -            checkArgument(length > 0, "A length greater than zero must be provided for a FixedBinary type.");
> -            return TypeProtos.MajorType.newBuilder().setMinorType(TypeProtos.MinorType.FIXEDBINARY)
> -                .setWidth(length).setMode(mode).build();
> +            if (convertedType == null) {
> +              checkArgument(length > 0, "A length greater than zero must be provided for a FixedBinary type.");
> +              return TypeProtos.MajorType.newBuilder().setMinorType(TypeProtos.MinorType.FIXEDBINARY)
> +                      .setWidth(length).setMode(mode).build();
> +            } else if (convertedType == ConvertedType.DECIMAL) {
> +              return Types.withScaleAndPrecision(getDecimalType(schemaElement), DataMode.OPTIONAL, schemaElement.getScale(), schemaElement.getPrecision());
> +            }
>            default:
>              throw new UnsupportedOperationException("Type not supported: " + primitiveTypeName);
>          }
>        case REQUIRED:
>          switch (primitiveTypeName) {
>            case BINARY:
> -            if (convertedType == ConvertedType.UTF8)
> -              return Types.required(TypeProtos.MinorType.VARCHAR);
> -            else
> +            if (convertedType == null) {
>                return Types.required(TypeProtos.MinorType.VARBINARY);
> +            }
> +            switch (convertedType) {
> +              case UTF8:
> +                return Types.required(MinorType.VARCHAR);
> +              case DECIMAL:
> +                return Types.withScaleAndPrecision(getDecimalType(schemaElement), DataMode.REQUIRED, schemaElement.getScale(), schemaElement.getPrecision());
> +              default:
> +                throw new UnsupportedOperationException(String.format("unsupported type: %s %s", primitiveTypeName, convertedType));
> +            }
>            case INT64:
> -            return Types.required(TypeProtos.MinorType.BIGINT);
> +            if (convertedType == null) {
> +              return Types.required(MinorType.BIGINT);
> +            }
> +            switch(convertedType) {
> +              case DECIMAL:
> +                return Types.withScaleAndPrecision(MinorType.DECIMAL18, DataMode.REQUIRED, schemaElement.getScale(), schemaElement.getPrecision());
> +              case FINETIME:
> +                throw new UnsupportedOperationException();
> +              case TIMESTAMP:
> +                return Types.required(MinorType.TIMESTAMP);
> +              default:
> +                throw new UnsupportedOperationException(String.format("unsupported type: %s %s", primitiveTypeName, convertedType));
> +            }
>            case INT32:
> -            return Types.required(TypeProtos.MinorType.INT);
> +            if (convertedType == null) {
> +              return Types.required(MinorType.INT);
> +            }
> +            switch(convertedType) {
> +              case DECIMAL:
> +                return Types.withScaleAndPrecision(MinorType.DECIMAL9, DataMode.REQUIRED, schemaElement.getScale(), schemaElement.getPrecision());
> +              case DATE:
> +                return Types.required(MinorType.DATE);
> +              case TIME:
> +                return Types.required(MinorType.TIME);
> +              default:
> +                throw new UnsupportedOperationException(String.format("unsupported type: %s %s", primitiveTypeName, convertedType));
> +            }
>            case BOOLEAN:
>              return Types.required(TypeProtos.MinorType.BIT);
>            case FLOAT:
> @@ -480,23 +543,58 @@ public class ParquetRecordReader implements RecordReader {
>              return TypeProtos.MajorType.newBuilder().setMinorType(TypeProtos.MinorType.FIXEDBINARY).setWidth(12)
>                  .setMode(mode).build();
>            case FIXED_LEN_BYTE_ARRAY:
> -            checkArgument(length > 0, "A length greater than zero must be provided for a FixedBinary type.");
> -            return TypeProtos.MajorType.newBuilder().setMinorType(TypeProtos.MinorType.FIXEDBINARY)
> -                .setWidth(length).setMode(mode).build();
> +            if (convertedType == null) {
> +              checkArgument(length > 0, "A length greater than zero must be provided for a FixedBinary type.");
> +              return TypeProtos.MajorType.newBuilder().setMinorType(TypeProtos.MinorType.FIXEDBINARY)
> +                      .setWidth(length).setMode(mode).build();
> +            } else if (convertedType == ConvertedType.DECIMAL) {
> +              return Types.withScaleAndPrecision(getDecimalType(schemaElement), DataMode.REQUIRED, schemaElement.getScale(), schemaElement.getPrecision());
> +            }
>            default:
>              throw new UnsupportedOperationException("Type not supported: " + primitiveTypeName);
>          }
>        case REPEATED:
>          switch (primitiveTypeName) {
>            case BINARY:
> -            if (convertedType == ConvertedType.UTF8)
> -              return Types.required(TypeProtos.MinorType.VARCHAR);
> -            else
> +            if (convertedType == null) {
>                return Types.repeated(TypeProtos.MinorType.VARBINARY);
> +            }
> +            switch (schemaElement.getConverted_type()) {
> +              case UTF8:
> +                return Types.repeated(MinorType.VARCHAR);
> +              case DECIMAL:
> +                return Types.withScaleAndPrecision(getDecimalType(schemaElement), DataMode.REPEATED, schemaElement.getScale(), schemaElement.getPrecision());
> +              default:
> +                throw new UnsupportedOperationException(String.format("unsupported type: %s %s", primitiveTypeName, convertedType));
> +            }
>            case INT64:
> -            return Types.repeated(TypeProtos.MinorType.BIGINT);
> +            if (convertedType == null) {
> +              return Types.repeated(MinorType.BIGINT);
> +            }
> +            switch(convertedType) {
> +              case DECIMAL:
> +                return Types.withScaleAndPrecision(MinorType.DECIMAL18, DataMode.REPEATED, schemaElement.getScale(), schemaElement.getPrecision());
> +              case FINETIME:
> +                throw new UnsupportedOperationException();
> +              case TIMESTAMP:
> +                return Types.repeated(MinorType.TIMESTAMP);
> +              default:
> +                throw new UnsupportedOperationException(String.format("unsupported type: %s %s", primitiveTypeName, convertedType));
> +            }
>            case INT32:
> -            return Types.repeated(TypeProtos.MinorType.INT);
> +            if (convertedType == null) {
> +              return Types.repeated(MinorType.INT);
> +            }
> +            switch(convertedType) {
> +              case DECIMAL:
> +                return Types.withScaleAndPrecision(MinorType.DECIMAL9, DataMode.REPEATED, schemaElement.getScale(), schemaElement.getPrecision());
> +              case DATE:
> +                return Types.repeated(MinorType.DATE);
> +              case TIME:
> +                return Types.repeated(MinorType.TIME);
> +              default:
> +                throw new UnsupportedOperationException(String.format("unsupported type: %s %s", primitiveTypeName, convertedType));
> +            }
>            case BOOLEAN:
>              return Types.repeated(TypeProtos.MinorType.BIT);
>            case FLOAT:
> @@ -509,9 +607,13 @@ public class ParquetRecordReader implements RecordReader {
>              return TypeProtos.MajorType.newBuilder().setMinorType(TypeProtos.MinorType.FIXEDBINARY).setWidth(12)
>                  .setMode(mode).build();
>            case FIXED_LEN_BYTE_ARRAY:
> -            checkArgument(length > 0, "A length greater than zero must be provided for a FixedBinary type.");
> -            return TypeProtos.MajorType.newBuilder().setMinorType(TypeProtos.MinorType.FIXEDBINARY)
> -                .setWidth(length).setMode(mode).build();
> +            if (convertedType == null) {
> +              checkArgument(length > 0, "A length greater than zero must be provided for a FixedBinary type.");
> +              return TypeProtos.MajorType.newBuilder().setMinorType(TypeProtos.MinorType.FIXEDBINARY)
> +                      .setWidth(length).setMode(mode).build();
> +            } else if (convertedType == ConvertedType.DECIMAL) {
> +              return Types.withScaleAndPrecision(getDecimalType(schemaElement), DataMode.REPEATED, schemaElement.getScale(), schemaElement.getPrecision());
> +            }
>            default:
>              throw new UnsupportedOperationException("Type not supported: " + primitiveTypeName);
>          }
> @@ -519,6 +621,58 @@ public class ParquetRecordReader implements RecordReader {
>      throw new UnsupportedOperationException("Type not supported: " + primitiveTypeName + " Mode: " + mode);
>    }
>
> +  private static void getReader(ParquetRecordReader parentReader, int allocateSize, ColumnDescriptor descriptor,
> +                                        ColumnChunkMetaData columnChunkMetaData, boolean fixedLength, ValueVector v,
> +                                        SchemaElement schemaElement, List<VarLengthColumn> varLengthColumns,
> +                                        List<NullableVarLengthColumn> nullableVarLengthColumns) throws ExecutionSetupException {
> +    ConvertedType convertedType = schemaElement.getConverted_type();
> +    switch (descriptor.getMaxDefinitionLevel()) {
> +      case 0:
> +        if (convertedType == null) {
> +          varLengthColumns.add(new VarBinaryColumn(parentReader, allocateSize, descriptor, columnChunkMetaData, fixedLength, (VarBinaryVector) v, schemaElement));
> +          return;
> +        }
> +        switch (convertedType) {
> +          case UTF8:
> +            varLengthColumns.add(new VarCharColumn(parentReader, allocateSize, descriptor, columnChunkMetaData, fixedLength, (VarCharVector) v, schemaElement));
> +            return;
> +          case DECIMAL:
> +            if (v instanceof Decimal28SparseVector) {
> +              varLengthColumns.add(new Decimal28Column(parentReader, allocateSize, descriptor, columnChunkMetaData, fixedLength, (Decimal28SparseVector) v, schemaElement));
> +              return;
> +            } else if (v instanceof Decimal38SparseVector) {
> +              varLengthColumns.add(new Decimal38Column(parentReader, allocateSize, descriptor, columnChunkMetaData, fixedLength, (Decimal38SparseVector) v, schemaElement));
> +              return;
> +            }
> +          default:
> +        }
> +      default:
> +        if (convertedType == null) {
> +          nullableVarLengthColumns.add(new NullableVarBinaryColumn(parentReader, allocateSize, descriptor, columnChunkMetaData, fixedLength, (NullableVarBinaryVector) v, schemaElement));
> +          return;
> +        }
> +        switch (convertedType) {
> +          case UTF8:
> +            nullableVarLengthColumns.add(new NullableVarCharColumn(parentReader, allocateSize, descriptor, columnChunkMetaData, fixedLength, (NullableVarCharVector) v, schemaElement));
> +            return;
> +          case DECIMAL:
> +            if (v instanceof NullableDecimal28SparseVector) {
> +              nullableVarLengthColumns.add(new NullableDecimal28Column(parentReader, allocateSize, descriptor, columnChunkMetaData, fixedLength, (NullableDecimal28SparseVector) v, schemaElement));
> +              return;
> +            } else if (v instanceof NullableDecimal38SparseVector) {
> +              nullableVarLengthColumns.add(new NullableDecimal38Column(parentReader, allocateSize, descriptor, columnChunkMetaData, fixedLength, (NullableDecimal38SparseVector) v, schemaElement));
> +              return;
> +            }
> +          default:
> +        }
> +    }
> +    throw new UnsupportedOperationException();
> +  }
> +
> +  private static MinorType getDecimalType(SchemaElement schemaElement) {
> +    return schemaElement.getPrecision() <= 28 ? MinorType.DECIMAL28SPARSE : MinorType.DECIMAL38SPARSE;
> +  }
> +
>    static String join(String delimiter, String... str) {
>      StringBuilder builder = new StringBuilder();
>      int i = 0;
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/63b03467/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetRecordWriter.java
> ----------------------------------------------------------------------
> diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetRecordWriter.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetRecordWriter.java
> new file mode 100644
> index 0000000..3ba13f4
> --- /dev/null
> +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetRecordWriter.java
> @@ -0,0 +1,185 @@
> +/**
> + * Licensed to the Apache Software Foundation (ASF) under one
> + * or more contributor license agreements.  See the NOTICE file
> + * distributed with this work for additional information
> + * regarding copyright ownership.  The ASF licenses this file
> + * to you under the Apache License, Version 2.0 (the
> + * "License"); you may not use this file except in compliance
> + * with the License.  You may obtain a copy of the License at
> + *
> + * http://www.apache.org/licenses/LICENSE-2.0
> + *
> + * Unless required by applicable law or agreed to in writing, software
> + * distributed under the License is distributed on an "AS IS" BASIS,
> + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
> + * See the License for the specific language governing permissions and
> + * limitations under the License.
> + */
> +package org.apache.drill.exec.store.parquet;
> +
> +import com.google.common.collect.Lists;
> +import org.apache.drill.common.types.TypeProtos.MinorType;
> +import org.apache.drill.exec.ExecConstants;
> +import org.apache.drill.exec.record.BatchSchema;
> +import org.apache.drill.exec.record.MaterializedField;
> +import org.apache.drill.exec.store.ParquetOutputRecordWriter;
> +import org.apache.hadoop.conf.Configuration;
> +import org.apache.hadoop.fs.FileSystem;
> +import org.apache.hadoop.fs.Path;
> +import parquet.column.ParquetProperties.WriterVersion;
> +import parquet.column.impl.ColumnWriteStoreImpl;
> +import parquet.column.page.PageWriteStore;
> +import parquet.hadoop.ColumnChunkPageWriteStoreExposer;
> +import parquet.hadoop.ParquetFileWriter;
> +import parquet.hadoop.metadata.CompressionCodecName;
> +import parquet.io.ColumnIOFactory;
> +import parquet.io.MessageColumnIO;
> +import parquet.io.api.RecordConsumer;
> +import parquet.schema.DecimalMetadata;
> +import parquet.schema.MessageType;
> +import parquet.schema.OriginalType;
> +import parquet.schema.PrimitiveType.PrimitiveTypeName;
> +import parquet.schema.Type;
> +import parquet.schema.Type.Repetition;
> +
> +import java.io.IOException;
> +import java.util.HashMap;
> +import java.util.List;
> +import java.util.Map;
> +
> +import static java.lang.Math.max;
> +import static java.lang.Math.min;
> +import static java.lang.String.format;
> +
> +public class ParquetRecordWriter extends ParquetOutputRecordWriter {
> +
> +  private static final int MINIMUM_BUFFER_SIZE = 64 * 1024;
> +  private static final int MINIMUM_RECORD_COUNT_FOR_CHECK = 100;
> +  private static final int MAXIMUM_RECORD_COUNT_FOR_CHECK = 10000;
> +
> +  private ParquetFileWriter w;
> +  private MessageType schema;
> +  private Map<String, String> extraMetaData = new HashMap();
> +  private int blockSize;
> +  private int pageSize = 1 * 1024 * 1024;
> +  private int dictionaryPageSize = pageSize;
> +  private boolean enableDictionary = false;
> +  private boolean validating = false;
> +  private CompressionCodecName codec = CompressionCodecName.SNAPPY;
> +  private WriterVersion writerVersion = WriterVersion.PARQUET_1_0;
> +
> +  private long recordCount = 0;
> +  private long recordCountForNextMemCheck = MINIMUM_RECORD_COUNT_FOR_CHECK;
> +
> +  private ColumnWriteStoreImpl store;
> +  private PageWriteStore pageStore;
> +
> +  private RecordConsumer consumer;
> +  private BatchSchema batchSchema;
> +
> +  private Configuration conf;
> +  private String location;
> +  private String prefix;
> +  private int index = 0;
> +
> +  @Override
> +  public void init(Map<String, String> writerOptions) throws IOException {
> +    this.location = writerOptions.get("location");
> +    this.prefix = writerOptions.get("prefix");
> +
> +    conf = new Configuration();
> +    conf.set(FileSystem.FS_DEFAULT_NAME_KEY, writerOptions.get(FileSystem.FS_DEFAULT_NAME_KEY));
> +    blockSize = Integer.parseInt(writerOptions.get(ExecConstants.PARQUET_BLOCK_SIZE));
> +  }
> +
> +  @Override
> +  public void updateSchema(BatchSchema batchSchema) throws IOException {
> +    if (this.batchSchema == null || !this.batchSchema.equals(batchSchema)) {
> +      if (this.batchSchema != null) {
> +        flush();
> +      }
> +      this.batchSchema = batchSchema;
> +      newSchema();
> +    }
> +  }
> +
> +  private void newSchema() throws IOException {
> +    List<Type> types = Lists.newArrayList();
> +    for (MaterializedField field : batchSchema) {
> +      String name = field.getAsSchemaPath().getAsUnescapedPath();
> +      MinorType minorType = field.getType().getMinorType();
> +      PrimitiveTypeName primitiveTypeName = ParquetTypeHelper.getPrimitiveTypeNameForMinorType(minorType);
> +      Repetition repetition = ParquetTypeHelper.getRepetitionForDataMode(field.getDataMode());
> +      OriginalType originalType = ParquetTypeHelper.getOriginalTypeForMinorType(minorType);
> +      DecimalMetadata decimalMetadata = ParquetTypeHelper.getDecimalMetadataForField(field);
> +      int length = ParquetTypeHelper.getLengthForMinorType(minorType);
> +      parquet.schema.Type type = new parquet.schema.PrimitiveType(repetition, primitiveTypeName, length, name, originalType, decimalMetadata);
> +      types.add(type);
> +    }
> +    schema = new MessageType("root", types);
> +
> +    Path fileName = new Path(location, prefix + "_" + index + ".parquet");
> +    w = new ParquetFileWriter(conf, schema, fileName);
> +    w.start();
> +
> +    int initialBlockBufferSize = max(MINIMUM_BUFFER_SIZE, blockSize / this.schema.getColumns().size() / 5);
> +    pageStore = ColumnChunkPageWriteStoreExposer.newColumnChunkPageWriteStore(codec, pageSize, this.schema, initialBlockBufferSize);
> +    int initialPageBufferSize = max(MINIMUM_BUFFER_SIZE, min(pageSize + pageSize / 10, initialBlockBufferSize));
> +    store = new ColumnWriteStoreImpl(pageStore, pageSize, initialPageBufferSize, dictionaryPageSize, enableDictionary, writerVersion);
> +    MessageColumnIO columnIO = new ColumnIOFactory(validating).getColumnIO(this.schema);
> +    consumer = columnIO.getRecordWriter(store);
> +    setUp(schema, consumer);
> +  }
> +
> +  private void flush() throws IOException {
> +    w.startBlock(recordCount);
> +    store.flush();
> +    ColumnChunkPageWriteStoreExposer.flushPageStore(pageStore, w);
> +    recordCount = 0;
> +    w.endBlock();
> +    w.end(extraMetaData);
> +    store = null;
> +    pageStore = null;
> +    index++;
> +  }
> +
> +  private void checkBlockSizeReached() throws IOException {
> +    if (recordCount >= recordCountForNextMemCheck) { // checking the memory size is relatively expensive, so let's not do it for every record.
> +      long memSize = store.memSize();
> +      if (memSize > blockSize) {
> +        System.out.println("Reached block size " + blockSize);
> +        flush();
> +        newSchema();
> +        recordCountForNextMemCheck = min(max(MINIMUM_RECORD_COUNT_FOR_CHECK, recordCount / 2), MAXIMUM_RECORD_COUNT_FOR_CHECK);
> +      } else {
> +        float recordSize = (float) memSize / recordCount;
> +        recordCountForNextMemCheck = min(
> +                max(MINIMUM_RECORD_COUNT_FOR_CHECK, (recordCount + (long)(blockSize / recordSize)) / 2), // will check halfway
> +                recordCount + MAXIMUM_RECORD_COUNT_FOR_CHECK // will not look more than max records ahead
> +        );
> +      }
> +    }
> +  }
> +
> +  @Override
> +  public void startRecord() throws IOException {
> +    consumer.startMessage();
> +  }
> +
> +  @Override
> +  public void endRecord() throws IOException {
> +    consumer.endMessage();
> +    recordCount++;
> +    checkBlockSizeReached();
> +  }
> +
> +  @Override
> +  public void abort() throws IOException {
> +    //To change body of implemented methods use File | Settings | File Templates.
> +  }
> +
> +  @Override
> +  public void cleanup() throws IOException {
> +    flush();
> +  }
> +}
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/63b03467/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetWriter.java
> ----------------------------------------------------------------------
> diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetWriter.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetWriter.java
> new file mode 100644
> index 0000000..ea9798a
> --- /dev/null
> +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetWriter.java
> @@ -0,0 +1,91 @@
> +/**
> + * Licensed to the Apache Software Foundation (ASF) under one
> + * or more contributor license agreements.  See the NOTICE file
> + * distributed with this work for additional information
> + * regarding copyright ownership.  The ASF licenses this file
> + * to you under the Apache License, Version 2.0 (the
> + * "License"); you may not use this file except in compliance
> + * with the License.  You may obtain a copy of the License at
> + *
> + * http://www.apache.org/licenses/LICENSE-2.0
> + *
> + * Unless required by applicable law or agreed to in writing, software
> + * distributed under the License is distributed on an "AS IS" BASIS,
> + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
> + * See the License for the specific language governing permissions and
> + * limitations under the License.
> + */
> +package org.apache.drill.exec.store.parquet;
> +
> +import com.fasterxml.jackson.annotation.*;
> +import com.google.common.base.Preconditions;
> +import org.apache.drill.common.exceptions.ExecutionSetupException;
> +import org.apache.drill.common.logical.FormatPluginConfig;
> +import org.apache.drill.common.logical.StoragePluginConfig;
> +import org.apache.drill.exec.physical.OperatorCost;
> +import org.apache.drill.exec.physical.base.AbstractWriter;
> +import org.apache.drill.exec.physical.base.PhysicalOperator;
> +import org.apache.drill.exec.store.StoragePluginRegistry;
> +
> +import java.io.IOException;
> +
> +@JsonTypeName("parquet-writer")
> +public class ParquetWriter extends AbstractWriter {
> +  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ParquetWriter.class);
> +
> +  private final String location;
> +  private final ParquetFormatPlugin formatPlugin;
> +
> +  @JsonCreator
> +  public ParquetWriter(
> +          @JsonProperty("child") PhysicalOperator child,
> +          @JsonProperty("location") String location,
> +          @JsonProperty("storage") StoragePluginConfig storageConfig,
> +          @JacksonInject StoragePluginRegistry engineRegistry) throws IOException, ExecutionSetupException {
> +
> +    super(child);
> +    this.formatPlugin = (ParquetFormatPlugin) engineRegistry.getFormatPlugin(storageConfig, new ParquetFormatConfig());
> +    Preconditions.checkNotNull(formatPlugin, "Unable to load format plugin for provided format config.");
> +    this.location = location;
> +  }
> +
> +  public ParquetWriter(PhysicalOperator child,
> +                       String location,
> +                       ParquetFormatPlugin formatPlugin) {
> +
> +    super(child);
> +    this.formatPlugin = formatPlugin;
> +    this.location = location;
> +  }
> +
> +  @JsonProperty("location")
> +  public String getLocation() {
> +    return location;
> +  }
> +
> +  @JsonProperty("storage")
> +  public StoragePluginConfig getStorageConfig(){
> +    return formatPlugin.getStorageConfig();
> +  }
> +
> +  @JsonProperty("format")
> +  public FormatPluginConfig getFormatConfig(){
> +    return formatPlugin.getConfig();
> +  }
> +
> +  @JsonIgnore
> +  public ParquetFormatPlugin getFormatPlugin(){
> +    return formatPlugin;
> +  }
> +
> +  @Override
> +  protected PhysicalOperator getNewWithChild(PhysicalOperator child) {
> +    return new ParquetWriter(child, location, formatPlugin);
> +  }
> +
> +  @Override
> +  public OperatorCost getCost() {
> +    // TODO:
> +    return new OperatorCost(1,1,1,1);
> +  }
> +}
>