diff --git a/CHANGES.md b/CHANGES.md
index cd34e0a0b0..6f2894dce9 100644
--- a/CHANGES.md
+++ b/CHANGES.md
@@ -1,5 +1,63 @@
# Parquet #
+### Version 1.2.10 ###
+* ISSUE [247](https://github.com/Parquet/parquet-mr/pull/247): fix bug: when field index is greater than zero
+* ISSUE [244](https://github.com/Parquet/parquet-mr/pull/244): Feature/error handler
+* ISSUE [187](https://github.com/Parquet/parquet-mr/pull/187): Plumb OriginalType
+* ISSUE [245](https://github.com/Parquet/parquet-mr/pull/245): integrate parquet format 2.0
+
+### Version 1.2.9 ###
+* ISSUE [242](https://github.com/Parquet/parquet-mr/pull/242): upgrade elephant-bird version to 4.3
+* ISSUE [240](https://github.com/Parquet/parquet-mr/pull/240): fix loader cache
+* ISSUE [233](https://github.com/Parquet/parquet-mr/pull/233): use latest stable release of cascading: 2.5.1
+* ISSUE [241](https://github.com/Parquet/parquet-mr/pull/241): Update reference to 0.10 in Hive012Binding javadoc
+* ISSUE [239](https://github.com/Parquet/parquet-mr/pull/239): Fix hive map and array inspectors with null containers
+* ISSUE [234](https://github.com/Parquet/parquet-mr/pull/234): optimize chunk scan; fix compressed size
+* ISSUE [237](https://github.com/Parquet/parquet-mr/pull/237): Handle codec not found
+* ISSUE [238](https://github.com/Parquet/parquet-mr/pull/238): fix pom version caused by bad merge
+* ISSUE [235](https://github.com/Parquet/parquet-mr/pull/235): Not write pig meta data only when pig is not avaliable
+* ISSUE [227](https://github.com/Parquet/parquet-mr/pull/227): Breaks parquet-hive up into several submodules, creating infrastructure ...
+* ISSUE [229](https://github.com/Parquet/parquet-mr/pull/229): add changelog tool
+* ISSUE [236](https://github.com/Parquet/parquet-mr/pull/236): Make cascading a provided dependency
+
+### Version 1.2.8 ###
+* ISSUE 228: enable globing files for parquetTupleScheme, refactor unit tests and rem...
+* ISSUE 224: Changing read and write methods in ParquetInputSplit so that they can de...
+
+### Version 1.2.8 ###
+* ISSUE 228: enable globing files for parquetTupleScheme, refactor unit tests and rem...
+* ISSUE 224: Changing read and write methods in ParquetInputSplit so that they can de...
+
+### Version 1.2.7 ###
+* ISSUE 223: refactor encoded values changes and test that resetDictionary works
+* ISSUE 222: fix bug: set raw data size to 0 after reset
+
+### Version 1.2.6 ###
+* ISSUE 221: make pig, hadoop and log4j jars provided
+* ISSUE 220: parquet-hive should ship and uber jar
+* ISSUE 213: group parquet-format version in one property
+* ISSUE 215: Fix Binary.equals().
+* ISSUE 210: ParquetWriter ignores enable dictionary and validating flags.
+* ISSUE 202: Fix requested schema when recreating splits in hive
+* ISSUE 208: Improve dic fall back
+* ISSUE 207: Fix offset
+* ISSUE 206: Create a "Powered by" page
+
+### Version 1.2.5 ###
+* ISSUE 204: ParquetLoader.inputFormatCache as WeakHashMap
+* ISSUE 203: add null check for EnumWriteProtocol
+* ISSUE 205: use cascading 2.2.0
+* ISSUE 199: simplify TupleWriteSupport constructor
+* ISSUE 164: Dictionary changes
+* ISSUE 196: Fixes to the Hive SerDe
+* ISSUE 197: RLE decoder reading past the end of the stream
+* ISSUE 188: Added ability to define arbitrary predicate functions
+* ISSUE 194: refactor serde to remove some unecessary boxing and include dictionary awareness
+* ISSUE 190: NPE in DictionaryValuesWriter.
+
+### Version 1.2.4 ###
+* ISSUE 191: Add compatibility checker for ThriftStruct to check for backward compatibility of two thrift structs
+
### Version 1.2.3 ###
* ISSUE 186: add parquet-pig-bundle
* ISSUE 184: Update ParquetReader to take Configuration as a constructor argument.
diff --git a/PoweredBy.md b/PoweredBy.md
new file mode 100644
index 0000000000..f9b5ac5293
--- /dev/null
+++ b/PoweredBy.md
@@ -0,0 +1,25 @@
+Who's using Parquet?
+======
+(in alphabetical order)
+
+## Cloudera Impala
+
+
+
+## Criteo
+
+
+
+## Salesforce.com
+
+
+
+## Stripe
+
+
+
+## Twitter
+
+
+
+
diff --git a/README.md b/README.md
index 22b1c1b425..8a281d5499 100644
--- a/README.md
+++ b/README.md
@@ -18,6 +18,7 @@ Parquet is a very active project, and new features are being added quickly; belo
Hive integration YES (28 ) 1.0
Pig integration YES 1.0
Cascading integration YES 1.0
+ Crunch integration YES (CRUNCH-277 ) 1.0
Impala integration YES (non-nested) 1.0
Java Map/Reduce API YES 1.0
Native Avro support YES 1.0
diff --git a/changelog.sh b/changelog.sh
new file mode 100755
index 0000000000..e7e5b1f960
--- /dev/null
+++ b/changelog.sh
@@ -0,0 +1,36 @@
+echo "github username:" >&2
+read username >&2
+echo "github password:" >&2
+read -s password >&2
+
+curl -f -u $username:$password -s "https://api.github.com" > /dev/null
+if [ $? == 0 ]
+then
+ echo "login successful" >&2
+else
+ echo "login failed" >&2
+ curl -u $username:$password -s "https://api.github.com"
+ exit 1
+fi
+
+echo "# Parquet #"
+
+git log | grep -E "Merge pull request|prepare release" | while read l
+do
+ release=`echo $l | grep "\[maven-release-plugin\] prepare release" | cut -d "-" -f 4`
+ PR=`echo $l| grep -E -o "Merge pull request #[^ ]*" | cut -d "#" -f 2`
+# echo $l
+ if [ -n "$release" ]
+ then
+ echo
+ echo "### Version $release ###"
+ fi
+ if [ -n "$PR" ]
+ then
+ JSON=`curl -u $username:$password -s https://api.github.com/repos/Parquet/parquet-mr/pulls/$PR | tr "\n" " "`
+ DESC_RAW=$(echo $JSON | grep -Po '"title":.*?[^\\]",' | cut -d "\"" -f 4- | head -n 1 | sed -e "s/\\\\//g")
+ DESC=$(echo ${DESC_RAW%\",})
+ echo "* ISSUE [$PR](https://github.com/Parquet/parquet-mr/pull/$PR): ${DESC}"
+ fi
+done
+
diff --git a/parquet-avro/pom.xml b/parquet-avro/pom.xml
index 33258e5b59..ea3d5effa9 100644
--- a/parquet-avro/pom.xml
+++ b/parquet-avro/pom.xml
@@ -3,7 +3,7 @@
com.twitter
parquet
../pom.xml
- 1.2.4-SNAPSHOT
+ 1.2.11-SNAPSHOT
4.0.0
@@ -32,7 +32,7 @@
com.twitter
parquet-format
- 1.0.0
+ ${parquet.format.version}
org.apache.avro
@@ -43,6 +43,7 @@
org.apache.hadoop
hadoop-client
${hadoop.version}
+ provided
com.google.guava
diff --git a/parquet-cascading/pom.xml b/parquet-cascading/pom.xml
index 38e03fc28d..de03a11387 100644
--- a/parquet-cascading/pom.xml
+++ b/parquet-cascading/pom.xml
@@ -3,7 +3,7 @@
com.twitter
parquet
../pom.xml
- 1.2.4-SNAPSHOT
+ 1.2.11-SNAPSHOT
4.0.0
@@ -41,11 +41,13 @@
org.apache.hadoop
hadoop-client
${hadoop.version}
+ provided
log4j
log4j
- 1.2.17
+ ${log4j.version}
+ provided
com.twitter
@@ -63,7 +65,8 @@
cascading
cascading-hadoop
- 2.1.5
+ ${cascading.version}
+ provided
diff --git a/parquet-cascading/src/main/java/parquet/cascading/ParquetTupleScheme.java b/parquet-cascading/src/main/java/parquet/cascading/ParquetTupleScheme.java
index d75b6e76d7..5d229c72d7 100644
--- a/parquet-cascading/src/main/java/parquet/cascading/ParquetTupleScheme.java
+++ b/parquet-cascading/src/main/java/parquet/cascading/ParquetTupleScheme.java
@@ -93,7 +93,8 @@ private MessageType readSchema(FlowProcess flowProcess, Tap tap) {
else
hfs = (Hfs) tap;
- List footers = ParquetFileReader.readFooters(flowProcess.getConfigCopy(), hfs.getPath());
+ List footers = getFooters(flowProcess, hfs);
+
if(footers.isEmpty()) {
throw new TapException("Could not read Parquet metadata at " + hfs.getPath());
} else {
@@ -104,7 +105,14 @@ private MessageType readSchema(FlowProcess flowProcess, Tap tap) {
}
}
- @SuppressWarnings("unchecked")
+ private List getFooters(FlowProcess flowProcess, Hfs hfs) throws IOException {
+ JobConf jobConf = flowProcess.getConfigCopy();
+ DeprecatedParquetInputFormat format = new DeprecatedParquetInputFormat();
+ format.addInputPath(jobConf, hfs.getPath());
+ return format.getFooters(jobConf);
+ }
+
+ @SuppressWarnings("unchecked")
@Override
public boolean source(FlowProcess fp, SourceCall sc)
throws IOException {
diff --git a/parquet-cascading/src/test/java/parquet/cascading/TestParquetTBaseScheme.java b/parquet-cascading/src/test/java/parquet/cascading/TestParquetTBaseScheme.java
index 60ad35c7d0..a69911fbf5 100644
--- a/parquet-cascading/src/test/java/parquet/cascading/TestParquetTBaseScheme.java
+++ b/parquet-cascading/src/test/java/parquet/cascading/TestParquetTBaseScheme.java
@@ -46,6 +46,7 @@
import static org.junit.Assert.*;
import parquet.hadoop.thrift.ThriftToParquetFileWriter;
+import parquet.hadoop.util.ContextUtil;
import parquet.thrift.test.Name;
import java.io.File;
@@ -117,7 +118,7 @@ private void createFileForRead() throws Exception {
TProtocolFactory protocolFactory = new TCompactProtocol.Factory();
TaskAttemptID taskId = new TaskAttemptID("local", 0, true, 0, 0);
- ThriftToParquetFileWriter w = new ThriftToParquetFileWriter(fileToCreate, new TaskAttemptContext(conf, taskId), protocolFactory, Name.class);
+ ThriftToParquetFileWriter w = new ThriftToParquetFileWriter(fileToCreate, ContextUtil.newTaskAttemptContext(conf, taskId), protocolFactory, Name.class);
final ByteArrayOutputStream baos = new ByteArrayOutputStream();
final TProtocol protocol = protocolFactory.getProtocol(new TIOStreamTransport(baos));
diff --git a/parquet-cascading/src/test/java/parquet/cascading/TestParquetTupleScheme.java b/parquet-cascading/src/test/java/parquet/cascading/TestParquetTupleScheme.java
index b53f24b7c2..067c227463 100644
--- a/parquet-cascading/src/test/java/parquet/cascading/TestParquetTupleScheme.java
+++ b/parquet-cascading/src/test/java/parquet/cascading/TestParquetTupleScheme.java
@@ -15,80 +15,165 @@
*/
package parquet.cascading;
+import cascading.flow.Flow;
import cascading.flow.FlowProcess;
-import cascading.scheme.hadoop.TextDelimited;
+import cascading.flow.hadoop.HadoopFlowConnector;
+import cascading.operation.BaseOperation;
+import cascading.operation.Function;
+import cascading.operation.FunctionCall;
+import cascading.pipe.Each;
+import cascading.pipe.Pipe;
+import cascading.scheme.Scheme;
+import cascading.scheme.hadoop.TextLine;
import cascading.tap.Tap;
import cascading.tap.hadoop.Hfs;
import cascading.tuple.Fields;
+import cascading.tuple.Tuple;
+import cascading.tuple.TupleEntry;
+import org.apache.commons.io.FileUtils;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.mapred.JobConf;
-import org.apache.hadoop.mapred.OutputCollector;
-import org.apache.hadoop.mapred.RecordReader;
-import org.junit.Before;
+import org.apache.hadoop.io.BytesWritable;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.hadoop.mapreduce.TaskAttemptID;
+import org.apache.thrift.protocol.TCompactProtocol;
+import org.apache.thrift.protocol.TProtocol;
+import org.apache.thrift.protocol.TProtocolFactory;
+import org.apache.thrift.transport.TIOStreamTransport;
import org.junit.Test;
-import static org.junit.Assert.*;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
-import parquet.hadoop.ParquetInputFormat;
-import java.net.URL;
+import parquet.hadoop.thrift.ThriftToParquetFileWriter;
+import parquet.hadoop.util.ContextUtil;
+import parquet.thrift.test.Name;
+
+import java.io.ByteArrayOutputStream;
+import java.io.File;
+
+import static org.junit.Assert.assertEquals;
public class TestParquetTupleScheme {
- Tap tap;
- FlowProcess fp;
+ final String parquetInputPath = "target/test/ParquetTupleIn/names-parquet-in";
+ final String txtOutputPath = "target/test/ParquetTupleOut/names-txt-out";
- @Before
- public void setUp() throws Exception {
- final Configuration conf = new Configuration();
- final Path parquetPath = new Path("target/test/thrift/TestInputOutputFormat/parquet");
- final FileSystem fileSystem = parquetPath.getFileSystem(conf);
- System.out.println(fileSystem);
- URL url = Thread.currentThread().getContextClassLoader().getResource("part-m-00000.gz.parquet");
- tap = new Hfs(new TextDelimited(true, "\t"), url.getPath());
- fp = mock(FlowProcess.class);
- when(fp.getConfigCopy()).thenReturn(new JobConf());
+ @Test
+ public void testReadPattern() throws Exception {
+ String sourceFolder = parquetInputPath;
+ testReadWrite(sourceFolder);
+
+ String sourceGlobPattern = parquetInputPath + "/*";
+ testReadWrite(sourceGlobPattern);
+
+ String multiLevelGlobPattern = "target/test/ParquetTupleIn/**/*";
+ testReadWrite(multiLevelGlobPattern);
}
@Test
- public void testReadSupportSetting() {
- ParquetTupleScheme scheme = new ParquetTupleScheme(Fields.ALL);
- FlowProcess fp = mock(FlowProcess.class);
- Tap tap = mock(Tap.class);
- JobConf jobConf = new JobConf();
-
- scheme.sourceConfInit(fp, tap, jobConf);
- System.out.println(scheme.getSourceFields());
- assertEquals(ParquetInputFormat.getReadSupportClass(jobConf), TupleReadSupport.class);
+ public void testFieldProjection() throws Exception {
+ createFileForRead();
+
+ Path path = new Path(txtOutputPath);
+ final FileSystem fs = path.getFileSystem(new Configuration());
+ if (fs.exists(path)) fs.delete(path, true);
+
+ Scheme sourceScheme = new ParquetTupleScheme(new Fields("last_name"));
+ Tap source = new Hfs(sourceScheme, parquetInputPath);
+
+ Scheme sinkScheme = new TextLine(new Fields("last_name"));
+ Tap sink = new Hfs(sinkScheme, txtOutputPath);
+
+ Pipe assembly = new Pipe("namecp");
+ assembly = new Each(assembly, new ProjectedTupleFunction());
+ Flow flow = new HadoopFlowConnector().connect("namecp", source, sink, assembly);
+
+ flow.complete();
+ String result = FileUtils.readFileToString(new File(txtOutputPath + "/part-00000"));
+ assertEquals("Practice\nHope\nHorse\n", result);
}
- @Test
- public void testRetrieveAllFields() throws Exception {
- ParquetTupleScheme scheme = new ParquetTupleScheme(Fields.ALL);
- scheme.sourceConfInit(fp,
- tap,
- new JobConf());
- Fields fs = scheme.retrieveSourceFields(fp, tap);
- assertEquals("persons", fs.get(0).toString());
+ public void testReadWrite(String inputPath) throws Exception {
+ createFileForRead();
+
+ Path path = new Path(txtOutputPath);
+ final FileSystem fs = path.getFileSystem(new Configuration());
+ if (fs.exists(path)) fs.delete(path, true);
+
+ Scheme sourceScheme = new ParquetTupleScheme(new Fields("first_name", "last_name"));
+ Tap source = new Hfs(sourceScheme, inputPath);
+
+ Scheme sinkScheme = new TextLine(new Fields("first", "last"));
+ Tap sink = new Hfs(sinkScheme, txtOutputPath);
+
+ Pipe assembly = new Pipe("namecp");
+ assembly = new Each(assembly, new UnpackTupleFunction());
+ Flow flow = new HadoopFlowConnector().connect("namecp", source, sink, assembly);
+
+ flow.complete();
+ String result = FileUtils.readFileToString(new File(txtOutputPath + "/part-00000"));
+ assertEquals("Alice\tPractice\nBob\tHope\nCharlie\tHorse\n", result);
}
- @Test
- public void testRetrieveDefaultAllFields() throws Exception {
- ParquetTupleScheme scheme = new ParquetTupleScheme();
- scheme.sourceConfInit(fp,
- tap,
- new JobConf());
- Fields fs = scheme.retrieveSourceFields(fp, tap);
- assertEquals("persons", fs.get(0).toString());
+ private void createFileForRead() throws Exception {
+ final Path fileToCreate = new Path(parquetInputPath + "/names.parquet");
+
+ final Configuration conf = new Configuration();
+ final FileSystem fs = fileToCreate.getFileSystem(conf);
+ if (fs.exists(fileToCreate)) fs.delete(fileToCreate, true);
+
+ TProtocolFactory protocolFactory = new TCompactProtocol.Factory();
+ TaskAttemptID taskId = new TaskAttemptID("local", 0, true, 0, 0);
+ ThriftToParquetFileWriter w = new ThriftToParquetFileWriter(fileToCreate, ContextUtil.newTaskAttemptContext(conf, taskId), protocolFactory, Name.class);
+
+ final ByteArrayOutputStream baos = new ByteArrayOutputStream();
+ final TProtocol protocol = protocolFactory.getProtocol(new TIOStreamTransport(baos));
+
+ Name n1 = new Name();
+ n1.setFirst_name("Alice");
+ n1.setLast_name("Practice");
+ Name n2 = new Name();
+ n2.setFirst_name("Bob");
+ n2.setLast_name("Hope");
+ Name n3 = new Name();
+ n3.setFirst_name("Charlie");
+ n3.setLast_name("Horse");
+
+ n1.write(protocol);
+ w.write(new BytesWritable(baos.toByteArray()));
+ baos.reset();
+ n2.write(protocol);
+ w.write(new BytesWritable(baos.toByteArray()));
+ baos.reset();
+ n3.write(protocol);
+ w.write(new BytesWritable(baos.toByteArray()));
+ w.close();
}
- @Test
- public void testRetrieveNoneFields() throws Exception {
- ParquetTupleScheme scheme = new ParquetTupleScheme(Fields.NONE);
- scheme.sourceConfInit(fp,
- tap,
- new JobConf());
- Fields fs = scheme.retrieveSourceFields(fp, tap);
- assertEquals(0, fs.size());
+ private static class UnpackTupleFunction extends BaseOperation implements Function {
+ @Override
+ public void operate(FlowProcess flowProcess, FunctionCall functionCall) {
+ TupleEntry arguments = functionCall.getArguments();
+ Tuple result = new Tuple();
+
+ Tuple name = new Tuple();
+ name.addString(arguments.getString(0));
+ name.addString(arguments.getString(1));
+
+ result.add(name);
+ functionCall.getOutputCollector().add(result);
+ }
+ }
+
+ private static class ProjectedTupleFunction extends BaseOperation implements Function {
+ @Override
+ public void operate(FlowProcess flowProcess, FunctionCall functionCall) {
+ TupleEntry arguments = functionCall.getArguments();
+ Tuple result = new Tuple();
+
+ Tuple name = new Tuple();
+ name.addString(arguments.getString(0));
+// name.addString(arguments.getString(1));
+
+ result.add(name);
+ functionCall.getOutputCollector().add(result);
+ }
}
}
diff --git a/parquet-cascading/src/test/resources/part-m-00000.gz.parquet b/parquet-cascading/src/test/resources/part-m-00000.gz.parquet
deleted file mode 100755
index 4e6542e376..0000000000
Binary files a/parquet-cascading/src/test/resources/part-m-00000.gz.parquet and /dev/null differ
diff --git a/parquet-column/pom.xml b/parquet-column/pom.xml
index 2d8d930620..bc5220edd8 100644
--- a/parquet-column/pom.xml
+++ b/parquet-column/pom.xml
@@ -3,7 +3,7 @@
com.twitter
parquet
../pom.xml
- 1.2.4-SNAPSHOT
+ 1.2.11-SNAPSHOT
4.0.0
@@ -46,6 +46,19 @@
fastutil
6.5.7
+
+
+ com.carrotsearch
+ junit-benchmarks
+ 0.7.0
+ test
+
+
+ com.h2database
+ h2
+ 1.3.149
+ test
+
@@ -58,6 +71,32 @@
org.apache.maven.plugins
maven-shade-plugin
+
+ org.apache.maven.plugins
+ maven-shade-plugin
+
+
+ package
+
+ shade
+
+
+ true
+
+
+ it.unimi.dsi:fastutil
+
+
+
+
+ it.unimi.dsi
+ parquet.it.unimi.dsi
+
+
+
+
+
+
diff --git a/parquet-column/src/main/java/parquet/column/Encoding.java b/parquet-column/src/main/java/parquet/column/Encoding.java
index a543b4f58e..7e0305ec06 100644
--- a/parquet-column/src/main/java/parquet/column/Encoding.java
+++ b/parquet-column/src/main/java/parquet/column/Encoding.java
@@ -16,6 +16,7 @@
package parquet.column;
import static parquet.column.values.bitpacking.Packer.BIG_ENDIAN;
+import static parquet.schema.PrimitiveType.PrimitiveTypeName.INT32;
import java.io.IOException;
@@ -24,6 +25,7 @@
import parquet.column.values.ValuesReader;
import parquet.column.values.bitpacking.ByteBitPackingValuesReader;
import parquet.column.values.boundedint.ZeroIntegerValuesReader;
+import parquet.column.values.delta.DeltaBinaryPackingValuesReader;
import parquet.column.values.dictionary.DictionaryValuesReader;
import parquet.column.values.dictionary.PlainValuesDictionary.PlainBinaryDictionary;
import parquet.column.values.dictionary.PlainValuesDictionary.PlainDoubleDictionary;
@@ -99,13 +101,6 @@ public ValuesReader getValuesReader(ColumnDescriptor descriptor, ValuesType valu
}
},
- GROUP_VAR_INT {
- @Override // TODO: GROUP VAR INT encoding
- public ValuesReader getValuesReader(ColumnDescriptor descriptor, ValuesType valuesType) {
- throw new UnsupportedOperationException("NYI");
- }
- },
-
PLAIN_DICTIONARY {
@Override
public ValuesReader getDictionaryBasedValuesReader(ColumnDescriptor descriptor, ValuesType valuesType, Dictionary dictionary) {
@@ -137,7 +132,7 @@ public Dictionary initDictionary(ColumnDescriptor descriptor, DictionaryPage dic
default:
throw new ParquetDecodingException("Dictionary encoding not supported for type: " + descriptor.getType());
}
-
+
}
@Override
@@ -145,7 +140,38 @@ public boolean usesDictionary() {
return true;
}
- };
+ },
+
+ /**
+ * Delta encoding for integers. This can be used for int columns and works best
+ * on sorted data
+ */
+ DELTA_BINARY_PACKED {
+ @Override
+ public ValuesReader getValuesReader(ColumnDescriptor descriptor, ValuesType valuesType) {
+ if(descriptor.getType() != INT32) {
+ throw new ParquetDecodingException("Encoding DELTA_BINARY_PACKED is only supported for type INT32");
+ }
+ return new DeltaBinaryPackingValuesReader();
+ }
+ },
+
+ /**
+ * Encoding for byte arrays to separate the length values and the data. The lengths
+ * are encoded using DELTA_BINARY_PACKED
+ */
+ DELTA_LENGTH_BYTE_ARRAY,
+
+ /**
+ * Incremental-encoded byte array. Prefix lengths are encoded using DELTA_BINARY_PACKED.
+ * Suffixes are stored as delta length byte arrays.
+ */
+ DELTA_BYTE_ARRAY,
+
+ /**
+ * Dictionary encoding: the ids are encoded using the RLE encoding
+ */
+ RLE_DICTIONARY;
int getMaxLevel(ColumnDescriptor descriptor, ValuesType valuesType) {
int maxLevel;
diff --git a/parquet-column/src/main/java/parquet/column/ParquetProperties.java b/parquet-column/src/main/java/parquet/column/ParquetProperties.java
new file mode 100644
index 0000000000..0e890409e6
--- /dev/null
+++ b/parquet-column/src/main/java/parquet/column/ParquetProperties.java
@@ -0,0 +1,135 @@
+package parquet.column;
+
+import parquet.bytes.BytesUtils;
+import parquet.column.values.ValuesWriter;
+import parquet.column.values.boundedint.DevNullValuesWriter;
+import parquet.column.values.dictionary.DictionaryValuesWriter.PlainBinaryDictionaryValuesWriter;
+import parquet.column.values.dictionary.DictionaryValuesWriter.PlainDoubleDictionaryValuesWriter;
+import parquet.column.values.dictionary.DictionaryValuesWriter.PlainFloatDictionaryValuesWriter;
+import parquet.column.values.dictionary.DictionaryValuesWriter.PlainIntegerDictionaryValuesWriter;
+import parquet.column.values.dictionary.DictionaryValuesWriter.PlainLongDictionaryValuesWriter;
+import parquet.column.values.plain.BooleanPlainValuesWriter;
+import parquet.column.values.plain.FixedLenByteArrayPlainValuesWriter;
+import parquet.column.values.plain.PlainValuesWriter;
+import parquet.column.values.rle.RunLengthBitPackingHybridValuesWriter;
+
+/**
+ * This class represents all the configurable Parquet properties.
+ *
+ * @author amokashi
+ *
+ */
+public class ParquetProperties {
+
+ public enum WriterVersion {
+ PARQUET_1_0 ("v1"),
+ PARQUET_2_0 ("v2");
+
+ private final String shortName;
+
+ WriterVersion(String shortname) {
+ this.shortName = shortname;
+ }
+
+ public static WriterVersion fromString(String name) {
+ for(WriterVersion v : WriterVersion.values()) {
+ if (v.shortName.equals(name)) {
+ return v;
+ }
+ }
+ // Throws IllegalArgumentException if name does not exact match with enum name
+ return WriterVersion.valueOf(name);
+ }
+ }
+ private final int dictionaryPageSizeThreshold;
+ private final WriterVersion writerVersion;
+ private final boolean enableDictionary;
+
+ public ParquetProperties(int dictPageSize, WriterVersion writerVersion, boolean enableDict) {
+ this.dictionaryPageSizeThreshold = dictPageSize;
+ this.writerVersion = writerVersion;
+ this.enableDictionary = enableDict;
+ }
+
+ public static ValuesWriter getColumnDescriptorValuesWriter(int maxLevel, int initialSizePerCol) {
+ if (maxLevel == 0) {
+ return new DevNullValuesWriter();
+ } else {
+ return new RunLengthBitPackingHybridValuesWriter(
+ BytesUtils.getWidthFromMaxInt(maxLevel), initialSizePerCol);
+ }
+ }
+
+ public ValuesWriter getValuesWriter(ColumnDescriptor path, int initialSizePerCol) {
+ switch (path.getType()) {
+ case BOOLEAN:
+ if(writerVersion == WriterVersion.PARQUET_1_0) {
+ return new BooleanPlainValuesWriter();
+ } else if (writerVersion == WriterVersion.PARQUET_2_0) {
+ return new RunLengthBitPackingHybridValuesWriter(1, initialSizePerCol);
+ }
+ break;
+ case BINARY:
+ if(enableDictionary) {
+ return new PlainBinaryDictionaryValuesWriter(dictionaryPageSizeThreshold, initialSizePerCol);
+ } else {
+ if (writerVersion == WriterVersion.PARQUET_1_0) {
+ return new PlainValuesWriter(initialSizePerCol);
+ } else if (writerVersion == WriterVersion.PARQUET_2_0) {
+ // TODO enable 2.0 encodings in another commit
+ // return new DeltaByteArrayWriter(initialSizePerCol);
+ return new PlainValuesWriter(initialSizePerCol);
+ }
+ }
+ break;
+ case INT32:
+ if(enableDictionary) {
+ return new PlainIntegerDictionaryValuesWriter(dictionaryPageSizeThreshold, initialSizePerCol);
+ } else {
+ if(writerVersion == WriterVersion.PARQUET_1_0) {
+ return new PlainValuesWriter(initialSizePerCol);
+ } else if(writerVersion == WriterVersion.PARQUET_2_0) {
+ // TODO enable 2.0 encodings in another commit
+ // return new DeltaBinaryPackingValuesWriter(initialSizePerCol);
+ return new PlainValuesWriter(initialSizePerCol);
+ }
+ }
+ break;
+ case INT64:
+ if(enableDictionary) {
+ return new PlainLongDictionaryValuesWriter(dictionaryPageSizeThreshold, initialSizePerCol);
+ } else {
+ return new PlainValuesWriter(initialSizePerCol);
+ }
+ case DOUBLE:
+ if(enableDictionary) {
+ return new PlainDoubleDictionaryValuesWriter(dictionaryPageSizeThreshold, initialSizePerCol);
+ } else {
+ return new PlainValuesWriter(initialSizePerCol);
+ }
+ case FLOAT:
+ if(enableDictionary) {
+ return new PlainFloatDictionaryValuesWriter(dictionaryPageSizeThreshold, initialSizePerCol);
+ } else {
+ return new PlainValuesWriter(initialSizePerCol);
+ }
+ case FIXED_LEN_BYTE_ARRAY:
+ return new FixedLenByteArrayPlainValuesWriter(path.getTypeLength(), initialSizePerCol);
+ default:
+ return new PlainValuesWriter(initialSizePerCol);
+ }
+ return null;
+ }
+
+ public int getDictionaryPageSizeThreshold() {
+ return dictionaryPageSizeThreshold;
+ }
+
+ public WriterVersion getWriterVersion() {
+ return writerVersion;
+ }
+
+ public boolean isEnableDictionary() {
+ return enableDictionary;
+ }
+}
diff --git a/parquet-column/src/main/java/parquet/column/impl/ColumnWriteStoreImpl.java b/parquet-column/src/main/java/parquet/column/impl/ColumnWriteStoreImpl.java
index abd93732ea..43a3bc58e3 100644
--- a/parquet-column/src/main/java/parquet/column/impl/ColumnWriteStoreImpl.java
+++ b/parquet-column/src/main/java/parquet/column/impl/ColumnWriteStoreImpl.java
@@ -24,6 +24,7 @@
import parquet.column.ColumnDescriptor;
import parquet.column.ColumnWriteStore;
import parquet.column.ColumnWriter;
+import parquet.column.ParquetProperties.WriterVersion;
import parquet.column.page.PageWriteStore;
import parquet.column.page.PageWriter;
@@ -33,15 +34,19 @@ public class ColumnWriteStoreImpl implements ColumnWriteStore {
private final Map columns = new TreeMap();
private final PageWriteStore pageWriteStore;
private final int pageSizeThreshold;
+ private final int dictionaryPageSizeThreshold;
private final boolean enableDictionary;
private final int initialSizePerCol;
+ private final WriterVersion writerVersion;
- public ColumnWriteStoreImpl(PageWriteStore pageWriteStore, int pageSizeThreshold, int initialSizePerCol, boolean enableDictionary) {
+ public ColumnWriteStoreImpl(PageWriteStore pageWriteStore, int pageSizeThreshold, int initialSizePerCol, int dictionaryPageSizeThreshold, boolean enableDictionary, WriterVersion writerVersion) {
super();
this.pageWriteStore = pageWriteStore;
this.pageSizeThreshold = pageSizeThreshold;
this.initialSizePerCol = initialSizePerCol;
+ this.dictionaryPageSizeThreshold = dictionaryPageSizeThreshold;
this.enableDictionary = enableDictionary;
+ this.writerVersion = writerVersion;
}
public ColumnWriter getColumnWriter(ColumnDescriptor path) {
@@ -55,7 +60,7 @@ public ColumnWriter getColumnWriter(ColumnDescriptor path) {
private ColumnWriterImpl newMemColumn(ColumnDescriptor path) {
PageWriter pageWriter = pageWriteStore.getPageWriter(path);
- return new ColumnWriterImpl(path, pageWriter, pageSizeThreshold, initialSizePerCol, enableDictionary);
+ return new ColumnWriterImpl(path, pageWriter, pageSizeThreshold, initialSizePerCol, dictionaryPageSizeThreshold, enableDictionary, writerVersion);
}
@Override
diff --git a/parquet-column/src/main/java/parquet/column/impl/ColumnWriterImpl.java b/parquet-column/src/main/java/parquet/column/impl/ColumnWriterImpl.java
index d49231880f..abf750a84a 100644
--- a/parquet-column/src/main/java/parquet/column/impl/ColumnWriterImpl.java
+++ b/parquet-column/src/main/java/parquet/column/impl/ColumnWriterImpl.java
@@ -23,6 +23,8 @@
import parquet.bytes.BytesUtils;
import parquet.column.ColumnDescriptor;
import parquet.column.ColumnWriter;
+import parquet.column.ParquetProperties;
+import parquet.column.ParquetProperties.WriterVersion;
import parquet.column.page.DictionaryPage;
import parquet.column.page.PageWriter;
import parquet.column.values.ValuesWriter;
@@ -47,9 +49,8 @@
*/
final class ColumnWriterImpl implements ColumnWriter {
private static final Log LOG = Log.getLog(ColumnWriterImpl.class);
- private static final boolean DEBUG = false; //Log.DEBUG;
+ private static final boolean DEBUG = Log.DEBUG;
private static final int INITIAL_COUNT_FOR_SIZE_CHECK = 100;
- private static final int DICTIONARY_PAGE_MAX_SIZE_PERCENT = 20;
private final ColumnDescriptor path;
private final PageWriter pageWriter;
@@ -60,72 +61,25 @@ final class ColumnWriterImpl implements ColumnWriter {
private int valueCount;
private int valueCountForNextSizeCheck;
- public ColumnWriterImpl(ColumnDescriptor path, PageWriter pageWriter, int pageSizeThreshold, int initialSizePerCol, boolean enableDictionary) {
+ public ColumnWriterImpl(
+ ColumnDescriptor path,
+ PageWriter pageWriter,
+ int pageSizeThreshold,
+ int initialSizePerCol,
+ int dictionaryPageSizeThreshold,
+ boolean enableDictionary,
+ WriterVersion writerVersion) {
this.path = path;
this.pageWriter = pageWriter;
this.pageSizeThreshold = pageSizeThreshold;
// initial check of memory usage. So that we have enough data to make an initial prediction
this.valueCountForNextSizeCheck = INITIAL_COUNT_FOR_SIZE_CHECK;
- repetitionLevelColumn = getColumnDescriptorValuesWriter(path.getMaxRepetitionLevel());
- definitionLevelColumn = getColumnDescriptorValuesWriter(path.getMaxDefinitionLevel());
- if (enableDictionary) {
- int maxDictByteSize = applyRatioInPercent(pageSizeThreshold, DICTIONARY_PAGE_MAX_SIZE_PERCENT);
- switch (path.getType()) {
- case BOOLEAN:
- this.dataColumn = new BooleanPlainValuesWriter();
- break;
- case BINARY:
- this.dataColumn = new PlainBinaryDictionaryValuesWriter(maxDictByteSize, initialSizePerCol);
- break;
- case INT64:
- this.dataColumn = new PlainLongDictionaryValuesWriter(maxDictByteSize, initialSizePerCol);
- break;
- case DOUBLE:
- this.dataColumn = new PlainDoubleDictionaryValuesWriter(maxDictByteSize, initialSizePerCol);
- break;
- case INT32:
- this.dataColumn = new PlainIntegerDictionaryValuesWriter(maxDictByteSize, initialSizePerCol);
- break;
- case FLOAT:
- this.dataColumn = new PlainFloatDictionaryValuesWriter(maxDictByteSize, initialSizePerCol);
- break;
- case FIXED_LEN_BYTE_ARRAY:
- this.dataColumn = new FixedLenByteArrayPlainValuesWriter(path.getTypeLength(), initialSizePerCol);
- break;
- default:
- this.dataColumn = new PlainValuesWriter(initialSizePerCol);
- }
- } else {
- switch (path.getType()) {
- case BOOLEAN:
- this.dataColumn = new BooleanPlainValuesWriter();
- break;
- case FIXED_LEN_BYTE_ARRAY:
- this.dataColumn = new FixedLenByteArrayPlainValuesWriter(path.getTypeLength(), initialSizePerCol);
- break;
- default:
- this.dataColumn = new PlainValuesWriter(initialSizePerCol);
- }
- }
- }
-
- private ValuesWriter getColumnDescriptorValuesWriter(int maxLevel) {
- if (maxLevel == 0) {
- return new DevNullValuesWriter();
- } else {
- // TODO: what is a good initialCapacity?
- return new RunLengthBitPackingHybridValuesWriter(
- BytesUtils.getWidthFromMaxInt(maxLevel),
- 64 * 1024);
- }
- }
-
- private int applyRatioInPercent(int value, int ratio) {
- if (100 % ratio != 0) {
- throw new IllegalArgumentException("ratio should be a diviser of 100: not " + ratio);
- }
- return value / (100 / ratio);
+ ParquetProperties parquetProps = new ParquetProperties(dictionaryPageSizeThreshold, writerVersion, enableDictionary);
+
+ this.repetitionLevelColumn = ParquetProperties.getColumnDescriptorValuesWriter(path.getMaxRepetitionLevel(), initialSizePerCol);
+ this.definitionLevelColumn = ParquetProperties.getColumnDescriptorValuesWriter(path.getMaxDefinitionLevel(), initialSizePerCol);
+ this.dataColumn = parquetProps.getValuesWriter(path, initialSizePerCol);
}
private void log(Object value, int r, int d) {
diff --git a/parquet-column/src/main/java/parquet/column/values/delta/DeltaBinaryPackingConfig.java b/parquet-column/src/main/java/parquet/column/values/delta/DeltaBinaryPackingConfig.java
new file mode 100644
index 0000000000..be054a0c58
--- /dev/null
+++ b/parquet-column/src/main/java/parquet/column/values/delta/DeltaBinaryPackingConfig.java
@@ -0,0 +1,54 @@
+/**
+ * Copyright 2012 Twitter, Inc.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package parquet.column.values.delta;
+
+
+import parquet.Preconditions;
+import parquet.bytes.BytesInput;
+import parquet.bytes.BytesUtils;
+
+import java.io.IOException;
+import java.io.InputStream;
+
+/**
+ * Config for delta binary packing
+ *
+ * @author Tianshuo Deng
+ */
+class DeltaBinaryPackingConfig {
+ final int blockSizeInValues;
+ final int miniBlockNumInABlock;
+ final int miniBlockSizeInValues;
+
+ public DeltaBinaryPackingConfig(int blockSizeInValues, int miniBlockNumInABlock) {
+ this.blockSizeInValues = blockSizeInValues;
+ this.miniBlockNumInABlock = miniBlockNumInABlock;
+ double miniSize = (double) blockSizeInValues / miniBlockNumInABlock;
+ Preconditions.checkArgument(miniSize % 8 == 0, "miniBlockSize must be multiple of 8, but it's " + miniSize);
+ this.miniBlockSizeInValues = (int) miniSize;
+ }
+
+ public static DeltaBinaryPackingConfig readConfig(InputStream in) throws IOException {
+ return new DeltaBinaryPackingConfig(BytesUtils.readUnsignedVarInt(in),
+ BytesUtils.readUnsignedVarInt(in));
+ }
+
+ public BytesInput toBytesInput() {
+ return BytesInput.concat(
+ BytesInput.fromUnsignedVarInt(blockSizeInValues),
+ BytesInput.fromUnsignedVarInt(miniBlockNumInABlock));
+ }
+}
diff --git a/parquet-column/src/main/java/parquet/column/values/delta/DeltaBinaryPackingValuesReader.java b/parquet-column/src/main/java/parquet/column/values/delta/DeltaBinaryPackingValuesReader.java
new file mode 100644
index 0000000000..9af01ec556
--- /dev/null
+++ b/parquet-column/src/main/java/parquet/column/values/delta/DeltaBinaryPackingValuesReader.java
@@ -0,0 +1,164 @@
+/**
+ * Copyright 2012 Twitter, Inc.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package parquet.column.values.delta;
+
+
+import parquet.bytes.BytesUtils;
+import parquet.column.values.ValuesReader;
+import parquet.column.values.bitpacking.BytePacker;
+import parquet.column.values.bitpacking.Packer;
+import parquet.io.ParquetDecodingException;
+
+import java.io.ByteArrayInputStream;
+import java.io.IOException;
+
+/**
+ * Read values written by {@link DeltaBinaryPackingValuesWriter}
+ *
+ * @author Tianshuo Deng
+ */
+public class DeltaBinaryPackingValuesReader extends ValuesReader {
+ private int totalValueCount;
+ /**
+ * values read by the caller
+ */
+ private int valuesRead;
+ private int minDeltaInCurrentBlock;
+ private byte[] page;
+ /**
+ * stores the decoded values including the first value which is written to the header
+ */
+ private int[] valuesBuffer;
+ /**
+ * values loaded to the buffer, it could be bigger than the totalValueCount
+ * when data is not aligned to mini block, which means padding 0s are in the buffer
+ */
+ private int valuesBuffered;
+ private ByteArrayInputStream in;
+ private int nextOffset;
+ private DeltaBinaryPackingConfig config;
+ private int[] bitWidths;
+
+ /**
+ * eagerly load all the data into memory
+ *
+ * @param valueCount count of values in this page
+ * @param page the array to read from containing the page data (repetition levels, definition levels, data)
+ * @param offset where to start reading from in the page
+ * @return the length read
+ * @throws IOException
+ */
+ @Override
+ public int initFromPage(long valueCount, byte[] page, int offset) throws IOException {
+ in = new ByteArrayInputStream(page, offset, page.length - offset);
+ this.config = DeltaBinaryPackingConfig.readConfig(in);
+ this.page = page;
+ this.totalValueCount = BytesUtils.readUnsignedVarInt(in);
+ allocateValuesBuffer();
+ bitWidths = new int[config.miniBlockNumInABlock];
+
+ //read first value from header
+ valuesBuffer[valuesBuffered++] = BytesUtils.readZigZagVarInt(in);
+
+ while (valuesBuffered < totalValueCount) { //values Buffered could be more than totalValueCount, since we flush on a mini block basis
+ loadNewBlockToBuffer();
+ }
+ this.nextOffset = page.length - in.available();
+ return nextOffset;
+ }
+
+ /**
+ * the value buffer is allocated so that the size of it is multiple of mini block
+ * because when writing, data is flushed on a mini block basis
+ */
+ private void allocateValuesBuffer() {
+ int totalMiniBlockCount = (int) Math.ceil((double) totalValueCount / config.miniBlockSizeInValues);
+ //+ 1 because first value written to header is also stored in values buffer
+ valuesBuffer = new int[totalMiniBlockCount * config.miniBlockSizeInValues + 1];
+ }
+
+ @Override
+ public void skip() {
+ checkRead();
+ valuesRead++;
+ }
+
+ @Override
+ public int readInteger() {
+ checkRead();
+ return valuesBuffer[valuesRead++];
+ }
+
+ private void checkRead() {
+ if (valuesRead >= totalValueCount) {
+ throw new ParquetDecodingException("no more value to read, total value count is " + totalValueCount);
+ }
+ }
+
+ private void loadNewBlockToBuffer() {
+ try {
+ minDeltaInCurrentBlock = BytesUtils.readZigZagVarInt(in);
+ } catch (IOException e) {
+ throw new ParquetDecodingException("can not read min delta in current block", e);
+ }
+
+ readBitWidthsForMiniBlocks();
+
+ // mini block is atomic for reading, we read a mini block when there are more values left
+ int i;
+ for (i = 0; i < config.miniBlockNumInABlock && valuesBuffered < totalValueCount; i++) {
+ BytePacker packer = Packer.LITTLE_ENDIAN.newBytePacker(bitWidths[i]);
+ unpackMiniBlock(packer);
+ }
+
+ //calculate values from deltas unpacked for current block
+ int valueUnpacked=i*config.miniBlockSizeInValues;
+ for (int j = valuesBuffered-valueUnpacked; j < valuesBuffered; j++) {
+ int index = j;
+ valuesBuffer[index] += minDeltaInCurrentBlock + valuesBuffer[index - 1];
+ }
+ }
+
+ /**
+ * mini block has a size of 8*n, unpack 8 value each time
+ *
+ * @param packer the packer created from bitwidth of current mini block
+ */
+ private void unpackMiniBlock(BytePacker packer) {
+ for (int j = 0; j < config.miniBlockSizeInValues; j += 8) {
+ unpack8Values(packer);
+ }
+ }
+
+ private void unpack8Values(BytePacker packer) {
+ //calculate the pos because the packer api uses array not stream
+ int pos = page.length - in.available();
+ packer.unpack8Values(page, pos, valuesBuffer, valuesBuffered);
+ this.valuesBuffered += 8;
+ //sync the pos in stream
+ in.skip(packer.getBitWidth());
+ }
+
+ private void readBitWidthsForMiniBlocks() {
+ for (int i = 0; i < config.miniBlockNumInABlock; i++) {
+ try {
+ bitWidths[i] = BytesUtils.readIntLittleEndianOnOneByte(in);
+ } catch (IOException e) {
+ throw new ParquetDecodingException("Can not decode bitwidth in block header", e);
+ }
+ }
+ }
+}
diff --git a/parquet-column/src/main/java/parquet/column/values/delta/DeltaBinaryPackingValuesWriter.java b/parquet-column/src/main/java/parquet/column/values/delta/DeltaBinaryPackingValuesWriter.java
new file mode 100644
index 0000000000..e98cf6749b
--- /dev/null
+++ b/parquet-column/src/main/java/parquet/column/values/delta/DeltaBinaryPackingValuesWriter.java
@@ -0,0 +1,266 @@
+/**
+ * Copyright 2012 Twitter, Inc.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package parquet.column.values.delta;
+
+import parquet.bytes.BytesInput;
+import parquet.bytes.BytesUtils;
+import parquet.bytes.CapacityByteArrayOutputStream;
+import parquet.column.Encoding;
+import parquet.column.values.ValuesWriter;
+import parquet.column.values.bitpacking.BytePacker;
+import parquet.column.values.bitpacking.Packer;
+import parquet.io.ParquetEncodingException;
+
+import java.io.IOException;
+
+/**
+ * Write integers with delta encoding and binary packing
+ * The format is as follows:
+ *
+ *
+ * {@code
+ * delta-binary-packing: *
+ * page-header :=
+ * block :=
+ *
+ * min delta : zig-zag var int encoded
+ * bitWidthsOfMiniBlock : 1 byte little endian
+ * blockSizeInValues,blockSizeInValues,totalValueCount,firstValue : unsigned varint
+ * }
+ *
+ *
+ * The algorithm and format is inspired by D. Lemire's paper: http://lemire.me/blog/archives/2012/09/12/fast-integer-compression-decoding-billions-of-integers-per-second/
+ *
+ * @author Tianshuo Deng
+ */
+public class DeltaBinaryPackingValuesWriter extends ValuesWriter {
+ /**
+ * max bitwidth for a mini block, it is used to allocate miniBlockByteBuffer which is
+ * reused between flushes.
+ */
+ public static final int MAX_BITWIDTH = 32;
+
+ public static final int DEFAULT_NUM_BLOCK_VALUES = 128;
+
+ public static final int DEFAULT_NUM_MINIBLOCKS = 4;
+
+ private final CapacityByteArrayOutputStream baos;
+
+ /**
+ * stores blockSizeInValues, miniBlockNumInABlock and miniBlockSizeInValues
+ */
+ private final DeltaBinaryPackingConfig config;
+
+ /**
+ * bit width for each mini block, reused between flushes
+ */
+ private final int[] bitWidths;
+
+ private int totalValueCount = 0;
+
+ /**
+ * a pointer to deltaBlockBuffer indicating the end of deltaBlockBuffer
+ * the number of values in the deltaBlockBuffer that haven't flushed to baos
+ * it will be reset after each flush
+ */
+ private int deltaValuesToFlush = 0;
+
+ /**
+ * stores delta values starting from the 2nd value written(1st value is stored in header).
+ * It's reused between flushes
+ */
+ private int[] deltaBlockBuffer;
+
+ /**
+ * bytes buffer for a mini block, it is reused for each mini block.
+ * Therefore the size of biggest miniblock with bitwith of MAX_BITWITH is allocated
+ */
+ private byte[] miniBlockByteBuffer;
+
+ /**
+ * firstValue is written to the header of the page
+ */
+ private int firstValue = 0;
+
+ /**
+ * cache previous written value for calculating delta
+ */
+ private int previousValue = 0;
+
+ /**
+ * min delta is written to the beginning of each block.
+ * it's zig-zag encoded. The deltas stored in each block is actually the difference to min delta,
+ * therefore are all positive
+ * it will be reset after each flush
+ */
+ private int minDeltaInCurrentBlock = Integer.MAX_VALUE;
+
+ public DeltaBinaryPackingValuesWriter(int slabSize) {
+ this(DEFAULT_NUM_BLOCK_VALUES, DEFAULT_NUM_MINIBLOCKS, slabSize);
+ }
+
+ public DeltaBinaryPackingValuesWriter(int blockSizeInValues, int miniBlockNum, int slabSize) {
+ this.config = new DeltaBinaryPackingConfig(blockSizeInValues, miniBlockNum);
+ bitWidths = new int[config.miniBlockNumInABlock];
+ deltaBlockBuffer = new int[blockSizeInValues];
+ miniBlockByteBuffer = new byte[config.miniBlockSizeInValues * MAX_BITWIDTH];
+ baos = new CapacityByteArrayOutputStream(slabSize);
+ }
+
+ @Override
+ public long getBufferedSize() {
+ return baos.size();
+ }
+
+ @Override
+ public void writeInteger(int v) {
+ totalValueCount++;
+
+ if (totalValueCount == 1) {
+ firstValue = v;
+ previousValue = firstValue;
+ return;
+ }
+
+ int delta = v - previousValue;//calculate delta
+ previousValue = v;
+
+ deltaBlockBuffer[deltaValuesToFlush++] = delta;
+
+ if (delta < minDeltaInCurrentBlock) {
+ minDeltaInCurrentBlock = delta;
+ }
+
+ if (config.blockSizeInValues == deltaValuesToFlush) {
+ flushBlockBuffer();
+ }
+ }
+
+ private void flushBlockBuffer() {
+ //since we store the min delta, the deltas will be converted to be the difference to min delta and all positive
+ for (int i = 0; i < deltaValuesToFlush; i++) {
+ deltaBlockBuffer[i] = deltaBlockBuffer[i] - minDeltaInCurrentBlock;
+ }
+
+ writeMinDelta();
+ int miniBlocksToFlush = getMiniBlockCountToFlush(deltaValuesToFlush);
+
+ calculateBitWidthsForDeltaBlockBuffer(miniBlocksToFlush);
+ for (int i = 0; i < config.miniBlockNumInABlock; i++) {
+ writeBitWidthForMiniBlock(i);
+ }
+
+ for (int i = 0; i < miniBlocksToFlush; i++) {
+ //writing i th miniblock
+ int currentBitWidth = bitWidths[i];
+ BytePacker packer = Packer.LITTLE_ENDIAN.newBytePacker(currentBitWidth);
+ int miniBlockStart = i * config.miniBlockSizeInValues;
+ for (int j = miniBlockStart; j < (i + 1) * config.miniBlockSizeInValues; j += 8) {//8 values per pack
+ // mini block is atomic in terms of flushing
+ // This may write more values when reach to the end of data writing to last mini block,
+ // since it may not be aligend to miniblock,
+ // but doesnt matter. The reader uses total count to see if reached the end.
+ packer.pack8Values(deltaBlockBuffer, j, miniBlockByteBuffer, 0);
+ baos.write(miniBlockByteBuffer, 0, currentBitWidth);
+ }
+ }
+
+ minDeltaInCurrentBlock = Integer.MAX_VALUE;
+ deltaValuesToFlush = 0;
+ }
+
+ private void writeBitWidthForMiniBlock(int i) {
+ try {
+ BytesUtils.writeIntLittleEndianOnOneByte(baos, bitWidths[i]);
+ } catch (IOException e) {
+ throw new ParquetEncodingException("can not write bitwith for miniblock", e);
+ }
+ }
+
+ private void writeMinDelta() {
+ try {
+ BytesUtils.writeZigZagVarInt(minDeltaInCurrentBlock, baos);
+ } catch (IOException e) {
+ throw new ParquetEncodingException("can not write min delta for block", e);
+ }
+ }
+
+ /**
+ * iterate through values in each mini block and calculate the bitWidths of max values.
+ *
+ * @param miniBlocksToFlush
+ */
+ private void calculateBitWidthsForDeltaBlockBuffer(int miniBlocksToFlush) {
+ for (int miniBlockIndex = 0; miniBlockIndex < miniBlocksToFlush; miniBlockIndex++) {
+
+ int mask = 0;
+ int miniStart = miniBlockIndex * config.miniBlockSizeInValues;
+
+ //The end of current mini block could be the end of current block(deltaValuesToFlush) buffer when data is not aligned to mini block
+ int miniEnd = Math.min((miniBlockIndex + 1) * config.miniBlockSizeInValues, deltaValuesToFlush);
+
+ for (int i = miniStart; i < miniEnd; i++) {
+ mask |= deltaBlockBuffer[i];
+ }
+ bitWidths[miniBlockIndex] = 32 - Integer.numberOfLeadingZeros(mask);
+ }
+ }
+
+ private int getMiniBlockCountToFlush(double numberCount) {
+ return (int) Math.ceil(numberCount / config.miniBlockSizeInValues);
+ }
+
+ /**
+ * getBytes will trigger flushing block buffer, DO NOT write after getBytes() is called without calling reset()
+ *
+ * @return
+ */
+ @Override
+ public BytesInput getBytes() {
+ //The Page Header should include: blockSizeInValues, numberOfMiniBlocks, totalValueCount
+ if (deltaValuesToFlush != 0) {
+ flushBlockBuffer();
+ }
+ return BytesInput.concat(
+ config.toBytesInput(),
+ BytesInput.fromUnsignedVarInt(totalValueCount),
+ BytesInput.fromZigZagVarInt(firstValue),
+ BytesInput.from(baos));
+ }
+
+ @Override
+ public Encoding getEncoding() {
+ return Encoding.DELTA_BINARY_PACKED;
+ }
+
+ @Override
+ public void reset() {
+ this.totalValueCount = 0;
+ this.baos.reset();
+ this.deltaValuesToFlush = 0;
+ this.minDeltaInCurrentBlock = Integer.MAX_VALUE;
+ }
+
+ @Override
+ public long getAllocatedSize() {
+ return baos.getCapacity();
+ }
+
+ @Override
+ public String memUsageString(String prefix) {
+ return String.format("%s DeltaBinaryPacking %d bytes", prefix, getAllocatedSize());
+ }
+}
diff --git a/parquet-column/src/main/java/parquet/column/values/dictionary/DictionaryValuesReader.java b/parquet-column/src/main/java/parquet/column/values/dictionary/DictionaryValuesReader.java
index 683bbabf58..5d5a67785a 100644
--- a/parquet-column/src/main/java/parquet/column/values/dictionary/DictionaryValuesReader.java
+++ b/parquet-column/src/main/java/parquet/column/values/dictionary/DictionaryValuesReader.java
@@ -38,7 +38,7 @@
public class DictionaryValuesReader extends ValuesReader {
private static final Log LOG = Log.getLog(DictionaryValuesReader.class);
- private InputStream in;
+ private ByteArrayInputStream in;
private Dictionary dictionary;
@@ -55,7 +55,7 @@ public int initFromPage(long valueCount, byte[] page, int offset)
this.in = new ByteArrayInputStream(page, offset, page.length - offset);
int bitWidth = BytesUtils.readIntLittleEndianOnOneByte(in);
if (DEBUG) LOG.debug("bit width " + bitWidth);
- decoder = new RunLengthBitPackingHybridDecoder((int)valueCount, bitWidth, in);
+ decoder = new RunLengthBitPackingHybridDecoder(bitWidth, in);
return page.length;
}
diff --git a/parquet-column/src/main/java/parquet/column/values/dictionary/DictionaryValuesWriter.java b/parquet-column/src/main/java/parquet/column/values/dictionary/DictionaryValuesWriter.java
index eb44354605..58e1d460e7 100644
--- a/parquet-column/src/main/java/parquet/column/values/dictionary/DictionaryValuesWriter.java
+++ b/parquet-column/src/main/java/parquet/column/values/dictionary/DictionaryValuesWriter.java
@@ -29,11 +29,12 @@
import it.unimi.dsi.fastutil.longs.Long2IntLinkedOpenHashMap;
import it.unimi.dsi.fastutil.longs.Long2IntMap;
import it.unimi.dsi.fastutil.longs.LongIterator;
+import it.unimi.dsi.fastutil.objects.Object2IntLinkedOpenHashMap;
+import it.unimi.dsi.fastutil.objects.Object2IntMap;
+import it.unimi.dsi.fastutil.objects.ObjectIterator;
import java.io.IOException;
import java.util.Iterator;
-import java.util.LinkedHashMap;
-import java.util.Map;
import parquet.Log;
import parquet.bytes.BytesInput;
@@ -56,19 +57,19 @@
*/
public abstract class DictionaryValuesWriter extends ValuesWriter {
private static final Log LOG = Log.getLog(DictionaryValuesWriter.class);
-
+
/* max entries allowed for the dictionary will fail over to plain encoding if reached */
- private static final int MAX_DICTIONARY_ENTRIES = 65535 /* 2^16 - 1 */;
-
+ private static final int MAX_DICTIONARY_ENTRIES = Integer.MAX_VALUE - 1;
+
/* maximum size in bytes allowed for the dictionary will fail over to plain encoding if reached */
protected final int maxDictionaryByteSize;
-
+
/* contains the values encoded in plain if the dictionary grows too big */
protected final PlainValuesWriter plainValuesWriter;
-
+
/* will become true if the dictionary becomes too big */
protected boolean dictionaryTooBig;
-
+
/* current size in bytes the dictionary will take once serialized */
protected int dictionaryByteSize;
@@ -81,7 +82,16 @@ public abstract class DictionaryValuesWriter extends ValuesWriter {
/* dictionary encoded values */
protected IntList encodedValues = new IntList();
- /*
+ /* size of raw data, even if dictionary is used, it will not have effect on raw data size, it is used to decide
+ * if fall back to plain encoding is better by comparing rawDataByteSize with Encoded data size
+ * It's also used in getBufferedSize, so the page will be written based on raw data size
+ */
+ protected long rawDataByteSize = 0;
+
+ /** indicates if this is the first page being processed */
+ protected boolean firstPage = true;
+
+ /**
* @param maxDictionaryByteSize
* @param initialSize
*/
@@ -90,52 +100,55 @@ protected DictionaryValuesWriter(int maxDictionaryByteSize, int initialSize) {
this.plainValuesWriter = new PlainValuesWriter(initialSize);
}
- /*
+ /**
* check the size constraints of the dictionary and fail over to plain values encoding if threshold reached
*/
protected void checkAndFallbackIfNeeded() {
if (dictionaryByteSize > maxDictionaryByteSize || getDictionarySize() > MAX_DICTIONARY_ENTRIES) {
- // if the dictionary reaches the max byte size or the values can not be encoded on two bytes anymore.
- if (DEBUG)
- LOG.debug("dictionary is now too big, falling back to plain: " + dictionaryByteSize + "B and " + getDictionarySize() + " entries");
- dictionaryTooBig = true;
- if (lastUsedDictionarySize == 0) {
- // if we never used the dictionary
- // we free dictionary encoded data
- clearDictionaryContent();
- dictionaryByteSize = 0;
- encodedValues = null;
- }
+ // if the dictionary reaches the max byte size or the values can not be encoded on 4 bytes anymore.
+ fallBackToPlainEncoding();
}
}
+ private void fallBackToPlainEncoding() {
+ if (DEBUG)
+ LOG.debug("dictionary is now too big, falling back to plain: " + dictionaryByteSize + "B and " + getDictionarySize() + " entries");
+ dictionaryTooBig = true;
+ fallBackDictionaryEncodedData();
+ if (lastUsedDictionarySize == 0) {
+ // if we never used the dictionary
+ // we free dictionary encoded data
+ clearDictionaryContent();
+ dictionaryByteSize = 0;
+ encodedValues = new IntList();
+ }
+ }
+
+ protected abstract void fallBackDictionaryEncodedData();
+
@Override
public long getBufferedSize() {
- // size that will be written to a page
- // not including the dictionary size
- return dictionaryTooBig ? plainValuesWriter.getBufferedSize() : encodedValues.size() * 4;
+ // use raw data size to decide if we want to flush the page
+ // so the acutual size of the page written could be much more smaller
+ // due to dictionary encoding. This prevents page being to big when fallback happens.
+ return rawDataByteSize;
}
@Override
public long getAllocatedSize() {
// size used in memory
- return (encodedValues == null ? 0 : encodedValues.size() * 4) + dictionaryByteSize + plainValuesWriter.getAllocatedSize();
+ return encodedValues.size() * 4 + dictionaryByteSize + plainValuesWriter.getAllocatedSize();
}
@Override
public BytesInput getBytes() {
if (!dictionaryTooBig && getDictionarySize() > 0) {
- // remember size of dictionary when we last wrote a page
- lastUsedDictionarySize = getDictionarySize();
- lastUsedDictionaryByteSize = dictionaryByteSize;
int maxDicId = getDictionarySize() - 1;
- if (DEBUG)
- LOG.debug("max dic id " + maxDicId);
+ if (DEBUG) LOG.debug("max dic id " + maxDicId);
int bitWidth = BytesUtils.getWidthFromMaxInt(maxDicId);
// TODO: what is a good initialCapacity?
- final RunLengthBitPackingHybridEncoder encoder = new RunLengthBitPackingHybridEncoder(bitWidth,
- 64 * 1024);
+ RunLengthBitPackingHybridEncoder encoder = new RunLengthBitPackingHybridEncoder(bitWidth, 64 * 1024);
IntIterator iterator = encodedValues.iterator();
try {
while (iterator.hasNext()) {
@@ -144,9 +157,16 @@ public BytesInput getBytes() {
// encodes the bit width
byte[] bytesHeader = new byte[] { (byte) bitWidth };
BytesInput rleEncodedBytes = encoder.toBytes();
- if (DEBUG)
- LOG.debug("rle encoded bytes " + rleEncodedBytes.size());
- return concat(BytesInput.from(bytesHeader), rleEncodedBytes);
+ if (DEBUG) LOG.debug("rle encoded bytes " + rleEncodedBytes.size());
+ BytesInput bytes = concat(BytesInput.from(bytesHeader), rleEncodedBytes);
+ if (firstPage && ((bytes.size() + dictionaryByteSize) > rawDataByteSize)) {
+ fallBackToPlainEncoding();
+ } else {
+ // remember size of dictionary when we last wrote a page
+ lastUsedDictionarySize = getDictionarySize();
+ lastUsedDictionaryByteSize = dictionaryByteSize;
+ return bytes;
+ }
} catch (IOException e) {
throw new ParquetEncodingException("could not encode the values", e);
}
@@ -156,6 +176,7 @@ public BytesInput getBytes() {
@Override
public Encoding getEncoding() {
+ firstPage = false;
if (!dictionaryTooBig && getDictionarySize() > 0) {
return PLAIN_DICTIONARY;
}
@@ -164,10 +185,9 @@ public Encoding getEncoding() {
@Override
public void reset() {
- if (encodedValues != null) {
- encodedValues = new IntList();
- }
+ encodedValues = new IntList();
plainValuesWriter.reset();
+ rawDataByteSize = 0;
}
@Override
@@ -182,7 +202,7 @@ public void resetDictionary() {
* clear/free the underlying dictionary content
*/
protected abstract void clearDictionaryContent();
-
+
/**
* @return size in items
*/
@@ -190,23 +210,24 @@ public void resetDictionary() {
@Override
public String memUsageString(String prefix) {
- return String.format("%s DictionaryValuesWriter{\n%s\n%s\n%s\n%s}\n",
- prefix,
+ return String.format(
+ "%s DictionaryValuesWriter{\n%s\n%s\n%s\n%s}\n",
+ prefix,
plainValuesWriter.
memUsageString(prefix + " plain:"),
- prefix + " dict:" + dictionaryByteSize,
- prefix + " values:" + (encodedValues.size() * 4),
+ prefix + " dict:" + dictionaryByteSize,
+ prefix + " values:" + String.valueOf(encodedValues.size() * 4),
prefix
);
}
-
+
/**
*
*/
public static class PlainBinaryDictionaryValuesWriter extends DictionaryValuesWriter {
/* type specific dictionary content */
- private Map binaryDictionaryContent = new LinkedHashMap();
+ private Object2IntMap binaryDictionaryContent = new Object2IntLinkedOpenHashMap();
/**
* @param maxDictionaryByteSize
@@ -214,23 +235,26 @@ public static class PlainBinaryDictionaryValuesWriter extends DictionaryValuesWr
*/
public PlainBinaryDictionaryValuesWriter(int maxDictionaryByteSize, int initialSize) {
super(maxDictionaryByteSize, initialSize);
+ binaryDictionaryContent.defaultReturnValue(-1);
}
@Override
public void writeBytes(Binary v) {
if (!dictionaryTooBig) {
- Integer id = binaryDictionaryContent.get(v);
- if (id == null) {
+ int id = binaryDictionaryContent.getInt(v);
+ if (id == -1) {
id = binaryDictionaryContent.size();
binaryDictionaryContent.put(v, id);
- // length as int (2 bytes) + actual bytes
- dictionaryByteSize += 2 + v.length();
+ // length as int (4 bytes) + actual bytes
+ dictionaryByteSize += 4 + v.length();
}
encodedValues.add(id);
checkAndFallbackIfNeeded();
+ } else {
+ plainValuesWriter.writeBytes(v);
}
- // write also to plain encoding if we need to fall back
- plainValuesWriter.writeBytes(v);
+ //for rawdata, length(4 bytes int) is stored, followed by the binary content itself
+ rawDataByteSize += v.length() + 4;
}
@Override
@@ -259,6 +283,23 @@ protected void clearDictionaryContent() {
binaryDictionaryContent.clear();
}
+ @Override
+ protected void fallBackDictionaryEncodedData() {
+ //build reverse dictionary
+ Binary[] reverseDictionary = new Binary[getDictionarySize()];
+ ObjectIterator> entryIterator = binaryDictionaryContent.object2IntEntrySet().iterator();
+ while (entryIterator.hasNext()) {
+ Object2IntMap.Entry entry = entryIterator.next();
+ reverseDictionary[entry.getIntValue()] = entry.getKey();
+ }
+
+ //fall back to plain encoding
+ IntIterator iterator = encodedValues.iterator();
+ while (iterator.hasNext()) {
+ int id = iterator.next();
+ plainValuesWriter.writeBytes(reverseDictionary[id]);
+ }
+ }
}
/**
@@ -289,9 +330,10 @@ public void writeLong(long v) {
}
encodedValues.add(id);
checkAndFallbackIfNeeded();
+ } else {
+ plainValuesWriter.writeLong(v);
}
- // write also to plain encoding if we need to fall back
- plainValuesWriter.writeLong(v);
+ rawDataByteSize += 8;
}
@Override
@@ -319,6 +361,23 @@ protected void clearDictionaryContent() {
longDictionaryContent.clear();
}
+ @Override
+ protected void fallBackDictionaryEncodedData() {
+ //build reverse dictionary
+ long[] reverseDictionary = new long[getDictionarySize()];
+ ObjectIterator entryIterator = longDictionaryContent.long2IntEntrySet().iterator();
+ while (entryIterator.hasNext()) {
+ Long2IntMap.Entry entry = entryIterator.next();
+ reverseDictionary[entry.getIntValue()] = entry.getLongKey();
+ }
+
+ //fall back to plain encoding
+ IntIterator iterator = encodedValues.iterator();
+ while (iterator.hasNext()) {
+ int id = iterator.next();
+ plainValuesWriter.writeLong(reverseDictionary[id]);
+ }
+ }
}
/**
@@ -349,9 +408,10 @@ public void writeDouble(double v) {
}
encodedValues.add(id);
checkAndFallbackIfNeeded();
+ } else {
+ plainValuesWriter.writeDouble(v);
}
- // write also to plain encoding if we need to fall back
- plainValuesWriter.writeDouble(v);
+ rawDataByteSize += 8;
}
@Override
@@ -379,6 +439,23 @@ protected void clearDictionaryContent() {
doubleDictionaryContent.clear();
}
+ @Override
+ protected void fallBackDictionaryEncodedData() {
+ //build reverse dictionary
+ double[] reverseDictionary = new double[getDictionarySize()];
+ ObjectIterator entryIterator = doubleDictionaryContent.double2IntEntrySet().iterator();
+ while (entryIterator.hasNext()) {
+ Double2IntMap.Entry entry = entryIterator.next();
+ reverseDictionary[entry.getIntValue()] = entry.getDoubleKey();
+ }
+
+ //fall back to plain encoding
+ IntIterator iterator = encodedValues.iterator();
+ while (iterator.hasNext()) {
+ int id = iterator.next();
+ plainValuesWriter.writeDouble(reverseDictionary[id]);
+ }
+ }
}
/**
@@ -409,9 +486,12 @@ public void writeInteger(int v) {
}
encodedValues.add(id);
checkAndFallbackIfNeeded();
+ } else {
+ plainValuesWriter.writeInteger(v);
}
- // write also to plain encoding if we need to fall back
- plainValuesWriter.writeInteger(v);
+
+ //Each integer takes 4 bytes as raw data(plain encoding)
+ rawDataByteSize += 4;
}
@Override
@@ -439,6 +519,23 @@ protected void clearDictionaryContent() {
intDictionaryContent.clear();
}
+ @Override
+ protected void fallBackDictionaryEncodedData() {
+ //build reverse dictionary
+ int[] reverseDictionary = new int[getDictionarySize()];
+ ObjectIterator entryIterator = intDictionaryContent.int2IntEntrySet().iterator();
+ while (entryIterator.hasNext()) {
+ Int2IntMap.Entry entry = entryIterator.next();
+ reverseDictionary[entry.getIntValue()] = entry.getIntKey();
+ }
+
+ //fall back to plain encoding
+ IntIterator iterator = encodedValues.iterator();
+ while (iterator.hasNext()) {
+ int id = iterator.next();
+ plainValuesWriter.writeInteger(reverseDictionary[id]);
+ }
+ }
}
/**
@@ -469,9 +566,10 @@ public void writeFloat(float v) {
}
encodedValues.add(id);
checkAndFallbackIfNeeded();
+ } else {
+ plainValuesWriter.writeFloat(v);
}
- // write also to plain encoding if we need to fall back
- plainValuesWriter.writeFloat(v);
+ rawDataByteSize += 4;
}
@Override
@@ -499,6 +597,23 @@ protected void clearDictionaryContent() {
floatDictionaryContent.clear();
}
+ @Override
+ protected void fallBackDictionaryEncodedData() {
+ //build reverse dictionary
+ float[] reverseDictionary = new float[getDictionarySize()];
+ ObjectIterator entryIterator = floatDictionaryContent.float2IntEntrySet().iterator();
+ while (entryIterator.hasNext()) {
+ Float2IntMap.Entry entry = entryIterator.next();
+ reverseDictionary[entry.getIntValue()] = entry.getFloatKey();
+ }
+
+ //fall back to plain encoding
+ IntIterator iterator = encodedValues.iterator();
+ while (iterator.hasNext()) {
+ int id = iterator.next();
+ plainValuesWriter.writeFloat(reverseDictionary[id]);
+ }
+ }
}
}
diff --git a/parquet-column/src/main/java/parquet/column/values/rle/RunLengthBitPackingHybridDecoder.java b/parquet-column/src/main/java/parquet/column/values/rle/RunLengthBitPackingHybridDecoder.java
index fc4133c782..04d3eeb3c8 100644
--- a/parquet-column/src/main/java/parquet/column/values/rle/RunLengthBitPackingHybridDecoder.java
+++ b/parquet-column/src/main/java/parquet/column/values/rle/RunLengthBitPackingHybridDecoder.java
@@ -17,6 +17,7 @@
import static parquet.Log.DEBUG;
+import java.io.ByteArrayInputStream;
import java.io.DataInputStream;
import java.io.IOException;
import java.io.InputStream;
@@ -40,31 +41,27 @@ private static enum MODE { RLE, PACKED }
private final int bitWidth;
private final BytePacker packer;
- private final InputStream in;
+ private final ByteArrayInputStream in;
private MODE mode;
- private int valuesRemaining;
private int currentCount;
private int currentValue;
private int[] currentBuffer;
- public RunLengthBitPackingHybridDecoder(int numValues, int bitWidth, InputStream in) {
+ public RunLengthBitPackingHybridDecoder(int bitWidth, ByteArrayInputStream in) {
if (DEBUG) LOG.debug("decoding bitWidth " + bitWidth);
Preconditions.checkArgument(bitWidth >= 0 && bitWidth <= 32, "bitWidth must be >= 0 and <= 32");
this.bitWidth = bitWidth;
this.packer = Packer.LITTLE_ENDIAN.newBytePacker(bitWidth);
this.in = in;
- this.valuesRemaining = numValues;
}
public int readInt() throws IOException {
- Preconditions.checkArgument(valuesRemaining > 0, "Reading past RLE/BitPacking stream.");
if (currentCount == 0) {
readNext();
}
-- currentCount;
- --valuesRemaining;
int result;
switch (mode) {
case RLE:
@@ -79,7 +76,8 @@ public int readInt() throws IOException {
return result;
}
- private void readNext() throws IOException {
+ private void readNext() throws IOException {
+ Preconditions.checkArgument(in.available() > 0, "Reading past RLE/BitPacking stream.");
final int header = BytesUtils.readUnsignedVarInt(in);
mode = (header & 1) == 0 ? MODE.RLE : MODE.PACKED;
switch (mode) {
@@ -95,12 +93,9 @@ private void readNext() throws IOException {
currentBuffer = new int[currentCount]; // TODO: reuse a buffer
byte[] bytes = new byte[numGroups * bitWidth];
// At the end of the file RLE data though, there might not be that many bytes left.
- // For example, if the number of values is 3, with bitwidth 2, this is encoded as 1
- // group (even though it's really 3/8 of a group). We only need 1 byte to encode the
- // group values (2 * 3 = 6 bits) but using the numGroups data, we'd think we needed
- // 2 bytes (1 * 2 = 2 Bytes).
- int valuesLeft = Math.min(currentCount, valuesRemaining);
- new DataInputStream(in).readFully(bytes, 0, (int)Math.ceil(valuesLeft * bitWidth / 8.0));
+ int bytesToRead = (int)Math.ceil(currentCount * bitWidth / 8.0);
+ bytesToRead = Math.min(bytesToRead, in.available());
+ new DataInputStream(in).readFully(bytes, 0, bytesToRead);
for (int valueIndex = 0, byteIndex = 0; valueIndex < currentCount; valueIndex += 8, byteIndex += bitWidth) {
packer.unpack8Values(bytes, byteIndex, currentBuffer, valueIndex);
}
diff --git a/parquet-column/src/main/java/parquet/column/values/rle/RunLengthBitPackingHybridValuesReader.java b/parquet-column/src/main/java/parquet/column/values/rle/RunLengthBitPackingHybridValuesReader.java
index 221f4c10a4..be4c1f7df0 100644
--- a/parquet-column/src/main/java/parquet/column/values/rle/RunLengthBitPackingHybridValuesReader.java
+++ b/parquet-column/src/main/java/parquet/column/values/rle/RunLengthBitPackingHybridValuesReader.java
@@ -49,10 +49,10 @@ public int initFromPage(long valueCountL, byte[] page, int offset) throws IOExce
return offset;
}
- ByteArrayInputStream in = new ByteArrayInputStream(page, offset, page.length);
+ ByteArrayInputStream in = new ByteArrayInputStream(page, offset, page.length - offset);
int length = BytesUtils.readIntLittleEndian(in);
- decoder = new RunLengthBitPackingHybridDecoder(valueCount, bitWidth, in);
+ decoder = new RunLengthBitPackingHybridDecoder(bitWidth, in);
// 4 is for the length which is stored as 4 bytes little endian
return offset + length + 4;
diff --git a/parquet-column/src/main/java/parquet/filter/ColumnPredicates.java b/parquet-column/src/main/java/parquet/filter/ColumnPredicates.java
index b8fc10b3fc..ab7a47c6a0 100644
--- a/parquet-column/src/main/java/parquet/filter/ColumnPredicates.java
+++ b/parquet-column/src/main/java/parquet/filter/ColumnPredicates.java
@@ -17,6 +17,7 @@
import parquet.Preconditions;
import parquet.column.ColumnReader;
+import parquet.io.api.Binary;
/**
* ColumnPredicates class provides checks for column values. Factory methods
@@ -29,6 +30,32 @@ public static interface Predicate {
boolean apply(ColumnReader input);
}
+ public static interface PredicateFunction {
+ boolean functionToApply(T input);
+ }
+
+ /* provide the following to avoid boxing primitives */
+
+ public static interface IntegerPredicateFunction {
+ boolean functionToApply(int input);
+ }
+
+ public static interface LongPredicateFunction {
+ boolean functionToApply(long input);
+ }
+
+ public static interface FloatPredicateFunction {
+ boolean functionToApply(float input);
+ }
+
+ public static interface DoublePredicateFunction {
+ boolean functionToApply(double input);
+ }
+
+ public static interface BooleanPredicateFunction {
+ boolean functionToApply(boolean input);
+ }
+
public static Predicate equalTo(final String target) {
Preconditions.checkNotNull(target,"target");
return new Predicate() {
@@ -39,6 +66,15 @@ public boolean apply(ColumnReader input) {
};
}
+ public static Predicate applyFunctionToString(final PredicateFunction fn) {
+ return new Predicate() {
+ @Override
+ public boolean apply(ColumnReader input) {
+ return fn.functionToApply(input.getBinary().toStringUsingUTF8());
+ }
+ };
+ }
+
public static Predicate equalTo(final int target) {
return new Predicate() {
@Override
@@ -48,6 +84,15 @@ public boolean apply(ColumnReader input) {
};
}
+ public static Predicate applyFunctionToInteger(final IntegerPredicateFunction fn) {
+ return new Predicate() {
+ @Override
+ public boolean apply(ColumnReader input) {
+ return fn.functionToApply(input.getInteger());
+ }
+ };
+ }
+
public static Predicate equalTo(final long target) {
return new Predicate() {
@Override
@@ -57,6 +102,15 @@ public boolean apply(ColumnReader input) {
};
}
+ public static Predicate applyFunctionToLong(final LongPredicateFunction fn) {
+ return new Predicate() {
+ @Override
+ public boolean apply(ColumnReader input) {
+ return fn.functionToApply(input.getLong());
+ }
+ };
+ }
+
public static Predicate equalTo(final float target) {
return new Predicate() {
@Override
@@ -66,6 +120,15 @@ public boolean apply(ColumnReader input) {
};
}
+ public static Predicate applyFunctionToFloat(final FloatPredicateFunction fn) {
+ return new Predicate() {
+ @Override
+ public boolean apply(ColumnReader input) {
+ return fn.functionToApply(input.getFloat());
+ }
+ };
+ }
+
public static Predicate equalTo(final double target) {
return new Predicate() {
@Override
@@ -75,6 +138,15 @@ public boolean apply(ColumnReader input) {
};
}
+ public static Predicate applyFunctionToDouble(final DoublePredicateFunction fn) {
+ return new Predicate() {
+ @Override
+ public boolean apply(ColumnReader input) {
+ return fn.functionToApply(input.getDouble());
+ }
+ };
+ }
+
public static Predicate equalTo(final boolean target) {
return new Predicate() {
@Override
@@ -84,6 +156,15 @@ public boolean apply(ColumnReader input) {
};
}
+ public static Predicate applyFunctionToBoolean (final BooleanPredicateFunction fn) {
+ return new Predicate() {
+ @Override
+ public boolean apply(ColumnReader input) {
+ return fn.functionToApply(input.getBoolean());
+ }
+ };
+ }
+
public static Predicate equalTo(final E target) {
Preconditions.checkNotNull(target,"target");
final String targetAsString = target.name();
@@ -94,4 +175,13 @@ public boolean apply(ColumnReader input) {
}
};
}
+
+ public static Predicate applyFunctionToBinary (final PredicateFunction fn) {
+ return new Predicate() {
+ @Override
+ public boolean apply(ColumnReader input) {
+ return fn.functionToApply(input.getBinary());
+ }
+ };
+ }
}
diff --git a/parquet-column/src/main/java/parquet/io/api/Binary.java b/parquet-column/src/main/java/parquet/io/api/Binary.java
index 365d39e528..72bb29cb74 100644
--- a/parquet-column/src/main/java/parquet/io/api/Binary.java
+++ b/parquet-column/src/main/java/parquet/io/api/Binary.java
@@ -17,6 +17,7 @@
import static parquet.bytes.BytesUtils.UTF8;
+import java.io.DataOutput;
import java.io.IOException;
import java.io.OutputStream;
import java.io.UnsupportedEncodingException;
@@ -78,6 +79,11 @@ public ByteBuffer toByteBuffer() {
return ByteBuffer.wrap(value, offset, length);
}
+ @Override
+ public void writeTo(DataOutput out) throws IOException {
+ out.write(value, offset, length);
+ }
+
};
}
@@ -122,6 +128,11 @@ boolean equals(byte[] other, int otherOffset, int otherLength) {
public ByteBuffer toByteBuffer() {
return ByteBuffer.wrap(value);
}
+
+ @Override
+ public void writeTo(DataOutput out) throws IOException {
+ out.write(value);
+ }
};
}
@@ -139,6 +150,7 @@ public int length() {
@Override
public void writeTo(OutputStream out) throws IOException {
+ // TODO: should not have to materialize those bytes
out.write(getBytes());
}
@@ -185,6 +197,12 @@ boolean equals(byte[] other, int otherOffset, int otherLength) {
public ByteBuffer toByteBuffer() {
return value;
}
+
+ @Override
+ public void writeTo(DataOutput out) throws IOException {
+ // TODO: should not have to materialize those bytes
+ out.write(getBytes());
+ }
};
}
@@ -223,9 +241,10 @@ private static final int hashCode(byte[] array, int offset, int length) {
* @return
*/
private static final boolean equals(byte[] array1, int offset1, int length1, byte[] array2, int offset2, int length2) {
- if (array1 == array2) return true;
+ if (array1 == null && array2 == null) return true;
if (array1 == null || array2 == null) return false;
if (length1 != length2) return false;
+ if (array1 == array2 && offset1 == offset2) return true;
for (int i = 0; i < length1; i++) {
if (array1[i + offset1] != array2[i + offset2]) {
return false;
@@ -240,6 +259,8 @@ private static final boolean equals(byte[] array1, int offset1, int length1, byt
abstract public void writeTo(OutputStream out) throws IOException;
+ abstract public void writeTo(DataOutput out) throws IOException;
+
abstract public byte[] getBytes();
abstract boolean equals(byte[] bytes, int offset, int length);
diff --git a/parquet-column/src/test/java/parquet/column/mem/TestMemColumn.java b/parquet-column/src/test/java/parquet/column/mem/TestMemColumn.java
index 843c639fee..46b3b8fa32 100644
--- a/parquet-column/src/test/java/parquet/column/mem/TestMemColumn.java
+++ b/parquet-column/src/test/java/parquet/column/mem/TestMemColumn.java
@@ -23,6 +23,8 @@
import parquet.column.ColumnDescriptor;
import parquet.column.ColumnReader;
import parquet.column.ColumnWriter;
+import parquet.column.ParquetProperties;
+import parquet.column.ParquetProperties.WriterVersion;
import parquet.column.impl.ColumnReadStoreImpl;
import parquet.column.impl.ColumnWriteStoreImpl;
import parquet.column.page.mem.MemPageStore;
@@ -54,7 +56,7 @@ public void testMemColumn() throws Exception {
}
private ColumnWriter getColumnWriter(ColumnDescriptor path, MemPageStore memPageStore) {
- ColumnWriteStoreImpl memColumnsStore = new ColumnWriteStoreImpl(memPageStore, 2048, 2048, false);
+ ColumnWriteStoreImpl memColumnsStore = newColumnWriteStoreImpl(memPageStore);
ColumnWriter columnWriter = memColumnsStore.getColumnWriter(path);
return columnWriter;
}
@@ -73,7 +75,7 @@ public void testMemColumnBinary() throws Exception {
String[] col = new String[]{"foo", "bar"};
MemPageStore memPageStore = new MemPageStore(10);
- ColumnWriteStoreImpl memColumnsStore = new ColumnWriteStoreImpl(memPageStore, 2048, 2048, false);
+ ColumnWriteStoreImpl memColumnsStore = newColumnWriteStoreImpl(memPageStore);
ColumnDescriptor path1 = mt.getColumnDescription(col);
ColumnDescriptor path = path1;
@@ -95,7 +97,7 @@ public void testMemColumnSeveralPages() throws Exception {
MessageType mt = MessageTypeParser.parseMessageType("message msg { required group foo { required int64 bar; } }");
String[] col = new String[]{"foo", "bar"};
MemPageStore memPageStore = new MemPageStore(10);
- ColumnWriteStoreImpl memColumnsStore = new ColumnWriteStoreImpl(memPageStore, 2048, 2048, false);
+ ColumnWriteStoreImpl memColumnsStore = newColumnWriteStoreImpl(memPageStore);
ColumnDescriptor path1 = mt.getColumnDescription(col);
ColumnDescriptor path = path1;
@@ -119,7 +121,7 @@ public void testMemColumnSeveralPagesRepeated() throws Exception {
MessageType mt = MessageTypeParser.parseMessageType("message msg { repeated group foo { repeated int64 bar; } }");
String[] col = new String[]{"foo", "bar"};
MemPageStore memPageStore = new MemPageStore(10);
- ColumnWriteStoreImpl memColumnsStore = new ColumnWriteStoreImpl(memPageStore, 2048, 2048, false);
+ ColumnWriteStoreImpl memColumnsStore = newColumnWriteStoreImpl(memPageStore);
ColumnDescriptor path1 = mt.getColumnDescription(col);
ColumnDescriptor path = path1;
@@ -153,4 +155,8 @@ public void testMemColumnSeveralPagesRepeated() throws Exception {
++ i;
}
}
+
+ private ColumnWriteStoreImpl newColumnWriteStoreImpl(MemPageStore memPageStore) {
+ return new ColumnWriteStoreImpl(memPageStore, 2048, 2048, 2048, false, WriterVersion.PARQUET_1_0);
+ }
}
diff --git a/parquet-column/src/test/java/parquet/column/values/delta/DeltaBinaryPackingValuesWriterTest.java b/parquet-column/src/test/java/parquet/column/values/delta/DeltaBinaryPackingValuesWriterTest.java
new file mode 100644
index 0000000000..833299ddd9
--- /dev/null
+++ b/parquet-column/src/test/java/parquet/column/values/delta/DeltaBinaryPackingValuesWriterTest.java
@@ -0,0 +1,262 @@
+/**
+ * Copyright 2012 Twitter, Inc.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package parquet.column.values.delta;
+
+import static junit.framework.Assert.assertTrue;
+import static org.junit.Assert.assertEquals;
+
+import java.io.IOException;
+import java.util.Random;
+
+import org.junit.Before;
+import org.junit.Test;
+
+import parquet.bytes.BytesInput;
+import parquet.column.values.ValuesReader;
+import parquet.column.values.ValuesWriter;
+import parquet.column.values.rle.RunLengthBitPackingHybridValuesWriter;
+import parquet.io.ParquetDecodingException;
+
+public class DeltaBinaryPackingValuesWriterTest {
+ DeltaBinaryPackingValuesReader reader;
+ private int blockSize;
+ private int miniBlockNum;
+ private ValuesWriter writer;
+ private Random random;
+
+ @Before
+ public void setUp() {
+ blockSize = 128;
+ miniBlockNum = 4;
+ writer = new DeltaBinaryPackingValuesWriter(blockSize, miniBlockNum, 100);
+ random = new Random();
+ }
+
+ @Test(expected = IllegalArgumentException.class)
+ public void miniBlockSizeShouldBeMultipleOf8() {
+ new DeltaBinaryPackingValuesWriter(1281, 4, 100);
+ }
+
+ /* When data size is multiple of Block*/
+ @Test
+ public void shouldWriteWhenDataIsAlignedWithBlock() throws IOException {
+ int[] data = new int[5 * blockSize];
+ for (int i = 0; i < blockSize * 5; i++) {
+ data[i] = random.nextInt();
+ }
+ shouldWriteAndRead(data);
+ }
+
+ @Test
+ public void shouldWriteAndReadWhenBlockIsNotFullyWritten() throws IOException {
+ int[] data = new int[blockSize - 3];
+ for (int i = 0; i < data.length; i++) {
+ data[i] = random.nextInt();
+ }
+ shouldWriteAndRead(data);
+ }
+
+ @Test
+ public void shouldWriteAndReadWhenAMiniBlockIsNotFullyWritten() throws IOException {
+ int miniBlockSize = blockSize / miniBlockNum;
+ int[] data = new int[miniBlockSize - 3];
+ for (int i = 0; i < data.length; i++) {
+ data[i] = random.nextInt();
+ }
+ shouldWriteAndRead(data);
+ }
+
+ @Test
+ public void shouldWriteNegativeDeltas() throws IOException {
+ int[] data = new int[blockSize];
+ for (int i = 0; i < data.length; i++) {
+ data[i] = 10 - (i * 32 - random.nextInt(6));
+ }
+ shouldWriteAndRead(data);
+ }
+
+ @Test
+ public void shouldWriteAndReadWhenDeltasAreSame() throws IOException {
+ int[] data = new int[2 * blockSize];
+ for (int i = 0; i < blockSize; i++) {
+ data[i] = i * 32;
+ }
+ shouldWriteAndRead(data);
+ }
+
+ @Test
+ public void shouldWriteAndReadWhenValuesAreSame() throws IOException {
+ int[] data = new int[2 * blockSize];
+ for (int i = 0; i < blockSize; i++) {
+ data[i] = 3;
+ }
+ shouldWriteAndRead(data);
+ }
+
+ @Test
+ public void shouldWriteWhenDeltaIs0ForEachBlock() throws IOException {
+ int[] data = new int[5 * blockSize + 1];
+ for (int i = 0; i < data.length; i++) {
+ data[i] = (i - 1) / blockSize;
+ }
+ shouldWriteAndRead(data);
+ }
+
+ @Test
+ public void shouldReadWriteWhenDataIsNotAlignedWithBlock() throws IOException {
+ int[] data = new int[5 * blockSize + 3];
+ for (int i = 0; i < data.length; i++) {
+ data[i] = random.nextInt(20) - 10;
+ }
+ shouldWriteAndRead(data);
+ }
+
+ @Test
+ public void shouldReadMaxMinValue() throws IOException {
+ int[] data = new int[10];
+ for (int i = 0; i < data.length; i++) {
+ if(i%2==0) {
+ data[i]=Integer.MIN_VALUE;
+ }else {
+ data[i]=Integer.MAX_VALUE;
+ }
+ }
+ shouldWriteAndRead(data);
+ }
+
+ @Test
+ public void shouldReturnCorrectOffsetAfterInitialization() throws IOException {
+ int[] data = new int[2 * blockSize + 3];
+ for (int i = 0; i < data.length; i++) {
+ data[i] = i * 32;
+ }
+ writeData(data);
+
+ reader = new DeltaBinaryPackingValuesReader();
+ BytesInput bytes = writer.getBytes();
+ byte[] valueContent = bytes.toByteArray();
+ byte[] pageContent = new byte[valueContent.length * 10];
+ int contentOffsetInPage = 33;
+ System.arraycopy(valueContent, 0, pageContent, contentOffsetInPage, valueContent.length);
+
+ //offset should be correct
+ reader.initFromPage(100, pageContent, contentOffsetInPage);
+ //TODO: uncomment following test for testing returning correct offset
+// int offset= reader.getNextOffset();
+// assertEquals(valueContent.length + contentOffsetInPage, offset);
+
+ //should be able to read data correclty
+ for (int i : data) {
+ assertEquals(i, reader.readInteger());
+ }
+ }
+
+ @Test
+ public void shouldThrowExceptionWhenReadMoreThanWritten() throws IOException {
+ int[] data = new int[5 * blockSize + 1];
+ for (int i = 0; i < data.length; i++) {
+ data[i] = i * 32;
+ }
+ shouldWriteAndRead(data);
+ try {
+ reader.readInteger();
+ } catch (ParquetDecodingException e) {
+ assertEquals("no more value to read, total value count is " + data.length, e.getMessage());
+ }
+
+ }
+
+ @Test
+ public void shouldSkip() throws IOException {
+ int[] data = new int[5 * blockSize + 1];
+ for (int i = 0; i < data.length; i++) {
+ data[i] = i * 32;
+ }
+ writeData(data);
+ reader = new DeltaBinaryPackingValuesReader();
+ reader.initFromPage(100, writer.getBytes().toByteArray(), 0);
+ for (int i = 0; i < data.length; i++) {
+ if (i % 3 == 0) {
+ reader.skip();
+ } else {
+ assertEquals(i * 32, reader.readInteger());
+ }
+ }
+ }
+
+ @Test
+ public void shouldReset() throws IOException {
+ shouldReadWriteWhenDataIsNotAlignedWithBlock();
+ int[] data = new int[5 * blockSize];
+ for (int i = 0; i < blockSize * 5; i++) {
+ data[i] = i * 2;
+ }
+ writer.reset();
+ shouldWriteAndRead(data);
+ }
+
+ @Test
+ public void randomDataTest() throws IOException {
+ int maxSize = 1000;
+ int[] data = new int[maxSize];
+
+ for (int round = 0; round < 100000; round++) {
+
+
+ int size = random.nextInt(maxSize);
+
+ for (int i = 0; i < size; i++) {
+ data[i] = random.nextInt();
+ }
+ shouldReadAndWrite(data, size);
+ writer.reset();
+ }
+ }
+
+ private void shouldWriteAndRead(int[] data) throws IOException {
+ shouldReadAndWrite(data, data.length);
+ }
+
+ private void shouldReadAndWrite(int[] data, int length) throws IOException {
+ writeData(data, length);
+ reader = new DeltaBinaryPackingValuesReader();
+ byte[] page = writer.getBytes().toByteArray();
+ int miniBlockSize = blockSize / miniBlockNum;
+
+ double miniBlockFlushed = Math.ceil(((double) length - 1) / miniBlockSize);
+ double blockFlushed = Math.ceil(((double) length - 1) / blockSize);
+ double estimatedSize = 4 * 5 //blockHeader
+ + 4 * miniBlockFlushed * miniBlockSize //data(aligned to miniBlock)
+ + blockFlushed * miniBlockNum //bitWidth of mini blocks
+ + (5.0 * blockFlushed);//min delta for each block
+ assertTrue(estimatedSize >= page.length);
+ reader.initFromPage(100, page, 0);
+
+ for (int i = 0; i < length; i++) {
+ assertEquals(data[i], reader.readInteger());
+ }
+ }
+
+ private void writeData(int[] data) {
+ writeData(data, data.length);
+ }
+
+ private void writeData(int[] data, int length) {
+ for (int i = 0; i < length; i++) {
+ writer.writeInteger(data[i]);
+ }
+ }
+}
diff --git a/parquet-column/src/test/java/parquet/column/values/delta/benchmark/BenchMarkTest.java b/parquet-column/src/test/java/parquet/column/values/delta/benchmark/BenchMarkTest.java
new file mode 100644
index 0000000000..ef32ae1b04
--- /dev/null
+++ b/parquet-column/src/test/java/parquet/column/values/delta/benchmark/BenchMarkTest.java
@@ -0,0 +1,40 @@
+/**
+ * Copyright 2012 Twitter, Inc.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package parquet.column.values.delta.benchmark;
+
+import parquet.column.values.ValuesWriter;
+
+public abstract class BenchMarkTest {
+ public static int[] data;
+
+ protected void runWriteTest(ValuesWriter writer){
+ int pageCount = 10;
+ double avg = 0.0;
+ for (int i = 0; i < pageCount ; i++) {
+ writer.reset();
+ long startTime = System.nanoTime();
+ for(int item:data){
+ writer.writeInteger(item);
+ }
+ long endTime = System.nanoTime();
+ long duration = endTime - startTime;
+ avg += (double) duration / pageCount;
+ }
+
+ System.out.println("size is "+writer.getBytes().size());
+ }
+
+}
diff --git a/parquet-column/src/test/java/parquet/column/values/delta/benchmark/BenchmarkIntegerOutputSize.java b/parquet-column/src/test/java/parquet/column/values/delta/benchmark/BenchmarkIntegerOutputSize.java
new file mode 100644
index 0000000000..08eaf397fd
--- /dev/null
+++ b/parquet-column/src/test/java/parquet/column/values/delta/benchmark/BenchmarkIntegerOutputSize.java
@@ -0,0 +1,99 @@
+/**
+ * Copyright 2012 Twitter, Inc.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package parquet.column.values.delta.benchmark;
+
+import org.junit.Test;
+import parquet.column.values.delta.DeltaBinaryPackingValuesWriter;
+import parquet.column.values.rle.RunLengthBitPackingHybridValuesWriter;
+import java.util.Random;
+
+public class BenchmarkIntegerOutputSize {
+ public static int blockSize=128;
+ public static int miniBlockNum=4;
+ public static int dataSize=10000 * blockSize;
+
+ private interface IntFunc {
+ public int getIntValue();
+ }
+
+ @Test
+ public void testBigNumbers() {
+ final Random r=new Random();
+ testRandomIntegers(new IntFunc() {
+ @Override
+ public int getIntValue() {
+ return r.nextInt();
+ }
+ },32);
+ }
+
+ @Test
+ public void testRangedNumbersWithSmallVariations() {
+ final Random r=new Random();
+ testRandomIntegers(new IntFunc() {
+ @Override
+ public int getIntValue() {
+ return 1000+r.nextInt(20);
+ }
+ },10);
+ }
+
+ @Test
+ public void testSmallNumbersWithSmallVariations() {
+ final Random r=new Random();
+ testRandomIntegers(new IntFunc() {
+ @Override
+ public int getIntValue() {
+ return 40+r.nextInt(20);
+ }
+ },6);
+ }
+
+ @Test
+ public void testSmallNumberVariation() {
+ final Random r=new Random();
+ testRandomIntegers(new IntFunc() {
+ @Override
+ public int getIntValue() {
+ return r.nextInt(20)-10;
+ }
+ },4);
+ }
+
+ public void testRandomIntegers(IntFunc func,int bitWidth) {
+ DeltaBinaryPackingValuesWriter delta=new DeltaBinaryPackingValuesWriter(blockSize,miniBlockNum,100);
+ RunLengthBitPackingHybridValuesWriter rle= new RunLengthBitPackingHybridValuesWriter(bitWidth,100);
+ for (int i = 0; i < dataSize; i++) {
+ int v = func.getIntValue();
+ delta.writeInteger(v);
+ rle.writeInteger(v);
+ }
+ System.out.println("delta size: "+delta.getBytes().size());
+ System.out.println("estimated size"+estimatedSize());
+ System.out.println("rle size: "+rle.getBytes().size());
+ }
+
+ private double estimatedSize(){
+ int miniBlockSize = blockSize / miniBlockNum;
+ double miniBlockFlushed = Math.ceil(((double) dataSize - 1) / miniBlockSize);
+ double blockFlushed = Math.ceil(((double) dataSize - 1) / blockSize);
+ double estimatedSize = 4 * 5 //blockHeader
+ + 4 * miniBlockFlushed * miniBlockSize //data(aligned to miniBlock)
+ + blockFlushed * miniBlockNum //bitWidth of mini blocks
+ + (5.0 * blockFlushed);//min delta for each block
+ return estimatedSize;
+ }
+}
diff --git a/parquet-column/src/test/java/parquet/column/values/delta/benchmark/BenchmarkReadingRandomIntegers.java b/parquet-column/src/test/java/parquet/column/values/delta/benchmark/BenchmarkReadingRandomIntegers.java
new file mode 100644
index 0000000000..d01a605dc4
--- /dev/null
+++ b/parquet-column/src/test/java/parquet/column/values/delta/benchmark/BenchmarkReadingRandomIntegers.java
@@ -0,0 +1,93 @@
+/**
+ * Copyright 2012 Twitter, Inc.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package parquet.column.values.delta.benchmark;
+
+import com.carrotsearch.junitbenchmarks.BenchmarkOptions;
+import com.carrotsearch.junitbenchmarks.BenchmarkRule;
+import com.carrotsearch.junitbenchmarks.annotation.AxisRange;
+import com.carrotsearch.junitbenchmarks.annotation.BenchmarkMethodChart;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+import parquet.column.values.ValuesReader;
+import parquet.column.values.ValuesWriter;
+import parquet.column.values.delta.DeltaBinaryPackingValuesReader;
+import parquet.column.values.delta.DeltaBinaryPackingValuesWriter;
+import parquet.column.values.rle.RunLengthBitPackingHybridValuesReader;
+import parquet.column.values.rle.RunLengthBitPackingHybridValuesWriter;
+
+import java.io.IOException;
+import java.util.Random;
+
+@AxisRange(min = 0, max = 1)
+@BenchmarkMethodChart(filePrefix = "benchmark-encoding-reading-random")
+public class BenchmarkReadingRandomIntegers {
+ public static int blockSize = 128;
+ public static int miniBlockNum = 4;
+ public static byte[] deltaBytes;
+ public static byte[] rleBytes;
+ public static int[] data;
+ @Rule
+ public org.junit.rules.TestRule benchmarkRun = new BenchmarkRule();
+
+ @BeforeClass
+ public static void prepare() throws IOException {
+ Random random = new Random();
+ data = new int[100000 * blockSize];
+ for (int i = 0; i < data.length; i++) {
+ data[i] = random.nextInt(100) - 200;
+ }
+
+ ValuesWriter delta = new DeltaBinaryPackingValuesWriter(blockSize, miniBlockNum, 100);
+ ValuesWriter rle = new RunLengthBitPackingHybridValuesWriter(32, 100);
+
+ for (int i = 0; i < data.length; i++) {
+ delta.writeInteger(data[i]);
+ rle.writeInteger(data[i]);
+ }
+ deltaBytes = delta.getBytes().toByteArray();
+ rleBytes = rle.getBytes().toByteArray();
+ }
+
+ @BenchmarkOptions(benchmarkRounds = 20, warmupRounds = 10)
+ @Test
+ public void readingDelta() throws IOException {
+ for (int j = 0; j < 10; j++) {
+
+ DeltaBinaryPackingValuesReader reader = new DeltaBinaryPackingValuesReader();
+ readData(reader, deltaBytes);
+ }
+ }
+
+ @BenchmarkOptions(benchmarkRounds = 20, warmupRounds = 10)
+ @Test
+ public void readingRLE() throws IOException {
+ for (int j = 0; j < 10; j++) {
+
+ ValuesReader reader = new RunLengthBitPackingHybridValuesReader(32);
+ readData(reader, rleBytes);
+ }
+ }
+
+ private void readData(ValuesReader reader, byte[] deltaBytes) throws IOException {
+ reader.initFromPage(data.length, deltaBytes, 0);
+ for (int i = 0; i < data.length; i++) {
+ reader.readInteger();
+ }
+ }
+
+}
+
diff --git a/parquet-column/src/test/java/parquet/column/values/delta/benchmark/RandomWritingBenchmarkTest.java b/parquet-column/src/test/java/parquet/column/values/delta/benchmark/RandomWritingBenchmarkTest.java
new file mode 100644
index 0000000000..0e4ec0b56c
--- /dev/null
+++ b/parquet-column/src/test/java/parquet/column/values/delta/benchmark/RandomWritingBenchmarkTest.java
@@ -0,0 +1,67 @@
+/**
+ * Copyright 2012 Twitter, Inc.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package parquet.column.values.delta.benchmark;
+
+import com.carrotsearch.junitbenchmarks.BenchmarkOptions;
+import com.carrotsearch.junitbenchmarks.BenchmarkRule;
+import com.carrotsearch.junitbenchmarks.annotation.AxisRange;
+import com.carrotsearch.junitbenchmarks.annotation.BenchmarkMethodChart;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+import parquet.column.values.ValuesWriter;
+import parquet.column.values.delta.DeltaBinaryPackingValuesWriter;
+import parquet.column.values.rle.RunLengthBitPackingHybridValuesWriter;
+import java.util.Random;
+
+@AxisRange(min = 0, max = 1)
+@BenchmarkMethodChart(filePrefix = "benchmark-encoding-writing-random")
+public class RandomWritingBenchmarkTest extends BenchMarkTest{
+ public static int blockSize=128;
+ public static int miniBlockNum=4;
+ @Rule
+ public org.junit.rules.TestRule benchmarkRun = new BenchmarkRule();
+
+ @BeforeClass
+ public static void prepare() {
+ Random random=new Random();
+ data = new int[10000 * blockSize];
+ for (int i = 0; i < data.length; i++) {
+ data[i] = random.nextInt(100) - 200;
+ }
+ }
+
+ @BenchmarkOptions(benchmarkRounds = 10, warmupRounds = 2)
+ @Test
+ public void writeDeltaPackingTest(){
+ DeltaBinaryPackingValuesWriter writer = new DeltaBinaryPackingValuesWriter(blockSize, miniBlockNum, 100);
+ runWriteTest(writer);
+ }
+
+ @BenchmarkOptions(benchmarkRounds = 10, warmupRounds = 2)
+ @Test
+ public void writeRLETest(){
+ ValuesWriter writer = new RunLengthBitPackingHybridValuesWriter(32,100);
+ runWriteTest(writer);
+ }
+
+ @BenchmarkOptions(benchmarkRounds = 10, warmupRounds = 2)
+ @Test
+ public void writeDeltaPackingTest2(){
+ DeltaBinaryPackingValuesWriter writer = new DeltaBinaryPackingValuesWriter(blockSize, miniBlockNum, 100);
+ runWriteTest(writer);
+ }
+}
diff --git a/parquet-column/src/test/java/parquet/column/values/delta/benchmark/SmallRangeWritingBenchmarkTest.java b/parquet-column/src/test/java/parquet/column/values/delta/benchmark/SmallRangeWritingBenchmarkTest.java
new file mode 100644
index 0000000000..f5ef85e228
--- /dev/null
+++ b/parquet-column/src/test/java/parquet/column/values/delta/benchmark/SmallRangeWritingBenchmarkTest.java
@@ -0,0 +1,45 @@
+/**
+ * Copyright 2012 Twitter, Inc.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package parquet.column.values.delta.benchmark;
+
+import com.carrotsearch.junitbenchmarks.BenchmarkOptions;
+import com.carrotsearch.junitbenchmarks.annotation.AxisRange;
+import com.carrotsearch.junitbenchmarks.annotation.BenchmarkMethodChart;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import parquet.column.values.ValuesWriter;
+import parquet.column.values.rle.RunLengthBitPackingHybridValuesWriter;
+import java.util.Random;
+
+@AxisRange(min = 0, max = 2)
+@BenchmarkMethodChart(filePrefix = "benchmark-encoding-writing-random-small")
+public class SmallRangeWritingBenchmarkTest extends RandomWritingBenchmarkTest {
+ @BeforeClass
+ public static void prepare() {
+ Random random=new Random();
+ data = new int[100000 * blockSize];
+ for (int i = 0; i < data.length; i++) {
+ data[i] = random.nextInt(2) - 1;
+ }
+ }
+
+ @BenchmarkOptions(benchmarkRounds = 10, warmupRounds = 2)
+ @Test
+ public void writeRLEWithSmallBitWidthTest(){
+ ValuesWriter writer = new RunLengthBitPackingHybridValuesWriter(2,100);
+ runWriteTest(writer);
+ }
+}
diff --git a/parquet-column/src/test/java/parquet/column/values/dictionary/TestDictionary.java b/parquet-column/src/test/java/parquet/column/values/dictionary/TestDictionary.java
index 64c0f10c63..f202305b0b 100644
--- a/parquet-column/src/test/java/parquet/column/values/dictionary/TestDictionary.java
+++ b/parquet-column/src/test/java/parquet/column/values/dictionary/TestDictionary.java
@@ -16,8 +16,12 @@
package parquet.column.values.dictionary;
import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
+import static parquet.column.Encoding.PLAIN;
import static parquet.column.Encoding.PLAIN_DICTIONARY;
+import static parquet.schema.PrimitiveType.PrimitiveTypeName.BINARY;
+import static parquet.schema.PrimitiveType.PrimitiveTypeName.DOUBLE;
+import static parquet.schema.PrimitiveType.PrimitiveTypeName.FLOAT;
+import static parquet.schema.PrimitiveType.PrimitiveTypeName.INT32;
import java.io.IOException;
@@ -37,11 +41,7 @@
import parquet.column.values.dictionary.DictionaryValuesWriter.PlainIntegerDictionaryValuesWriter;
import parquet.column.values.dictionary.DictionaryValuesWriter.PlainLongDictionaryValuesWriter;
import parquet.column.values.plain.BinaryPlainValuesReader;
-import parquet.column.values.plain.PlainValuesReader.DoublePlainValuesReader;
-import parquet.column.values.plain.PlainValuesReader.FloatPlainValuesReader;
-import parquet.column.values.plain.PlainValuesReader.IntegerPlainValuesReader;
-import parquet.column.values.plain.PlainValuesReader.LongPlainValuesReader;
-import parquet.column.values.plain.PlainValuesWriter;
+import parquet.column.values.plain.PlainValuesReader;
import parquet.io.api.Binary;
import parquet.schema.PrimitiveType.PrimitiveTypeName;
@@ -49,125 +49,111 @@ public class TestDictionary {
@Test
public void testBinaryDictionary() throws IOException {
-
int COUNT = 100;
- ValuesWriter cw = new PlainBinaryDictionaryValuesWriter(10000, 10000);
- for (int i = 0; i < COUNT; i++) {
- cw.writeBytes(Binary.fromString("a" + i % 10));
- }
- assertEquals(PLAIN_DICTIONARY, cw.getEncoding());
- final BytesInput bytes1 = BytesInput.copy(cw.getBytes());
- cw.reset();
- for (int i = 0; i < COUNT; i++) {
- cw.writeBytes(Binary.fromString("b" + i % 10));
- }
- assertEquals(PLAIN_DICTIONARY, cw.getEncoding());
- final BytesInput bytes2 = BytesInput.copy(cw.getBytes());
- cw.reset();
-
- final DictionaryPage dictionaryPage = cw.createDictionaryPage().copy();
- final ColumnDescriptor descriptor = new ColumnDescriptor(new String[] {"foo"}, PrimitiveTypeName.BINARY, 0, 0);
- final Dictionary dictionary = PLAIN_DICTIONARY.initDictionary(descriptor, dictionaryPage);
- final DictionaryValuesReader cr = new DictionaryValuesReader(dictionary);
-
- cr.initFromPage(COUNT, bytes1.toByteArray(), 0);
- for (int i = 0; i < COUNT; i++) {
- final String str = cr.readBytes().toStringUsingUTF8();
- Assert.assertEquals("a" + i % 10, str);
- }
-
- cr.initFromPage(COUNT, bytes2.toByteArray(), 0);
- for (int i = 0; i < COUNT; i++) {
- final String str = cr.readBytes().toStringUsingUTF8();
- Assert.assertEquals("b" + i % 10, str);
- }
-
+ ValuesWriter cw = new PlainBinaryDictionaryValuesWriter(200, 10000);
+ writeRepeated(COUNT, cw, "a");
+ BytesInput bytes1 = getBytesAndCheckEncoding(cw, PLAIN_DICTIONARY);
+ writeRepeated(COUNT, cw, "b");
+ BytesInput bytes2 = getBytesAndCheckEncoding(cw, PLAIN_DICTIONARY);
+ // now we will fall back
+ writeDistinct(COUNT, cw, "c");
+ BytesInput bytes3 = getBytesAndCheckEncoding(cw, PLAIN);
+
+ DictionaryValuesReader cr = initDicReader(cw, BINARY);
+ checkRepeated(COUNT, bytes1, cr, "a");
+ checkRepeated(COUNT, bytes2, cr, "b");
+ BinaryPlainValuesReader cr2 = new BinaryPlainValuesReader();
+ checkDistinct(COUNT, bytes3, cr2, "c");
}
@Test
- public void testBinaryDictionaryInefficiency() throws IOException {
-
- int COUNT = 40000;
- ValuesWriter cw = new PlainBinaryDictionaryValuesWriter(2000000, 10000);
- for (int i = 0; i < COUNT; i++) {
- cw.writeBytes(Binary.fromString("a" + i ));
- }
- final BytesInput bytes1 = BytesInput.copy(cw.getBytes());
- final Encoding encoding1 = cw.getEncoding();
- System.out.println(PrimitiveTypeName.BINARY.name() + " " + encoding1 + " " + bytes1.size());
- cw.reset();
- for (int i = 0; i < COUNT; i++) {
- cw.writeBytes(Binary.fromString("b" + i ));
- }
- final BytesInput bytes2 = BytesInput.copy(cw.getBytes());
- final Encoding encoding2 = cw.getEncoding();
- System.out.println(PrimitiveTypeName.BINARY.name() + " " + encoding2 + " " + bytes2.size());
+ public void testBinaryDictionaryFallBack() throws IOException {
+ int slabSize = 100;
+ int maxDictionaryByteSize = 50;
+ final DictionaryValuesWriter cw = new PlainBinaryDictionaryValuesWriter(maxDictionaryByteSize, slabSize);
+ int fallBackThreshold = maxDictionaryByteSize;
+ int dataSize=0;
+ for (long i = 0; i < 100; i++) {
+ Binary binary = Binary.fromString("str" + i);
+ cw.writeBytes(binary);
+ dataSize+=(binary.length()+4);
+ if (dataSize < fallBackThreshold) {
+ assertEquals( PLAIN_DICTIONARY,cw.getEncoding());
+ } else {
+ assertEquals(PLAIN,cw.getEncoding());
+ }
+ }
+
+ //Fallbacked to Plain encoding, therefore use PlainValuesReader to read it back
+ ValuesReader reader = new BinaryPlainValuesReader();
+ reader.initFromPage(100, cw.getBytes().toByteArray(), 0);
+
+ for (long i = 0; i < 100; i++) {
+ assertEquals(Binary.fromString("str" + i), reader.readBytes());
+ }
+
+ //simulate cutting the page
cw.reset();
+ assertEquals(0,cw.getBufferedSize());
+ }
- final DictionaryPage dictionaryPage = cw.createDictionaryPage();
- Dictionary dictionary = null;
- long dictPageSize = 0;
- ValuesReader cr;
- if (dictionaryPage != null) {
- dictPageSize = dictionaryPage.getBytes().size();
- System.out.println(PrimitiveTypeName.BINARY.name() + " dict byte size: " + dictPageSize);
- final ColumnDescriptor descriptor = new ColumnDescriptor(new String[] {"foo"}, PrimitiveTypeName.BINARY, 0, 0);
- dictionary = Encoding.PLAIN_DICTIONARY.initDictionary(descriptor, dictionaryPage);
- cr = new DictionaryValuesReader(dictionary);
- } else {
- cr = new BinaryPlainValuesReader();
- }
+ @Test
+ public void testFirstPageFallBack() throws IOException {
+ int COUNT = 1000;
+ ValuesWriter cw = new PlainBinaryDictionaryValuesWriter(10000, 10000);
+ writeDistinct(COUNT, cw, "a");
+ // not efficient so falls back
+ BytesInput bytes1 = getBytesAndCheckEncoding(cw, PLAIN);
+ writeRepeated(COUNT, cw, "b");
+ // still plain because we fell back on first page
+ BytesInput bytes2 = getBytesAndCheckEncoding(cw, PLAIN);
- cr.initFromPage(COUNT, bytes1.toByteArray(), 0);
- for (int i = 0; i < COUNT; i++) {
- final String str = cr.readBytes().toStringUsingUTF8();
- Assert.assertEquals("a" + i, str);
- }
+ ValuesReader cr = new BinaryPlainValuesReader();
+ checkDistinct(COUNT, bytes1, cr, "a");
+ checkRepeated(COUNT, bytes2, cr, "b");
- if (dictionary != null && encoding2 == Encoding.PLAIN_DICTIONARY) {
- cr = new DictionaryValuesReader(dictionary);
- } else {
- cr = new BinaryPlainValuesReader();
- }
- cr.initFromPage(COUNT, bytes2.toByteArray(), 0);
- for (int i = 0; i < COUNT; i++) {
- final String str = cr.readBytes().toStringUsingUTF8();
- Assert.assertEquals("b" + i, str);
- }
-
- assertTrue(bytes1.size() < bytes2.size()); // encoded int values smaller (w/o considering dictionary size)
- assertEquals(dictPageSize, bytes2.size()); // but dictionary is same size as full plain when no repeated values
+ }
+ @Test
+ public void testSecondPageFallBack() throws IOException {
+
+ int COUNT = 1000;
+ ValuesWriter cw = new PlainBinaryDictionaryValuesWriter(1000, 10000);
+ writeRepeated(COUNT, cw, "a");
+ BytesInput bytes1 = getBytesAndCheckEncoding(cw, PLAIN_DICTIONARY);
+ writeDistinct(COUNT, cw, "b");
+ // not efficient so falls back
+ BytesInput bytes2 = getBytesAndCheckEncoding(cw, PLAIN);
+ writeRepeated(COUNT, cw, "a");
+ // still plain because we fell back on previous page
+ BytesInput bytes3 = getBytesAndCheckEncoding(cw, PLAIN);
+
+ ValuesReader cr = initDicReader(cw, BINARY);
+ checkRepeated(COUNT, bytes1, cr, "a");
+ cr = new BinaryPlainValuesReader();
+ checkDistinct(COUNT, bytes2, cr, "b");
+ checkRepeated(COUNT, bytes3, cr, "a");
}
-
+
@Test
public void testLongDictionary() throws IOException {
-
+
int COUNT = 1000;
int COUNT2 = 2000;
final DictionaryValuesWriter cw = new PlainLongDictionaryValuesWriter(10000, 10000);
-
for (long i = 0; i < COUNT; i++) {
cw.writeLong(i % 50);
}
- assertEquals(PLAIN_DICTIONARY, cw.getEncoding());
+ BytesInput bytes1 = getBytesAndCheckEncoding(cw, PLAIN_DICTIONARY);
assertEquals(50, cw.getDictionarySize());
-
- final BytesInput bytes1 = BytesInput.copy(cw.getBytes());
- cw.reset();
-
+
for (long i = COUNT2; i > 0; i--) {
cw.writeLong(i % 50);
}
- assertEquals(PLAIN_DICTIONARY, cw.getEncoding());
+ BytesInput bytes2 = getBytesAndCheckEncoding(cw, PLAIN_DICTIONARY);
assertEquals(50, cw.getDictionarySize());
- final BytesInput bytes2 = BytesInput.copy(cw.getBytes());
- cw.reset();
- final DictionaryPage dictionaryPage = cw.createDictionaryPage().copy();
- final ColumnDescriptor descriptor = new ColumnDescriptor(new String[] {"along"}, PrimitiveTypeName.INT64, 0, 0);
- final Dictionary dictionary = PLAIN_DICTIONARY.initDictionary(descriptor, dictionaryPage);
- final DictionaryValuesReader cr = new DictionaryValuesReader(dictionary);
+ DictionaryValuesReader cr = initDicReader(cw, PrimitiveTypeName.INT64);
cr.initFromPage(COUNT, bytes1.toByteArray(), 0);
for (long i = 0; i < COUNT; i++) {
@@ -180,82 +166,64 @@ public void testLongDictionary() throws IOException {
long back = cr.readLong();
assertEquals(i % 50, back);
}
-
}
- @Test
- public void testLongDictionaryInefficiency() throws IOException {
-
- int COUNT = 50000;
- final DictionaryValuesWriter cw = new PlainLongDictionaryValuesWriter(2000000, 10000);
- for (long i = 0; i < COUNT; i++) {
+ private void roundTripLong(DictionaryValuesWriter cw, ValuesReader reader, int maxDictionaryByteSize) throws IOException {
+ int fallBackThreshold = maxDictionaryByteSize / 8;
+ for (long i = 0; i < 100; i++) {
cw.writeLong(i);
- }
-
- final BytesInput bytes1 = BytesInput.copy(cw.getBytes());
- final Encoding encoding1 = cw.getEncoding();
- System.out.println(PrimitiveTypeName.INT64.name() + " " + encoding1 + " " + bytes1.size());
-
- final PlainValuesWriter pw = new PlainValuesWriter(64 * 1024);
- for (long i = 0; i < COUNT; i++) {
- pw.writeLong(i);
- }
- final BytesInput bytes2 = pw.getBytes();
- System.out.println(PrimitiveTypeName.INT64.name() + " " + pw.getEncoding() + " " + bytes2.size());
-
- final DictionaryPage dictionaryPage = cw.createDictionaryPage();
- Dictionary dictionary = null;
- long dictPageSize = 0;
- ValuesReader cr;
- if (dictionaryPage != null) {
- dictPageSize = dictionaryPage.getBytes().size();
- System.out.println(PrimitiveTypeName.INT64.name() + " dict byte size: " + dictPageSize);
- final ColumnDescriptor descriptor = new ColumnDescriptor(new String[] {"along"}, PrimitiveTypeName.INT64, 0, 0);
- dictionary = Encoding.PLAIN_DICTIONARY.initDictionary(descriptor, dictionaryPage);
- cr = new DictionaryValuesReader(dictionary);
- } else {
- cr = new LongPlainValuesReader();
+ if (i < fallBackThreshold) {
+ assertEquals(cw.getEncoding(), PLAIN_DICTIONARY);
+ } else {
+ assertEquals(cw.getEncoding(), PLAIN);
+ }
}
- cr.initFromPage(COUNT, bytes1.toByteArray(), 0);
- for (long i = 0; i < COUNT; i++) {
- long back = cr.readLong();
- assertEquals(i, back);
+ reader.initFromPage(100, cw.getBytes().toByteArray(), 0);
+
+ for (long i = 0; i < 100; i++) {
+ assertEquals(i, reader.readLong());
}
-
- assertTrue(bytes1.size() < bytes2.size()); // encoded int values smaller (w/o considering dictionary size)
- assertEquals(dictPageSize, bytes2.size()); // but dictionary is same size as full plain when no repeated values
+ }
+ @Test
+ public void testLongDictionaryFallBack() throws IOException {
+ int slabSize = 100;
+ int maxDictionaryByteSize = 50;
+ final DictionaryValuesWriter cw = new PlainLongDictionaryValuesWriter(maxDictionaryByteSize, slabSize);
+ // Fallbacked to Plain encoding, therefore use PlainValuesReader to read it back
+ ValuesReader reader = new PlainValuesReader.LongPlainValuesReader();
+
+ roundTripLong(cw, reader, maxDictionaryByteSize);
+ //simulate cutting the page
+ cw.reset();
+ assertEquals(0,cw.getBufferedSize());
+ cw.resetDictionary();
+
+ roundTripLong(cw, reader, maxDictionaryByteSize);
}
-
+
@Test
public void testDoubleDictionary() throws IOException {
-
+
int COUNT = 1000;
int COUNT2 = 2000;
final DictionaryValuesWriter cw = new PlainDoubleDictionaryValuesWriter(10000, 10000);
-
+
for (double i = 0; i < COUNT; i++) {
cw.writeDouble(i % 50);
}
- assertEquals(PLAIN_DICTIONARY, cw.getEncoding());
+
+ BytesInput bytes1 = getBytesAndCheckEncoding(cw, PLAIN_DICTIONARY);
assertEquals(50, cw.getDictionarySize());
-
- final BytesInput bytes1 = BytesInput.copy(cw.getBytes());
- cw.reset();
-
+
for (double i = COUNT2; i > 0; i--) {
cw.writeDouble(i % 50);
}
- assertEquals(PLAIN_DICTIONARY, cw.getEncoding());
+ BytesInput bytes2 = getBytesAndCheckEncoding(cw, PLAIN_DICTIONARY);
assertEquals(50, cw.getDictionarySize());
- final BytesInput bytes2 = BytesInput.copy(cw.getBytes());
- cw.reset();
- final DictionaryPage dictionaryPage = cw.createDictionaryPage().copy();
- final ColumnDescriptor descriptor = new ColumnDescriptor(new String[] {"adouble"}, PrimitiveTypeName.DOUBLE, 0, 0);
- final Dictionary dictionary = PLAIN_DICTIONARY.initDictionary(descriptor, dictionaryPage);
- final DictionaryValuesReader cr = new DictionaryValuesReader(dictionary);
+ final DictionaryValuesReader cr = initDicReader(cw, DOUBLE);
cr.initFromPage(COUNT, bytes1.toByteArray(), 0);
for (double i = 0; i < COUNT; i++) {
@@ -271,79 +239,62 @@ public void testDoubleDictionary() throws IOException {
}
- @Test
- public void testDoubleDictionaryInefficiency() throws IOException {
-
- int COUNT = 30000;
- final DictionaryValuesWriter cw = new PlainDoubleDictionaryValuesWriter(2000000, 10000);
- for (double i = 0; i < COUNT; i++) {
+ private void roundTripDouble(DictionaryValuesWriter cw, ValuesReader reader, int maxDictionaryByteSize) throws IOException {
+ int fallBackThreshold = maxDictionaryByteSize / 8;
+ for (double i = 0; i < 100; i++) {
cw.writeDouble(i);
- }
-
- final BytesInput bytes1 = BytesInput.copy(cw.getBytes());
- final Encoding encoding1 = cw.getEncoding();
- System.out.println(PrimitiveTypeName.DOUBLE.name() + " " + encoding1 + " " + bytes1.size());
-
- final PlainValuesWriter pw = new PlainValuesWriter(64 * 1024);
- for (double i = 0; i < COUNT; i++) {
- pw.writeDouble(i);
- }
- final BytesInput bytes2 = pw.getBytes();
- System.out.println(PrimitiveTypeName.DOUBLE.name() + " " + pw.getEncoding() + " " + bytes2.size());
-
- final DictionaryPage dictionaryPage = cw.createDictionaryPage();
- Dictionary dictionary = null;
- long dictPageSize = 0;
- ValuesReader cr;
- if (dictionaryPage != null) {
- dictPageSize = dictionaryPage.getBytes().size();
- System.out.println(PrimitiveTypeName.DOUBLE.name() + " dict byte size: " + dictPageSize);
- final ColumnDescriptor descriptor = new ColumnDescriptor(new String[] {"adouble"}, PrimitiveTypeName.DOUBLE, 0, 0);
- dictionary = Encoding.PLAIN_DICTIONARY.initDictionary(descriptor, dictionaryPage);
- cr = new DictionaryValuesReader(dictionary);
- } else {
- cr = new DoublePlainValuesReader();
+ if (i < fallBackThreshold) {
+ assertEquals(cw.getEncoding(), PLAIN_DICTIONARY);
+ } else {
+ assertEquals(cw.getEncoding(), PLAIN);
+ }
}
- cr.initFromPage(COUNT, bytes1.toByteArray(), 0);
- for (double i = 0; i < COUNT; i++) {
- double back = cr.readDouble();
- assertEquals(i, back, 0.0);
- }
-
- assertTrue(bytes1.size() < bytes2.size()); // encoded int values smaller (w/o considering dictionary size)
- assertEquals(dictPageSize, bytes2.size()); // but dictionary is same size as full plain when no repeated values
+ reader.initFromPage(100, cw.getBytes().toByteArray(), 0);
+ for (double i = 0; i < 100; i++) {
+ assertEquals(i, reader.readDouble(), 0.00001);
+ }
}
@Test
- public void testIntDictionary() throws IOException {
+ public void testDoubleDictionaryFallBack() throws IOException {
+ int slabSize = 100;
+ int maxDictionaryByteSize = 50;
+ final DictionaryValuesWriter cw = new PlainDoubleDictionaryValuesWriter(maxDictionaryByteSize, slabSize);
+
+ // Fallbacked to Plain encoding, therefore use PlainValuesReader to read it back
+ ValuesReader reader = new PlainValuesReader.DoublePlainValuesReader();
+ roundTripDouble(cw, reader, maxDictionaryByteSize);
+ //simulate cutting the page
+ cw.reset();
+ assertEquals(0,cw.getBufferedSize());
+ cw.resetDictionary();
+
+ roundTripDouble(cw, reader, maxDictionaryByteSize);
+ }
+
+ @Test
+ public void testIntDictionary() throws IOException {
+
int COUNT = 2000;
int COUNT2 = 4000;
final DictionaryValuesWriter cw = new PlainIntegerDictionaryValuesWriter(10000, 10000);
-
+
for (int i = 0; i < COUNT; i++) {
cw.writeInteger(i % 50);
}
- assertEquals(PLAIN_DICTIONARY, cw.getEncoding());
+ BytesInput bytes1 = getBytesAndCheckEncoding(cw, PLAIN_DICTIONARY);
assertEquals(50, cw.getDictionarySize());
-
- final BytesInput bytes1 = BytesInput.copy(cw.getBytes());
- cw.reset();
-
+
for (int i = COUNT2; i > 0; i--) {
cw.writeInteger(i % 50);
}
- assertEquals(PLAIN_DICTIONARY, cw.getEncoding());
+ BytesInput bytes2 = getBytesAndCheckEncoding(cw, PLAIN_DICTIONARY);
assertEquals(50, cw.getDictionarySize());
- final BytesInput bytes2 = BytesInput.copy(cw.getBytes());
- cw.reset();
- final DictionaryPage dictionaryPage = cw.createDictionaryPage().copy();
- final ColumnDescriptor descriptor = new ColumnDescriptor(new String[] {"anint"}, PrimitiveTypeName.INT32, 0, 0);
- final Dictionary dictionary = PLAIN_DICTIONARY.initDictionary(descriptor, dictionaryPage);
- final DictionaryValuesReader cr = new DictionaryValuesReader(dictionary);
+ DictionaryValuesReader cr = initDicReader(cw, INT32);
cr.initFromPage(COUNT, bytes1.toByteArray(), 0);
for (int i = 0; i < COUNT; i++) {
@@ -359,79 +310,62 @@ public void testIntDictionary() throws IOException {
}
- @Test
- public void testIntDictionaryInefficiency() throws IOException {
-
- int COUNT = 20000;
- final DictionaryValuesWriter cw = new PlainIntegerDictionaryValuesWriter(2000000, 10000);
- for (int i = 0; i < COUNT; i++) {
+ private void roundTripInt(DictionaryValuesWriter cw, ValuesReader reader, int maxDictionaryByteSize) throws IOException {
+ int fallBackThreshold = maxDictionaryByteSize / 4;
+ for (int i = 0; i < 100; i++) {
cw.writeInteger(i);
- }
-
- final BytesInput bytes1 = BytesInput.copy(cw.getBytes());
- final Encoding encoding1 = cw.getEncoding();
- System.out.println(PrimitiveTypeName.INT32.name() + " " + encoding1 + " " + bytes1.size());
-
- final PlainValuesWriter pw = new PlainValuesWriter(64 * 1024);
- for (int i = 0; i < COUNT; i++) {
- pw.writeInteger(i);
- }
- final BytesInput bytes2 = pw.getBytes();
- System.out.println(PrimitiveTypeName.INT32.name() + " " + pw.getEncoding() + " " + bytes2.size());
-
- final DictionaryPage dictionaryPage = cw.createDictionaryPage();
- Dictionary dictionary = null;
- long dictPageSize = 0;
- ValuesReader cr;
- if (dictionaryPage != null) {
- dictPageSize = dictionaryPage.getBytes().size();
- System.out.println(PrimitiveTypeName.INT32.name() + " dict byte size: " + dictPageSize);
- final ColumnDescriptor descriptor = new ColumnDescriptor(new String[] {"anint"}, PrimitiveTypeName.INT32, 0, 0);
- dictionary = Encoding.PLAIN_DICTIONARY.initDictionary(descriptor, dictionaryPage);
- cr = new DictionaryValuesReader(dictionary);
- } else {
- cr = new IntegerPlainValuesReader();
+ if (i < fallBackThreshold) {
+ assertEquals(cw.getEncoding(), PLAIN_DICTIONARY);
+ } else {
+ assertEquals(cw.getEncoding(), PLAIN);
+ }
}
- cr.initFromPage(COUNT, bytes1.toByteArray(), 0);
- for (int i = 0; i < COUNT; i++) {
- int back = cr.readInteger();
- assertEquals(i, back);
- }
-
- assertTrue(bytes1.size() < bytes2.size()); // encoded int values smaller (w/o considering dictionary size)
- assertEquals(dictPageSize, bytes2.size()); // but dictionary is same size as full plain when no repeated values
+ reader.initFromPage(100, cw.getBytes().toByteArray(), 0);
+ for (int i = 0; i < 100; i++) {
+ assertEquals(i, reader.readInteger());
+ }
}
@Test
- public void testFloatDictionary() throws IOException {
+ public void testIntDictionaryFallBack() throws IOException {
+ int slabSize = 100;
+ int maxDictionaryByteSize = 50;
+ final DictionaryValuesWriter cw = new PlainIntegerDictionaryValuesWriter(maxDictionaryByteSize, slabSize);
+ // Fallbacked to Plain encoding, therefore use PlainValuesReader to read it back
+ ValuesReader reader = new PlainValuesReader.IntegerPlainValuesReader();
+
+ roundTripInt(cw, reader, maxDictionaryByteSize);
+ //simulate cutting the page
+ cw.reset();
+ assertEquals(0,cw.getBufferedSize());
+ cw.resetDictionary();
+
+ roundTripInt(cw, reader, maxDictionaryByteSize);
+ }
+
+ @Test
+ public void testFloatDictionary() throws IOException {
+
int COUNT = 2000;
int COUNT2 = 4000;
final DictionaryValuesWriter cw = new PlainFloatDictionaryValuesWriter(10000, 10000);
-
+
for (float i = 0; i < COUNT; i++) {
cw.writeFloat(i % 50);
}
- assertEquals(PLAIN_DICTIONARY, cw.getEncoding());
+ BytesInput bytes1 = getBytesAndCheckEncoding(cw, PLAIN_DICTIONARY);
assertEquals(50, cw.getDictionarySize());
-
- final BytesInput bytes1 = BytesInput.copy(cw.getBytes());
- cw.reset();
-
+
for (float i = COUNT2; i > 0; i--) {
cw.writeFloat(i % 50);
}
- assertEquals(PLAIN_DICTIONARY, cw.getEncoding());
+ BytesInput bytes2 = getBytesAndCheckEncoding(cw, PLAIN_DICTIONARY);
assertEquals(50, cw.getDictionarySize());
- final BytesInput bytes2 = BytesInput.copy(cw.getBytes());
- cw.reset();
- final DictionaryPage dictionaryPage = cw.createDictionaryPage().copy();
- final ColumnDescriptor descriptor = new ColumnDescriptor(new String[] {"afloat"}, PrimitiveTypeName.FLOAT, 0, 0);
- final Dictionary dictionary = PLAIN_DICTIONARY.initDictionary(descriptor, dictionaryPage);
- final DictionaryValuesReader cr = new DictionaryValuesReader(dictionary);
+ DictionaryValuesReader cr = initDicReader(cw, FLOAT);
cr.initFromPage(COUNT, bytes1.toByteArray(), 0);
for (float i = 0; i < COUNT; i++) {
@@ -447,48 +381,82 @@ public void testFloatDictionary() throws IOException {
}
- @Test
- public void testFloatDictionaryInefficiency() throws IOException {
-
- int COUNT = 60000;
- final DictionaryValuesWriter cw = new PlainFloatDictionaryValuesWriter(2000000, 10000);
- for (float i = 0; i < COUNT; i++) {
+ private void roundTripFloat(DictionaryValuesWriter cw, ValuesReader reader, int maxDictionaryByteSize) throws IOException {
+ int fallBackThreshold = maxDictionaryByteSize / 4;
+ for (float i = 0; i < 100; i++) {
cw.writeFloat(i);
+ if (i < fallBackThreshold) {
+ assertEquals(cw.getEncoding(), PLAIN_DICTIONARY);
+ } else {
+ assertEquals(cw.getEncoding(), PLAIN);
+ }
+ }
+
+ reader.initFromPage(100, cw.getBytes().toByteArray(), 0);
+
+ for (float i = 0; i < 100; i++) {
+ assertEquals(i, reader.readFloat(), 0.00001);
}
+ }
+
+ @Test
+ public void testFloatDictionaryFallBack() throws IOException {
+ int slabSize = 100;
+ int maxDictionaryByteSize = 50;
+ final DictionaryValuesWriter cw = new PlainFloatDictionaryValuesWriter(maxDictionaryByteSize, slabSize);
- final BytesInput bytes1 = BytesInput.copy(cw.getBytes());
- final Encoding encoding1 = cw.getEncoding();
- System.out.println(PrimitiveTypeName.FLOAT.name() + " " + encoding1 + " " + bytes1.size());
+ // Fallbacked to Plain encoding, therefore use PlainValuesReader to read it back
+ ValuesReader reader = new PlainValuesReader.FloatPlainValuesReader();
- final PlainValuesWriter pw = new PlainValuesWriter(64 * 1024);
- for (float i = 0; i < COUNT; i++) {
- pw.writeFloat(i);
+ roundTripFloat(cw, reader, maxDictionaryByteSize);
+ //simulate cutting the page
+ cw.reset();
+ assertEquals(0,cw.getBufferedSize());
+ cw.resetDictionary();
+
+ roundTripFloat(cw, reader, maxDictionaryByteSize);
+ }
+
+ private DictionaryValuesReader initDicReader(ValuesWriter cw, PrimitiveTypeName type)
+ throws IOException {
+ final DictionaryPage dictionaryPage = cw.createDictionaryPage().copy();
+ final ColumnDescriptor descriptor = new ColumnDescriptor(new String[] {"foo"}, type, 0, 0);
+ final Dictionary dictionary = PLAIN_DICTIONARY.initDictionary(descriptor, dictionaryPage);
+ final DictionaryValuesReader cr = new DictionaryValuesReader(dictionary);
+ return cr;
+ }
+
+ private void checkDistinct(int COUNT, BytesInput bytes, ValuesReader cr, String prefix) throws IOException {
+ cr.initFromPage(COUNT, bytes.toByteArray(), 0);
+ for (int i = 0; i < COUNT; i++) {
+ Assert.assertEquals(prefix + i, cr.readBytes().toStringUsingUTF8());
}
- final BytesInput bytes2 = pw.getBytes();
- System.out.println(PrimitiveTypeName.FLOAT.name() + " " + pw.getEncoding() + " " + bytes2.size());
-
- final DictionaryPage dictionaryPage = cw.createDictionaryPage();
- Dictionary dictionary = null;
- long dictPageSize = 0;
- ValuesReader cr;
- if (dictionaryPage != null) {
- dictPageSize = dictionaryPage.getBytes().size();
- System.out.println(PrimitiveTypeName.FLOAT.name() + " dict byte size: " + dictPageSize);
- final ColumnDescriptor descriptor = new ColumnDescriptor(new String[] {"afloat"}, PrimitiveTypeName.FLOAT, 0, 0);
- dictionary = Encoding.PLAIN_DICTIONARY.initDictionary(descriptor, dictionaryPage);
- cr = new DictionaryValuesReader(dictionary);
- } else {
- cr = new FloatPlainValuesReader();
+ }
+
+ private void checkRepeated(int COUNT, BytesInput bytes, ValuesReader cr, String prefix) throws IOException {
+ cr.initFromPage(COUNT, bytes.toByteArray(), 0);
+ for (int i = 0; i < COUNT; i++) {
+ Assert.assertEquals(prefix + i % 10, cr.readBytes().toStringUsingUTF8());
}
+ }
- cr.initFromPage(COUNT, bytes1.toByteArray(), 0);
- for (float i = 0; i < COUNT; i++) {
- float back = cr.readFloat();
- assertEquals(i, back, 0.0f);
+ private void writeDistinct(int COUNT, ValuesWriter cw, String prefix) {
+ for (int i = 0; i < COUNT; i++) {
+ cw.writeBytes(Binary.fromString(prefix + i));
}
-
- assertTrue(bytes1.size() < bytes2.size()); // encoded int values smaller (w/o considering dictionary size)
- assertEquals(dictPageSize, bytes2.size()); // but dictionary is same size as full plain when no repeated values
+ }
+
+ private void writeRepeated(int COUNT, ValuesWriter cw, String prefix) {
+ for (int i = 0; i < COUNT; i++) {
+ cw.writeBytes(Binary.fromString(prefix + i % 10));
+ }
+ }
+ private BytesInput getBytesAndCheckEncoding(ValuesWriter cw, Encoding encoding)
+ throws IOException {
+ BytesInput bytes = BytesInput.copy(cw.getBytes());
+ assertEquals(encoding, cw.getEncoding());
+ cw.reset();
+ return bytes;
}
}
diff --git a/parquet-column/src/test/java/parquet/column/values/rle/RunLengthBitPackingHybridIntegrationTest.java b/parquet-column/src/test/java/parquet/column/values/rle/RunLengthBitPackingHybridIntegrationTest.java
index 6743610871..2359d8de72 100644
--- a/parquet-column/src/test/java/parquet/column/values/rle/RunLengthBitPackingHybridIntegrationTest.java
+++ b/parquet-column/src/test/java/parquet/column/values/rle/RunLengthBitPackingHybridIntegrationTest.java
@@ -67,9 +67,10 @@ private void doIntegrationTest(int bitWidth) throws Exception {
}
numValues += 1000;
- InputStream in = new ByteArrayInputStream(encoder.toBytes().toByteArray());
+ byte[] encodedBytes = encoder.toBytes().toByteArray();
+ ByteArrayInputStream in = new ByteArrayInputStream(encodedBytes);
- RunLengthBitPackingHybridDecoder decoder = new RunLengthBitPackingHybridDecoder(numValues, bitWidth, in);
+ RunLengthBitPackingHybridDecoder decoder = new RunLengthBitPackingHybridDecoder(bitWidth, in);
for (int i = 0; i < 100; i++) {
assertEquals(i % modValue, decoder.readInt());
diff --git a/parquet-column/src/test/java/parquet/column/values/rle/TestRunLengthBitPackingHybridEncoder.java b/parquet-column/src/test/java/parquet/column/values/rle/TestRunLengthBitPackingHybridEncoder.java
index 29930bb342..0859cb10d1 100644
--- a/parquet-column/src/test/java/parquet/column/values/rle/TestRunLengthBitPackingHybridEncoder.java
+++ b/parquet-column/src/test/java/parquet/column/values/rle/TestRunLengthBitPackingHybridEncoder.java
@@ -285,7 +285,7 @@ public void testGroupBoundary() throws Exception {
bytes[0] = (1 << 1 )| 1;
bytes[1] = (1 << 0) | (2 << 2) | (3 << 4);
ByteArrayInputStream stream = new ByteArrayInputStream(bytes);
- RunLengthBitPackingHybridDecoder decoder = new RunLengthBitPackingHybridDecoder(3, 2, stream);
+ RunLengthBitPackingHybridDecoder decoder = new RunLengthBitPackingHybridDecoder(2, stream);
assertEquals(decoder.readInt(), 1);
assertEquals(decoder.readInt(), 2);
assertEquals(decoder.readInt(), 3);
diff --git a/parquet-column/src/test/java/parquet/io/PerfTest.java b/parquet-column/src/test/java/parquet/io/PerfTest.java
index c08d689407..da46b51a56 100644
--- a/parquet-column/src/test/java/parquet/io/PerfTest.java
+++ b/parquet-column/src/test/java/parquet/io/PerfTest.java
@@ -24,6 +24,7 @@
import java.util.logging.Level;
import parquet.Log;
+import parquet.column.ParquetProperties.WriterVersion;
import parquet.column.impl.ColumnWriteStoreImpl;
import parquet.column.page.mem.MemPageStore;
import parquet.example.DummyRecordConverter;
@@ -73,7 +74,7 @@ private static void read(MemPageStore memPageStore, MessageType myschema,
private static void write(MemPageStore memPageStore) {
- ColumnWriteStoreImpl columns = new ColumnWriteStoreImpl(memPageStore, 50*1024*1024, 50*1024*1024, false);
+ ColumnWriteStoreImpl columns = new ColumnWriteStoreImpl(memPageStore, 50*1024*1024, 50*1024*1024, 50*1024*1024, false, WriterVersion.PARQUET_1_0);
MessageColumnIO columnIO = newColumnFactory(schema);
GroupWriter groupWriter = new GroupWriter(columnIO.getRecordWriter(columns), schema);
diff --git a/parquet-column/src/test/java/parquet/io/TestColumnIO.java b/parquet-column/src/test/java/parquet/io/TestColumnIO.java
index e0e41111f3..15e4ce32ec 100644
--- a/parquet-column/src/test/java/parquet/io/TestColumnIO.java
+++ b/parquet-column/src/test/java/parquet/io/TestColumnIO.java
@@ -40,6 +40,8 @@
import parquet.column.ColumnDescriptor;
import parquet.column.ColumnWriteStore;
import parquet.column.ColumnWriter;
+import parquet.column.ParquetProperties;
+import parquet.column.ParquetProperties.WriterVersion;
import parquet.column.impl.ColumnWriteStoreImpl;
import parquet.column.page.PageReadStore;
import parquet.column.page.mem.MemPageStore;
@@ -260,7 +262,7 @@ private List readGroups(MemPageStore memPageStore, MessageType fileSchema
private void writeGroups(MessageType writtenSchema, MemPageStore memPageStore, Group... groups) {
ColumnIOFactory columnIOFactory = new ColumnIOFactory(true);
- ColumnWriteStoreImpl columns = new ColumnWriteStoreImpl(memPageStore, 800, 800, false);
+ ColumnWriteStoreImpl columns = newColumnWriteStore(memPageStore);
MessageColumnIO columnIO = columnIOFactory.getColumnIO(writtenSchema);
GroupWriter groupWriter = new GroupWriter(columnIO.getRecordWriter(columns), writtenSchema);
for (Group group : groups) {
@@ -278,7 +280,7 @@ public void testColumnIO() {
log(r2);
MemPageStore memPageStore = new MemPageStore(2);
- ColumnWriteStoreImpl columns = new ColumnWriteStoreImpl(memPageStore, 800, 800, false);
+ ColumnWriteStoreImpl columns = newColumnWriteStore(memPageStore);
ColumnIOFactory columnIOFactory = new ColumnIOFactory(true);
{
@@ -426,15 +428,14 @@ public void testOptionalRequiredInteraction() {
private void testSchema(MessageType messageSchema, List groups) {
MemPageStore memPageStore = new MemPageStore(groups.size());
- ColumnWriteStoreImpl columns =
- new ColumnWriteStoreImpl(memPageStore, 800, 800, false);
+ ColumnWriteStoreImpl columns = newColumnWriteStore(memPageStore);
ColumnIOFactory columnIOFactory = new ColumnIOFactory(true);
MessageColumnIO columnIO = columnIOFactory.getColumnIO(messageSchema);
log(columnIO);
-
+
// Write groups.
- GroupWriter groupWriter =
+ GroupWriter groupWriter =
new GroupWriter(columnIO.getRecordWriter(columns), messageSchema);
for (Group group : groups) {
groupWriter.write(group);
@@ -442,11 +443,11 @@ private void testSchema(MessageType messageSchema, List groups) {
columns.flush();
// Read groups and verify.
- RecordReaderImplementation recordReader =
+ RecordReaderImplementation recordReader =
getRecordReader(columnIO, messageSchema, memPageStore);
for (Group group : groups) {
final Group got = recordReader.read();
- assertEquals("deserialization does not display the same result",
+ assertEquals("deserialization does not display the same result",
group.toString(), got.toString());
}
}
@@ -479,7 +480,7 @@ private void validateFSA(int[][] expectedFSA, MessageColumnIO columnIO, RecordRe
@Test
public void testPushParser() {
MemPageStore memPageStore = new MemPageStore(1);
- ColumnWriteStoreImpl columns = new ColumnWriteStoreImpl(memPageStore, 800, 800, false);
+ ColumnWriteStoreImpl columns = newColumnWriteStore(memPageStore);
MessageColumnIO columnIO = new ColumnIOFactory().getColumnIO(schema);
new GroupWriter(columnIO.getRecordWriter(columns), schema).write(r1);
columns.flush();
@@ -489,10 +490,14 @@ public void testPushParser() {
}
+ private ColumnWriteStoreImpl newColumnWriteStore(MemPageStore memPageStore) {
+ return new ColumnWriteStoreImpl(memPageStore, 800, 800, 800, false, WriterVersion.PARQUET_1_0);
+ }
+
@Test
public void testEmptyField() {
MemPageStore memPageStore = new MemPageStore(1);
- ColumnWriteStoreImpl columns = new ColumnWriteStoreImpl(memPageStore, 800, 800, false);
+ ColumnWriteStoreImpl columns = newColumnWriteStore(memPageStore);
MessageColumnIO columnIO = new ColumnIOFactory(true).getColumnIO(schema);
final RecordConsumer recordWriter = columnIO.getRecordWriter(columns);
recordWriter.startMessage();
@@ -573,7 +578,7 @@ public void writeNull(int repetitionLevel, int definitionLevel) {
public void write(Binary value, int repetitionLevel, int definitionLevel) {
validate(value.toStringUsingUTF8(), repetitionLevel, definitionLevel);
}
-
+
@Override
public void write(boolean value, int repetitionLevel, int definitionLevel) {
validate(value, repetitionLevel, definitionLevel);
diff --git a/parquet-column/src/test/java/parquet/io/TestFiltered.java b/parquet-column/src/test/java/parquet/io/TestFiltered.java
index 3846d328e3..66fe6a0248 100644
--- a/parquet-column/src/test/java/parquet/io/TestFiltered.java
+++ b/parquet-column/src/test/java/parquet/io/TestFiltered.java
@@ -15,33 +15,52 @@
*/
package parquet.io;
+import static org.junit.Assert.assertEquals;
+import static parquet.example.Paper.r1;
+import static parquet.example.Paper.r2;
+import static parquet.example.Paper.schema;
+import static parquet.filter.AndRecordFilter.and;
+import static parquet.filter.ColumnPredicates.applyFunctionToLong;
+import static parquet.filter.ColumnPredicates.applyFunctionToString;
+import static parquet.filter.ColumnPredicates.equalTo;
+import static parquet.filter.ColumnRecordFilter.column;
+import static parquet.filter.NotRecordFilter.not;
+import static parquet.filter.OrRecordFilter.or;
+import static parquet.filter.PagedRecordFilter.page;
+
import java.util.ArrayList;
import java.util.List;
import org.junit.Test;
+
+import parquet.column.ParquetProperties.WriterVersion;
import parquet.column.impl.ColumnWriteStoreImpl;
import parquet.column.page.mem.MemPageStore;
import parquet.example.data.Group;
import parquet.example.data.GroupWriter;
-import parquet.example.data.simple.SimpleGroup;
import parquet.example.data.simple.convert.GroupRecordConverter;
+import parquet.filter.ColumnPredicates.LongPredicateFunction;
+import parquet.filter.ColumnPredicates.PredicateFunction;
import parquet.io.api.RecordMaterializer;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNull;
-import static parquet.example.Paper.r1;
-import static parquet.example.Paper.r2;
-import static parquet.example.Paper.schema;
-import static parquet.filter.AndRecordFilter.and;
-import static parquet.filter.OrRecordFilter.or;
-import static parquet.filter.NotRecordFilter.not;
-import static parquet.filter.PagedRecordFilter.page;
-import static parquet.filter.ColumnPredicates.equalTo;
-import static parquet.filter.ColumnRecordFilter.column;
-
public class TestFiltered {
+ /* Class that implements applyFunction filter for long. Checks for long greater than 15. */
+ public class LongGreaterThan15Predicate implements LongPredicateFunction {
+ @Override
+ public boolean functionToApply(long input) {
+ return input > 15;
+ }
+ };
+
+ /* Class that implements applyFunction filter for string. Checks for string ending in 'A'. */
+ public class StringEndsWithAPredicate implements PredicateFunction {
+ @Override
+ public boolean functionToApply(String input) {
+ return input.endsWith("A");
+ }
+ };
+
private List readAll(RecordReader reader) {
List result = new ArrayList();
Group g;
@@ -79,6 +98,27 @@ public void testFilterOnInteger() {
}
+ @Test
+ public void testApplyFunctionFilterOnLong() {
+ MessageColumnIO columnIO = new ColumnIOFactory(true).getColumnIO(schema);
+ MemPageStore memPageStore = writeTestRecords(columnIO, 1);
+
+ // Get first record
+ RecordMaterializer recordConverter = new GroupRecordConverter(schema);
+ RecordReaderImplementation recordReader = (RecordReaderImplementation)
+ columnIO.getRecordReader(memPageStore, recordConverter,
+ column("DocId", equalTo(10l)));
+
+ readOne(recordReader, "r2 filtered out", r1);
+
+ // Get second record
+ recordReader = (RecordReaderImplementation)
+ columnIO.getRecordReader(memPageStore, recordConverter,
+ column("DocId", applyFunctionToLong (new LongGreaterThan15Predicate())));
+
+ readOne(recordReader, "r1 filtered out", r2);
+ }
+
@Test
public void testFilterOnString() {
MessageColumnIO columnIO = new ColumnIOFactory(true).getColumnIO(schema);
@@ -110,6 +150,37 @@ public void testFilterOnString() {
}
+ @Test
+ public void testApplyFunctionFilterOnString() {
+ MessageColumnIO columnIO = new ColumnIOFactory(true).getColumnIO(schema);
+ MemPageStore memPageStore = writeTestRecords(columnIO, 1);
+
+ // First try matching against the A url in record 1
+ RecordMaterializer recordConverter = new GroupRecordConverter(schema);
+ RecordReaderImplementation recordReader = (RecordReaderImplementation)
+ columnIO.getRecordReader(memPageStore, recordConverter,
+ column("Name.Url", applyFunctionToString (new StringEndsWithAPredicate ())));
+
+ readOne(recordReader, "r2 filtered out", r1);
+
+ // Second try matching against the B url in record 1 - it should fail as we only match
+ // against the first instance of a
+ recordReader = (RecordReaderImplementation)
+ columnIO.getRecordReader(memPageStore, recordConverter,
+ column("Name.Url", equalTo("http://B")));
+
+ List all = readAll(recordReader);
+ assertEquals("There should be no matching records: " + all , 0, all.size());
+
+ // Finally try matching against the C url in record 2
+ recordReader = (RecordReaderImplementation)
+ columnIO.getRecordReader(memPageStore, recordConverter,
+ column("Name.Url", equalTo("http://C")));
+
+ readOne(recordReader, "r1 filtered out", r2);
+
+ }
+
@Test
public void testPaged() {
MessageColumnIO columnIO = new ColumnIOFactory(true).getColumnIO(schema);
@@ -153,8 +224,8 @@ public void testFilteredOrPaged() {
RecordMaterializer recordConverter = new GroupRecordConverter(schema);
RecordReaderImplementation recordReader = (RecordReaderImplementation)
columnIO.getRecordReader(memPageStore, recordConverter,
- or(column("DocId", equalTo(10l)),
- column("DocId", equalTo(20l))));
+ or(column("DocId", equalTo(10l)),
+ column("DocId", equalTo(20l))));
List all = readAll(recordReader);
assertEquals("expecting 8 records " + all, 16, all.size());
@@ -172,7 +243,7 @@ public void testFilteredNotPaged() {
RecordMaterializer recordConverter = new GroupRecordConverter(schema);
RecordReaderImplementation recordReader = (RecordReaderImplementation)
columnIO.getRecordReader(memPageStore, recordConverter,
- not(column("DocId", equalTo(10l))));
+ not(column("DocId", equalTo(10l))));
List all = readAll(recordReader);
assertEquals("expecting 8 records " + all, 8, all.size());
@@ -183,7 +254,7 @@ public void testFilteredNotPaged() {
private MemPageStore writeTestRecords(MessageColumnIO columnIO, int number) {
MemPageStore memPageStore = new MemPageStore(number * 2);
- ColumnWriteStoreImpl columns = new ColumnWriteStoreImpl(memPageStore, 800, 800, false);
+ ColumnWriteStoreImpl columns = new ColumnWriteStoreImpl(memPageStore, 800, 800, 800, false, WriterVersion.PARQUET_1_0);
GroupWriter groupWriter = new GroupWriter(columnIO.getRecordWriter(columns), schema);
for ( int i = 0; i < number; i++ ) {
diff --git a/parquet-common/pom.xml b/parquet-common/pom.xml
index cacb1ef59e..63771e5a81 100644
--- a/parquet-common/pom.xml
+++ b/parquet-common/pom.xml
@@ -3,7 +3,7 @@
com.twitter
parquet
../pom.xml
- 1.2.4-SNAPSHOT
+ 1.2.11-SNAPSHOT
4.0.0
diff --git a/parquet-common/src/main/java/parquet/bytes/BytesUtils.java b/parquet-common/src/main/java/parquet/bytes/BytesUtils.java
index 1ef8bf34ac..50919f1462 100644
--- a/parquet-common/src/main/java/parquet/bytes/BytesUtils.java
+++ b/parquet-common/src/main/java/parquet/bytes/BytesUtils.java
@@ -182,6 +182,18 @@ public static int readUnsignedVarInt(InputStream in) throws IOException {
return value | (b << i);
}
+ /**
+ * uses a trick mentioned in https://developers.google.com/protocol-buffers/docs/encoding to read zigZag encoded data
+ * @param in
+ * @return
+ * @throws IOException
+ */
+ public static int readZigZagVarInt(InputStream in) throws IOException {
+ int raw = readUnsignedVarInt(in);
+ int temp = (((raw << 31) >> 31) ^ raw) >> 1;
+ return temp ^ (raw & (1 << 31));
+ }
+
public static void writeUnsignedVarInt(int value, OutputStream out) throws IOException {
while ((value & 0xFFFFFF80) != 0L) {
out.write((value & 0x7F) | 0x80);
@@ -190,6 +202,10 @@ public static void writeUnsignedVarInt(int value, OutputStream out) throws IOExc
out.write(value & 0x7F);
}
+ public static void writeZigZagVarInt(int intValue, OutputStream out) throws IOException{
+ writeUnsignedVarInt((intValue << 1) ^ (intValue >> 31), out);
+ }
+
/**
* @param bitLength a count of bits
* @return the corresponding byte count padded to the next byte
diff --git a/parquet-encoding/pom.xml b/parquet-encoding/pom.xml
index 6e686b33c8..43b7ec95bb 100644
--- a/parquet-encoding/pom.xml
+++ b/parquet-encoding/pom.xml
@@ -3,7 +3,7 @@
com.twitter
parquet
../pom.xml
- 1.2.4-SNAPSHOT
+ 1.2.11-SNAPSHOT
4.0.0
diff --git a/parquet-encoding/src/main/java/parquet/bytes/BytesInput.java b/parquet-encoding/src/main/java/parquet/bytes/BytesInput.java
index 3298a94cdd..fc8abfd51d 100644
--- a/parquet-encoding/src/main/java/parquet/bytes/BytesInput.java
+++ b/parquet-encoding/src/main/java/parquet/bytes/BytesInput.java
@@ -92,6 +92,23 @@ public static BytesInput fromInt(int intValue) {
return new IntBytesInput(intValue);
}
+ /**
+ * @param intValue the int to write
+ * @return a BytesInput that will write var int
+ */
+ public static BytesInput fromUnsignedVarInt(int intValue) {
+ return new UnsignedVarIntBytesInput(intValue);
+ }
+
+ /**
+ *
+ * @param intValue the int to write
+ */
+ public static BytesInput fromZigZagVarInt(int intValue) {
+ int zigZag = (intValue << 1) ^ (intValue >> 31);
+ return new UnsignedVarIntBytesInput(zigZag);
+ }
+
/**
* @param arrayOut
* @return a BytesInput that will write the content of the buffer
@@ -245,6 +262,26 @@ public long size() {
}
+ private static class UnsignedVarIntBytesInput extends BytesInput {
+
+ private final int intValue;
+
+ public UnsignedVarIntBytesInput(int intValue) {
+ this.intValue = intValue;
+ }
+
+ @Override
+ public void writeAllTo(OutputStream out) throws IOException {
+ BytesUtils.writeUnsignedVarInt(intValue, out);
+ }
+
+ @Override
+ public long size() {
+ int s = 5 - ((Integer.numberOfLeadingZeros(intValue) + 3) / 7);
+ return s == 0 ? 1 : s;
+ }
+ }
+
private static class EmptyBytesInput extends BytesInput {
@Override
diff --git a/parquet-generator/pom.xml b/parquet-generator/pom.xml
index 23c4395f28..c532f3134b 100644
--- a/parquet-generator/pom.xml
+++ b/parquet-generator/pom.xml
@@ -3,7 +3,7 @@
com.twitter
parquet
../pom.xml
- 1.2.4-SNAPSHOT
+ 1.2.11-SNAPSHOT
4.0.0
diff --git a/parquet-hadoop-bundle/README b/parquet-hadoop-bundle/README
new file mode 100644
index 0000000000..95f57be516
--- /dev/null
+++ b/parquet-hadoop-bundle/README
@@ -0,0 +1,2 @@
+This contains all classes required to use Parquet within a Hadoop
+environment.
diff --git a/parquet-hadoop-bundle/pom.xml b/parquet-hadoop-bundle/pom.xml
new file mode 100644
index 0000000000..dfe192c672
--- /dev/null
+++ b/parquet-hadoop-bundle/pom.xml
@@ -0,0 +1,63 @@
+
+
+ com.twitter
+ parquet
+ ../pom.xml
+ 1.2.11-SNAPSHOT
+
+
+ 4.0.0
+
+ parquet-hadoop-bundle
+ jar
+
+ Parquet Hadoop Bundle
+ https://github.com/Parquet/parquet-mr
+
+
+
+
+
+
+ com.twitter
+ parquet-common
+ ${project.version}
+
+
+ com.twitter
+ parquet-column
+ ${project.version}
+
+
+ com.twitter
+ parquet-hadoop
+ ${project.version}
+
+
+
+
+
+
+ org.apache.maven.plugins
+ maven-shade-plugin
+
+
+ package
+
+ shade
+
+
+ false
+
+
+ com.twitter:parquet-*
+
+
+
+
+
+
+
+
+
+
diff --git a/parquet-hadoop-bundle/src/main/resources/parquet/bundle b/parquet-hadoop-bundle/src/main/resources/parquet/bundle
new file mode 100644
index 0000000000..e69de29bb2
diff --git a/parquet-hadoop/pom.xml b/parquet-hadoop/pom.xml
index fb8ba4bd92..31859b096e 100644
--- a/parquet-hadoop/pom.xml
+++ b/parquet-hadoop/pom.xml
@@ -3,7 +3,7 @@
com.twitter
parquet
../pom.xml
- 1.2.4-SNAPSHOT
+ 1.2.11-SNAPSHOT
4.0.0
@@ -26,17 +26,19 @@
com.twitter
parquet-format
- 1.0.0
+ ${parquet.format.version}
org.apache.hadoop
hadoop-client
${hadoop.version}
+ provided
log4j
log4j
- 1.2.17
+ ${log4j.version}
+ provided
org.codehaus.jackson
diff --git a/parquet-hadoop/src/main/java/parquet/format/converter/ParquetMetadataConverter.java b/parquet-hadoop/src/main/java/parquet/format/converter/ParquetMetadataConverter.java
index 9148e2f0b8..8f0b0c0cf7 100644
--- a/parquet-hadoop/src/main/java/parquet/format/converter/ParquetMetadataConverter.java
+++ b/parquet-hadoop/src/main/java/parquet/format/converter/ParquetMetadataConverter.java
@@ -33,6 +33,7 @@
import java.util.Set;
import parquet.Log;
+import parquet.format.ConvertedType;
import parquet.format.ColumnChunk;
import parquet.format.DataPageHeader;
import parquet.format.DictionaryPageHeader;
@@ -53,6 +54,7 @@
import parquet.io.ParquetDecodingException;
import parquet.schema.GroupType;
import parquet.schema.MessageType;
+import parquet.schema.OriginalType;
import parquet.schema.PrimitiveType;
import parquet.schema.PrimitiveType.PrimitiveTypeName;
import parquet.schema.Type.Repetition;
@@ -97,6 +99,9 @@ public void visit(PrimitiveType primitiveType) {
SchemaElement element = new SchemaElement(primitiveType.getName());
element.setRepetition_type(toParquetRepetition(primitiveType.getRepetition()));
element.setType(getType(primitiveType.getPrimitiveTypeName()));
+ if (primitiveType.getOriginalType() != null) {
+ element.setConverted_type(getConvertedType(primitiveType.getOriginalType()));
+ }
if (primitiveType.getTypeLength() > 0) {
element.setType_length(primitiveType.getTypeLength());
}
@@ -113,6 +118,9 @@ public void visit(MessageType messageType) {
public void visit(GroupType groupType) {
SchemaElement element = new SchemaElement(groupType.getName());
element.setRepetition_type(toParquetRepetition(groupType.getRepetition()));
+ if (groupType.getOriginalType() != null) {
+ element.setConverted_type(getConvertedType(groupType.getOriginalType()));
+ }
visitChildren(result, groupType, element);
}
@@ -254,10 +262,44 @@ Type getType(PrimitiveTypeName type) {
case FIXED_LEN_BYTE_ARRAY:
return Type.FIXED_LEN_BYTE_ARRAY;
default:
- throw new RuntimeException("Unknown type " + type);
+ throw new RuntimeException("Unknown primitive type " + type);
}
}
+ OriginalType getOriginalType(ConvertedType type) {
+ switch (type) {
+ case UTF8:
+ return OriginalType.UTF8;
+ case MAP:
+ return OriginalType.MAP;
+ case MAP_KEY_VALUE:
+ return OriginalType.MAP_KEY_VALUE;
+ case LIST:
+ return OriginalType.LIST;
+ case ENUM:
+ return OriginalType.ENUM;
+ default:
+ throw new RuntimeException("Unknown converted type " + type);
+ }
+ }
+
+ ConvertedType getConvertedType(OriginalType type) {
+ switch (type) {
+ case UTF8:
+ return ConvertedType.UTF8;
+ case MAP:
+ return ConvertedType.MAP;
+ case MAP_KEY_VALUE:
+ return ConvertedType.MAP_KEY_VALUE;
+ case LIST:
+ return ConvertedType.LIST;
+ case ENUM:
+ return ConvertedType.ENUM;
+ default:
+ throw new RuntimeException("Unknown original type " + type);
+ }
+ }
+
private void addKeyValue(FileMetaData fileMetaData, String key, String value) {
KeyValue keyValue = new KeyValue(key);
keyValue.value = value;
@@ -340,23 +382,32 @@ private parquet.schema.Type[] convertChildren(Iterator schema, in
}
Repetition repetition = fromParquetRepetition(schemaElement.getRepetition_type());
String name = schemaElement.getName();
+ OriginalType originalType = null;
+ if (schemaElement.isSetConverted_type()) {
+ originalType = getOriginalType(schemaElement.converted_type);
+ }
+
+ // Create Parquet Type.
if (schemaElement.type != null) {
if (schemaElement.isSetType_length()) {
result[i] = new PrimitiveType(
repetition,
getPrimitive(schemaElement.getType()),
schemaElement.type_length,
- name);
+ name,
+ originalType);
} else {
result[i] = new PrimitiveType(
repetition,
getPrimitive(schemaElement.getType()),
- name);
+ name,
+ originalType);
}
} else {
result[i] = new GroupType(
repetition,
name,
+ originalType,
convertChildren(schema, schemaElement.getNum_children()));
}
}
diff --git a/parquet-hadoop/src/main/java/parquet/hadoop/CodecFactory.java b/parquet-hadoop/src/main/java/parquet/hadoop/CodecFactory.java
index d491aeac3a..1a87493566 100644
--- a/parquet-hadoop/src/main/java/parquet/hadoop/CodecFactory.java
+++ b/parquet-hadoop/src/main/java/parquet/hadoop/CodecFactory.java
@@ -140,7 +140,7 @@ public CodecFactory(Configuration configuration) {
* @return the corresponding hadoop codec. null if UNCOMPRESSED
*/
private CompressionCodec getCodec(CompressionCodecName codecName) {
- String codecClassName = codecName.getHadoopCompressionCodecClass();
+ String codecClassName = codecName.getHadoopCompressionCodecClassName();
if (codecClassName == null) {
return null;
}
diff --git a/parquet-hadoop/src/main/java/parquet/hadoop/InternalParquetRecordWriter.java b/parquet-hadoop/src/main/java/parquet/hadoop/InternalParquetRecordWriter.java
index 5ae627979c..c0a66a87ec 100644
--- a/parquet-hadoop/src/main/java/parquet/hadoop/InternalParquetRecordWriter.java
+++ b/parquet-hadoop/src/main/java/parquet/hadoop/InternalParquetRecordWriter.java
@@ -15,9 +15,17 @@
*/
package parquet.hadoop;
+import static java.lang.Math.max;
+import static java.lang.Math.min;
+import static java.lang.String.format;
+import static parquet.Log.DEBUG;
+import static parquet.Preconditions.checkNotNull;
+
import java.io.IOException;
import java.util.Map;
+
import parquet.Log;
+import parquet.column.ParquetProperties.WriterVersion;
import parquet.column.impl.ColumnWriteStoreImpl;
import parquet.hadoop.CodecFactory.BytesCompressor;
import parquet.hadoop.api.WriteSupport;
@@ -25,11 +33,6 @@
import parquet.io.MessageColumnIO;
import parquet.schema.MessageType;
-import static java.lang.Math.max;
-import static java.lang.Math.min;
-import static java.lang.String.format;
-import static parquet.Log.DEBUG;
-
class InternalParquetRecordWriter {
private static final Log LOG = Log.getLog(InternalParquetRecordWriter.class);
@@ -44,8 +47,10 @@ class InternalParquetRecordWriter {
private final int blockSize;
private final int pageSize;
private final BytesCompressor compressor;
+ private final int dictionaryPageSize;
private final boolean enableDictionary;
private final boolean validating;
+ private final WriterVersion writerVersion;
private long recordCount = 0;
private long recordCountForNextMemCheck = MINIMUM_RECORD_COUNT_FOR_CHECK;
@@ -53,10 +58,7 @@ class InternalParquetRecordWriter {
private ColumnWriteStoreImpl store;
private ColumnChunkPageWriteStore pageStore;
-
-
/**
- *
* @param w the file to write to
* @param writeSupport the class to convert incoming records
* @param schema the schema of the records
@@ -64,21 +66,29 @@ class InternalParquetRecordWriter {
* @param blockSize the size of a block in the file (this will be approximate)
* @param codec the codec used to compress
*/
- public InternalParquetRecordWriter(ParquetFileWriter w, WriteSupport writeSupport,
- MessageType schema, Map extraMetaData, int blockSize,
- int pageSize, BytesCompressor compressor, boolean enableDictionary, boolean validating) {
- if (writeSupport == null) {
- throw new NullPointerException("writeSupport");
- }
+ public InternalParquetRecordWriter(
+ ParquetFileWriter w,
+ WriteSupport writeSupport,
+ MessageType schema,
+ Map extraMetaData,
+ int blockSize,
+ int pageSize,
+ BytesCompressor compressor,
+ int dictionaryPageSize,
+ boolean enableDictionary,
+ boolean validating,
+ WriterVersion writerVersion) {
this.w = w;
- this.writeSupport = writeSupport;
+ this.writeSupport = checkNotNull(writeSupport, "writeSupport");
this.schema = schema;
this.extraMetaData = extraMetaData;
this.blockSize = blockSize;
this.pageSize = pageSize;
this.compressor = compressor;
+ this.dictionaryPageSize = dictionaryPageSize;
this.enableDictionary = enableDictionary;
this.validating = validating;
+ this.writerVersion = writerVersion;
initStore();
}
@@ -91,7 +101,7 @@ private void initStore() {
// we don't want this number to be too small either
// ideally, slightly bigger than the page size, but not bigger than the block buffer
int initialPageBufferSize = max(MINIMUM_BUFFER_SIZE, min(pageSize + pageSize / 10, initialBlockBufferSize));
- store = new ColumnWriteStoreImpl(pageStore, pageSize, initialPageBufferSize, enableDictionary);
+ store = new ColumnWriteStoreImpl(pageStore, pageSize, initialPageBufferSize, dictionaryPageSize, enableDictionary, writerVersion);
MessageColumnIO columnIO = new ColumnIOFactory(validating).getColumnIO(schema);
writeSupport.prepareForWrite(columnIO.getRecordWriter(store));
}
diff --git a/parquet-hadoop/src/main/java/parquet/hadoop/ParquetFileReader.java b/parquet-hadoop/src/main/java/parquet/hadoop/ParquetFileReader.java
index 77e8b10a25..5d7d320a86 100644
--- a/parquet-hadoop/src/main/java/parquet/hadoop/ParquetFileReader.java
+++ b/parquet-hadoop/src/main/java/parquet/hadoop/ParquetFileReader.java
@@ -21,6 +21,7 @@
import static parquet.hadoop.ParquetFileWriter.MAGIC;
import static parquet.hadoop.ParquetFileWriter.PARQUET_METADATA_FILE;
+import java.io.ByteArrayInputStream;
import java.io.Closeable;
import java.io.IOException;
import java.util.ArrayList;
@@ -335,6 +336,37 @@ public PageReadStore readNextRowGroup() throws IOException {
return columnChunkPageReadStore;
}
+ private static class Chunk extends ByteArrayInputStream{
+
+ private final FSDataInputStream f;
+
+ private Chunk(FSDataInputStream f, int size) throws IOException {
+ super(new byte[size]);
+ this.f = f;
+ f.readFully(this.buf);
+ }
+
+ public int pos() {
+ return this.pos;
+ }
+
+ public BytesInput readAsBytesInput(int size) throws IOException {
+ if (pos + size > count) {
+ // this is to workaround a bug where the compressedLength
+ // of the chunk is missing the size of the header of the dictionary
+ // to allow reading older files (using dictionary) we need this.
+ // usually 13 to 19 bytes are missing
+ int l1 = count - pos;
+ int l2 = size - l1;
+ LOG.info("completed the column chunk with " + l2 + " bytes");
+ return BytesInput.concat(this.readAsBytesInput(l1), BytesInput.copy(BytesInput.from(f, l2)));
+ }
+ final BytesInput r = BytesInput.from(this.buf, this.pos, size);
+ this.pos += size;
+ return r;
+ }
+
+ }
/**
* Read all of the pages in a given column chunk.
* @return the list of pages
@@ -351,14 +383,15 @@ private void readColumnChunkPages(ColumnDescriptor columnDescriptor, ColumnChunk
LOG.debug(f.getPos() + ": start column chunk " + metadata.getPath() +
" " + metadata.getType() + " count=" + metadata.getValueCount());
}
+ Chunk chunk = new Chunk(f, (int)metadata.getTotalSize());
long valuesCountReadSoFar = 0;
while (valuesCountReadSoFar < metadata.getValueCount()) {
- PageHeader pageHeader = readPageHeader(f);
+ PageHeader pageHeader = readPageHeader(chunk);
switch (pageHeader.type) {
case DICTIONARY_PAGE:
dictionaryPagesInChunk.add(
new DictionaryPage(
- BytesInput.copy(BytesInput.from(f, pageHeader.compressed_page_size)),
+ chunk.readAsBytesInput(pageHeader.compressed_page_size),
pageHeader.uncompressed_page_size,
pageHeader.dictionary_page_header.num_values,
parquetMetadataConverter.getEncoding(pageHeader.dictionary_page_header.encoding)
@@ -367,7 +400,7 @@ private void readColumnChunkPages(ColumnDescriptor columnDescriptor, ColumnChunk
case DATA_PAGE:
pagesInChunk.add(
new Page(
- BytesInput.copy(BytesInput.from(f, pageHeader.compressed_page_size)),
+ chunk.readAsBytesInput(pageHeader.compressed_page_size),
pageHeader.data_page_header.num_values,
pageHeader.uncompressed_page_size,
parquetMetadataConverter.getEncoding(pageHeader.data_page_header.repetition_level_encoding),
@@ -378,7 +411,7 @@ private void readColumnChunkPages(ColumnDescriptor columnDescriptor, ColumnChunk
break;
default:
if (DEBUG) LOG.debug("skipping page of type " + pageHeader.type + " of size " + pageHeader.compressed_page_size);
- f.skip(pageHeader.compressed_page_size);
+ chunk.skip(pageHeader.compressed_page_size);
break;
}
}
@@ -388,7 +421,7 @@ private void readColumnChunkPages(ColumnDescriptor columnDescriptor, ColumnChunk
"Expected " + metadata.getValueCount() + " values in column chunk at " +
filePath + " offset " + metadata.getFirstDataPageOffset() +
" but got " + valuesCountReadSoFar + " values instead over " + pagesInChunk.size()
- + " pages ending at file offset " + f.getPos());
+ + " pages ending at file offset " + (startingPos + chunk.pos()));
}
}
diff --git a/parquet-hadoop/src/main/java/parquet/hadoop/ParquetFileWriter.java b/parquet-hadoop/src/main/java/parquet/hadoop/ParquetFileWriter.java
index acb4f9d73a..e45ba06e08 100644
--- a/parquet-hadoop/src/main/java/parquet/hadoop/ParquetFileWriter.java
+++ b/parquet-hadoop/src/main/java/parquet/hadoop/ParquetFileWriter.java
@@ -205,8 +205,9 @@ public void writeDictionaryPage(DictionaryPage dictionaryPage) throws IOExceptio
dictionaryPage.getDictionarySize(),
dictionaryPage.getEncoding(),
out);
- this.uncompressedLength += uncompressedSize;
- this.compressedLength += compressedPageSize;
+ long headerSize = out.getPos() - currentChunkDictionaryPageOffset;
+ this.uncompressedLength += uncompressedSize + headerSize;
+ this.compressedLength += compressedPageSize + headerSize;
if (DEBUG) LOG.debug(out.getPos() + ": write dictionary page content " + compressedPageSize);
dictionaryPage.getBytes().writeAllTo(out);
currentEncodings.add(dictionaryPage.getEncoding());
diff --git a/parquet-hadoop/src/main/java/parquet/hadoop/ParquetInputFormat.java b/parquet-hadoop/src/main/java/parquet/hadoop/ParquetInputFormat.java
index 16fa13eab8..fcb4986fb5 100644
--- a/parquet-hadoop/src/main/java/parquet/hadoop/ParquetInputFormat.java
+++ b/parquet-hadoop/src/main/java/parquet/hadoop/ParquetInputFormat.java
@@ -301,7 +301,7 @@ private static List getAllFileRecursively(
if (file.isDir()) {
Path p = file.getPath();
FileSystem fs = p.getFileSystem(conf);
- addInputPathRecursively(result, fs, p, hiddenFileFilter);
+ staticAddInputPathRecursively(result, fs, p, hiddenFileFilter);
} else {
result.add(file);
}
@@ -310,12 +310,12 @@ private static List getAllFileRecursively(
return result;
}
- private static void addInputPathRecursively(List result,
+ private static void staticAddInputPathRecursively(List result,
FileSystem fs, Path path, PathFilter inputFilter)
throws IOException {
for (FileStatus stat: fs.listStatus(path, inputFilter)) {
if (stat.isDir()) {
- addInputPathRecursively(result, fs, stat.getPath(), inputFilter);
+ staticAddInputPathRecursively(result, fs, stat.getPath(), inputFilter);
} else {
result.add(stat);
}
diff --git a/parquet-hadoop/src/main/java/parquet/hadoop/ParquetInputSplit.java b/parquet-hadoop/src/main/java/parquet/hadoop/ParquetInputSplit.java
index 788a5e0bfe..0d44f33142 100644
--- a/parquet-hadoop/src/main/java/parquet/hadoop/ParquetInputSplit.java
+++ b/parquet-hadoop/src/main/java/parquet/hadoop/ParquetInputSplit.java
@@ -30,6 +30,7 @@
import java.util.Set;
import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.Text;
import org.apache.hadoop.io.Writable;
import org.apache.hadoop.mapreduce.InputSplit;
@@ -186,8 +187,8 @@ public void readFields(DataInput in) throws IOException {
for (int i = 0; i < blocksSize; i++) {
blocks.add(readBlock(in));
}
- this.requestedSchema = in.readUTF().intern();
- this.fileSchema = in.readUTF().intern();
+ this.requestedSchema = Text.readString(in);
+ this.fileSchema = Text.readString(in);
this.extraMetadata = readKeyValues(in);
this.readSupportMetadata = readKeyValues(in);
}
@@ -208,8 +209,8 @@ public void write(DataOutput out) throws IOException {
for (BlockMetaData block : blocks) {
writeBlock(out, block);
}
- out.writeUTF(requestedSchema);
- out.writeUTF(fileSchema);
+ Text.writeString(out, requestedSchema);
+ Text.writeString(out, fileSchema);
writeKeyValues(out, extraMetadata);
writeKeyValues(out, readSupportMetadata);
}
diff --git a/parquet-hadoop/src/main/java/parquet/hadoop/ParquetOutputFormat.java b/parquet-hadoop/src/main/java/parquet/hadoop/ParquetOutputFormat.java
index d6f286768e..759699006c 100644
--- a/parquet-hadoop/src/main/java/parquet/hadoop/ParquetOutputFormat.java
+++ b/parquet-hadoop/src/main/java/parquet/hadoop/ParquetOutputFormat.java
@@ -18,15 +18,12 @@
import static parquet.Log.INFO;
import static parquet.hadoop.ParquetWriter.DEFAULT_BLOCK_SIZE;
import static parquet.hadoop.ParquetWriter.DEFAULT_PAGE_SIZE;
-import static parquet.hadoop.metadata.CompressionCodecName.UNCOMPRESSED;
-import static parquet.hadoop.metadata.CompressionCodecName.fromConf;
import static parquet.hadoop.util.ContextUtil.getConfiguration;
import java.io.IOException;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.compress.DefaultCodec;
import org.apache.hadoop.mapreduce.Job;
import org.apache.hadoop.mapreduce.JobContext;
import org.apache.hadoop.mapreduce.OutputCommitter;
@@ -35,8 +32,10 @@
import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
import parquet.Log;
+import parquet.column.ParquetProperties.WriterVersion;
import parquet.hadoop.api.WriteSupport;
import parquet.hadoop.api.WriteSupport.WriteContext;
+import parquet.hadoop.codec.CodecConfig;
import parquet.hadoop.metadata.CompressionCodecName;
/**
@@ -58,6 +57,10 @@
* # If this value is too small, the compression will deteriorate
* parquet.page.size=1048576 # in bytes, default = 1 * 1024 * 1024
*
+ * # There is one dictionary page per column per row group when dictionary encoding is used.
+ * # The dictionary page size works like the page size but for dictionary
+ * parquet.dictionary.page.size=1048576 # in bytes, default = 1 * 1024 * 1024
+ *
* # The compression algorithm used to compress pages
* parquet.compression=UNCOMPRESSED # one of: UNCOMPRESSED, SNAPPY, GZIP, LZO. Default: UNCOMPRESSED. Supersedes mapred.output.compress*
*
@@ -65,8 +68,8 @@
* # Usually provided by a specific ParquetOutputFormat subclass
* parquet.write.support.class= # fully qualified name
*
- * # To enable dictionary encoding
- * parquet.enable.dictionary=false # true to enable dictionary encoding
+ * # To enable/disable dictionary encoding
+ * parquet.enable.dictionary=true # false to disable dictionary encoding
*
*
* If parquet.compression is not set, the following properties are checked (FileOutputFormat behavior).
@@ -85,12 +88,14 @@
public class ParquetOutputFormat extends FileOutputFormat {
private static final Log LOG = Log.getLog(ParquetOutputFormat.class);
- public static final String BLOCK_SIZE = "parquet.block.size";
- public static final String PAGE_SIZE = "parquet.page.size";
- public static final String COMPRESSION = "parquet.compression";
- public static final String WRITE_SUPPORT_CLASS = "parquet.write.support.class";
- public static final String ENABLE_DICTIONARY = "parquet.enable.dictionary";
- public static final String VALIDATION = "parquet.validation";
+ public static final String BLOCK_SIZE = "parquet.block.size";
+ public static final String PAGE_SIZE = "parquet.page.size";
+ public static final String COMPRESSION = "parquet.compression";
+ public static final String WRITE_SUPPORT_CLASS = "parquet.write.support.class";
+ public static final String DICTIONARY_PAGE_SIZE = "parquet.dictionary.page.size";
+ public static final String ENABLE_DICTIONARY = "parquet.enable.dictionary";
+ public static final String VALIDATION = "parquet.validation";
+ public static final String WRITER_VERSION = "parquet.writer.version";
public static void setWriteSupportClass(Job job, Class> writeSupportClass) {
getConfiguration(job).set(WRITE_SUPPORT_CLASS, writeSupportClass.getName());
@@ -120,6 +125,10 @@ public static void setPageSize(Job job, int pageSize) {
getConfiguration(job).setInt(PAGE_SIZE, pageSize);
}
+ public static void setDictionaryPageSize(Job job, int pageSize) {
+ getConfiguration(job).setInt(DICTIONARY_PAGE_SIZE, pageSize);
+ }
+
public static void setCompression(Job job, CompressionCodecName compression) {
getConfiguration(job).set(COMPRESSION, compression.name());
}
@@ -140,6 +149,10 @@ public static int getPageSize(JobContext jobContext) {
return getPageSize(getConfiguration(jobContext));
}
+ public static int getDictionaryPageSize(JobContext jobContext) {
+ return getDictionaryPageSize(getConfiguration(jobContext));
+ }
+
public static CompressionCodecName getCompression(JobContext jobContext) {
return getCompression(getConfiguration(jobContext));
}
@@ -157,7 +170,7 @@ public static boolean getValidation(JobContext jobContext) {
}
public static boolean getEnableDictionary(Configuration configuration) {
- return configuration.getBoolean(ENABLE_DICTIONARY, false);
+ return configuration.getBoolean(ENABLE_DICTIONARY, true);
}
public static int getBlockSize(Configuration configuration) {
@@ -168,12 +181,21 @@ public static int getPageSize(Configuration configuration) {
return configuration.getInt(PAGE_SIZE, DEFAULT_PAGE_SIZE);
}
+ public static int getDictionaryPageSize(Configuration configuration) {
+ return configuration.getInt(DICTIONARY_PAGE_SIZE, DEFAULT_PAGE_SIZE);
+ }
+
+ public static WriterVersion getWriterVersion(Configuration configuration) {
+ String writerVersion = configuration.get(WRITER_VERSION, WriterVersion.PARQUET_1_0.toString());
+ return WriterVersion.fromString(writerVersion);
+ }
+
public static CompressionCodecName getCompression(Configuration configuration) {
- return fromConf(configuration.get(COMPRESSION, UNCOMPRESSED.name()));
+ return CodecConfig.getParquetCompressionCodec(configuration);
}
public static boolean isCompressionSet(Configuration configuration) {
- return configuration.get(COMPRESSION) != null;
+ return CodecConfig.isParquetCompressionSet(configuration);
}
public static void setValidation(Configuration configuration, boolean validating) {
@@ -184,27 +206,12 @@ public static boolean getValidation(Configuration configuration) {
return configuration.getBoolean(VALIDATION, false);
}
- private static CompressionCodecName getCodec(TaskAttemptContext taskAttemptContext) {
- Configuration conf = getConfiguration(taskAttemptContext);
- CompressionCodecName codec;
-
- if (isCompressionSet(conf)) { // explicit parquet config
- codec = getCompression(conf);
- } else if (getCompressOutput(taskAttemptContext)) { // from hadoop config
- // find the right codec
- Class> codecClass = getOutputCompressorClass(taskAttemptContext, DefaultCodec.class);
- if (INFO) LOG.info("Compression set through hadoop codec: " + codecClass.getName());
- codec = CompressionCodecName.fromCompressionCodec(codecClass);
- } else {
- if (INFO) LOG.info("Compression set to false");
- codec = CompressionCodecName.UNCOMPRESSED;
- }
-
- if (INFO) LOG.info("Compression: " + codec.name());
- return codec;
+ private CompressionCodecName getCodec(TaskAttemptContext taskAttemptContext) {
+ return CodecConfig.from(taskAttemptContext).getCodec();
}
+
private WriteSupport writeSupport;
private ParquetOutputCommitter committer;
@@ -245,7 +252,6 @@ public RecordWriter getRecordWriter(TaskAttemptContext taskAttemptConte
return getRecordWriter(getConfiguration(taskAttemptContext), file, getCodec(taskAttemptContext));
}
- @SuppressWarnings("unchecked") // writeSupport instantiation
public RecordWriter getRecordWriter(Configuration conf, Path file, CompressionCodecName codec)
throws IOException, InterruptedException {
final WriteSupport writeSupport = getWriteSupport(conf);
@@ -255,14 +261,19 @@ public RecordWriter getRecordWriter(Configuration conf, Path file, Comp
if (INFO) LOG.info("Parquet block size to " + blockSize);
int pageSize = getPageSize(conf);
if (INFO) LOG.info("Parquet page size to " + pageSize);
-
+ int dictionaryPageSize = getDictionaryPageSize(conf);
+ if (INFO) LOG.info("Parquet dictionary page size to " + dictionaryPageSize);
boolean enableDictionary = getEnableDictionary(conf);
- WriteContext init = writeSupport.init(conf);
- ParquetFileWriter w = new ParquetFileWriter(conf, init.getSchema(), file);
- w.start();
+ if (INFO) LOG.info("Dictionary is " + (enableDictionary ? "on" : "off"));
boolean validating = getValidation(conf);
if (INFO) LOG.info("Validation is " + (validating ? "on" : "off"));
+ WriterVersion writerVersion = getWriterVersion(conf);
+ if (INFO) LOG.info("Writer version is: " + writerVersion);
+ WriteContext init = writeSupport.init(conf);
+ ParquetFileWriter w = new ParquetFileWriter(conf, init.getSchema(), file);
+ w.start();
+
return new ParquetRecordWriter(
w,
writeSupport,
@@ -270,8 +281,10 @@ public RecordWriter getRecordWriter(Configuration conf, Path file, Comp
init.getExtraMetaData(),
blockSize, pageSize,
codecFactory.getCompressor(codec, pageSize),
+ dictionaryPageSize,
enableDictionary,
- validating);
+ validating,
+ writerVersion);
}
/**
diff --git a/parquet-hadoop/src/main/java/parquet/hadoop/ParquetRecordWriter.java b/parquet-hadoop/src/main/java/parquet/hadoop/ParquetRecordWriter.java
index 054bc53e2a..fac67fee4a 100644
--- a/parquet-hadoop/src/main/java/parquet/hadoop/ParquetRecordWriter.java
+++ b/parquet-hadoop/src/main/java/parquet/hadoop/ParquetRecordWriter.java
@@ -19,6 +19,8 @@
import java.util.Map;
import org.apache.hadoop.mapreduce.RecordWriter;
import org.apache.hadoop.mapreduce.TaskAttemptContext;
+
+import parquet.column.ParquetProperties.WriterVersion;
import parquet.hadoop.CodecFactory.BytesCompressor;
import parquet.hadoop.api.WriteSupport;
import parquet.schema.MessageType;
@@ -43,10 +45,24 @@ public class ParquetRecordWriter extends RecordWriter {
* @param schema the schema of the records
* @param extraMetaData extra meta data to write in the footer of the file
* @param blockSize the size of a block in the file (this will be approximate)
+ * @param compressor the compressor used to compress the pages
+ * @param dictionaryPageSize the threshold for dictionary size
+ * @param enableDictionary to enable the dictionary
+ * @param validating if schema validation should be turned on
*/
- public ParquetRecordWriter(ParquetFileWriter w, WriteSupport writeSupport, MessageType schema, Map extraMetaData, int blockSize, int pageSize, BytesCompressor compressor, boolean enableDictionary, boolean validating) {
+ public ParquetRecordWriter(
+ ParquetFileWriter w,
+ WriteSupport writeSupport,
+ MessageType schema,
+ Map extraMetaData,
+ int blockSize, int pageSize,
+ BytesCompressor compressor,
+ int dictionaryPageSize,
+ boolean enableDictionary,
+ boolean validating,
+ WriterVersion writerVersion) {
internalWriter = new InternalParquetRecordWriter(w, writeSupport, schema,
- extraMetaData, blockSize, pageSize, compressor, enableDictionary, validating);
+ extraMetaData, blockSize, pageSize, compressor, dictionaryPageSize, enableDictionary, validating, writerVersion);
}
/**
diff --git a/parquet-hadoop/src/main/java/parquet/hadoop/ParquetWriter.java b/parquet-hadoop/src/main/java/parquet/hadoop/ParquetWriter.java
index a31d9e8e83..9b9cf4fcd5 100644
--- a/parquet-hadoop/src/main/java/parquet/hadoop/ParquetWriter.java
+++ b/parquet-hadoop/src/main/java/parquet/hadoop/ParquetWriter.java
@@ -21,6 +21,8 @@
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
+import parquet.column.ParquetProperties;
+import parquet.column.ParquetProperties.WriterVersion;
import parquet.hadoop.api.WriteSupport;
import parquet.hadoop.metadata.CompressionCodecName;
import parquet.schema.MessageType;
@@ -37,7 +39,7 @@ public class ParquetWriter implements Closeable {
/**
* Create a new ParquetWriter.
- * (with dictionary encoding disabled and validation off)
+ * (with dictionary encoding enabled and validation off)
* @see ParquetWriter#ParquetWriter(Path, WriteSupport, CompressionCodecName, int, int, boolean)
*
* @param file the file to create
@@ -48,7 +50,30 @@ public class ParquetWriter implements Closeable {
* @throws IOException
*/
public ParquetWriter(Path file, WriteSupport writeSupport, CompressionCodecName compressionCodecName, int blockSize, int pageSize) throws IOException {
- this(file, writeSupport, compressionCodecName, blockSize, pageSize, false, false);
+ this(file, writeSupport, compressionCodecName, blockSize, pageSize, true, false);
+ }
+
+ /**
+ * Create a new ParquetWriter.
+ *
+ * @param file the file to create
+ * @param writeSupport the implementation to write a record to a RecordConsumer
+ * @param compressionCodecName the compression codec to use
+ * @param blockSize the block size threshold
+ * @param pageSize the page size threshold (both data and dictionary)
+ * @param enableDictionary to turn dictionary encoding on
+ * @param validating to turn on validation using the schema
+ * @throws IOException
+ */
+ public ParquetWriter(
+ Path file,
+ WriteSupport writeSupport,
+ CompressionCodecName compressionCodecName,
+ int blockSize,
+ int pageSize,
+ boolean enableDictionary,
+ boolean validating) throws IOException {
+ this(file, writeSupport, compressionCodecName, blockSize, pageSize, pageSize, enableDictionary, validating);
}
/**
@@ -59,6 +84,7 @@ public ParquetWriter(Path file, WriteSupport writeSupport, CompressionCodecNa
* @param compressionCodecName the compression codec to use
* @param blockSize the block size threshold
* @param pageSize the page size threshold
+ * @param dictionaryPageSize the page size threshold for the dictionary pages
* @param enableDictionary to turn dictionary encoding on
* @param validating to turn on validation using the schema
* @throws IOException
@@ -69,8 +95,36 @@ public ParquetWriter(
CompressionCodecName compressionCodecName,
int blockSize,
int pageSize,
+ int dictionaryPageSize,
boolean enableDictionary,
boolean validating) throws IOException {
+ this(file, writeSupport, compressionCodecName, blockSize, pageSize, dictionaryPageSize, enableDictionary, validating, WriterVersion.PARQUET_1_0);
+ }
+
+ /**
+ * Create a new ParquetWriter.
+ *
+ * @param file the file to create
+ * @param writeSupport the implementation to write a record to a RecordConsumer
+ * @param compressionCodecName the compression codec to use
+ * @param blockSize the block size threshold
+ * @param pageSize the page size threshold
+ * @param dictionaryPageSize the page size threshold for the dictionary pages
+ * @param enableDictionary to turn dictionary encoding on
+ * @param validating to turn on validation using the schema
+ * @poram writerVersion version of parquetWriter from {@link ParquetProperties.WriterVersion}
+ * @throws IOException
+ */
+ public ParquetWriter(
+ Path file,
+ WriteSupport writeSupport,
+ CompressionCodecName compressionCodecName,
+ int blockSize,
+ int pageSize,
+ int dictionaryPageSize,
+ boolean enableDictionary,
+ boolean validating,
+ WriterVersion writerVersion) throws IOException {
Configuration conf = new Configuration();
WriteSupport.WriteContext writeContext = writeSupport.init(conf);
@@ -81,8 +135,18 @@ public ParquetWriter(
CodecFactory codecFactory = new CodecFactory(conf);
CodecFactory.BytesCompressor compressor = codecFactory.getCompressor(compressionCodecName, 0);
- this.writer = new InternalParquetRecordWriter(fileWriter, writeSupport, schema, writeContext.getExtraMetaData(), blockSize, pageSize, compressor, enableDictionary, validating);
-
+ this.writer = new InternalParquetRecordWriter(
+ fileWriter,
+ writeSupport,
+ schema,
+ writeContext.getExtraMetaData(),
+ blockSize,
+ pageSize,
+ compressor,
+ dictionaryPageSize,
+ enableDictionary,
+ validating,
+ writerVersion);
}
/**
diff --git a/parquet-hadoop/src/main/java/parquet/hadoop/codec/CodecConfig.java b/parquet-hadoop/src/main/java/parquet/hadoop/codec/CodecConfig.java
new file mode 100644
index 0000000000..84673db357
--- /dev/null
+++ b/parquet-hadoop/src/main/java/parquet/hadoop/codec/CodecConfig.java
@@ -0,0 +1,166 @@
+/**
+ * Copyright 2012 Twitter, Inc.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package parquet.hadoop.codec;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
+import parquet.Log;
+import parquet.hadoop.ParquetOutputFormat;
+import parquet.hadoop.metadata.CompressionCodecName;
+import parquet.hadoop.util.ContextUtil;
+
+import static parquet.Log.INFO;
+import static parquet.Log.WARN;
+import static parquet.hadoop.metadata.CompressionCodecName.UNCOMPRESSED;
+
+/**
+ * Template class and factory for accessing codec related configurations in different APIs(mapreduce or mapred),
+ * use {@link #from(org.apache.hadoop.mapred.JobConf)} for mapred API,
+ * use {@link #from(org.apache.hadoop.mapreduce.TaskAttemptContext)} for mapreduce API
+ *
+ * @author Tianshuo Deng
+ */
+public abstract class CodecConfig {
+ private static final Log LOG = Log.getLog(CodecConfig.class);
+
+ /**
+ * @return if a compress flag is set from hadoop
+ */
+ public abstract boolean isHadoopCompressionSet();
+
+ /**
+ * @param defaultCodec the codec to use when codec is not set in conf
+ * @return codec specified in hadoop config
+ */
+ public abstract Class getHadoopOutputCompressorClass(Class defaultCodec);
+
+ /**
+ * @return configuration of the job
+ */
+ public abstract Configuration getConfiguration();
+
+ /**
+ * use mapred api to read codec config
+ * @return MapredCodecConfig
+ */
+ public static CodecConfig from(JobConf jobConf) {
+ return new MapredCodecConfig(jobConf);
+ }
+
+ /**
+ * use mapreduce api to read codec config
+ * @return MapreduceCodecConfig
+ */
+ public static CodecConfig from(TaskAttemptContext context) {
+ return new MapreduceCodecConfig(context);
+ }
+
+ public static boolean isParquetCompressionSet(Configuration conf) {
+ return conf.get(ParquetOutputFormat.COMPRESSION) != null;
+ }
+
+ public static CompressionCodecName getParquetCompressionCodec(Configuration configuration) {
+ return CompressionCodecName.fromConf(configuration.get(ParquetOutputFormat.COMPRESSION, UNCOMPRESSED.name()));
+ }
+
+ public CompressionCodecName getCodec() {
+ CompressionCodecName codec;
+ Configuration configuration = getConfiguration();
+ if (isParquetCompressionSet(configuration)) { // explicit parquet config
+ codec = getParquetCompressionCodec(configuration);
+ } else if (isHadoopCompressionSet()) { // from hadoop config
+ codec = getHadoopCompressionCodec();
+ } else {
+ if (INFO) LOG.info("Compression set to false");
+ codec = CompressionCodecName.UNCOMPRESSED;
+ }
+
+ if (INFO) LOG.info("Compression: " + codec.name());
+ return codec;
+ }
+
+ private CompressionCodecName getHadoopCompressionCodec() {
+ CompressionCodecName codec;
+ try {
+ // find the right codec
+ Class> codecClass = getHadoopOutputCompressorClass(CompressionCodecName.UNCOMPRESSED.getHadoopCompressionCodecClass());
+ if (INFO) LOG.info("Compression set through hadoop codec: " + codecClass.getName());
+ codec = CompressionCodecName.fromCompressionCodec(codecClass);
+ } catch (CompressionCodecNotSupportedException e) {
+ if (WARN)
+ LOG.warn("codec defined in hadoop config is not supported by parquet [" + e.getCodecClass().getName() + "] and will use UNCOMPRESSED", e);
+ codec = CompressionCodecName.UNCOMPRESSED;
+ } catch (IllegalArgumentException e) {
+ if (WARN) LOG.warn("codec class not found: " + e.getMessage(), e);
+ codec = CompressionCodecName.UNCOMPRESSED;
+ }
+ return codec;
+ }
+
+ /**
+ * Access codec related configurations in mapreduce API
+ */
+ private static class MapreduceCodecConfig extends CodecConfig {
+ private final TaskAttemptContext context;
+
+ public MapreduceCodecConfig(TaskAttemptContext context) {
+ this.context = context;
+ }
+
+ @Override
+ public boolean isHadoopCompressionSet() {
+ return FileOutputFormat.getCompressOutput(context);
+ }
+
+ @Override
+ public Class getHadoopOutputCompressorClass(Class defaultCodec) {
+ return FileOutputFormat.getOutputCompressorClass(context, defaultCodec);
+ }
+
+ @Override
+ public Configuration getConfiguration() {
+ return ContextUtil.getConfiguration(context);
+ }
+ }
+
+ /**
+ * Access codec related configurations in mapred API
+ */
+ private static class MapredCodecConfig extends CodecConfig {
+ private final JobConf conf;
+
+ public MapredCodecConfig(JobConf conf) {
+ this.conf = conf;
+ }
+
+ @Override
+ public boolean isHadoopCompressionSet() {
+ return org.apache.hadoop.mapred.FileOutputFormat.getCompressOutput(conf);
+ }
+
+ @Override
+ public Class getHadoopOutputCompressorClass(Class defaultCodec) {
+ return org.apache.hadoop.mapred.FileOutputFormat.getOutputCompressorClass(conf, defaultCodec);
+ }
+
+ @Override
+ public Configuration getConfiguration() {
+ return conf;
+ }
+ }
+}
diff --git a/parquet-hadoop/src/main/java/parquet/hadoop/codec/CompressionCodecNotSupportedException.java b/parquet-hadoop/src/main/java/parquet/hadoop/codec/CompressionCodecNotSupportedException.java
new file mode 100644
index 0000000000..0050e74ce1
--- /dev/null
+++ b/parquet-hadoop/src/main/java/parquet/hadoop/codec/CompressionCodecNotSupportedException.java
@@ -0,0 +1,33 @@
+/**
+ * Copyright 2012 Twitter, Inc.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package parquet.hadoop.codec;
+
+/**
+ * This exception will be thrown when the codec is not supported by parquet, meaning there is no
+ * matching codec defined in {@link parquet.hadoop.metadata.CompressionCodecName}
+ */
+public class CompressionCodecNotSupportedException extends RuntimeException {
+ private final Class codecClass;
+
+ public CompressionCodecNotSupportedException(Class codecClass) {
+ super("codec not supported: " + codecClass.getName());
+ this.codecClass = codecClass;
+ }
+
+ public Class getCodecClass() {
+ return codecClass;
+ }
+}
diff --git a/parquet-hadoop/src/main/java/parquet/hadoop/mapred/DeprecatedParquetInputFormat.java b/parquet-hadoop/src/main/java/parquet/hadoop/mapred/DeprecatedParquetInputFormat.java
index a4c11671da..f7196e1153 100644
--- a/parquet-hadoop/src/main/java/parquet/hadoop/mapred/DeprecatedParquetInputFormat.java
+++ b/parquet-hadoop/src/main/java/parquet/hadoop/mapred/DeprecatedParquetInputFormat.java
@@ -49,8 +49,8 @@ public RecordReader> getRecordReader(InputSplit split, JobCon
@Override
public InputSplit[] getSplits(JobConf job, int numSplits) throws IOException {
- List footers = realInputFormat.getFooters(job, Arrays.asList(super.listStatus(job)));
- List splits = realInputFormat.getSplits(job, footers);
+ List footers = getFooters(job);
+ List splits = realInputFormat.getSplits(job, footers);
if (splits == null) {
return null;
@@ -65,6 +65,10 @@ public InputSplit[] getSplits(JobConf job, int numSplits) throws IOException {
return resultSplits;
}
+ public List getFooters(JobConf job) throws IOException {
+ return realInputFormat.getFooters(job, Arrays.asList(super.listStatus(job)));
+ }
+
private static class RecordReaderWrapper implements RecordReader> {
private ParquetRecordReader realReader;
diff --git a/parquet-hadoop/src/main/java/parquet/hadoop/mapred/DeprecatedParquetOutputFormat.java b/parquet-hadoop/src/main/java/parquet/hadoop/mapred/DeprecatedParquetOutputFormat.java
index cb907e38ef..69bf36b68e 100644
--- a/parquet-hadoop/src/main/java/parquet/hadoop/mapred/DeprecatedParquetOutputFormat.java
+++ b/parquet-hadoop/src/main/java/parquet/hadoop/mapred/DeprecatedParquetOutputFormat.java
@@ -15,10 +15,10 @@
*/
package parquet.hadoop.mapred;
-import static parquet.Log.INFO;
import parquet.Log;
import parquet.hadoop.ParquetOutputFormat;
import parquet.hadoop.ParquetRecordWriter;
+import parquet.hadoop.codec.CodecConfig;
import parquet.hadoop.metadata.CompressionCodecName;
import java.io.IOException;
@@ -26,7 +26,6 @@
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.compress.DefaultCodec;
import org.apache.hadoop.mapred.JobConf;
import org.apache.hadoop.mapred.RecordWriter;
import org.apache.hadoop.mapred.Reporter;
@@ -34,7 +33,7 @@
@SuppressWarnings("deprecation")
public class DeprecatedParquetOutputFormat extends org.apache.hadoop.mapred.FileOutputFormat {
- private static final Log LOG = Log.getLog(ParquetOutputFormat.class);
+ private static final Log LOG = Log.getLog(DeprecatedParquetOutputFormat.class);
public static void setWriteSupportClass(Configuration configuration, Class> writeSupportClass) {
configuration.set(ParquetOutputFormat.WRITE_SUPPORT_CLASS, writeSupportClass.getName());
@@ -56,23 +55,8 @@ public static void setEnableDictionary(Configuration configuration, boolean enab
configuration.setBoolean(ParquetOutputFormat.ENABLE_DICTIONARY, enableDictionary);
}
- private static CompressionCodecName getCodec(JobConf conf) {
- CompressionCodecName codec;
-
- if (ParquetOutputFormat.isCompressionSet(conf)) { // explicit parquet config
- codec = ParquetOutputFormat.getCompression(conf);
- } else if (getCompressOutput(conf)) { // from hadoop config
- // find the right codec
- Class> codecClass = getOutputCompressorClass(conf, DefaultCodec.class);
- if (INFO) LOG.info("Compression set through hadoop codec: " + codecClass.getName());
- codec = CompressionCodecName.fromCompressionCodec(codecClass);
- } else {
- if (INFO) LOG.info("Compression set to false");
- codec = CompressionCodecName.UNCOMPRESSED;
- }
-
- if (INFO) LOG.info("Compression: " + codec.name());
- return codec;
+ private CompressionCodecName getCodec(final JobConf conf) {
+ return CodecConfig.from(conf).getCodec();
}
private static Path getDefaultWorkFile(JobConf conf, String name, String extension) {
@@ -88,7 +72,7 @@ public RecordWriter getRecordWriter(FileSystem fs,
return new RecordWriterWrapper(realOutputFormat, fs, conf, name, progress);
}
- private static class RecordWriterWrapper implements RecordWriter {
+ private class RecordWriterWrapper implements RecordWriter {
private ParquetRecordWriter realWriter;
diff --git a/parquet-hadoop/src/main/java/parquet/hadoop/metadata/CompressionCodecName.java b/parquet-hadoop/src/main/java/parquet/hadoop/metadata/CompressionCodecName.java
index 195d59340a..808beb33c3 100644
--- a/parquet-hadoop/src/main/java/parquet/hadoop/metadata/CompressionCodecName.java
+++ b/parquet-hadoop/src/main/java/parquet/hadoop/metadata/CompressionCodecName.java
@@ -16,6 +16,7 @@
package parquet.hadoop.metadata;
import parquet.format.CompressionCodec;
+import parquet.hadoop.codec.CompressionCodecNotSupportedException;
public enum CompressionCodecName {
UNCOMPRESSED(null, CompressionCodec.UNCOMPRESSED, ""),
@@ -36,11 +37,11 @@ public static CompressionCodecName fromCompressionCodec(Class> clazz) {
}
String name = clazz.getName();
for (CompressionCodecName codec : CompressionCodecName.values()) {
- if (name.equals(codec.getHadoopCompressionCodecClass())) {
+ if (name.equals(codec.getHadoopCompressionCodecClassName())) {
return codec;
}
}
- throw new IllegalArgumentException("Unknown compression codec " + clazz);
+ throw new CompressionCodecNotSupportedException(clazz);
}
public static CompressionCodecName fromParquet(CompressionCodec codec) {
@@ -62,10 +63,22 @@ private CompressionCodecName(String hadoopCompressionCodecClass, CompressionCode
this.extension = extension;
}
- public String getHadoopCompressionCodecClass() {
+ public String getHadoopCompressionCodecClassName() {
return hadoopCompressionCodecClass;
}
+ public Class getHadoopCompressionCodecClass() {
+ String codecClassName = getHadoopCompressionCodecClassName();
+ if (codecClassName==null) {
+ return null;
+ }
+ try {
+ return Class.forName(codecClassName);
+ } catch (ClassNotFoundException e) {
+ return null;
+ }
+ }
+
public CompressionCodec getParquetCompressionCodec() {
return parquetCompressionCodec;
}
diff --git a/parquet-hadoop/src/main/java/parquet/hadoop/metadata/EncodingList.java b/parquet-hadoop/src/main/java/parquet/hadoop/metadata/EncodingList.java
index 4385492a4f..9d375c5e9b 100644
--- a/parquet-hadoop/src/main/java/parquet/hadoop/metadata/EncodingList.java
+++ b/parquet-hadoop/src/main/java/parquet/hadoop/metadata/EncodingList.java
@@ -15,7 +15,6 @@
*/
package parquet.hadoop.metadata;
-import java.util.Collection;
import java.util.Collections;
import java.util.HashMap;
import java.util.Iterator;
diff --git a/parquet-hadoop/src/test/java/parquet/hadoop/codec/CodecConfigTest.java b/parquet-hadoop/src/test/java/parquet/hadoop/codec/CodecConfigTest.java
new file mode 100644
index 0000000000..e5d3f7019e
--- /dev/null
+++ b/parquet-hadoop/src/test/java/parquet/hadoop/codec/CodecConfigTest.java
@@ -0,0 +1,75 @@
+/**
+ * Copyright 2012 Twitter, Inc.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package parquet.hadoop.codec;
+
+
+import junit.framework.Assert;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapreduce.*;
+import org.junit.Test;
+import parquet.hadoop.ParquetOutputFormat;
+import parquet.hadoop.metadata.CompressionCodecName;
+
+import java.io.IOException;
+import parquet.hadoop.util.ContextUtil;
+
+public class CodecConfigTest {
+ @Test
+ public void testReadingCodecs() throws IOException {
+ shouldUseParquetFlagToSetCodec("gzip", CompressionCodecName.GZIP);
+ shouldUseHadoopFlagToSetCodec(CompressionCodecName.GZIP.getHadoopCompressionCodecClassName(), CompressionCodecName.GZIP);
+ shouldUseParquetFlagToSetCodec("snappy", CompressionCodecName.SNAPPY);
+ shouldUseHadoopFlagToSetCodec(CompressionCodecName.SNAPPY.getHadoopCompressionCodecClassName(), CompressionCodecName.SNAPPY);
+ //When codec is unrecognized, use uncompressed
+ shouldUseHadoopFlagToSetCodec("unexistedCodec", CompressionCodecName.UNCOMPRESSED);
+ //For unsupported codec, use uncompressed
+ shouldUseHadoopFlagToSetCodec("org.apache.hadoop.io.compress.DefaultCodec", CompressionCodecName.UNCOMPRESSED);
+ }
+
+ public void shouldUseParquetFlagToSetCodec(String codecNameStr, CompressionCodecName expectedCodec) throws IOException {
+
+ //Test mapreduce API
+ Job job = new Job();
+ Configuration conf = job.getConfiguration();
+ conf.set(ParquetOutputFormat.COMPRESSION, codecNameStr);
+ TaskAttemptContext task = ContextUtil.newTaskAttemptContext(conf, new TaskAttemptID(new TaskID(new JobID("test", 1), false, 1), 1));
+ Assert.assertEquals(CodecConfig.from(task).getCodec(), expectedCodec);
+
+ //Test mapred API
+ JobConf jobConf = new JobConf();
+ jobConf.set(ParquetOutputFormat.COMPRESSION, codecNameStr);
+ Assert.assertEquals(CodecConfig.from(jobConf).getCodec(), expectedCodec);
+ }
+
+ public void shouldUseHadoopFlagToSetCodec(String codecClassStr, CompressionCodecName expectedCodec) throws IOException {
+ //Test mapreduce API
+ Job job = new Job();
+ Configuration conf = job.getConfiguration();
+ conf.setBoolean("mapred.output.compress", true);
+ conf.set("mapred.output.compression.codec", codecClassStr);
+ TaskAttemptContext task = ContextUtil.newTaskAttemptContext(conf, new TaskAttemptID(new TaskID(new JobID("test", 1), false, 1), 1));
+ Assert.assertEquals(expectedCodec, CodecConfig.from(task).getCodec());
+
+ //Test mapred API
+ JobConf jobConf = new JobConf();
+ jobConf.setBoolean("mapred.output.compress", true);
+ jobConf.set("mapred.output.compression.codec", codecClassStr);
+ Assert.assertEquals(CodecConfig.from(jobConf).getCodec(), expectedCodec);
+ }
+
+
+}
diff --git a/parquet-hive-bundle/pom.xml b/parquet-hive-bundle/pom.xml
new file mode 100644
index 0000000000..250fa77ebf
--- /dev/null
+++ b/parquet-hive-bundle/pom.xml
@@ -0,0 +1,58 @@
+
+
+ com.twitter
+ parquet
+ ../pom.xml
+ 1.2.11-SNAPSHOT
+
+
+ 4.0.0
+
+ parquet-hive-bundle
+ jar
+
+ Parquet Hive Bundle
+ https://github.com/Parquet/parquet-mr
+
+
+
+
+
+
+ com.twitter
+ parquet-hive-binding-bundle
+ ${project.version}
+
+
+ com.twitter
+ parquet-hive-storage-handler
+ ${project.version}
+
+
+
+
+
+
+ org.apache.maven.plugins
+ maven-shade-plugin
+
+
+ package
+
+ shade
+
+
+ false
+
+
+ com.twitter:parquet-*
+
+
+
+
+
+
+
+
+
+
diff --git a/parquet-hive-bundle/src/main/resources/parquet/bundle b/parquet-hive-bundle/src/main/resources/parquet/bundle
new file mode 100644
index 0000000000..e69de29bb2
diff --git a/parquet-hive/parquet-hive-binding/parquet-hive-0.10-binding/pom.xml b/parquet-hive/parquet-hive-binding/parquet-hive-0.10-binding/pom.xml
new file mode 100644
index 0000000000..2d43b8b7c7
--- /dev/null
+++ b/parquet-hive/parquet-hive-binding/parquet-hive-0.10-binding/pom.xml
@@ -0,0 +1,65 @@
+
+
+ com.twitter
+ parquet-hive-binding
+ ../pom.xml
+ 1.2.11-SNAPSHOT
+
+
+ 4.0.0
+
+ parquet-hive-0.10-binding
+ Parquet Hive 0.10 Binding
+ jar
+
+
+ 0.10.0
+
+
+
+
+ com.twitter
+ parquet-common
+ ${project.version}
+
+
+ com.twitter
+ parquet-hive-binding-interface
+ ${project.version}
+
+
+ org.apache.hadoop
+ hadoop-core
+ ${hadoop.version}
+ provided
+
+
+ org.apache.hive
+ hive-common
+ ${hive.version}
+ provided
+
+
+ org.apache.hive
+ hive-exec
+ ${hive.version}
+ provided
+
+
+ org.apache.hive
+ hive-serde
+ ${hive.version}
+ provided
+
+
+
+
+
+
+ org.apache.maven.plugins
+ maven-jar-plugin
+
+
+
+
+
diff --git a/parquet-hive/src/main/java/parquet/hive/ManageJobConfig.java b/parquet-hive/parquet-hive-binding/parquet-hive-0.10-binding/src/main/java/parquet/hive/internal/Hive010Binding.java
similarity index 56%
rename from parquet-hive/src/main/java/parquet/hive/ManageJobConfig.java
rename to parquet-hive/parquet-hive-binding/parquet-hive-0.10-binding/src/main/java/parquet/hive/internal/Hive010Binding.java
index 02286aec48..06af888163 100644
--- a/parquet-hive/src/main/java/parquet/hive/ManageJobConfig.java
+++ b/parquet-hive/parquet-hive-binding/parquet-hive-0.10-binding/src/main/java/parquet/hive/internal/Hive010Binding.java
@@ -1,26 +1,10 @@
-/**
- * Copyright 2013 Criteo.
- *
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package parquet.hive;
+package parquet.hive.internal;
import java.io.IOException;
import java.io.Serializable;
import java.util.ArrayList;
import java.util.Iterator;
import java.util.LinkedHashMap;
-import java.util.List;
import java.util.Map;
import java.util.Map.Entry;
@@ -35,41 +19,23 @@
import org.apache.hadoop.hive.ql.plan.TableScanDesc;
import org.apache.hadoop.hive.serde2.ColumnProjectionUtils;
import org.apache.hadoop.mapred.JobConf;
-import org.apache.hadoop.util.StringUtils;
+
+import parquet.Log;
/**
- *
- * ManageJobConfig quick workaround to initialize the job with column needed
- *
- * We are going to get the Table from a partition in order to get all the aliases from it.
- * Once we have them, we take a look at the different columns needed for each of them, and we update the job
- * by appending these columns.
- *
- * At the end, the new JobConf will contain all the wanted columns.
- *
- * @author Mickaël Lacour
- *
+ * Hive 0.10 implementation of {@link parquet.hive.HiveBinding HiveBinding}.
+ * This class is a renamed version of
+ * ManageJobConfig class.
*/
-public class ManageJobConfig {
-
- MapredWork mrwork;
- private Map pathToPartitionInfo;
-
+public class Hive010Binding extends AbstractHiveBinding {
+ private static final Log LOG = Log.getLog(Hive010Binding.class);
+ private final Map pathToPartitionInfo =
+ new LinkedHashMap();
/**
- * From a string which columns names (including hive column), return a list of string columns
- *
- * @param columns
- * @return
+ * MapredWork is the Hive object which describes input files,
+ * columns projections, and filters.
*/
- public static List getColumns(final String columns) {
- final List listColumns = (List) StringUtils.getStringCollection(columns);
- // Remove virtual Hive columns, hardcoded for compatibility
- listColumns.remove("INPUT__FILE__NAME");
- listColumns.remove("BLOCK__OFFSET__INSIDE__FILE");
- listColumns.remove("ROW__OFFSET__INSIDE__BLOCK");
- listColumns.remove("RAW__DATA__SIZE");
- return listColumns;
- }
+ private MapredWork mrwork;
/**
* Initialize the mrwork variable in order to get all the partition and start to update the jobconf
@@ -80,28 +46,26 @@ private void init(final JobConf job) {
final String plan = HiveConf.getVar(job, HiveConf.ConfVars.PLAN);
if (mrwork == null && plan != null && plan.length() > 0) {
mrwork = Utilities.getMapRedWork(job);
- pathToPartitionInfo = new LinkedHashMap();
+ pathToPartitionInfo.clear();
for (final Map.Entry entry : mrwork.getPathToPartitionInfo().entrySet()) {
pathToPartitionInfo.put(new Path(entry.getKey()).toUri().getPath().toString(), entry.getValue());
}
}
}
- private PartitionDesc getPartition(final String path) {
- if (pathToPartitionInfo == null) {
- return null;
- }
- return pathToPartitionInfo.get(path);
- }
+ private void pushProjectionsAndFilters(final JobConf jobConf,
+ final String splitPath, final String splitPathWithNoSchema) {
- private void pushProjectionsAndFilters(final JobConf jobConf, final String splitPath, final String splitPathWithNoSchema) {
-
- if (mrwork == null || mrwork.getPathToAliases() == null) {
+ if (mrwork == null) {
+ LOG.debug("Not pushing projections and filters because MapredWork is null");
+ return;
+ } else if (mrwork.getPathToAliases() == null) {
+ LOG.debug("Not pushing projections and filters because pathToAliases is null");
return;
}
final ArrayList aliases = new ArrayList();
- final Iterator>> iterator = this.mrwork.getPathToAliases().entrySet().iterator();
+ final Iterator>> iterator = mrwork.getPathToAliases().entrySet().iterator();
while (iterator.hasNext()) {
final Entry> entry = iterator.next();
@@ -116,7 +80,7 @@ private void pushProjectionsAndFilters(final JobConf jobConf, final String split
}
for (final String alias : aliases) {
- final Operator extends Serializable> op = this.mrwork.getAliasToWork().get(
+ final Operator extends Serializable> op = mrwork.getAliasToWork().get(
alias);
if (op != null && op instanceof TableScanOperator) {
final TableScanOperator tableScan = (TableScanOperator) op;
@@ -139,6 +103,7 @@ private void pushFilters(final JobConf jobConf, final TableScanOperator tableSca
final TableScanDesc scanDesc = tableScan.getConf();
if (scanDesc == null) {
+ LOG.debug("Not pushing filters because TableScanDesc is null");
return;
}
@@ -148,6 +113,7 @@ private void pushFilters(final JobConf jobConf, final TableScanOperator tableSca
// push down filters
final ExprNodeDesc filterExpr = scanDesc.getFilterExpr();
if (filterExpr == null) {
+ LOG.debug("Not pushing filters because FilterExpr is null");
return;
}
@@ -162,18 +128,14 @@ private void pushFilters(final JobConf jobConf, final TableScanOperator tableSca
}
/**
- *
- * Clone the job and try to update the new one with the requested columns
- *
- * @param jobConf
- * @param path : needed to be able to
- * @return the
- * @throws IOException
+ * {@inheritDoc}
*/
- public JobConf cloneJobAndInit(final JobConf jobConf, final Path path) throws IOException {
+ @Override
+ public JobConf pushProjectionsAndFilters(JobConf jobConf, Path path)
+ throws IOException {
init(jobConf);
final JobConf cloneJobConf = new JobConf(jobConf);
- final PartitionDesc part = getPartition(path.toString());
+ final PartitionDesc part = pathToPartitionInfo.get(path.toString());
if ((part != null) && (part.getTableDesc() != null)) {
Utilities.copyTableJobPropertiesToConf(part.getTableDesc(), cloneJobConf);
@@ -182,4 +144,5 @@ public JobConf cloneJobAndInit(final JobConf jobConf, final Path path) throws IO
pushProjectionsAndFilters(cloneJobConf, path.toString(), path.toUri().toString());
return cloneJobConf;
}
+
}
diff --git a/parquet-hive/parquet-hive-binding/parquet-hive-0.12-binding/pom.xml b/parquet-hive/parquet-hive-binding/parquet-hive-0.12-binding/pom.xml
new file mode 100644
index 0000000000..a5f7733aaa
--- /dev/null
+++ b/parquet-hive/parquet-hive-binding/parquet-hive-0.12-binding/pom.xml
@@ -0,0 +1,65 @@
+
+
+ com.twitter
+ parquet-hive-binding
+ ../pom.xml
+ 1.2.11-SNAPSHOT
+
+
+ 4.0.0
+
+ parquet-hive-0.12-binding
+ Parquet Hive 0.12 Binding
+ jar
+
+
+ 0.12.0
+
+
+
+
+ com.twitter
+ parquet-common
+ ${project.version}
+
+
+ com.twitter
+ parquet-hive-binding-interface
+ ${project.version}
+
+
+ org.apache.hadoop
+ hadoop-core
+ ${hadoop.version}
+ provided
+
+
+ org.apache.hive
+ hive-common
+ ${hive.version}
+ provided
+
+
+ org.apache.hive
+ hive-exec
+ ${hive.version}
+ provided
+
+
+ org.apache.hive
+ hive-serde
+ ${hive.version}
+ provided
+
+
+
+
+
+
+ org.apache.maven.plugins
+ maven-jar-plugin
+
+
+
+
+
diff --git a/parquet-hive/parquet-hive-binding/parquet-hive-0.12-binding/src/main/java/parquet/hive/internal/Hive012Binding.java b/parquet-hive/parquet-hive-binding/parquet-hive-0.12-binding/src/main/java/parquet/hive/internal/Hive012Binding.java
new file mode 100644
index 0000000000..a88254b3b4
--- /dev/null
+++ b/parquet-hive/parquet-hive-binding/parquet-hive-0.12-binding/src/main/java/parquet/hive/internal/Hive012Binding.java
@@ -0,0 +1,149 @@
+package parquet.hive.internal;
+
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.LinkedHashMap;
+import java.util.Map;
+import java.util.Map.Entry;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.ql.exec.Operator;
+import org.apache.hadoop.hive.ql.exec.TableScanOperator;
+import org.apache.hadoop.hive.ql.exec.Utilities;
+import org.apache.hadoop.hive.ql.plan.ExprNodeDesc;
+import org.apache.hadoop.hive.ql.plan.MapWork;
+import org.apache.hadoop.hive.ql.plan.PartitionDesc;
+import org.apache.hadoop.hive.ql.plan.TableScanDesc;
+import org.apache.hadoop.hive.serde2.ColumnProjectionUtils;
+import org.apache.hadoop.mapred.JobConf;
+
+import parquet.Log;
+
+/**
+ * Hive 0.12 implementation of {@link parquet.hive.HiveBinding HiveBinding}.
+ * This class is a copied and slightly modified version of
+ * ManageJobConfig class.
+ */
+public class Hive012Binding extends AbstractHiveBinding {
+ private static final Log LOG = Log.getLog(Hive012Binding.class);
+ private final Map pathToPartitionInfo =
+ new LinkedHashMap();
+ /**
+ * MapWork is the Hive object which describes input files,
+ * columns projections, and filters.
+ */
+ private MapWork mapWork;
+
+
+ /**
+ * Initialize the mapWork variable in order to get all the partition and start to update the jobconf
+ *
+ * @param job
+ */
+ private void init(final JobConf job) {
+ final String plan = HiveConf.getVar(job, HiveConf.ConfVars.PLAN);
+ if (mapWork == null && plan != null && plan.length() > 0) {
+ mapWork = Utilities.getMapWork(job);
+ pathToPartitionInfo.clear();
+ for (final Map.Entry entry : mapWork.getPathToPartitionInfo().entrySet()) {
+ pathToPartitionInfo.put(new Path(entry.getKey()).toUri().getPath().toString(), entry.getValue());
+ }
+ }
+ }
+
+ private void pushProjectionsAndFilters(final JobConf jobConf,
+ final String splitPath, final String splitPathWithNoSchema) {
+
+ if (mapWork == null) {
+ LOG.debug("Not pushing projections and filters because MapWork is null");
+ return;
+ } else if (mapWork.getPathToAliases() == null) {
+ LOG.debug("Not pushing projections and filters because pathToAliases is null");
+ return;
+ }
+
+ final ArrayList aliases = new ArrayList();
+ final Iterator>> iterator = mapWork.getPathToAliases().entrySet().iterator();
+
+ while (iterator.hasNext()) {
+ final Entry> entry = iterator.next();
+ final String key = new Path(entry.getKey()).toUri().getPath();
+
+ if (splitPath.equals(key) || splitPathWithNoSchema.equals(key)) {
+ final ArrayList list = entry.getValue();
+ for (final String val : list) {
+ aliases.add(val);
+ }
+ }
+ }
+
+ for (final String alias : aliases) {
+ final Operator extends Serializable> op = mapWork.getAliasToWork().get(
+ alias);
+ if (op != null && op instanceof TableScanOperator) {
+ final TableScanOperator tableScan = (TableScanOperator) op;
+
+ // push down projections
+ final ArrayList list = tableScan.getNeededColumnIDs();
+
+ if (list != null) {
+ ColumnProjectionUtils.appendReadColumnIDs(jobConf, list);
+ } else {
+ ColumnProjectionUtils.setFullyReadColumns(jobConf);
+ }
+
+ pushFilters(jobConf, tableScan);
+ }
+ }
+ }
+
+ private void pushFilters(final JobConf jobConf, final TableScanOperator tableScan) {
+
+ final TableScanDesc scanDesc = tableScan.getConf();
+ if (scanDesc == null) {
+ LOG.debug("Not pushing filters because TableScanDesc is null");
+ return;
+ }
+
+ // construct column name list for reference by filter push down
+ Utilities.setColumnNameList(jobConf, tableScan);
+
+ // push down filters
+ final ExprNodeDesc filterExpr = scanDesc.getFilterExpr();
+ if (filterExpr == null) {
+ LOG.debug("Not pushing filters because FilterExpr is null");
+ return;
+ }
+
+ final String filterText = filterExpr.getExprString();
+ final String filterExprSerialized = Utilities.serializeExpression(filterExpr);
+ jobConf.set(
+ TableScanDesc.FILTER_TEXT_CONF_STR,
+ filterText);
+ jobConf.set(
+ TableScanDesc.FILTER_EXPR_CONF_STR,
+ filterExprSerialized);
+ }
+
+ /**
+ * {@inheritDoc}
+ */
+ @Override
+ public JobConf pushProjectionsAndFilters(JobConf jobConf, Path path)
+ throws IOException {
+ init(jobConf);
+ final JobConf cloneJobConf = new JobConf(jobConf);
+ final PartitionDesc part = pathToPartitionInfo.get(path.toString());
+
+ if ((part != null) && (part.getTableDesc() != null)) {
+ Utilities.copyTableJobPropertiesToConf(part.getTableDesc(), cloneJobConf);
+ }
+
+ pushProjectionsAndFilters(cloneJobConf, path.toString(), path.toUri().toString());
+ return cloneJobConf;
+ }
+}
diff --git a/parquet-hive/parquet-hive-binding/parquet-hive-binding-bundle/pom.xml b/parquet-hive/parquet-hive-binding/parquet-hive-binding-bundle/pom.xml
new file mode 100644
index 0000000000..e85d5e777f
--- /dev/null
+++ b/parquet-hive/parquet-hive-binding/parquet-hive-binding-bundle/pom.xml
@@ -0,0 +1,86 @@
+
+
+
+
+ com.twitter
+ parquet-hive-binding
+ ../pom.xml
+ 1.2.11-SNAPSHOT
+
+
+ 4.0.0
+
+ parquet-hive-binding-bundle
+ Parquet Hive Binding Bundle
+ jar
+
+
+
+
+ com.twitter
+ parquet-common
+ ${project.version}
+
+
+ com.twitter
+ parquet-hive-binding-factory
+ ${project.version}
+ provided
+
+
+ com.twitter
+ parquet-hive-binding-interface
+ ${project.version}
+ provided
+
+
+ com.twitter
+ parquet-hive-0.10-binding
+ ${project.version}
+ provided
+
+
+ com.twitter
+ parquet-hive-0.12-binding
+ ${project.version}
+ provided
+
+
+
+
+
+
+ org.apache.maven.plugins
+ maven-assembly-plugin
+
+ false
+
+ src/assemble/uberjar.xml
+
+
+
+
+ uberjar
+ package
+
+ single
+
+
+
+
+
+
+
diff --git a/parquet-hive/parquet-hive-binding/parquet-hive-binding-bundle/src/assemble/uberjar.xml b/parquet-hive/parquet-hive-binding/parquet-hive-binding-bundle/src/assemble/uberjar.xml
new file mode 100644
index 0000000000..68326d46d8
--- /dev/null
+++ b/parquet-hive/parquet-hive-binding/parquet-hive-binding-bundle/src/assemble/uberjar.xml
@@ -0,0 +1,29 @@
+
+
+
+ uberjar
+
+ jar
+
+ false
+
+
+ true
+ provided
+ true
+ false
+
+
+
diff --git a/parquet-hive/parquet-hive-binding/parquet-hive-binding-bundle/src/main/resources/parquet/bundle b/parquet-hive/parquet-hive-binding/parquet-hive-binding-bundle/src/main/resources/parquet/bundle
new file mode 100644
index 0000000000..e69de29bb2
diff --git a/parquet-hive/parquet-hive-binding/parquet-hive-binding-factory/pom.xml b/parquet-hive/parquet-hive-binding/parquet-hive-binding-factory/pom.xml
new file mode 100644
index 0000000000..5e191caca8
--- /dev/null
+++ b/parquet-hive/parquet-hive-binding/parquet-hive-binding-factory/pom.xml
@@ -0,0 +1,53 @@
+
+
+ com.twitter
+ parquet-hive-binding
+ ../pom.xml
+ 1.2.11-SNAPSHOT
+
+
+ 4.0.0
+
+ parquet-hive-binding-factory
+ Parquet Hive Binding Factory
+ jar
+
+
+
+ com.twitter
+ parquet-common
+ ${project.version}
+
+
+ com.twitter
+ parquet-hive-0.10-binding
+ ${project.version}
+
+
+ com.twitter
+ parquet-hive-0.12-binding
+ ${project.version}
+
+
+ com.twitter
+ parquet-hive-binding-interface
+ ${project.version}
+
+
+ org.apache.hadoop
+ hadoop-core
+ ${hadoop.version}
+ test
+
+
+
+
+
+
+ org.apache.maven.plugins
+ maven-jar-plugin
+
+
+
+
+
diff --git a/parquet-hive/parquet-hive-binding/parquet-hive-binding-factory/src/main/java/parquet/hive/HiveBindingFactory.java b/parquet-hive/parquet-hive-binding/parquet-hive-binding-factory/src/main/java/parquet/hive/HiveBindingFactory.java
new file mode 100644
index 0000000000..0c88ef04a0
--- /dev/null
+++ b/parquet-hive/parquet-hive-binding/parquet-hive-binding-factory/src/main/java/parquet/hive/HiveBindingFactory.java
@@ -0,0 +1,147 @@
+/**
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package parquet.hive;
+
+import java.lang.reflect.Method;
+
+import parquet.Log;
+import parquet.hive.internal.Hive010Binding;
+import parquet.hive.internal.Hive012Binding;
+
+/**
+ * Factory for creating HiveBinding objects based on the version of Hive
+ * available in the classpath. This class does not provide static methods
+ * to enable mocking.
+ */
+public class HiveBindingFactory {
+ private static final Log LOG = Log.getLog(HiveBindingFactory.class);
+ private static final String HIVE_VERSION_CLASS_NAME = "org.apache.hive.common.util.HiveVersionInfo";
+ private static final String HIVE_VERSION_METHOD_NAME = "getVersion";
+ private static final String HIVE_UTILITIES_CLASS_NAME = "org.apache.hadoop.hive.ql.exec.Utilities";
+ private static final String HIVE_012_INDICATOR_UTILITIES_GETMAPWORK = "getMapWork";
+ private static final String HIVE_VERSION_NULL = "";
+ private static final Class extends HiveBinding> LATEST_BINDING = Hive012Binding.class;
+ static final String HIVE_VERSION_UNKNOWN = "Unknown";
+ static final String HIVE_VERSION_010 = "0.10";
+ static final String HIVE_VERSION_011 = "0.11";
+ static final String HIVE_VERSION_012 = "0.12";
+ static final String HIVE_VERSION_013 = "0.13";
+
+ /**
+ * @return HiveBinding based on the Hive version in the classpath
+ */
+ public HiveBinding create() {
+ Class extends HiveBinding> bindingClazz = create(HiveBindingFactory.class
+ .getClassLoader());
+ try {
+ return bindingClazz.newInstance();
+ } catch (Exception e) {
+ throw new HiveBindingInstantiationError("Unexpected error creating instance"
+ + " of " + bindingClazz.getCanonicalName(), e);
+ }
+ }
+
+ /**
+ * Internal method visible for testing purposes
+ */
+ @SuppressWarnings("rawtypes")
+ Class extends HiveBinding> create(ClassLoader classLoader) {
+ // HiveVersionInfo was added in 0.11, if the class does
+ // not exist then return the hive binding for 0.10
+ Class hiveVersionInfo;
+ try {
+ hiveVersionInfo = Class.forName(HIVE_VERSION_CLASS_NAME, true, classLoader);
+ } catch (ClassNotFoundException e) {
+ LOG.debug("Class " + HIVE_VERSION_CLASS_NAME + ", not found, returning " +
+ Hive010Binding.class.getSimpleName());
+ return Hive010Binding.class;
+ }
+ return createInternal(hiveVersionInfo);
+ }
+
+ /**
+ * Internal method visible for testing purposes
+ */
+ @SuppressWarnings({"unchecked", "rawtypes"})
+ Class extends HiveBinding> createInternal(Class hiveVersionInfo) {
+ String hiveVersion;
+ try {
+ Method getVersionMethod = hiveVersionInfo.
+ getMethod(HIVE_VERSION_METHOD_NAME, (Class[])null);
+ String rawVersion = (String)getVersionMethod.invoke(null, (Object[])null);
+ LOG.debug("Raw Version from " + hiveVersionInfo.getSimpleName() + " is '" +
+ rawVersion + "'");
+ hiveVersion = trimVersion(rawVersion);
+ } catch (Exception e) {
+ throw new UnexpectedHiveVersionProviderError("Unexpected error whilst " +
+ "determining Hive version", e);
+ }
+ if(hiveVersion.equalsIgnoreCase(HIVE_VERSION_UNKNOWN)) {
+ LOG.debug("Unknown hive version, attempting to guess");
+ return createBindingForUnknownVersion();
+ }
+ if(hiveVersion.startsWith(HIVE_VERSION_010)) {
+ LOG.debug("Hive version " + hiveVersion + ", returning " +
+ Hive010Binding.class.getSimpleName());
+ return Hive010Binding.class;
+ } else if(hiveVersion.startsWith(HIVE_VERSION_011)) {
+ LOG.debug("Hive version " + hiveVersion + ", returning " +
+ Hive010Binding.class.getSimpleName() + " as it's expected the 0.10 " +
+ "binding will work with 0.11");
+ return Hive010Binding.class;
+ }
+ LOG.debug("Hive version " + hiveVersion + ", returning " +
+ Hive012Binding.class.getSimpleName());
+ // as of 11/26/2013 it looks like the 0.12 binding will work for 0.13
+ return Hive012Binding.class;
+ }
+
+ private Class extends HiveBinding> createBindingForUnknownVersion() {
+ try {
+ Class> utilitiesClass = Class.forName(HIVE_UTILITIES_CLASS_NAME);
+ for(Method method : utilitiesClass.getDeclaredMethods()) {
+ if(HIVE_012_INDICATOR_UTILITIES_GETMAPWORK.equals(method.getName())) {
+ LOG.debug("Found " + HIVE_UTILITIES_CLASS_NAME + "." +
+ HIVE_012_INDICATOR_UTILITIES_GETMAPWORK + " returning 0.12 binding");
+ return Hive012Binding.class;
+ }
+ }
+ // if the getMapWork method does not exist then it must be 0.10 or 0.11
+ return Hive010Binding.class;
+ } catch (ClassNotFoundException e) {
+ LOG.debug("Could not find " + HIVE_UTILITIES_CLASS_NAME + ", returning" +
+ " the latest binding since this class existed in 0.10, 0.11, and 0.12");
+ return LATEST_BINDING;
+ }
+ }
+
+ private static String trimVersion(String s) {
+ if(s == null) {
+ return HIVE_VERSION_NULL;
+ }
+ return s.trim();
+ }
+ static class HiveBindingInstantiationError extends Error {
+ private static final long serialVersionUID = -9348060142128L;
+ public HiveBindingInstantiationError(String msg, Exception e) {
+ super(msg, e);
+ }
+ }
+ static class UnexpectedHiveVersionProviderError extends Error {
+ private static final long serialVersionUID = -7344858060142118L;
+ public UnexpectedHiveVersionProviderError(String msg, Exception e) {
+ super(msg, e);
+ }
+ }
+}
diff --git a/parquet-hive/parquet-hive-binding/parquet-hive-binding-factory/src/test/java/parquet/hive/TestHiveBindingFactory.java b/parquet-hive/parquet-hive-binding/parquet-hive-binding-factory/src/test/java/parquet/hive/TestHiveBindingFactory.java
new file mode 100644
index 0000000000..6385781b91
--- /dev/null
+++ b/parquet-hive/parquet-hive-binding/parquet-hive-binding-factory/src/test/java/parquet/hive/TestHiveBindingFactory.java
@@ -0,0 +1,121 @@
+package parquet.hive;
+
+import org.junit.Assert;
+
+import org.junit.Before;
+import org.junit.Test;
+
+import parquet.hive.internal.Hive010Binding;
+import parquet.hive.internal.Hive012Binding;
+import parquet.hive.HiveBindingFactory.UnexpectedHiveVersionProviderError;
+
+public class TestHiveBindingFactory {
+ private HiveBindingFactory hiveBindingFactory;
+
+ @Before
+ public void setup() {
+ hiveBindingFactory = new HiveBindingFactory();
+ }
+ @Test
+ public void testMissingHiveVersionInfoClass() {
+ Assert.assertEquals(Hive010Binding.class, hiveBindingFactory.
+ create(new NoopClassLoader()));
+ }
+ @Test(expected=UnexpectedHiveVersionProviderError.class)
+ public void testNoHiveVersion() {
+ hiveBindingFactory.createInternal(NoHiveVersion.class);
+ }
+ @Test
+ public void testBlankHiveVersion() {
+ hiveBindingFactory.createInternal(BlankHiveVersion.class);
+ Assert.assertEquals(Hive012Binding.class, hiveBindingFactory.
+ createInternal(BlankHiveVersion.class));
+ }
+ @Test
+ public void testUnknownHiveVersion() {
+ hiveBindingFactory.createInternal(BlankHiveVersion.class);
+ // returns 0.12 because we don't have hive in classpath
+ Assert.assertEquals(Hive012Binding.class, hiveBindingFactory.
+ createInternal(BlankHiveVersion.class));
+ }
+ @Test
+ public void testNullHiveVersion() {
+ hiveBindingFactory.createInternal(NullHiveVersion.class);
+ Assert.assertEquals(Hive012Binding.class, hiveBindingFactory.
+ createInternal(NullHiveVersion.class));
+ }
+ @Test
+ public void testHive010() {
+ Assert.assertEquals(Hive010Binding.class, hiveBindingFactory.
+ createInternal(Hive010Version.class));
+ }
+ @Test
+ public void testHive010WithSpaces() {
+ Assert.assertEquals(Hive010Binding.class, hiveBindingFactory.
+ createInternal(Hive010VersionWithSpaces.class));
+ }
+ @Test
+ public void testHive011() {
+ Assert.assertEquals(Hive010Binding.class, hiveBindingFactory.
+ createInternal(Hive011Version.class));
+ }
+ @Test
+ public void testHive012() {
+ Assert.assertEquals(Hive012Binding.class, hiveBindingFactory.
+ createInternal(Hive012Version.class));
+ }
+ @Test
+ public void testHive013() {
+ Assert.assertEquals(Hive012Binding.class, hiveBindingFactory.
+ createInternal(Hive013Version.class));
+ }
+
+ static class NoopClassLoader extends ClassLoader {
+ public Class> loadClass(String name) throws ClassNotFoundException {
+ throw new ClassNotFoundException(name);
+ }
+ }
+ static class NoHiveVersion {
+
+ }
+ static class BlankHiveVersion {
+ public static String getVersion() {
+ return "";
+ }
+ }
+ static class UnknownHiveVersion {
+ public static String getVersion() {
+ return HiveBindingFactory.HIVE_VERSION_UNKNOWN;
+ }
+ }
+ static class NullHiveVersion {
+ public static String getVersion() {
+ return null;
+ }
+ }
+ static class Hive010Version {
+ public static String getVersion() {
+ return HiveBindingFactory.HIVE_VERSION_010;
+ }
+ }
+ static class Hive010VersionWithSpaces {
+ public static String getVersion() {
+ return " " + HiveBindingFactory.HIVE_VERSION_010 + " ";
+ }
+ }
+ static class Hive011Version {
+ public static String getVersion() {
+ return HiveBindingFactory.HIVE_VERSION_011;
+ }
+ }
+ static class Hive012Version {
+ public static String getVersion() {
+ return HiveBindingFactory.HIVE_VERSION_012;
+ }
+ }
+ static class Hive013Version {
+ public static String getVersion() {
+ return HiveBindingFactory.HIVE_VERSION_013;
+ }
+ }
+}
diff --git a/parquet-hive/parquet-hive-binding/parquet-hive-binding-interface/pom.xml b/parquet-hive/parquet-hive-binding/parquet-hive-binding-interface/pom.xml
new file mode 100644
index 0000000000..6bf365e87b
--- /dev/null
+++ b/parquet-hive/parquet-hive-binding/parquet-hive-binding-interface/pom.xml
@@ -0,0 +1,33 @@
+
+
+ com.twitter
+ parquet-hive-binding
+ ../pom.xml
+ 1.2.11-SNAPSHOT
+
+
+ 4.0.0
+
+ parquet-hive-binding-interface
+ Parquet Hive Binding Interface
+ jar
+
+
+
+ org.apache.hadoop
+ hadoop-core
+ ${hadoop.version}
+ provided
+
+
+
+
+
+
+ org.apache.maven.plugins
+ maven-jar-plugin
+
+
+
+
+
diff --git a/parquet-hive/parquet-hive-binding/parquet-hive-binding-interface/src/main/java/parquet/hive/HiveBinding.java b/parquet-hive/parquet-hive-binding/parquet-hive-binding-interface/src/main/java/parquet/hive/HiveBinding.java
new file mode 100644
index 0000000000..858e93b313
--- /dev/null
+++ b/parquet-hive/parquet-hive-binding/parquet-hive-binding-interface/src/main/java/parquet/hive/HiveBinding.java
@@ -0,0 +1,52 @@
+/**
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package parquet.hive;
+
+import java.io.IOException;
+import java.util.List;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapred.JobConf;
+
+public interface HiveBinding {
+
+ /**
+ * From a string which columns names (including hive column), return a list
+ * of string columns
+ *
+ * @param comma separated list of columns
+ * @return list with virtual columns removed
+ */
+ public List getColumns(final String columns);
+
+ /**
+ * Processes the JobConf object pushing down projections and filters.
+ *
+ * We are going to get the Table from a partition in order to get all the
+ * aliases from it. Once we have them, we take a look at the different
+ * columns needed for each of them, and we update the job by appending
+ * these columns.
+ *
+ * The JobConf is modified and therefore is cloned first to ensure
+ * other owners are not impacted by the changes here. This is a standard
+ * practice when modifying JobConf objects in InputFormats, for example
+ * HCatalog does this.
+ *
+ * @param jobConf
+ * @param path
+ * @return cloned jobConf which can be used to read Parquet files
+ * @throws IOException
+ */
+ public JobConf pushProjectionsAndFilters(final JobConf jobConf, final Path path) throws IOException;
+}
diff --git a/parquet-hive/parquet-hive-binding/parquet-hive-binding-interface/src/main/java/parquet/hive/internal/AbstractHiveBinding.java b/parquet-hive/parquet-hive-binding/parquet-hive-binding-interface/src/main/java/parquet/hive/internal/AbstractHiveBinding.java
new file mode 100644
index 0000000000..649ece03b4
--- /dev/null
+++ b/parquet-hive/parquet-hive-binding/parquet-hive-binding-interface/src/main/java/parquet/hive/internal/AbstractHiveBinding.java
@@ -0,0 +1,36 @@
+package parquet.hive.internal;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.hadoop.util.StringUtils;
+
+import parquet.hive.HiveBinding;
+
+/**
+ * Common code among implementations of {@link parquet.hive.HiveBinding HiveBinding}.
+ */
+public abstract class AbstractHiveBinding implements HiveBinding {
+ private static final List virtualColumns;
+
+ static {
+ List vcols = new ArrayList();
+ vcols.add("INPUT__FILE__NAME");
+ vcols.add("BLOCK__OFFSET__INSIDE__FILE");
+ vcols.add("ROW__OFFSET__INSIDE__BLOCK");
+ vcols.add("RAW__DATA__SIZE");
+ virtualColumns = Collections.unmodifiableList(vcols);
+ }
+
+ /**
+ * {@inheritDoc}
+ */
+ @Override
+ public List getColumns(final String columns) {
+ final List result = (List) StringUtils.getStringCollection(columns);
+ result.removeAll(virtualColumns);
+ return result;
+ }
+
+}
diff --git a/parquet-hive/parquet-hive-binding/pom.xml b/parquet-hive/parquet-hive-binding/pom.xml
new file mode 100644
index 0000000000..c602490425
--- /dev/null
+++ b/parquet-hive/parquet-hive-binding/pom.xml
@@ -0,0 +1,32 @@
+
+
+ com.twitter
+ parquet-hive
+ ../pom.xml
+ 1.2.11-SNAPSHOT
+
+
+ 4.0.0
+
+ parquet-hive-binding
+ Parquet Hive Binding Parent
+ pom
+
+
+
+
+ repository.cloudera.com
+ repository.cloudera.com-releases
+ https://repository.cloudera.com/artifactory/datanucleus
+
+
+
+
+ parquet-hive-binding-interface
+ parquet-hive-0.10-binding
+ parquet-hive-0.12-binding
+ parquet-hive-binding-factory
+ parquet-hive-binding-bundle
+
+
+
diff --git a/parquet-hive/parquet-hive-storage-handler/pom.xml b/parquet-hive/parquet-hive-storage-handler/pom.xml
new file mode 100644
index 0000000000..b47d68b867
--- /dev/null
+++ b/parquet-hive/parquet-hive-storage-handler/pom.xml
@@ -0,0 +1,111 @@
+
+
+ com.twitter
+ parquet-hive
+ ../pom.xml
+ 1.2.11-SNAPSHOT
+
+
+ 4.0.0
+
+ parquet-hive-storage-handler
+ Parquet Hive Storage Handler
+ jar
+
+
+
+ repository.cloudera.com
+ repository.cloudera.com-releases
+ https://repository.cloudera.com/artifactory/datanucleus
+
+
+
+
+ 0.12.0
+
+
+
+
+ com.twitter
+ parquet-hive-binding-bundle
+ ${project.version}
+
+
+ com.twitter
+ parquet-column
+ ${project.version}
+
+
+ com.twitter
+ parquet-hadoop
+ ${project.version}
+
+
+ org.apache.hadoop
+ hadoop-core
+ ${hadoop.version}
+ provided
+
+
+ org.apache.hive
+ hive-serde
+ ${hive.version}
+ provided
+
+
+ org.apache.hive
+ hive-exec
+ ${hive.version}
+ provided
+
+
+ com.twitter
+ parquet-column
+ ${project.version}
+ test-jar
+ test
+
+
+
+ com.twitter
+ parquet-hive-binding-factory
+ ${project.version}
+ test
+
+
+ com.twitter
+ parquet-hive-binding-interface
+ ${project.version}
+ test
+
+
+ com.twitter
+ parquet-hive-0.10-binding
+ ${project.version}
+ test
+
+
+ com.twitter
+ parquet-hive-0.12-binding
+ ${project.version}
+ test
+
+
+ commons-lang
+ commons-lang
+ 2.4
+ test
+
+
+
+
+
+
+ org.apache.maven.plugins
+ maven-jar-plugin
+
+
+
+
+
diff --git a/parquet-hive/parquet-hive-storage-handler/src/main/java/parquet/hive/DeprecatedParquetInputFormat.java b/parquet-hive/parquet-hive-storage-handler/src/main/java/parquet/hive/DeprecatedParquetInputFormat.java
new file mode 100644
index 0000000000..fd9e0d5d45
--- /dev/null
+++ b/parquet-hive/parquet-hive-storage-handler/src/main/java/parquet/hive/DeprecatedParquetInputFormat.java
@@ -0,0 +1,35 @@
+/**
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package parquet.hive;
+
+import org.apache.hadoop.io.ArrayWritable;
+import org.apache.hadoop.mapreduce.InputFormat;
+
+/**
+ * Deprecated name of the parquet-hive input format. This class exists
+ * simply to provide backwards compatibility with users who specified
+ * this name in the Hive metastore. All users should now use
+ * {@link MapredParquetInputFormat MapredParquetInputFormat}
+ */
+@Deprecated
+public class DeprecatedParquetInputFormat extends MapredParquetInputFormat {
+
+ public DeprecatedParquetInputFormat() {
+ super();
+ }
+
+ public DeprecatedParquetInputFormat(final InputFormat realInputFormat) {
+ super(realInputFormat);
+ }
+}
diff --git a/parquet-hive/parquet-hive-storage-handler/src/main/java/parquet/hive/DeprecatedParquetOutputFormat.java b/parquet-hive/parquet-hive-storage-handler/src/main/java/parquet/hive/DeprecatedParquetOutputFormat.java
new file mode 100644
index 0000000000..07c3f6c2e8
--- /dev/null
+++ b/parquet-hive/parquet-hive-storage-handler/src/main/java/parquet/hive/DeprecatedParquetOutputFormat.java
@@ -0,0 +1,35 @@
+/**
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package parquet.hive;
+
+import org.apache.hadoop.io.ArrayWritable;
+import org.apache.hadoop.mapreduce.OutputFormat;
+
+/**
+ * Deprecated name of the parquet-hive output format. This class exists
+ * simply to provide backwards compatibility with users who specified
+ * this name in the Hive metastore. All users should now use
+ * {@link MapredParquetOutputFormat MapredParquetOutputFormat}
+ */
+@Deprecated
+public class DeprecatedParquetOutputFormat extends MapredParquetOutputFormat {
+
+ public DeprecatedParquetOutputFormat() {
+ super();
+ }
+
+ public DeprecatedParquetOutputFormat(final OutputFormat mapreduceOutputFormat) {
+ super(mapreduceOutputFormat);
+ }
+}
diff --git a/parquet-hive/src/main/java/parquet/hive/DeprecatedParquetInputFormat.java b/parquet-hive/parquet-hive-storage-handler/src/main/java/parquet/hive/MapredParquetInputFormat.java
similarity index 82%
rename from parquet-hive/src/main/java/parquet/hive/DeprecatedParquetInputFormat.java
rename to parquet-hive/parquet-hive-storage-handler/src/main/java/parquet/hive/MapredParquetInputFormat.java
index 072d702e59..7f2dae0e4a 100644
--- a/parquet-hive/src/main/java/parquet/hive/DeprecatedParquetInputFormat.java
+++ b/parquet-hive/parquet-hive-storage-handler/src/main/java/parquet/hive/MapredParquetInputFormat.java
@@ -1,13 +1,17 @@
/**
* Copyright 2013 Criteo.
*
- * Licensed 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
+ * Licensed 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.
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
*/
package parquet.hive;
@@ -17,8 +21,6 @@
import java.util.ArrayList;
import java.util.List;
-import org.apache.hadoop.fs.BlockLocation;
-import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.io.ArrayWritable;
@@ -51,37 +53,46 @@
*
* A Parquet InputFormat for Hive (with the deprecated package mapred)
*
- * TODO : Refactor all of the wrappers here
- * Talk about it on : https://github.com/Parquet/parquet-mr/pull/28s
+ * TODO : Refactor all of the wrappers here Talk about it on : https://github.com/Parquet/parquet-mr/pull/28s
*
* @author Mickaël Lacour
* @author Rémy Pecqueur
*
*/
-public class DeprecatedParquetInputFormat extends FileInputFormat {
+public class MapredParquetInputFormat extends FileInputFormat {
- private static final Log LOG = Log.getLog(DeprecatedParquetInputFormat.class);
- protected ParquetInputFormat realInput;
+ private static final Log LOG = Log.getLog(MapredParquetInputFormat.class);
+ private final ParquetInputFormat realInput;
+ private final HiveBinding hiveBinding;
- public DeprecatedParquetInputFormat() {
- this.realInput = new ParquetInputFormat(DataWritableReadSupport.class);
+ public MapredParquetInputFormat() {
+ this(new ParquetInputFormat(DataWritableReadSupport.class),
+ new HiveBindingFactory().create());
}
- public DeprecatedParquetInputFormat(final InputFormat realInputFormat) {
- this.realInput = (ParquetInputFormat) realInputFormat;
+ public MapredParquetInputFormat(final InputFormat realInputFormat) {
+ this((ParquetInputFormat) realInputFormat, new HiveBindingFactory().create());
}
- private final ManageJobConfig manageJob = new ManageJobConfig();
+
+ private MapredParquetInputFormat(final ParquetInputFormat inputFormat,
+ final HiveBinding hiveBinding) {
+ this.realInput = inputFormat;
+ this.hiveBinding = hiveBinding;
+ }
+
@Override
- public org.apache.hadoop.mapred.InputSplit[] getSplits(final org.apache.hadoop.mapred.JobConf job, final int numSplits) throws IOException {
+ public org.apache.hadoop.mapred.InputSplit[] getSplits(final org.apache.hadoop.mapred.JobConf job,
+ final int numSplits) throws IOException {
final Path[] dirs = FileInputFormat.getInputPaths(job);
if (dirs.length == 0) {
throw new IOException("No input paths specified in job");
}
final Path tmpPath = new Path((dirs[dirs.length - 1]).makeQualified(FileSystem.get(job)).toUri().getPath());
- final JobConf cloneJobConf = manageJob.cloneJobAndInit(job, tmpPath);
- final List splits = realInput.getSplits(ContextUtil.newJobContext(cloneJobConf, null));
+ final JobConf cloneJobConf = hiveBinding.pushProjectionsAndFilters(job, tmpPath);
+ final List splits = realInput.
+ getSplits(ContextUtil.newJobContext(cloneJobConf, null));
if (splits == null) {
return null;
@@ -107,7 +118,7 @@ public org.apache.hadoop.mapred.RecordReader getRecordReade
try {
return (RecordReader) new RecordReaderWrapper(realInput, split, job, reporter);
} catch (final InterruptedException e) {
- throw new RuntimeException("Cannot create a ReacordReaderWrapper", e);
+ throw new RuntimeException("Cannot create a RecordReaderWrapper", e);
}
}
@@ -137,9 +148,9 @@ public long getLength() {
try {
return realSplit.getLength();
} catch (IOException ex) {
- throw new RuntimeException("Cannot get the length of the ParquetInputSlipt", ex);
+ throw new RuntimeException("Cannot get the length of the ParquetInputSplit: " + realSplit, ex);
} catch (InterruptedException ex) {
- throw new RuntimeException("Cannot get the length of the ParquetInputSlipt", ex);
+ throw new RuntimeException("Cannot get the length of the ParquetInputSplit: " + realSplit, ex);
}
}
}
@@ -185,21 +196,23 @@ public long getStart() {
}
}
- private static class RecordReaderWrapper implements RecordReader {
+ protected static class RecordReaderWrapper implements RecordReader {
- private org.apache.hadoop.mapreduce.RecordReader realReader;
+ private final HiveBinding hiveBinding;
private final long splitLen; // for getPos()
+
+ private org.apache.hadoop.mapreduce.RecordReader realReader;
// expect readReader return same Key & Value objects (common case)
// this avoids extra serialization & deserialization of these objects
private ArrayWritable valueObj = null;
- private final ManageJobConfig manageJob = new ManageJobConfig();
private boolean firstRecord = false;
private boolean eof = false;
private int schemaSize;
- public RecordReaderWrapper(final ParquetInputFormat newInputFormat, final InputSplit oldSplit, final JobConf oldJobConf, final Reporter reporter)
+ public RecordReaderWrapper(final ParquetInputFormat newInputFormat,
+ final InputSplit oldSplit, final JobConf oldJobConf, final Reporter reporter)
throws IOException, InterruptedException {
-
+ hiveBinding = new HiveBindingFactory().create();
splitLen = oldSplit.getLength();
final ParquetInputSplit split = getSplit(oldSplit, oldJobConf);
@@ -311,18 +324,18 @@ public boolean next(final Void key, final ArrayWritable value) throws IOExceptio
* gets a ParquetInputSplit corresponding to a split given by Hive
*
* @param oldSplit The split given by Hive
- * @param conf The JobConf of the Hive job
+ * @param conf The JobConf of the Hive job
* @return a ParquetInputSplit corresponding to the oldSplit
* @throws IOException if the config cannot be enhanced or if the footer cannot be read from the file
*/
- private ParquetInputSplit getSplit(final InputSplit oldSplit, final JobConf conf) throws IOException {
+ protected ParquetInputSplit getSplit(final InputSplit oldSplit, final JobConf conf) throws IOException {
ParquetInputSplit split;
if (oldSplit instanceof InputSplitWrapper) {
split = ((InputSplitWrapper) oldSplit).getRealSplit();
} else if (oldSplit instanceof FileSplit) {
final Path finalPath = ((FileSplit) oldSplit).getPath();
- final JobConf cloneJob = manageJob.cloneJobAndInit(conf, finalPath.getParent());
+ final JobConf cloneJob = hiveBinding.pushProjectionsAndFilters(conf, finalPath.getParent());
final ParquetMetadata parquetMetadata = ParquetFileReader.readFooter(cloneJob, finalPath);
final List blocks = parquetMetadata.getBlocks();
@@ -350,14 +363,14 @@ private ParquetInputSplit getSplit(final InputSplit oldSplit, final JobConf conf
splitLength,
((FileSplit) oldSplit).getLocations(),
splitGroup,
- fileMetaData.getSchema().toString(),
readContext.getRequestedSchema().toString(),
+ fileMetaData.getSchema().toString(),
fileMetaData.getKeyValueMetaData(),
readContext.getReadSupportMetadata());
}
} else {
- throw new RuntimeException("Unknown split type: " + oldSplit);
+ throw new IllegalArgumentException("Unknown split type: " + oldSplit);
}
return split;
diff --git a/parquet-hive/src/main/java/parquet/hive/DeprecatedParquetOutputFormat.java b/parquet-hive/parquet-hive-storage-handler/src/main/java/parquet/hive/MapredParquetOutputFormat.java
similarity index 86%
rename from parquet-hive/src/main/java/parquet/hive/DeprecatedParquetOutputFormat.java
rename to parquet-hive/parquet-hive-storage-handler/src/main/java/parquet/hive/MapredParquetOutputFormat.java
index 46b86940ae..4cd811a9e3 100644
--- a/parquet-hive/src/main/java/parquet/hive/DeprecatedParquetOutputFormat.java
+++ b/parquet-hive/parquet-hive-storage-handler/src/main/java/parquet/hive/MapredParquetOutputFormat.java
@@ -1,13 +1,17 @@
/**
* Copyright 2013 Criteo.
*
- * Licensed 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
+ * Licensed 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.
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
*/
package parquet.hive;
@@ -51,25 +55,26 @@
*
*/
@SuppressWarnings({"unchecked", "rawtypes"})
-public class DeprecatedParquetOutputFormat extends FileOutputFormat implements HiveOutputFormat {
+public class MapredParquetOutputFormat extends FileOutputFormat implements HiveOutputFormat {
protected ParquetOutputFormat realOutputFormat;
- public DeprecatedParquetOutputFormat() {
+ public MapredParquetOutputFormat() {
realOutputFormat = new ParquetOutputFormat(new DataWritableWriteSupport());
}
- public DeprecatedParquetOutputFormat(final OutputFormat mapreduceOutputFormat) {
+ public MapredParquetOutputFormat(final OutputFormat mapreduceOutputFormat) {
realOutputFormat = (ParquetOutputFormat) mapreduceOutputFormat;
}
@Override
public void checkOutputSpecs(final FileSystem ignored, final JobConf job) throws IOException {
- realOutputFormat.checkOutputSpecs(new JobContext(job, null));
+ realOutputFormat.checkOutputSpecs(ContextUtil.newJobContext(job, null));
}
@Override
- public RecordWriter getRecordWriter(final FileSystem ignored, final JobConf job, final String name, final Progressable progress) throws IOException {
+ public RecordWriter getRecordWriter(final FileSystem ignored,
+ final JobConf job, final String name, final Progressable progress) throws IOException {
throw new RuntimeException("Should never be used");
}
diff --git a/parquet-hive/src/main/java/parquet/hive/convert/ArrayWritableGroupConverter.java b/parquet-hive/parquet-hive-storage-handler/src/main/java/parquet/hive/convert/ArrayWritableGroupConverter.java
similarity index 100%
rename from parquet-hive/src/main/java/parquet/hive/convert/ArrayWritableGroupConverter.java
rename to parquet-hive/parquet-hive-storage-handler/src/main/java/parquet/hive/convert/ArrayWritableGroupConverter.java
diff --git a/parquet-hive/src/main/java/parquet/hive/convert/DataWritableGroupConverter.java b/parquet-hive/parquet-hive-storage-handler/src/main/java/parquet/hive/convert/DataWritableGroupConverter.java
similarity index 97%
rename from parquet-hive/src/main/java/parquet/hive/convert/DataWritableGroupConverter.java
rename to parquet-hive/parquet-hive-storage-handler/src/main/java/parquet/hive/convert/DataWritableGroupConverter.java
index 42ef78d43e..f21e18ee8d 100644
--- a/parquet-hive/src/main/java/parquet/hive/convert/DataWritableGroupConverter.java
+++ b/parquet-hive/parquet-hive-storage-handler/src/main/java/parquet/hive/convert/DataWritableGroupConverter.java
@@ -30,7 +30,6 @@
*/
public class DataWritableGroupConverter extends HiveGroupConverter {
- private final GroupType groupType;
private final Converter[] converters;
private final HiveGroupConverter parent;
private final int index;
@@ -48,7 +47,6 @@ public DataWritableGroupConverter(final GroupType groupType, final HiveGroupConv
}
public DataWritableGroupConverter(final GroupType selectedGroupType, final HiveGroupConverter parent, final int index, final GroupType containingGroupType) {
- this.groupType = selectedGroupType;
this.parent = parent;
this.index = index;
final int totalFieldCount = containingGroupType.getFieldCount();
@@ -79,7 +77,7 @@ final public ArrayWritable getCurrentArray() {
for (int i = 0; i < currentArr.length; i++) {
final Object obj = currentArr[i];
if (obj instanceof List) {
- final List objList = (List) (obj);
+ final List> objList = (List>)obj;
final ArrayWritable arr = new ArrayWritable(Writable.class, objList.toArray(new Writable[objList.size()]));
writableArr[i] = arr;
} else {
diff --git a/parquet-hive/src/main/java/parquet/hive/convert/DataWritableRecordConverter.java b/parquet-hive/parquet-hive-storage-handler/src/main/java/parquet/hive/convert/DataWritableRecordConverter.java
similarity index 100%
rename from parquet-hive/src/main/java/parquet/hive/convert/DataWritableRecordConverter.java
rename to parquet-hive/parquet-hive-storage-handler/src/main/java/parquet/hive/convert/DataWritableRecordConverter.java
diff --git a/parquet-hive/parquet-hive-storage-handler/src/main/java/parquet/hive/convert/ETypeConverter.java b/parquet-hive/parquet-hive-storage-handler/src/main/java/parquet/hive/convert/ETypeConverter.java
new file mode 100644
index 0000000000..81e893862c
--- /dev/null
+++ b/parquet-hive/parquet-hive-storage-handler/src/main/java/parquet/hive/convert/ETypeConverter.java
@@ -0,0 +1,165 @@
+/**
+ * Copyright 2013 Criteo.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package parquet.hive.convert;
+
+import java.math.BigDecimal;
+
+import org.apache.hadoop.hive.serde2.io.DoubleWritable;
+import org.apache.hadoop.io.BooleanWritable;
+import org.apache.hadoop.io.FloatWritable;
+import org.apache.hadoop.io.IntWritable;
+import org.apache.hadoop.io.LongWritable;
+
+import parquet.column.Dictionary;
+import parquet.hive.writable.BinaryWritable;
+import parquet.hive.writable.BinaryWritable.DicBinaryWritable;
+import parquet.io.api.Binary;
+import parquet.io.api.Converter;
+import parquet.io.api.PrimitiveConverter;
+
+/**
+ *
+ * ETypeConverter is an easy way to set the converter for the right type.
+ *
+ *
+ * @author Mickaël Lacour
+ *
+ */
+public enum ETypeConverter {
+
+ EDOUBLE_CONVERTER(Double.TYPE) {
+ @Override
+ Converter getConverter(final Class> type, final int index, final HiveGroupConverter parent) {
+ return new PrimitiveConverter() {
+ @Override
+ final public void addDouble(final double value) {
+ parent.set(index, new DoubleWritable(value));
+ }
+ };
+ }
+ },
+ EBOOLEAN_CONVERTER(Boolean.TYPE) {
+ @Override
+ Converter getConverter(final Class> type, final int index, final HiveGroupConverter parent) {
+ return new PrimitiveConverter() {
+ @Override
+ final public void addBoolean(final boolean value) {
+ parent.set(index, new BooleanWritable(value));
+ }
+ };
+ }
+ },
+ EFLOAT_CONVERTER(Float.TYPE) {
+ @Override
+ Converter getConverter(final Class> type, final int index, final HiveGroupConverter parent) {
+ return new PrimitiveConverter() {
+ @Override
+ final public void addFloat(final float value) {
+ parent.set(index, new FloatWritable(value));
+ }
+ };
+ }
+ },
+ EINT32_CONVERTER(Integer.TYPE) {
+ @Override
+ Converter getConverter(final Class> type, final int index, final HiveGroupConverter parent) {
+ return new PrimitiveConverter() {
+ @Override
+ final public void addInt(final int value) {
+ parent.set(index, new IntWritable(value));
+ }
+ };
+ }
+ },
+ EINT64_CONVERTER(Long.TYPE) {
+ @Override
+ Converter getConverter(final Class> type, final int index, final HiveGroupConverter parent) {
+ return new PrimitiveConverter() {
+ @Override
+ final public void addLong(final long value) {
+ parent.set(index, new LongWritable(value));
+ }
+ };
+ }
+ },
+ EINT96_CONVERTER(BigDecimal.class) {
+ @Override
+ Converter getConverter(final Class> type, final int index, final HiveGroupConverter parent) {
+ return new PrimitiveConverter() {
+ @Override
+ final public void addDouble(final double value) {
+ parent.set(index, new DoubleWritable(value));
+ }
+ };
+ }
+ },
+ EBINARY_CONVERTER(Binary.class) {
+ @Override
+ Converter getConverter(final Class> type, final int index, final HiveGroupConverter parent) {
+ return new PrimitiveConverter() {
+ private Binary[] dictBinary;
+ private String[] dict;
+
+ @Override
+ public boolean hasDictionarySupport() {
+ return true;
+ }
+
+ @Override
+ public void setDictionary(Dictionary dictionary) {
+ dictBinary = new Binary[dictionary.getMaxId() + 1];
+ dict = new String[dictionary.getMaxId() + 1];
+ for (int i = 0; i <= dictionary.getMaxId(); i++) {
+ Binary binary = dictionary.decodeToBinary(i);
+ dictBinary[i] = binary;
+ dict[i] = binary.toStringUsingUTF8();
+ }
+ }
+
+ @Override
+ public void addValueFromDictionary(int dictionaryId) {
+ parent.set(index, new DicBinaryWritable(dictBinary[dictionaryId], dict[dictionaryId]));
+ }
+
+ @Override
+ final public void addBinary(Binary value) {
+ parent.set(index, new BinaryWritable(value));
+ }
+ };
+ }
+ };
+ final Class> _type;
+
+ private ETypeConverter(final Class> type) {
+ this._type = type;
+ }
+
+ private Class> getType() {
+ return _type;
+ }
+
+ abstract Converter getConverter(final Class> type, final int index, final HiveGroupConverter parent);
+
+ static public Converter getNewConverter(final Class> type, final int index, final HiveGroupConverter parent) {
+ for (final ETypeConverter eConverter : values()) {
+ if (eConverter.getType() == type) {
+ return eConverter.getConverter(type, index, parent);
+ }
+ }
+ throw new RuntimeException("Converter not found ... for type : " + type);
+ }
+
+}
diff --git a/parquet-hive/src/main/java/parquet/hive/convert/HiveGroupConverter.java b/parquet-hive/parquet-hive-storage-handler/src/main/java/parquet/hive/convert/HiveGroupConverter.java
similarity index 100%
rename from parquet-hive/src/main/java/parquet/hive/convert/HiveGroupConverter.java
rename to parquet-hive/parquet-hive-storage-handler/src/main/java/parquet/hive/convert/HiveGroupConverter.java
diff --git a/parquet-hive/src/main/java/parquet/hive/convert/HiveSchemaConverter.java b/parquet-hive/parquet-hive-storage-handler/src/main/java/parquet/hive/convert/HiveSchemaConverter.java
similarity index 93%
rename from parquet-hive/src/main/java/parquet/hive/convert/HiveSchemaConverter.java
rename to parquet-hive/parquet-hive-storage-handler/src/main/java/parquet/hive/convert/HiveSchemaConverter.java
index 1e268b544b..dd70469812 100644
--- a/parquet-hive/src/main/java/parquet/hive/convert/HiveSchemaConverter.java
+++ b/parquet-hive/parquet-hive-storage-handler/src/main/java/parquet/hive/convert/HiveSchemaConverter.java
@@ -17,7 +17,6 @@
import java.util.List;
-import org.apache.commons.lang.NotImplementedException;
import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector.Category;
import org.apache.hadoop.hive.serde2.typeinfo.ListTypeInfo;
import org.apache.hadoop.hive.serde2.typeinfo.MapTypeInfo;
@@ -87,13 +86,13 @@ static private Type convertType(final String name, final TypeInfo typeInfo, fina
return new PrimitiveType(repetition, PrimitiveTypeName.BOOLEAN, name);
} else if (typeInfo.equals(TypeInfoFactory.binaryTypeInfo)) {
// TODO : binaryTypeInfo is a byte array. Need to map it
- throw new NotImplementedException("Binary type not implemented");
+ throw new UnsupportedOperationException("Binary type not implemented");
} else if (typeInfo.equals(TypeInfoFactory.timestampTypeInfo)) {
- throw new NotImplementedException("Timestamp type not implemented");
+ throw new UnsupportedOperationException("Timestamp type not implemented");
} else if (typeInfo.equals(TypeInfoFactory.voidTypeInfo)) {
- throw new NotImplementedException("Void type not implemented");
+ throw new UnsupportedOperationException("Void type not implemented");
} else if (typeInfo.equals(TypeInfoFactory.unknownTypeInfo)) {
- throw new NotImplementedException("Unknown type not implemented");
+ throw new UnsupportedOperationException("Unknown type not implemented");
} else {
throw new RuntimeException("Unknown type: " + typeInfo);
}
@@ -104,7 +103,7 @@ static private Type convertType(final String name, final TypeInfo typeInfo, fina
} else if (typeInfo.getCategory().equals(Category.MAP)) {
return convertMapType(name, (MapTypeInfo) typeInfo);
} else if (typeInfo.getCategory().equals(Category.UNION)) {
- throw new NotImplementedException("Union type not implemented");
+ throw new UnsupportedOperationException("Union type not implemented");
} else {
throw new RuntimeException("Unknown type: " + typeInfo);
}
diff --git a/parquet-hive/src/main/java/parquet/hive/read/DataWritableReadSupport.java b/parquet-hive/parquet-hive-storage-handler/src/main/java/parquet/hive/read/DataWritableReadSupport.java
similarity index 56%
rename from parquet-hive/src/main/java/parquet/hive/read/DataWritableReadSupport.java
rename to parquet-hive/parquet-hive-storage-handler/src/main/java/parquet/hive/read/DataWritableReadSupport.java
index 82e730f10a..77ecd00671 100644
--- a/parquet-hive/src/main/java/parquet/hive/read/DataWritableReadSupport.java
+++ b/parquet-hive/parquet-hive-storage-handler/src/main/java/parquet/hive/read/DataWritableReadSupport.java
@@ -1,13 +1,14 @@
/**
* Copyright 2013 Criteo.
*
- * Licensed 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
+ * Licensed 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.
+ * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
+ * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations under the License.
*/
package parquet.hive.read;
@@ -21,7 +22,7 @@
import org.apache.hadoop.io.ArrayWritable;
import parquet.hadoop.api.ReadSupport;
-import parquet.hive.ManageJobConfig;
+import parquet.hive.HiveBindingFactory;
import parquet.hive.convert.DataWritableRecordConverter;
import parquet.io.api.RecordMaterializer;
import parquet.schema.MessageType;
@@ -48,49 +49,53 @@ public class DataWritableReadSupport extends ReadSupport {
*
* It creates the readContext for Parquet side with the requested schema during the init phase.
*
- * @param configuration needed to get the wanted columns
+ * @param configuration needed to get the wanted columns
* @param keyValueMetaData // unused
- * @param fileSchema parquet file schema
+ * @param fileSchema parquet file schema
* @return the parquet ReadContext
*/
@Override
public parquet.hadoop.api.ReadSupport.ReadContext init(final Configuration configuration, final Map keyValueMetaData, final MessageType fileSchema) {
final String columns = configuration.get("columns");
- final List listColumns = ManageJobConfig.getColumns(columns);
final Map contextMetadata = new HashMap();
+ if (columns != null) {
+ final List listColumns = (new HiveBindingFactory()).create().getColumns(columns);
- final List typeListTable = new ArrayList();
- for (final String col : listColumns) {
- if (fileSchema.containsField(col)) {
- typeListTable.add(fileSchema.getType(col));
- } else { // dummy type, should not be called
- typeListTable.add(new PrimitiveType(Repetition.OPTIONAL, PrimitiveTypeName.BINARY, col));
+ final List typeListTable = new ArrayList();
+ for (final String col : listColumns) {
+ if (fileSchema.containsField(col)) {
+ typeListTable.add(fileSchema.getType(col));
+ } else { // dummy type, should not be called
+ typeListTable.add(new PrimitiveType(Repetition.OPTIONAL, PrimitiveTypeName.BINARY, col));
+ }
}
- }
- final MessageType tableSchema = new MessageType("table_schema", typeListTable);
- contextMetadata.put(HIVE_SCHEMA_KEY, tableSchema.toString());
-
- MessageType requestedSchemaByUser = tableSchema;
- final List indexColumnsWanted = ColumnProjectionUtils.getReadColumnIDs(configuration);
+ MessageType tableSchema = new MessageType("table_schema", typeListTable);
+ contextMetadata.put(HIVE_SCHEMA_KEY, tableSchema.toString());
- final List typeListWanted = new ArrayList();
- for (final Integer idx : indexColumnsWanted) {
- typeListWanted.add(tableSchema.getType(listColumns.get(idx)));
- }
- requestedSchemaByUser = new MessageType(fileSchema.getName(), typeListWanted);
+ MessageType requestedSchemaByUser = tableSchema;
+ final List indexColumnsWanted = ColumnProjectionUtils.getReadColumnIDs(configuration);
+ final List typeListWanted = new ArrayList();
+ for (final Integer idx : indexColumnsWanted) {
+ typeListWanted.add(tableSchema.getType(listColumns.get(idx)));
+ }
+ requestedSchemaByUser = new MessageType(fileSchema.getName(), typeListWanted);
- return new ReadContext(requestedSchemaByUser, contextMetadata);
+ return new ReadContext(requestedSchemaByUser, contextMetadata);
+ } else {
+ contextMetadata.put(HIVE_SCHEMA_KEY, fileSchema.toString());
+ return new ReadContext(fileSchema, contextMetadata);
+ }
}
/**
*
* It creates the hive read support to interpret data from parquet to hive
*
- * @param configuration // unused
+ * @param configuration // unused
* @param keyValueMetaData
- * @param fileSchema // unused
- * @param readContext containing the requested schema and the schema of the hive table
+ * @param fileSchema // unused
+ * @param readContext containing the requested schema and the schema of the hive table
* @return Record Materialize for Hive
*/
@Override
diff --git a/parquet-hive/parquet-hive-storage-handler/src/main/java/parquet/hive/serde/AbstractParquetMapInspector.java b/parquet-hive/parquet-hive-storage-handler/src/main/java/parquet/hive/serde/AbstractParquetMapInspector.java
new file mode 100644
index 0000000000..83bf00f5eb
--- /dev/null
+++ b/parquet-hive/parquet-hive-storage-handler/src/main/java/parquet/hive/serde/AbstractParquetMapInspector.java
@@ -0,0 +1,163 @@
+/**
+ * Copyright 2013 Criteo.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
+ * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations under the License.
+ */
+package parquet.hive.serde;
+
+import java.util.HashMap;
+import java.util.Map;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.SettableMapObjectInspector;
+import org.apache.hadoop.io.ArrayWritable;
+import org.apache.hadoop.io.Writable;
+
+/**
+ *
+ * @author Rémy Pecqueur
+ */
+public abstract class AbstractParquetMapInspector implements SettableMapObjectInspector {
+
+ protected final ObjectInspector keyInspector;
+ protected final ObjectInspector valueInspector;
+
+ public AbstractParquetMapInspector(final ObjectInspector keyInspector, final ObjectInspector valueInspector) {
+ this.keyInspector = keyInspector;
+ this.valueInspector = valueInspector;
+ }
+
+ @Override
+ public String getTypeName() {
+ return "map<" + keyInspector.getTypeName() + "," + valueInspector.getTypeName() + ">";
+ }
+
+ @Override
+ public Category getCategory() {
+ return Category.MAP;
+ }
+
+ @Override
+ public ObjectInspector getMapKeyObjectInspector() {
+ return keyInspector;
+ }
+
+ @Override
+ public ObjectInspector getMapValueObjectInspector() {
+ return valueInspector;
+ }
+
+ @Override
+ public Map, ?> getMap(final Object data) {
+ if (data == null) {
+ return null;
+ }
+
+ if (data instanceof ArrayWritable) {
+ final Writable[] mapContainer = ((ArrayWritable) data).get();
+
+ if (mapContainer == null || mapContainer.length == 0) {
+ return null;
+ }
+
+ final Writable[] mapArray = ((ArrayWritable) mapContainer[0]).get();
+ final Map map = new HashMap();
+
+ for (final Writable obj : mapArray) {
+ final ArrayWritable mapObj = (ArrayWritable) obj;
+ final Writable[] arr = mapObj.get();
+ map.put(arr[0], arr[1]);
+ }
+
+ return map;
+ }
+
+ if (data instanceof Map) {
+ return (Map) data;
+ }
+
+ throw new UnsupportedOperationException("Cannot inspect " + data.getClass().getCanonicalName());
+ }
+
+ @Override
+ public int getMapSize(final Object data) {
+ if (data == null) {
+ return -1;
+ }
+
+ if (data instanceof ArrayWritable) {
+ final Writable[] mapContainer = ((ArrayWritable) data).get();
+
+ if (mapContainer == null || mapContainer.length == 0) {
+ return -1;
+ } else {
+ return ((ArrayWritable) mapContainer[0]).get().length;
+ }
+ }
+
+ if (data instanceof Map) {
+ return ((Map) data).size();
+ }
+
+ throw new UnsupportedOperationException("Cannot inspect " + data.getClass().getCanonicalName());
+ }
+
+ @Override
+ public Object create() {
+ Map m = new HashMap();
+ return m;
+ }
+
+ @Override
+ public Object put(Object map, Object key, Object value) {
+ Map m = (HashMap) map;
+ m.put(key, value);
+ return m;
+ }
+
+ @Override
+ public Object remove(Object map, Object key) {
+ Map m = (HashMap) map;
+ m.remove(key);
+ return m;
+ }
+
+ @Override
+ public Object clear(Object map) {
+ Map m = (HashMap) map;
+ m.clear();
+ return m;
+ }
+
+ @Override
+ public boolean equals(Object obj) {
+ if (obj == null) {
+ return false;
+ }
+ if (getClass() != obj.getClass()) {
+ return false;
+ }
+ final StandardParquetHiveMapInspector other = (StandardParquetHiveMapInspector) obj;
+ if (this.keyInspector != other.keyInspector && (this.keyInspector == null || !this.keyInspector.equals(other.keyInspector))) {
+ return false;
+ }
+ if (this.valueInspector != other.valueInspector && (this.valueInspector == null || !this.valueInspector.equals(other.valueInspector))) {
+ return false;
+ }
+ return true;
+ }
+
+ @Override
+ public int hashCode() {
+ int hash = 7;
+ hash = 59 * hash + (this.keyInspector != null ? this.keyInspector.hashCode() : 0);
+ hash = 59 * hash + (this.valueInspector != null ? this.valueInspector.hashCode() : 0);
+ return hash;
+ }
+}
diff --git a/parquet-hive/parquet-hive-storage-handler/src/main/java/parquet/hive/serde/ArrayWritableObjectInspector.java b/parquet-hive/parquet-hive-storage-handler/src/main/java/parquet/hive/serde/ArrayWritableObjectInspector.java
new file mode 100644
index 0000000000..2f933831cc
--- /dev/null
+++ b/parquet-hive/parquet-hive-storage-handler/src/main/java/parquet/hive/serde/ArrayWritableObjectInspector.java
@@ -0,0 +1,224 @@
+/**
+ * Copyright 2013 Criteo.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
+ * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations under the License.
+ */
+package parquet.hive.serde;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.SettableStructObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.StructField;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory;
+import org.apache.hadoop.hive.serde2.typeinfo.ListTypeInfo;
+import org.apache.hadoop.hive.serde2.typeinfo.MapTypeInfo;
+import org.apache.hadoop.hive.serde2.typeinfo.StructTypeInfo;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory;
+import org.apache.hadoop.io.ArrayWritable;
+import parquet.hive.serde.primitive.ParquetPrimitiveInspectorFactory;
+
+/**
+ *
+ * The ArrayWritableObjectInspector will inspect an ArrayWritable, considering it as a Hive struct.
+ * It can also inspect a List if Hive decides to inspect the result of an inspection.
+ *
+ * @author Mickaël Lacour
+ * @author Rémy Pecqueur
+ *
+ */
+public class ArrayWritableObjectInspector extends SettableStructObjectInspector {
+
+ private final TypeInfo typeInfo;
+ private final List fieldInfos;
+ private final List fieldNames;
+ private final List fields;
+ private final HashMap fieldsByName;
+
+ public ArrayWritableObjectInspector(final StructTypeInfo rowTypeInfo) {
+
+ typeInfo = rowTypeInfo;
+ fieldNames = rowTypeInfo.getAllStructFieldNames();
+ fieldInfos = rowTypeInfo.getAllStructFieldTypeInfos();
+ fields = new ArrayList(fieldNames.size());
+ fieldsByName = new HashMap();
+
+ for (int i = 0; i < fieldNames.size(); ++i) {
+ final String name = fieldNames.get(i);
+ final TypeInfo fieldInfo = fieldInfos.get(i);
+
+ final StructFieldImpl field = new StructFieldImpl(name, getObjectInspector(fieldInfo), i);
+ fields.add(field);
+ fieldsByName.put(name, field);
+ }
+ }
+
+ private ObjectInspector getObjectInspector(final TypeInfo typeInfo) {
+ if (typeInfo.equals(TypeInfoFactory.doubleTypeInfo)) {
+ return PrimitiveObjectInspectorFactory.writableDoubleObjectInspector;
+ } else if (typeInfo.equals(TypeInfoFactory.booleanTypeInfo)) {
+ return PrimitiveObjectInspectorFactory.writableBooleanObjectInspector;
+ } else if (typeInfo.equals(TypeInfoFactory.floatTypeInfo)) {
+ return PrimitiveObjectInspectorFactory.writableFloatObjectInspector;
+ } else if (typeInfo.equals(TypeInfoFactory.intTypeInfo)) {
+ return PrimitiveObjectInspectorFactory.writableIntObjectInspector;
+ } else if (typeInfo.equals(TypeInfoFactory.longTypeInfo)) {
+ return PrimitiveObjectInspectorFactory.writableLongObjectInspector;
+ } else if (typeInfo.equals(TypeInfoFactory.stringTypeInfo)) {
+ return ParquetPrimitiveInspectorFactory.parquetStringInspector;
+ } else if (typeInfo.getCategory().equals(Category.STRUCT)) {
+ return new ArrayWritableObjectInspector((StructTypeInfo) typeInfo);
+ } else if (typeInfo.getCategory().equals(Category.LIST)) {
+ final TypeInfo subTypeInfo = ((ListTypeInfo) typeInfo).getListElementTypeInfo();
+ return new ParquetHiveArrayInspector(getObjectInspector(subTypeInfo));
+ } else if (typeInfo.getCategory().equals(Category.MAP)) {
+ final TypeInfo keyTypeInfo = ((MapTypeInfo) typeInfo).getMapKeyTypeInfo();
+ final TypeInfo valueTypeInfo = ((MapTypeInfo) typeInfo).getMapValueTypeInfo();
+ if (keyTypeInfo.equals(TypeInfoFactory.stringTypeInfo) || keyTypeInfo.equals(TypeInfoFactory.byteTypeInfo)
+ || keyTypeInfo.equals(TypeInfoFactory.shortTypeInfo)) {
+ return new DeepParquetHiveMapInspector(getObjectInspector(keyTypeInfo), getObjectInspector(valueTypeInfo));
+ } else {
+ return new StandardParquetHiveMapInspector(getObjectInspector(keyTypeInfo), getObjectInspector(valueTypeInfo));
+ }
+ } else if (typeInfo.equals(TypeInfoFactory.timestampTypeInfo)) {
+ throw new UnsupportedOperationException("timestamp not implemented yet");
+ } else if (typeInfo.equals(TypeInfoFactory.byteTypeInfo)) {
+ return ParquetPrimitiveInspectorFactory.parquetByteInspector;
+ } else if (typeInfo.equals(TypeInfoFactory.shortTypeInfo)) {
+ return ParquetPrimitiveInspectorFactory.parquetShortInspector;
+ } else {
+ throw new RuntimeException("Unknown field info: " + typeInfo);
+ }
+
+ }
+
+ @Override
+ public Category getCategory() {
+ return Category.STRUCT;
+ }
+
+ @Override
+ public String getTypeName() {
+ return typeInfo.getTypeName();
+ }
+
+ @Override
+ public List extends StructField> getAllStructFieldRefs() {
+ return fields;
+ }
+
+ @Override
+ public Object getStructFieldData(final Object data, final StructField fieldRef) {
+ if (data == null) {
+ return null;
+ }
+
+ if (data instanceof ArrayWritable) {
+ final ArrayWritable arr = (ArrayWritable) data;
+ return arr.get()[((StructFieldImpl) fieldRef).getIndex()];
+ }
+
+ throw new UnsupportedOperationException("Cannot inspect " + data.getClass().getCanonicalName());
+ }
+
+ @Override
+ public StructField getStructFieldRef(final String name) {
+ return fieldsByName.get(name);
+ }
+
+ @Override
+ public List getStructFieldsDataAsList(final Object data) {
+ if (data == null) {
+ return null;
+ }
+
+ if (data instanceof ArrayWritable) {
+ final ArrayWritable arr = (ArrayWritable) data;
+ final Object[] arrWritable = arr.get();
+ return new ArrayList(Arrays.asList(arrWritable));
+ }
+
+ throw new UnsupportedOperationException("Cannot inspect " + data.getClass().getCanonicalName());
+ }
+
+ @Override
+ public Object create() {
+ final ArrayList list = new ArrayList(fields.size());
+ for (int i = 0; i < fields.size(); ++i) {
+ list.add(null);
+ }
+ return list;
+ }
+
+ @Override
+ public Object setStructFieldData(Object struct, StructField field, Object fieldValue) {
+ final ArrayList list = (ArrayList) struct;
+ list.set(((StructFieldImpl) field).getIndex(), fieldValue);
+ return list;
+ }
+
+ @Override
+ public boolean equals(Object obj) {
+ if (obj == null) {
+ return false;
+ }
+ if (getClass() != obj.getClass()) {
+ return false;
+ }
+ final ArrayWritableObjectInspector other = (ArrayWritableObjectInspector) obj;
+ if (this.typeInfo != other.typeInfo && (this.typeInfo == null || !this.typeInfo.equals(other.typeInfo))) {
+ return false;
+ }
+ return true;
+ }
+
+ @Override
+ public int hashCode() {
+ int hash = 5;
+ hash = 29 * hash + (this.typeInfo != null ? this.typeInfo.hashCode() : 0);
+ return hash;
+ }
+
+ class StructFieldImpl implements StructField {
+
+ private final String name;
+ private final ObjectInspector inspector;
+ private final int index;
+
+ public StructFieldImpl(final String name, final ObjectInspector inspector, final int index) {
+ this.name = name;
+ this.inspector = inspector;
+ this.index = index;
+ }
+
+ @Override
+ public String getFieldComment() {
+ return "";
+ }
+
+ @Override
+ public String getFieldName() {
+ return name;
+ }
+
+ public int getIndex() {
+ return index;
+ }
+
+ @Override
+ public ObjectInspector getFieldObjectInspector() {
+ return inspector;
+ }
+ }
+}
diff --git a/parquet-hive/parquet-hive-storage-handler/src/main/java/parquet/hive/serde/DeepParquetHiveMapInspector.java b/parquet-hive/parquet-hive-storage-handler/src/main/java/parquet/hive/serde/DeepParquetHiveMapInspector.java
new file mode 100644
index 0000000000..2a3894966a
--- /dev/null
+++ b/parquet-hive/parquet-hive-storage-handler/src/main/java/parquet/hive/serde/DeepParquetHiveMapInspector.java
@@ -0,0 +1,82 @@
+/**
+ * Copyright 2013 Criteo.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
+ * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations under the License.
+ */
+package parquet.hive.serde;
+
+import java.util.Map;
+
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector;
+import org.apache.hadoop.io.ArrayWritable;
+import org.apache.hadoop.io.Writable;
+
+/**
+ * The DeepParquetHiveMapInspector will inspect an ArrayWritable, considering it as a Hive map.
+ * It can also inspect a Map if Hive decides to inspect the result of an inspection.
+ * When trying to access elements from the map it will iterate over all keys, inspecting them and comparing them to the
+ * desired key.
+ *
+ * @author Rémy Pecqueur
+ */
+public class DeepParquetHiveMapInspector extends AbstractParquetMapInspector {
+
+ public DeepParquetHiveMapInspector(final ObjectInspector keyInspector, final ObjectInspector valueInspector) {
+ super(keyInspector, valueInspector);
+ }
+
+ @Override
+ public Object getMapValueElement(final Object data, final Object key) {
+ if (data == null || key == null) {
+ return null;
+ }
+
+ if (data instanceof ArrayWritable) {
+ final Writable[] mapContainer = ((ArrayWritable) data).get();
+
+ if (mapContainer == null || mapContainer.length == 0) {
+ return null;
+ }
+
+ final Writable[] mapArray = ((ArrayWritable) mapContainer[0]).get();
+
+ for (final Writable obj : mapArray) {
+ final ArrayWritable mapObj = (ArrayWritable) obj;
+ final Writable[] arr = mapObj.get();
+ if (key.equals(arr[0]) || key.equals(((PrimitiveObjectInspector) keyInspector).getPrimitiveJavaObject(arr[0]))
+ || key.equals(((PrimitiveObjectInspector) keyInspector).getPrimitiveWritableObject(arr[0]))) {
+ return arr[1];
+ }
+ }
+
+ return null;
+ }
+
+ if (data instanceof Map) {
+ final Map, ?> map = (Map, ?>) data;
+
+ if (map.containsKey(key)) {
+ return map.get(key);
+ }
+
+ for (final Map.Entry, ?> entry : map.entrySet()) {
+ if (key.equals(((PrimitiveObjectInspector) keyInspector).getPrimitiveJavaObject(entry.getKey()))
+ || key.equals(((PrimitiveObjectInspector) keyInspector).getPrimitiveWritableObject(entry.getKey()))) {
+ return entry.getValue();
+ }
+ }
+
+ return null;
+ }
+
+ throw new UnsupportedOperationException("Cannot inspect " + data.getClass().getCanonicalName());
+ }
+}
diff --git a/parquet-hive/parquet-hive-storage-handler/src/main/java/parquet/hive/serde/ParquetHiveArrayInspector.java b/parquet-hive/parquet-hive-storage-handler/src/main/java/parquet/hive/serde/ParquetHiveArrayInspector.java
new file mode 100644
index 0000000000..1b0094f29f
--- /dev/null
+++ b/parquet-hive/parquet-hive-storage-handler/src/main/java/parquet/hive/serde/ParquetHiveArrayInspector.java
@@ -0,0 +1,185 @@
+/**
+ * Copyright 2013 Criteo.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
+ * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations under the License.
+ */
+package parquet.hive.serde;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.SettableListObjectInspector;
+import org.apache.hadoop.io.ArrayWritable;
+import org.apache.hadoop.io.Writable;
+
+/**
+ * The ParquetHiveArrayInspector will inspect an ArrayWritable, considering it as an Hive array.
+ * It can also inspect a List if Hive decides to inspect the result of an inspection.
+ *
+ * @author Rémy Pecqueur
+ */
+public class ParquetHiveArrayInspector implements SettableListObjectInspector {
+
+ ObjectInspector arrayElementInspector;
+
+ public ParquetHiveArrayInspector(final ObjectInspector arrayElementInspector) {
+ this.arrayElementInspector = arrayElementInspector;
+ }
+
+ @Override
+ public String getTypeName() {
+ return "array<" + arrayElementInspector.getTypeName() + ">";
+ }
+
+ @Override
+ public Category getCategory() {
+ return Category.LIST;
+ }
+
+ @Override
+ public ObjectInspector getListElementObjectInspector() {
+ return arrayElementInspector;
+ }
+
+ @Override
+ public Object getListElement(final Object data, final int index) {
+ if (data == null) {
+ return null;
+ }
+
+ if (data instanceof ArrayWritable) {
+ final Writable[] listContainer = ((ArrayWritable) data).get();
+
+ if (listContainer == null || listContainer.length == 0) {
+ return null;
+ }
+
+ final Writable subObj = listContainer[0];
+
+ if (subObj == null) {
+ return null;
+ }
+
+ if (index >= 0 && index < ((ArrayWritable) subObj).get().length) {
+ return ((ArrayWritable) subObj).get()[index];
+ } else {
+ return null;
+ }
+ }
+
+ throw new UnsupportedOperationException("Cannot inspect " + data.getClass().getCanonicalName());
+ }
+
+ @Override
+ public int getListLength(final Object data) {
+ if (data == null) {
+ return -1;
+ }
+
+ if (data instanceof ArrayWritable) {
+ final Writable[] listContainer = ((ArrayWritable) data).get();
+
+ if (listContainer == null || listContainer.length == 0) {
+ return -1;
+ }
+
+ final Writable subObj = listContainer[0];
+
+ if (subObj == null) {
+ return 0;
+ }
+
+ return ((ArrayWritable) subObj).get().length;
+ }
+
+ throw new UnsupportedOperationException("Cannot inspect " + data.getClass().getCanonicalName());
+ }
+
+ @Override
+ public List> getList(final Object data) {
+ if (data == null) {
+ return null;
+ }
+
+ if (data instanceof ArrayWritable) {
+ final Writable[] listContainer = ((ArrayWritable) data).get();
+
+ if (listContainer == null || listContainer.length == 0) {
+ return null;
+ }
+
+ final Writable subObj = listContainer[0];
+
+ if (subObj == null) {
+ return null;
+ }
+
+ final Writable[] array = ((ArrayWritable) subObj).get();
+ final List list = new ArrayList();
+
+ for (final Writable obj : array) {
+ list.add(obj);
+ }
+
+ return list;
+ }
+
+ throw new UnsupportedOperationException("Cannot inspect " + data.getClass().getCanonicalName());
+ }
+
+ @Override
+ public Object create(final int size) {
+ final ArrayList result = new ArrayList(size);
+ for (int i = 0; i < size; ++i) {
+ result.add(null);
+ }
+ return result;
+ }
+
+ @Override
+ public Object set(final Object list, final int index, final Object element) {
+ final ArrayList l = (ArrayList) list;
+ l.set(index, element);
+ return list;
+ }
+
+ @Override
+ public Object resize(final Object list, final int newSize) {
+ final ArrayList l = (ArrayList) list;
+ l.ensureCapacity(newSize);
+ while (l.size() < newSize) {
+ l.add(null);
+ }
+ while (l.size() > newSize) {
+ l.remove(l.size() - 1);
+ }
+ return list;
+ }
+
+ @Override
+ public boolean equals(final Object o) {
+ if (o == null || o.getClass() != getClass()) {
+ return false;
+ } else if (o == this) {
+ return true;
+ } else {
+ final ObjectInspector other = ((ParquetHiveArrayInspector) o).arrayElementInspector;
+ return other.equals(arrayElementInspector);
+ }
+ }
+
+ @Override
+ public int hashCode() {
+ int hash = 3;
+ hash = 29 * hash + (this.arrayElementInspector != null ? this.arrayElementInspector.hashCode() : 0);
+ return hash;
+ }
+}
diff --git a/parquet-hive/src/main/java/parquet/hive/serde/ParquetHiveSerDe.java b/parquet-hive/parquet-hive-storage-handler/src/main/java/parquet/hive/serde/ParquetHiveSerDe.java
similarity index 98%
rename from parquet-hive/src/main/java/parquet/hive/serde/ParquetHiveSerDe.java
rename to parquet-hive/parquet-hive-storage-handler/src/main/java/parquet/hive/serde/ParquetHiveSerDe.java
index 904bbfc826..f216e5edc6 100644
--- a/parquet-hive/src/main/java/parquet/hive/serde/ParquetHiveSerDe.java
+++ b/parquet-hive/parquet-hive-storage-handler/src/main/java/parquet/hive/serde/ParquetHiveSerDe.java
@@ -57,6 +57,7 @@
import org.apache.hadoop.io.Writable;
import parquet.hive.writable.BinaryWritable;
+import parquet.io.api.Binary;
/**
*
@@ -253,7 +254,7 @@ private Writable createPrimitive(final Object obj, final PrimitiveObjectInspecto
case SHORT:
return new ShortWritable((short) ((ShortObjectInspector) inspector).get(obj));
case STRING:
- return new BinaryWritable(((StringObjectInspector) inspector).getPrimitiveJavaObject(obj));
+ return new BinaryWritable(Binary.fromString(((StringObjectInspector) inspector).getPrimitiveJavaObject(obj)));
default:
throw new SerDeException("Unknown primitive : " + inspector.getPrimitiveCategory());
}
diff --git a/parquet-hive/parquet-hive-storage-handler/src/main/java/parquet/hive/serde/StandardParquetHiveMapInspector.java b/parquet-hive/parquet-hive-storage-handler/src/main/java/parquet/hive/serde/StandardParquetHiveMapInspector.java
new file mode 100644
index 0000000000..be703df40b
--- /dev/null
+++ b/parquet-hive/parquet-hive-storage-handler/src/main/java/parquet/hive/serde/StandardParquetHiveMapInspector.java
@@ -0,0 +1,65 @@
+/**
+ * Copyright 2013 Criteo.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
+ * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations under the License.
+ */
+package parquet.hive.serde;
+
+import java.util.Map;
+
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
+import org.apache.hadoop.io.ArrayWritable;
+import org.apache.hadoop.io.Writable;
+
+/**
+ * The StandardParquetHiveMapInspector will inspect an ArrayWritable, considering it as a Hive map.
+ * It can also inspect a Map if Hive decides to inspect the result of an inspection.
+ *
+ * @author Rémy Pecqueur
+ */
+public class StandardParquetHiveMapInspector extends AbstractParquetMapInspector {
+
+ public StandardParquetHiveMapInspector(final ObjectInspector keyInspector, final ObjectInspector valueInspector) {
+ super(keyInspector, valueInspector);
+ }
+
+ @Override
+ public Object getMapValueElement(final Object data, final Object key) {
+ if (data == null || key == null) {
+ return null;
+ }
+
+ if (data instanceof ArrayWritable) {
+ final Writable[] mapContainer = ((ArrayWritable) data).get();
+
+ if (mapContainer == null || mapContainer.length == 0) {
+ return null;
+ }
+
+ final Writable[] mapArray = ((ArrayWritable) mapContainer[0]).get();
+
+ for (final Writable obj : mapArray) {
+ final ArrayWritable mapObj = (ArrayWritable) obj;
+ final Writable[] arr = mapObj.get();
+ if (key.equals(arr[0])) {
+ return arr[1];
+ }
+ }
+
+ return null;
+ }
+
+ if (data instanceof Map) {
+ return ((Map) data).get(key);
+ }
+
+ throw new UnsupportedOperationException("Cannot inspect " + data.getClass().getCanonicalName());
+ }
+}
diff --git a/parquet-hive/parquet-hive-storage-handler/src/main/java/parquet/hive/serde/primitive/ParquetByteInspector.java b/parquet-hive/parquet-hive-storage-handler/src/main/java/parquet/hive/serde/primitive/ParquetByteInspector.java
new file mode 100644
index 0000000000..f2570330a3
--- /dev/null
+++ b/parquet-hive/parquet-hive-storage-handler/src/main/java/parquet/hive/serde/primitive/ParquetByteInspector.java
@@ -0,0 +1,59 @@
+/*
+ * Copyright 2013 Criteo.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package parquet.hive.serde.primitive;
+
+import org.apache.hadoop.hive.serde2.io.ByteWritable;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.AbstractPrimitiveJavaObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorUtils;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.SettableByteObjectInspector;
+import org.apache.hadoop.io.IntWritable;
+
+/**
+ * The ParquetByteInspector can inspect both ByteWritables and IntWritables into bytes.
+ *
+ * @author Nong Li
+ */
+public class ParquetByteInspector extends AbstractPrimitiveJavaObjectInspector implements SettableByteObjectInspector {
+
+ ParquetByteInspector() {
+ super(PrimitiveObjectInspectorUtils.byteTypeEntry);
+ }
+
+ @Override
+ public Object getPrimitiveWritableObject(final Object o) {
+ return o == null ? null : new ByteWritable(get(o));
+ }
+
+ @Override
+ public Object create(final byte val) {
+ return new ByteWritable(val);
+ }
+
+ @Override
+ public Object set(final Object o, final byte val) {
+ ((ByteWritable) o).set(val);
+ return o;
+ }
+
+ @Override
+ public byte get(Object o) {
+ // Accept int writables and convert them.
+ if (o instanceof IntWritable) {
+ return (byte) ((IntWritable) o).get();
+ }
+ return ((ByteWritable) o).get();
+ }
+}
diff --git a/parquet-hive/parquet-hive-storage-handler/src/main/java/parquet/hive/serde/primitive/ParquetPrimitiveInspectorFactory.java b/parquet-hive/parquet-hive-storage-handler/src/main/java/parquet/hive/serde/primitive/ParquetPrimitiveInspectorFactory.java
new file mode 100644
index 0000000000..fd4e0070a9
--- /dev/null
+++ b/parquet-hive/parquet-hive-storage-handler/src/main/java/parquet/hive/serde/primitive/ParquetPrimitiveInspectorFactory.java
@@ -0,0 +1,32 @@
+/*
+ * Copyright 2013 Criteo.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package parquet.hive.serde.primitive;
+
+/**
+ * The ParquetPrimitiveInspectorFactory allows us to be sure that the same object is inspected by the same inspector.
+ *
+ * @author Rémy Pecqueur
+ */
+public class ParquetPrimitiveInspectorFactory {
+
+ public static final ParquetByteInspector parquetByteInspector = new ParquetByteInspector();
+ public static final ParquetShortInspector parquetShortInspector = new ParquetShortInspector();
+ public static final ParquetStringInspector parquetStringInspector = new ParquetStringInspector();
+
+ private ParquetPrimitiveInspectorFactory() {
+ // prevent instantiation
+ }
+}
diff --git a/parquet-hive/parquet-hive-storage-handler/src/main/java/parquet/hive/serde/primitive/ParquetShortInspector.java b/parquet-hive/parquet-hive-storage-handler/src/main/java/parquet/hive/serde/primitive/ParquetShortInspector.java
new file mode 100644
index 0000000000..5094dc34fb
--- /dev/null
+++ b/parquet-hive/parquet-hive-storage-handler/src/main/java/parquet/hive/serde/primitive/ParquetShortInspector.java
@@ -0,0 +1,59 @@
+/*
+ * Copyright 2013 Criteo.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package parquet.hive.serde.primitive;
+
+import org.apache.hadoop.hive.serde2.io.ShortWritable;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.AbstractPrimitiveJavaObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorUtils;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.SettableShortObjectInspector;
+import org.apache.hadoop.io.IntWritable;
+
+/**
+ * The ParquetShortInspector can inspect both ShortWritables and IntWritables into shorts.
+ *
+ * @author Nong Li
+ */
+public class ParquetShortInspector extends AbstractPrimitiveJavaObjectInspector implements SettableShortObjectInspector {
+
+ ParquetShortInspector() {
+ super(PrimitiveObjectInspectorUtils.shortTypeEntry);
+ }
+
+ @Override
+ public Object getPrimitiveWritableObject(final Object o) {
+ return o == null ? null : new ShortWritable(get(o));
+ }
+
+ @Override
+ public Object create(final short val) {
+ return new ShortWritable(val);
+ }
+
+ @Override
+ public Object set(final Object o, final short val) {
+ ((ShortWritable) o).set(val);
+ return o;
+ }
+
+ @Override
+ public short get(Object o) {
+ // Accept int writables and convert them.
+ if (o instanceof IntWritable) {
+ return (short) ((IntWritable) o).get();
+ }
+ return ((ShortWritable) o).get();
+ }
+}
diff --git a/parquet-hive/parquet-hive-storage-handler/src/main/java/parquet/hive/serde/primitive/ParquetStringInspector.java b/parquet-hive/parquet-hive-storage-handler/src/main/java/parquet/hive/serde/primitive/ParquetStringInspector.java
new file mode 100644
index 0000000000..42040667ad
--- /dev/null
+++ b/parquet-hive/parquet-hive-storage-handler/src/main/java/parquet/hive/serde/primitive/ParquetStringInspector.java
@@ -0,0 +1,100 @@
+/*
+ * Copyright 2013 Criteo.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package parquet.hive.serde.primitive;
+
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.AbstractPrimitiveJavaObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorUtils;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.SettableStringObjectInspector;
+import org.apache.hadoop.io.Text;
+import parquet.hive.writable.BinaryWritable;
+import parquet.io.api.Binary;
+
+/**
+ * The ParquetStringInspector inspects a BinaryWritable to give a Text or String.
+ *
+ * @author Rémy Pecqueur
+ */
+public class ParquetStringInspector extends AbstractPrimitiveJavaObjectInspector implements SettableStringObjectInspector {
+
+ ParquetStringInspector() {
+ super(PrimitiveObjectInspectorUtils.stringTypeEntry);
+ }
+
+ @Override
+ public Text getPrimitiveWritableObject(final Object o) {
+ if (o == null) {
+ return null;
+ }
+
+ if (o instanceof BinaryWritable) {
+ return new Text(((BinaryWritable) o).getBytes());
+ }
+
+ if (o instanceof Text) {
+ return (Text) o;
+ }
+
+ if (o instanceof String) {
+ return new Text((String) o);
+ }
+
+ throw new UnsupportedOperationException("Cannot inspect " + o.getClass().getCanonicalName());
+ }
+
+ @Override
+ public String getPrimitiveJavaObject(final Object o) {
+ if (o == null) {
+ return null;
+ }
+
+ if (o instanceof BinaryWritable) {
+ return ((BinaryWritable) o).getString();
+ }
+
+ if (o instanceof Text) {
+ return ((Text) o).toString();
+ }
+
+ if (o instanceof String) {
+ return (String) o;
+ }
+
+ throw new UnsupportedOperationException("Cannot inspect " + o.getClass().getCanonicalName());
+ }
+
+ @Override
+ public Object set(final Object o, final Text text) {
+ return new BinaryWritable(text == null ? null : Binary.fromByteArray(text.getBytes()));
+ }
+
+ @Override
+ public Object set(final Object o, final String string) {
+ return new BinaryWritable(string == null ? null : Binary.fromString(string));
+ }
+
+ @Override
+ public Object create(final Text text) {
+ if (text == null) {
+ return null;
+ }
+ return text.toString();
+ }
+
+ @Override
+ public Object create(final String string) {
+ return string;
+ }
+}
diff --git a/parquet-hive/src/main/java/parquet/hive/writable/BigDecimalWritable.java b/parquet-hive/parquet-hive-storage-handler/src/main/java/parquet/hive/writable/BigDecimalWritable.java
similarity index 100%
rename from parquet-hive/src/main/java/parquet/hive/writable/BigDecimalWritable.java
rename to parquet-hive/parquet-hive-storage-handler/src/main/java/parquet/hive/writable/BigDecimalWritable.java
diff --git a/parquet-hive/parquet-hive-storage-handler/src/main/java/parquet/hive/writable/BinaryWritable.java b/parquet-hive/parquet-hive-storage-handler/src/main/java/parquet/hive/writable/BinaryWritable.java
new file mode 100644
index 0000000000..f187e31098
--- /dev/null
+++ b/parquet-hive/parquet-hive-storage-handler/src/main/java/parquet/hive/writable/BinaryWritable.java
@@ -0,0 +1,98 @@
+/**
+ * Copyright 2013 Criteo.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package parquet.hive.writable;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+import org.apache.hadoop.io.Writable;
+
+import parquet.io.api.Binary;
+
+/**
+ *
+ * A Wrapper to support constructor with Binary and String
+ *
+ * TODO : remove it, and call BytesWritable with the getBytes()
+ *
+ *
+ * @author Mickaël Lacour
+ * @author Rémy Pecqueur
+ *
+ */
+public class BinaryWritable implements Writable {
+
+ private Binary binary;
+
+ public BinaryWritable(final Binary binary) {
+ this.binary = binary;
+ }
+
+ public Binary getBinary() {
+ return binary;
+ }
+
+ public byte[] getBytes() {
+ return binary.getBytes();
+ }
+
+ public String getString() {
+ return binary.toStringUsingUTF8();
+ }
+
+ @Override
+ public void readFields(DataInput input) throws IOException {
+ byte[] bytes = new byte[input.readInt()];
+ input.readFully(bytes);
+ binary = Binary.fromByteArray(bytes);
+ }
+
+ @Override
+ public void write(DataOutput output) throws IOException {
+ output.writeInt(binary.length());
+ binary.writeTo(output);
+ }
+
+ @Override
+ public int hashCode() {
+ return binary == null ? 0 : binary.hashCode();
+ }
+
+ @Override
+ public boolean equals(Object obj) {
+ if (obj instanceof BinaryWritable) {
+ final BinaryWritable other = (BinaryWritable)obj;
+ return binary.equals(other.binary);
+ }
+ return false;
+ }
+
+ public static class DicBinaryWritable extends BinaryWritable {
+
+ private String string;
+
+ public DicBinaryWritable(Binary binary, String string) {
+ super(binary);
+ this.string = string;
+ }
+
+ public String getString() {
+ return string;
+ }
+ }
+
+}
diff --git a/parquet-hive/src/main/java/parquet/hive/write/DataWritableWriteSupport.java b/parquet-hive/parquet-hive-storage-handler/src/main/java/parquet/hive/write/DataWritableWriteSupport.java
similarity index 100%
rename from parquet-hive/src/main/java/parquet/hive/write/DataWritableWriteSupport.java
rename to parquet-hive/parquet-hive-storage-handler/src/main/java/parquet/hive/write/DataWritableWriteSupport.java
diff --git a/parquet-hive/src/main/java/parquet/hive/write/DataWritableWriter.java b/parquet-hive/parquet-hive-storage-handler/src/main/java/parquet/hive/write/DataWritableWriter.java
similarity index 96%
rename from parquet-hive/src/main/java/parquet/hive/write/DataWritableWriter.java
rename to parquet-hive/parquet-hive-storage-handler/src/main/java/parquet/hive/write/DataWritableWriter.java
index c3fcdd91f5..32873b74dc 100644
--- a/parquet-hive/src/main/java/parquet/hive/write/DataWritableWriter.java
+++ b/parquet-hive/parquet-hive-storage-handler/src/main/java/parquet/hive/write/DataWritableWriter.java
@@ -11,7 +11,6 @@
*/
package parquet.hive.write;
-import org.apache.commons.lang.NotImplementedException;
import org.apache.hadoop.hive.serde2.io.ByteWritable;
import org.apache.hadoop.hive.serde2.io.DoubleWritable;
import org.apache.hadoop.hive.serde2.io.ShortWritable;
@@ -152,9 +151,9 @@ private void writePrimitive(final Writable value) {
} else if (value instanceof ByteWritable) {
recordConsumer.addInteger(((ByteWritable) value).get());
} else if (value instanceof BigDecimalWritable) {
- throw new NotImplementedException("BigDecimal writing not implemented");
+ throw new UnsupportedOperationException("BigDecimal writing not implemented");
} else if (value instanceof BinaryWritable) {
- recordConsumer.addBinary(Binary.fromByteArray(((BinaryWritable) value).getBytes()));
+ recordConsumer.addBinary(((BinaryWritable) value).getBinary());
} else {
throw new RuntimeException("Unknown value type: " + value + " " + value.getClass());
}
diff --git a/parquet-hive/src/test/java/parquet/hive/TestHiveSchemaConverter.java b/parquet-hive/parquet-hive-storage-handler/src/test/java/parquet/hive/TestHiveSchemaConverter.java
similarity index 100%
rename from parquet-hive/src/test/java/parquet/hive/TestHiveSchemaConverter.java
rename to parquet-hive/parquet-hive-storage-handler/src/test/java/parquet/hive/TestHiveSchemaConverter.java
diff --git a/parquet-hive/src/test/java/parquet/hive/TestDeprecatedParquetInputFormat.java b/parquet-hive/parquet-hive-storage-handler/src/test/java/parquet/hive/TestMapredParquetInputFormat.java
similarity index 68%
rename from parquet-hive/src/test/java/parquet/hive/TestDeprecatedParquetInputFormat.java
rename to parquet-hive/parquet-hive-storage-handler/src/test/java/parquet/hive/TestMapredParquetInputFormat.java
index b26c45382c..96bee78c9a 100644
--- a/parquet-hive/src/test/java/parquet/hive/TestDeprecatedParquetInputFormat.java
+++ b/parquet-hive/parquet-hive-storage-handler/src/test/java/parquet/hive/TestMapredParquetInputFormat.java
@@ -1,16 +1,22 @@
/**
* Copyright 2013 Criteo.
*
- * Licensed 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
+ * Licensed 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.
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
*/
package parquet.hive;
+import static org.junit.Assert.*;
+
import java.io.File;
import java.io.IOException;
import java.util.ArrayList;
@@ -18,33 +24,38 @@
import java.util.List;
import java.util.Map;
-import junit.framework.TestCase;
+import org.apache.commons.lang.StringUtils;
import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.serde2.ColumnProjectionUtils;
import org.apache.hadoop.io.ArrayWritable;
import org.apache.hadoop.io.IntWritable;
import org.apache.hadoop.io.Writable;
import org.apache.hadoop.mapred.FileInputFormat;
+import org.apache.hadoop.mapred.FileSplit;
import org.apache.hadoop.mapred.JobConf;
import org.apache.hadoop.mapred.RecordReader;
import org.apache.hadoop.mapred.Reporter;
+import org.junit.Before;
+import org.junit.Test;
+import parquet.column.ParquetProperties.WriterVersion;
import parquet.column.impl.ColumnWriteStoreImpl;
import parquet.column.page.mem.MemPageStore;
import parquet.hadoop.ParquetFileReader;
import parquet.hadoop.ParquetInputSplit;
import parquet.hadoop.metadata.BlockMetaData;
import parquet.hadoop.metadata.ParquetMetadata;
-import parquet.hive.DeprecatedParquetInputFormat.InputSplitWrapper;
+import parquet.hive.MapredParquetInputFormat.InputSplitWrapper;
import parquet.hive.read.DataWritableReadSupport;
import parquet.io.ColumnIOFactory;
import parquet.io.MessageColumnIO;
import parquet.io.api.RecordConsumer;
import parquet.schema.GroupType;
import parquet.schema.MessageType;
+import parquet.schema.MessageTypeParser;
import parquet.schema.OriginalType;
import parquet.schema.PrimitiveType;
import parquet.schema.PrimitiveType.PrimitiveTypeName;
@@ -58,17 +69,17 @@
* @author Mickaël Lacour
*
*/
-public class TestDeprecatedParquetInputFormat extends TestCase {
+public class TestMapredParquetInputFormat {
- Configuration conf;
- JobConf job;
- FileSystem fs;
- Path dir;
- File testFile;
- Reporter reporter;
- FSDataOutputStream ds;
- Map mapData;
+ private Configuration conf;
+ private JobConf job;
+ private FileSystem fs;
+ private Path dir;
+ private File testFile;
+ private Reporter reporter;
+ private Map mapData;
+ @Test
public void testParquetHiveInputFormatWithoutSpecificSchema() throws Exception {
final String schemaRequested = "message customer {\n"
+ " optional int32 c_custkey;\n"
@@ -94,6 +105,7 @@ public void testParquetHiveInputFormatWithoutSpecificSchema() throws Exception {
readParquetHiveInputFormat(schemaRequested, new Integer[] {0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10});
}
+ @Test
public void testParquetHiveInputFormatWithSpecificSchema() throws Exception {
final String schemaRequested = "message customer {\n"
+ " optional int32 c_custkey;\n"
@@ -105,6 +117,7 @@ public void testParquetHiveInputFormatWithSpecificSchema() throws Exception {
readParquetHiveInputFormat(schemaRequested, new Integer[] {0, 1, 5, 6, 7});
}
+ @Test
public void testParquetHiveInputFormatWithSpecificSchemaRandomColumn() throws Exception {
final String schemaRequested = "message customer {\n"
+ " optional int32 c_custkey;\n"
@@ -113,6 +126,7 @@ public void testParquetHiveInputFormatWithSpecificSchemaRandomColumn() throws Ex
readParquetHiveInputFormat(schemaRequested, new Integer[] {0, 6});
}
+ @Test
public void testParquetHiveInputFormatWithSpecificSchemaFirstColumn() throws Exception {
final String schemaRequested = "message customer {\n"
+ " optional int32 c_custkey;\n"
@@ -120,6 +134,7 @@ public void testParquetHiveInputFormatWithSpecificSchemaFirstColumn() throws Exc
readParquetHiveInputFormat(schemaRequested, new Integer[] {0});
}
+ @Test
public void testParquetHiveInputFormatWithSpecificSchemaUnknownColumn() throws Exception {
final String schemaRequested = "message customer {\n"
+ " optional int32 c_custkey;\n"
@@ -128,8 +143,89 @@ public void testParquetHiveInputFormatWithSpecificSchemaUnknownColumn() throws E
readParquetHiveInputFormat(schemaRequested, new Integer[] {0, Integer.MIN_VALUE});
}
- @Override
- protected void setUp() throws Exception {
+ @Test
+ public void testGetSplit() throws Exception {
+ final ParquetMetadata readFooter = ParquetFileReader.readFooter(conf, new Path(testFile.getAbsolutePath()));
+
+ final MessageType fileSchema = readFooter.getFileMetaData().getSchema();
+ final MessageType requestedSchema = MessageTypeParser.parseMessageType("message customer {\n"
+ + " optional int32 c_custkey;\n"
+ + " optional binary c_name;\n"
+ + " optional double c_acctbal;\n"
+ + " optional binary c_mktsegment;\n"
+ + " optional binary c_comment;\n"
+ + "}");
+ final MessageType hiveSchema = MessageTypeParser.parseMessageType("message customer {\n"
+ + " optional int32 c_custkey;\n"
+ + " optional binary c_name;\n"
+ + " optional binary c_address;\n"
+ + " optional int32 c_nationkey;\n"
+ + " optional binary c_phone;\n"
+ + " optional double c_acctbal;\n"
+ + " optional binary c_mktsegment;\n"
+ + " optional binary c_comment;\n"
+ + " optional group c_map (MAP_KEY_VALUE) {\n"
+ + " repeated group map {\n"
+ + " required binary key;\n"
+ + " optional binary value;\n"
+ + " }\n"
+ + " }\n"
+ + " optional group c_list (LIST) {\n"
+ + " repeated group bag {\n"
+ + " optional int32 array_element;\n"
+ + " }\n"
+ + " }\n"
+ + " optional binary unknown;\n"
+ + "}");
+
+ // Put columns and projection info in the conf
+ List columns = new ArrayList();
+ List readColumns = new ArrayList();
+ for (int i = 0; i < hiveSchema.getFieldCount(); ++i) {
+ final String name = hiveSchema.getType(i).getName();
+ columns.add(name);
+ if (requestedSchema.containsField(name)) {
+ readColumns.add(i);
+ }
+ }
+ job.set("columns", StringUtils.join(columns, ","));
+ ColumnProjectionUtils.setReadColumnIDs(job, readColumns);
+
+ long size = 0;
+ final List blocks = readFooter.getBlocks();
+ for (final BlockMetaData block : blocks) {
+ size += block.getTotalByteSize();
+ }
+
+ final FileInputFormat format = new MapredParquetInputFormat();
+ final String[] locations = new String[] {"localhost"};
+
+ final Map readSupportMetaData = new HashMap();
+ readSupportMetaData.put(DataWritableReadSupport.HIVE_SCHEMA_KEY, hiveSchema.toString());
+ final ParquetInputSplit realSplit = new ParquetInputSplit(new Path(testFile.getAbsolutePath()), 0, size, locations, blocks,
+ fileSchema.toString(), requestedSchema.toString(), readFooter.getFileMetaData().getKeyValueMetaData(), readSupportMetaData);
+
+ final MapredParquetInputFormat.InputSplitWrapper splitWrapper = new InputSplitWrapper(realSplit);
+
+ // construct the record reader
+ final RecordReader reader = format.getRecordReader(splitWrapper, job, reporter);
+
+ assertEquals("Wrong real split inside wrapper", realSplit,
+ ((MapredParquetInputFormat.RecordReaderWrapper) reader).getSplit(splitWrapper, job));
+
+ // Recreate the split using getSplit, as Hive would
+ final FileSplit fileSplit = new FileSplit(splitWrapper.getPath(), splitWrapper.getStart(), splitWrapper.getLength(), splitWrapper.getLocations());
+ final ParquetInputSplit recreatedSplit = ((MapredParquetInputFormat.RecordReaderWrapper) reader).getSplit(fileSplit, job);
+ assertTrue("Wrong file schema", UtilitiesTestMethods.smartCheckSchema(fileSchema,
+ MessageTypeParser.parseMessageType(recreatedSplit.getFileSchema())));
+ assertTrue("Wrong requested schema", UtilitiesTestMethods.smartCheckSchema(requestedSchema,
+ MessageTypeParser.parseMessageType(recreatedSplit.getRequestedSchema())));
+ assertTrue("Wrong hive schema", UtilitiesTestMethods.smartCheckSchema(hiveSchema,
+ MessageTypeParser.parseMessageType(recreatedSplit.getReadSupportMetadata().get(DataWritableReadSupport.HIVE_SCHEMA_KEY))));
+ }
+
+ @Before
+ public void setUp() throws Exception {
//
// create job and filesystem and reporter and such.
//
@@ -167,7 +263,7 @@ private void writeFile() throws IOException {
new GroupType(Repetition.OPTIONAL, "c_list", OriginalType.LIST, new GroupType(Repetition.REPEATED, "bag", new PrimitiveType(Repetition.OPTIONAL, PrimitiveTypeName.INT32, "array_element"))));
final MemPageStore pageStore = new MemPageStore(1000);
- final ColumnWriteStoreImpl store = new ColumnWriteStoreImpl(pageStore, 8 * 1024, 8 * 1024, false);
+ final ColumnWriteStoreImpl store = new ColumnWriteStoreImpl(pageStore, 8 * 1024, 8 * 1024, 8 * 1024, false, WriterVersion.PARQUET_1_0);
final MessageColumnIO columnIO = new ColumnIOFactory().getColumnIO(schema);
final RecordConsumer recordWriter = columnIO.getRecordWriter(store);
@@ -223,7 +319,7 @@ private void readParquetHiveInputFormat(final String schemaRequested, final Inte
size += block.getTotalByteSize();
}
- final FileInputFormat format = new DeprecatedParquetInputFormat();
+ final FileInputFormat format = new MapredParquetInputFormat();
final String[] locations = new String[] {"localhost"};
final String schemaToString = schema.toString();
System.out.println(schemaToString);
@@ -260,7 +356,7 @@ private void readParquetHiveInputFormat(final String schemaRequested, final Inte
final ParquetInputSplit realSplit = new ParquetInputSplit(new Path(testFile.getAbsolutePath()), 0, size, locations, blocks,
schemaToString, specificSchema, readFooter.getFileMetaData().getKeyValueMetaData(), readSupportMetaData);
- final DeprecatedParquetInputFormat.InputSplitWrapper splitWrapper = new InputSplitWrapper(realSplit);
+ final MapredParquetInputFormat.InputSplitWrapper splitWrapper = new InputSplitWrapper(realSplit);
// construct the record reader
final RecordReader reader = format.getRecordReader(splitWrapper, job, reporter);
diff --git a/parquet-hive/src/test/java/parquet/hive/TestDeprecatedParquetOuputFormat.java b/parquet-hive/parquet-hive-storage-handler/src/test/java/parquet/hive/TestMapredParquetOuputFormat.java
similarity index 85%
rename from parquet-hive/src/test/java/parquet/hive/TestDeprecatedParquetOuputFormat.java
rename to parquet-hive/parquet-hive-storage-handler/src/test/java/parquet/hive/TestMapredParquetOuputFormat.java
index 0bdc4b526d..e5938f817c 100644
--- a/parquet-hive/src/test/java/parquet/hive/TestDeprecatedParquetOuputFormat.java
+++ b/parquet-hive/parquet-hive-storage-handler/src/test/java/parquet/hive/TestMapredParquetOuputFormat.java
@@ -1,16 +1,22 @@
/**
* Copyright 2013 Criteo.
*
- * Licensed 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
+ * Licensed 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.
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
*/
package parquet.hive;
+import static org.junit.Assert.*;
+
import java.io.File;
import java.io.IOException;
import java.util.ArrayList;
@@ -19,11 +25,7 @@
import java.util.Map;
import java.util.Properties;
-import junit.framework.TestCase;
-
import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FSDataOutputStream;
-import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hive.ql.io.HiveOutputFormat;
import org.apache.hadoop.io.ArrayWritable;
@@ -36,6 +38,8 @@
import org.apache.hadoop.mapred.Reporter;
import org.apache.hadoop.mapreduce.Counter;
import org.apache.hadoop.util.Progressable;
+import org.junit.Before;
+import org.junit.Test;
import parquet.hadoop.ParquetFileReader;
import parquet.hadoop.ParquetInputSplit;
@@ -52,22 +56,19 @@
* @author Mickaël Lacour
*
*/
-public class TestDeprecatedParquetOuputFormat extends TestCase {
-
- Map mapData;
- Configuration conf;
- JobConf job;
- FileSystem fs;
- Path dir;
- File testFile;
- Reporter reporter;
- FSDataOutputStream ds;
-
- @Override
- protected void setUp() throws Exception {
+public class TestMapredParquetOuputFormat {
+
+ private Map mapData;
+ private Configuration conf;
+ private JobConf job;
+ private Path dir;
+ private File testFile;
+ private Reporter reporter;
+
+ @Before
+ public void setUp() throws Exception {
conf = new Configuration();
job = new JobConf(conf);
- fs = FileSystem.getLocal(conf);
dir = new Path("target/tests/from_java/deprecatedoutputformat/");
testFile = new File(dir.toString(), "customer");
if (testFile.exists()) {
@@ -95,8 +96,9 @@ protected void setUp() throws Exception {
}
}
+ @Test
public void testParquetHiveOutputFormat() throws Exception {
- final HiveOutputFormat format = new DeprecatedParquetOutputFormat();
+ final HiveOutputFormat format = new MapredParquetOutputFormat();
final Properties tableProperties = new Properties();
// Set the configuration parameters
@@ -109,7 +111,7 @@ public void testParquetHiveOutputFormat() throws Exception {
System.out.println("First part, write the data");
job.set("mapred.task.id", "attempt_201304241759_32973_m_000002_0"); // FAKE ID
- final fakeStatus reporter = new fakeStatus();
+ final FakeStatus reporter = new FakeStatus();
final org.apache.hadoop.hive.ql.exec.FileSinkOperator.RecordWriter recordWriter = format.getHiveRecordWriter(
job,
new Path(testFile.getAbsolutePath()),
@@ -140,7 +142,7 @@ private void checkWrite() throws IOException, InterruptedException {
}
- final FileInputFormat format = new DeprecatedParquetInputFormat();
+ final FileInputFormat format = new MapredParquetInputFormat();
final String[] locations = new String[] {"localhost"};
final String schemaToString = schema.toString();
final String columnsStr = "message customer {\n"
@@ -171,7 +173,7 @@ private void checkWrite() throws IOException, InterruptedException {
final ParquetInputSplit realSplit = new ParquetInputSplit(new Path(testFile.getAbsolutePath()), 0, size, locations, blocks,
schemaToString, schemaToString, readFooter.getFileMetaData().getKeyValueMetaData(), readSupportMetaData);
- final DeprecatedParquetInputFormat.InputSplitWrapper splitWrapper = new DeprecatedParquetInputFormat.InputSplitWrapper(realSplit);
+ final MapredParquetInputFormat.InputSplitWrapper splitWrapper = new MapredParquetInputFormat.InputSplitWrapper(realSplit);
// construct the record reader
final RecordReader reader = format.getRecordReader(splitWrapper, job, reporter);
@@ -203,7 +205,7 @@ private void checkWrite() throws IOException, InterruptedException {
}
// FAKE Class in order to compile
- private class fakeStatus extends org.apache.hadoop.mapreduce.StatusReporter implements Progressable {
+ private class FakeStatus extends org.apache.hadoop.mapreduce.StatusReporter implements Progressable {
@Override
public Counter getCounter(final Enum> e) {
diff --git a/parquet-hive/src/test/java/parquet/hive/TestParquetSerDe.java b/parquet-hive/parquet-hive-storage-handler/src/test/java/parquet/hive/TestParquetSerDe.java
similarity index 94%
rename from parquet-hive/src/test/java/parquet/hive/TestParquetSerDe.java
rename to parquet-hive/parquet-hive-storage-handler/src/test/java/parquet/hive/TestParquetSerDe.java
index cc08bccbc5..0522b6baeb 100644
--- a/parquet-hive/src/test/java/parquet/hive/TestParquetSerDe.java
+++ b/parquet-hive/parquet-hive-storage-handler/src/test/java/parquet/hive/TestParquetSerDe.java
@@ -28,6 +28,7 @@
import parquet.hive.serde.ParquetHiveSerDe;
import parquet.hive.writable.BinaryWritable;
+import parquet.io.api.Binary;
/**
*
@@ -57,13 +58,13 @@ public void testParquetHiveSerDe() throws Throwable {
arr[2] = new IntWritable(789);
arr[3] = new LongWritable(1000l);
arr[4] = new DoubleWritable((double) 5.3);
- arr[5] = new BinaryWritable("hive and hadoop and parquet. Big family.");
+ arr[5] = new BinaryWritable(Binary.fromString("hive and hadoop and parquet. Big family."));
final Writable[] mapContainer = new Writable[1];
final Writable[] map = new Writable[3];
for (int i = 0; i < 3; ++i) {
final Writable[] pair = new Writable[2];
- pair[0] = new BinaryWritable("key_" + i);
+ pair[0] = new BinaryWritable(Binary.fromString("key_" + i));
pair[1] = new IntWritable(i);
map[i] = new ArrayWritable(Writable.class, pair);
}
@@ -73,7 +74,7 @@ public void testParquetHiveSerDe() throws Throwable {
final Writable[] arrayContainer = new Writable[1];
final Writable[] array = new Writable[5];
for (int i = 0; i < 5; ++i) {
- array[i] = new BinaryWritable("elem_" + i);
+ array[i] = new BinaryWritable(Binary.fromString("elem_" + i));
}
arrayContainer[0] = new ArrayWritable(Writable.class, array);
arr[7] = new ArrayWritable(Writable.class, arrayContainer);
diff --git a/parquet-hive/src/test/java/parquet/hive/UtilitiesTestMethods.java b/parquet-hive/parquet-hive-storage-handler/src/test/java/parquet/hive/UtilitiesTestMethods.java
similarity index 79%
rename from parquet-hive/src/test/java/parquet/hive/UtilitiesTestMethods.java
rename to parquet-hive/parquet-hive-storage-handler/src/test/java/parquet/hive/UtilitiesTestMethods.java
index f14f77d052..d1d40ad64d 100644
--- a/parquet-hive/src/test/java/parquet/hive/UtilitiesTestMethods.java
+++ b/parquet-hive/parquet-hive-storage-handler/src/test/java/parquet/hive/UtilitiesTestMethods.java
@@ -22,7 +22,9 @@
import parquet.hive.writable.BinaryWritable;
import parquet.io.api.Binary;
import parquet.io.api.RecordConsumer;
+import parquet.schema.GroupType;
import parquet.schema.MessageType;
+import parquet.schema.Type;
public class UtilitiesTestMethods {
@@ -37,6 +39,36 @@ public static void endFile(final ParquetFileWriter w) throws IOException {
w.end(new HashMap());
}
+ public static boolean smartCheckSchema(final GroupType expectedSchema, final GroupType actualSchema) {
+ if (expectedSchema.getFieldCount() != actualSchema.getFieldCount()) {
+ return false;
+ }
+
+ for (int i = 0; i < expectedSchema.getFieldCount(); ++i) {
+ Type expectedType = expectedSchema.getType(i);
+ Type actualType = actualSchema.getType(i);
+
+ if (!expectedType.getName().equals(actualType.getName())
+ || expectedType.getRepetition() != actualType.getRepetition()
+ || expectedType.isPrimitive() != actualType.isPrimitive()) {
+ return false;
+ }
+
+ if (expectedType.isPrimitive()) {
+ if (expectedType.asPrimitiveType().getPrimitiveTypeName() != actualType.asPrimitiveType().getPrimitiveTypeName()
+ || expectedType.asPrimitiveType().getTypeLength() != actualType.asPrimitiveType().getTypeLength()) {
+ return false;
+ }
+ } else {
+ if (!smartCheckSchema(expectedType.asGroupType(), actualType.asGroupType())) {
+ return false;
+ }
+ }
+ }
+
+ return true;
+ }
+
public static boolean smartCheckArray(final Writable[] arrValue, final Writable[] arrExpected, final Integer[] arrCheckIndexValues) {
int i = 0;
@@ -93,25 +125,25 @@ static public ArrayWritable createArrayWritable(final Integer custkey, final Str
final Writable[] arr = new Writable[11]; // The last one is for the unknown column
arr[0] = new IntWritable(custkey);
if (name != null) {
- arr[1] = new BinaryWritable(name);
+ arr[1] = new BinaryWritable(Binary.fromString(name));
}
if (address != null) {
- arr[2] = new BinaryWritable(address);
+ arr[2] = new BinaryWritable(Binary.fromString(address));
}
if (nationkey != null) {
arr[3] = new IntWritable(nationkey);
}
if (phone != null) {
- arr[4] = new BinaryWritable(phone);
+ arr[4] = new BinaryWritable(Binary.fromString(phone));
}
if (acctbal != null) {
arr[5] = new DoubleWritable(acctbal);
}
if (mktsegment != null) {
- arr[6] = new BinaryWritable(mktsegment);
+ arr[6] = new BinaryWritable(Binary.fromString(mktsegment));
}
if (comment != null) {
- arr[7] = new BinaryWritable(comment);
+ arr[7] = new BinaryWritable(Binary.fromString(comment));
}
if (map != null) {
final Writable[] mapContainer = new Writable[1];
@@ -119,8 +151,8 @@ static public ArrayWritable createArrayWritable(final Integer custkey, final Str
int i = 0;
for (Map.Entry entry : map.entrySet()) {
final Writable[] pair = new Writable[2];
- pair[0] = new BinaryWritable(entry.getKey());
- pair[1] = new BinaryWritable(entry.getValue());
+ pair[0] = new BinaryWritable(Binary.fromString(entry.getKey()));
+ pair[1] = new BinaryWritable(Binary.fromString(entry.getValue()));
mapArr[i] = new ArrayWritable(Writable.class, pair);
++i;
}
@@ -185,10 +217,10 @@ public static void writeField(final RecordConsumer recordWriter, final int index
} else if (value instanceof Map) {
recordWriter.startGroup();
recordWriter.startField("map", 0);
- for (Object entry : ((Map) value).entrySet()) {
+ for (Map.Entry, ?> entry : ((Map, ?>) value).entrySet()) {
recordWriter.startGroup();
- writeField(recordWriter, 0, "key", ((Map.Entry) entry).getKey());
- writeField(recordWriter, 1, "value", ((Map.Entry) entry).getValue());
+ writeField(recordWriter, 0, "key", entry.getKey());
+ writeField(recordWriter, 1, "value", entry.getValue());
recordWriter.endGroup();
}
recordWriter.endField("map", 0);
@@ -196,7 +228,7 @@ public static void writeField(final RecordConsumer recordWriter, final int index
} else if (value instanceof List) {
recordWriter.startGroup();
recordWriter.startField("bag", 0);
- for (Object element : (List) value) {
+ for (Object element : (List>) value) {
recordWriter.startGroup();
writeField(recordWriter, 0, "array_element", element);
recordWriter.endGroup();
diff --git a/parquet-hive/parquet-hive-storage-handler/src/test/java/parquet/hive/serde/TestAbstractParquetMapInspector.java b/parquet-hive/parquet-hive-storage-handler/src/test/java/parquet/hive/serde/TestAbstractParquetMapInspector.java
new file mode 100644
index 0000000000..f0edd8d490
--- /dev/null
+++ b/parquet-hive/parquet-hive-storage-handler/src/test/java/parquet/hive/serde/TestAbstractParquetMapInspector.java
@@ -0,0 +1,102 @@
+/*
+ * Copyright 2013 Criteo.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package parquet.hive.serde;
+
+import java.util.HashMap;
+import java.util.Map;
+import junit.framework.TestCase;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory;
+import org.apache.hadoop.io.ArrayWritable;
+import org.apache.hadoop.io.IntWritable;
+import org.apache.hadoop.io.Writable;
+import org.junit.Test;
+
+/**
+ *
+ * @author Rémy Pecqueur
+ */
+public class TestAbstractParquetMapInspector extends TestCase {
+
+ class TestableAbstractParquetMapInspector extends AbstractParquetMapInspector {
+
+ public TestableAbstractParquetMapInspector(ObjectInspector keyInspector, ObjectInspector valueInspector) {
+ super(keyInspector, valueInspector);
+ }
+
+ @Override
+ public Object getMapValueElement(Object o, Object o1) {
+ throw new UnsupportedOperationException("Should not be called");
+ }
+ }
+ private TestableAbstractParquetMapInspector inspector;
+
+ @Override
+ public void setUp() {
+ inspector = new TestableAbstractParquetMapInspector(PrimitiveObjectInspectorFactory.javaIntObjectInspector,
+ PrimitiveObjectInspectorFactory.javaIntObjectInspector);
+ }
+
+ @Test
+ public void testNullMap() {
+ assertEquals("Wrong size", -1, inspector.getMapSize(null));
+ assertNull("Should be null", inspector.getMap(null));
+ }
+
+ @Test
+ public void testNullContainer() {
+ final ArrayWritable map = new ArrayWritable(ArrayWritable.class, null);
+ assertEquals("Wrong size", -1, inspector.getMapSize(map));
+ assertNull("Should be null", inspector.getMap(map));
+ }
+
+ @Test
+ public void testEmptyContainer() {
+ final ArrayWritable map = new ArrayWritable(ArrayWritable.class, new ArrayWritable[0]);
+ assertEquals("Wrong size", -1, inspector.getMapSize(map));
+ assertNull("Should be null", inspector.getMap(map));
+ }
+
+ @Test
+ public void testRegularMap() {
+ final Writable[] entry1 = new Writable[]{new IntWritable(0), new IntWritable(1)};
+ final Writable[] entry2 = new Writable[]{new IntWritable(2), new IntWritable(3)};
+
+ final ArrayWritable internalMap = new ArrayWritable(ArrayWritable.class, new Writable[]{
+ new ArrayWritable(Writable.class, entry1), new ArrayWritable(Writable.class, entry2)});
+
+ final ArrayWritable map = new ArrayWritable(ArrayWritable.class, new Writable[]{internalMap});
+
+ final Map expected = new HashMap();
+ expected.put(new IntWritable(0), new IntWritable(1));
+ expected.put(new IntWritable(2), new IntWritable(3));
+
+ assertEquals("Wrong size", 2, inspector.getMapSize(map));
+ assertEquals("Wrong result of inspection", expected, inspector.getMap(map));
+ }
+
+ @Test
+ public void testHashMap() {
+ final Map map = new HashMap();
+ map.put(new IntWritable(0), new IntWritable(1));
+ map.put(new IntWritable(2), new IntWritable(3));
+ map.put(new IntWritable(4), new IntWritable(5));
+ map.put(new IntWritable(6), new IntWritable(7));
+
+ assertEquals("Wrong size", 4, inspector.getMapSize(map));
+ assertEquals("Wrong result of inspection", map, inspector.getMap(map));
+ }
+}
diff --git a/parquet-hive/parquet-hive-storage-handler/src/test/java/parquet/hive/serde/TestDeepParquetHiveMapInspector.java b/parquet-hive/parquet-hive-storage-handler/src/test/java/parquet/hive/serde/TestDeepParquetHiveMapInspector.java
new file mode 100644
index 0000000000..838f264166
--- /dev/null
+++ b/parquet-hive/parquet-hive-storage-handler/src/test/java/parquet/hive/serde/TestDeepParquetHiveMapInspector.java
@@ -0,0 +1,94 @@
+/*
+ * Copyright 2013 Criteo.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package parquet.hive.serde;
+
+import java.util.HashMap;
+import java.util.Map;
+import junit.framework.TestCase;
+import org.apache.hadoop.hive.serde2.io.ShortWritable;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory;
+import org.apache.hadoop.io.ArrayWritable;
+import org.apache.hadoop.io.IntWritable;
+import org.apache.hadoop.io.Writable;
+import org.junit.Test;
+import parquet.hive.serde.primitive.ParquetPrimitiveInspectorFactory;
+
+/**
+ *
+ * @author Rémy Pecqueur