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);
> + }
> +}
>