This is an automated email from the ASF dual-hosted git repository.
dzamo pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/drill.git
The following commit(s) were added to refs/heads/master by this push:
new 0224ee3 DRILL-8007: Fix reading time / timestamp micros and binary
types (#2370)
0224ee3 is described below
commit 0224ee33f8fb916067a64b997899957cde38af24
Author: Volodymyr Vysotskyi <[email protected]>
AuthorDate: Sun Nov 14 07:30:59 2021 +0200
DRILL-8007: Fix reading time / timestamp micros and binary types (#2370)
---
.../exec/store/parquet/ParquetReaderUtility.java | 2 +
.../parquet/columnreaders/ColumnReaderFactory.java | 15 ++-
.../NullableFixedByteAlignedReaders.java | 38 ++++++++
.../ParquetFixedWidthDictionaryReaders.java | 56 +++++++++++
.../columnreaders/ParquetToDrillTypeConverter.java | 5 +-
.../store/parquet2/DrillParquetGroupConverter.java | 104 +++++++++++++++------
.../store/parquet/TestParquetLogicalTypes.java | 72 +++++++-------
.../store/parquet2/TestDrillParquetReader.java | 76 ++++++++++++---
.../src/test/resources/parquet2/allTypes.parquet | Bin 0 -> 7804 bytes
9 files changed, 285 insertions(+), 83 deletions(-)
diff --git
a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetReaderUtility.java
b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetReaderUtility.java
index d4267dd..ef44340 100644
---
a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetReaderUtility.java
+++
b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetReaderUtility.java
@@ -619,8 +619,10 @@ public class ParquetReaderUtility {
case DATE:
return TypeProtos.MinorType.DATE;
case TIME_MILLIS:
+ case TIME_MICROS:
return TypeProtos.MinorType.TIME;
case TIMESTAMP_MILLIS:
+ case TIMESTAMP_MICROS:
return TypeProtos.MinorType.TIMESTAMP;
case UTF8:
return TypeProtos.MinorType.VARCHAR;
diff --git
a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/ColumnReaderFactory.java
b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/ColumnReaderFactory.java
index e084c2d..077ad92 100644
---
a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/ColumnReaderFactory.java
+++
b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/ColumnReaderFactory.java
@@ -176,11 +176,15 @@ public class ColumnReaderFactory {
columnChunkMetaData, fixedLength, (BigIntVector) v,
schemaElement);
}
switch (convertedType) {
- // DRILL-6670: handle TIMESTAMP_MICROS as INT64 with no logical type
case INT_64:
- case TIMESTAMP_MICROS:
return new
ParquetFixedWidthDictionaryReaders.DictionaryBigIntReader(recordReader,
descriptor,
- columnChunkMetaData, fixedLength, (BigIntVector) v,
schemaElement);
+ columnChunkMetaData, fixedLength, (BigIntVector) v,
schemaElement);
+ case TIMESTAMP_MICROS:
+ return new
ParquetFixedWidthDictionaryReaders.DictionaryTimeStampMicrosReader(recordReader,
descriptor,
+ columnChunkMetaData, fixedLength, (TimeStampVector) v,
schemaElement);
+ case TIME_MICROS:
+ return new
ParquetFixedWidthDictionaryReaders.DictionaryTimeMicrosReader(recordReader,
descriptor,
+ columnChunkMetaData, fixedLength, (TimeVector) v, schemaElement);
case UINT_64:
return new
ParquetFixedWidthDictionaryReaders.DictionaryUInt8Reader(recordReader,
descriptor,
columnChunkMetaData, fixedLength, (UInt8Vector) v,
schemaElement);
@@ -294,8 +298,11 @@ public class ColumnReaderFactory {
columnDescriptor, columnChunkMetaData, fixedLength,
(NullableVarDecimalVector) valueVec, schemaElement);
case TIMESTAMP_MILLIS:
return new
NullableFixedByteAlignedReaders.NullableDictionaryTimeStampReader(parentReader,
columnDescriptor, columnChunkMetaData, fixedLength,
(NullableTimeStampVector)valueVec, schemaElement);
- // DRILL-6670: handle TIMESTAMP_MICROS as INT64 with no logical type
+ case TIME_MICROS:
+ return new
NullableFixedByteAlignedReaders.NullableDictionaryTimeMicrosReader(parentReader,
columnDescriptor, columnChunkMetaData, fixedLength, (NullableTimeVector)
valueVec, schemaElement);
case TIMESTAMP_MICROS:
+ return new
NullableFixedByteAlignedReaders.NullableDictionaryTimeStampMicrosReader(parentReader,
+ columnDescriptor, columnChunkMetaData, fixedLength,
(NullableTimeStampVector) valueVec, schemaElement);
case INT_64:
return new
NullableFixedByteAlignedReaders.NullableDictionaryBigIntReader(parentReader,
columnDescriptor, columnChunkMetaData, fixedLength,
(NullableBigIntVector) valueVec, schemaElement);
diff --git
a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/NullableFixedByteAlignedReaders.java
b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/NullableFixedByteAlignedReaders.java
index 54705cc..e264536 100644
---
a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/NullableFixedByteAlignedReaders.java
+++
b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/NullableFixedByteAlignedReaders.java
@@ -185,6 +185,25 @@ public class NullableFixedByteAlignedReaders {
}
}
+ static class NullableDictionaryTimeMicrosReader extends
NullableColumnReader<NullableTimeVector> {
+
+ NullableDictionaryTimeMicrosReader(ParquetRecordReader parentReader,
ColumnDescriptor descriptor,
+ ColumnChunkMetaData columnChunkMetaData, boolean fixedLength,
NullableTimeVector v,
+ SchemaElement schemaElement) throws ExecutionSetupException {
+ super(parentReader, descriptor, columnChunkMetaData, fixedLength, v,
schemaElement);
+ }
+
+ // this method is called by its superclass during a read loop
+ @Override
+ protected void readField(long recordsToReadInThisPass) {
+ ValuesReader valReader = usingDictionary ?
pageReader.getDictionaryValueReader() : pageReader.getValueReader();
+ for (int i = 0; i < recordsToReadInThisPass; i++) {
+ valueVec.getMutator().setSafe(valuesReadInCurrentPass + i,
valReader.readInteger() / 1000);
+ }
+ advanceWriterIndex();
+ }
+ }
+
static class NullableDictionaryBigIntReader extends
NullableColumnReader<NullableBigIntVector> {
NullableDictionaryBigIntReader(ParquetRecordReader parentReader,
ColumnDescriptor descriptor,
@@ -242,6 +261,25 @@ public class NullableFixedByteAlignedReaders {
}
}
+ static class NullableDictionaryTimeStampMicrosReader extends
NullableColumnReader<NullableTimeStampVector> {
+
+ NullableDictionaryTimeStampMicrosReader(ParquetRecordReader parentReader,
ColumnDescriptor descriptor,
+ ColumnChunkMetaData columnChunkMetaData, boolean fixedLength,
NullableTimeStampVector v,
+ SchemaElement schemaElement) throws ExecutionSetupException {
+ super(parentReader, descriptor, columnChunkMetaData, fixedLength, v,
schemaElement);
+ }
+
+ // this method is called by its superclass during a read loop
+ @Override
+ protected void readField(long recordsToReadInThisPass) {
+ ValuesReader valReader = usingDictionary ?
pageReader.getDictionaryValueReader() : pageReader.getValueReader();
+ for (int i = 0; i < recordsToReadInThisPass; i++) {
+ valueVec.getMutator().setSafe(valuesReadInCurrentPass + i,
valReader.readLong() / 1000);
+ }
+ advanceWriterIndex();
+ }
+ }
+
static class NullableDictionaryVarDecimalReader extends
NullableColumnReader<NullableVarDecimalVector> {
NullableDictionaryVarDecimalReader(ParquetRecordReader parentReader,
ColumnDescriptor descriptor,
diff --git
a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/ParquetFixedWidthDictionaryReaders.java
b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/ParquetFixedWidthDictionaryReaders.java
index 845afe8..3ad9a20 100644
---
a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/ParquetFixedWidthDictionaryReaders.java
+++
b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/ParquetFixedWidthDictionaryReaders.java
@@ -157,6 +157,34 @@ public class ParquetFixedWidthDictionaryReaders {
}
}
+ static class DictionaryTimeMicrosReader extends
FixedByteAlignedReader<TimeVector> {
+ DictionaryTimeMicrosReader(ParquetRecordReader parentReader,
ColumnDescriptor descriptor,
+ ColumnChunkMetaData columnChunkMetaData, boolean fixedLength, TimeVector
v,
+ SchemaElement schemaElement) throws ExecutionSetupException {
+ super(parentReader, descriptor, columnChunkMetaData, fixedLength, v,
schemaElement);
+ }
+
+ // this method is called by its superclass during a read loop
+ @Override
+ protected void readField(long recordsToReadInThisPass) {
+ recordsReadInThisIteration = Math.min(pageReader.pageValueCount
+ - pageReader.valuesRead, recordsToReadInThisPass -
valuesReadInCurrentPass);
+ if (recordsRequireDecoding()) {
+ ValuesReader valReader = usingDictionary ?
pageReader.getDictionaryValueReader() : pageReader.getValueReader();
+ for (int i = 0; i < recordsReadInThisIteration; i++) {
+ valueVec.getMutator().setSafe(valuesReadInCurrentPass + i,
valReader.readInteger() / 1000);
+ }
+ } else {
+ int dataTypeLengthInBytes = (int) Math.ceil(dataTypeLengthInBits /
8.0);
+ for (int i = 0; i < recordsReadInThisIteration; i++) {
+ int value = pageReader.pageData.getInt((int) readStartInBytes + i *
dataTypeLengthInBytes);
+ valueVec.getMutator().setSafe(valuesReadInCurrentPass + i, value /
1000);
+ }
+ }
+ advanceWriterIndex(valueVec.getBuffer(),
BITS_COUNT_IN_BYTE_DOUBLE_VALUE);
+ }
+ }
+
static class DictionaryBigIntReader extends
FixedByteAlignedReader<BigIntVector> {
DictionaryBigIntReader(ParquetRecordReader parentReader, ColumnDescriptor
descriptor,
ColumnChunkMetaData columnChunkMetaData,
boolean fixedLength, BigIntVector v,
@@ -309,6 +337,34 @@ public class ParquetFixedWidthDictionaryReaders {
}
}
+ static class DictionaryTimeStampMicrosReader extends
FixedByteAlignedReader<TimeStampVector> {
+ DictionaryTimeStampMicrosReader(ParquetRecordReader parentReader,
ColumnDescriptor descriptor,
+ ColumnChunkMetaData columnChunkMetaData, boolean fixedLength,
TimeStampVector v,
+ SchemaElement schemaElement) throws ExecutionSetupException {
+ super(parentReader, descriptor, columnChunkMetaData, fixedLength, v,
schemaElement);
+ }
+
+ // this method is called by its superclass during a read loop
+ @Override
+ protected void readField(long recordsToReadInThisPass) {
+ recordsReadInThisIteration = Math.min(pageReader.pageValueCount
+ - pageReader.valuesRead, recordsToReadInThisPass -
valuesReadInCurrentPass);
+ if (recordsRequireDecoding()) {
+ ValuesReader valReader = usingDictionary ?
pageReader.getDictionaryValueReader() : pageReader.getValueReader();
+ for (int i = 0; i < recordsReadInThisIteration; i++) {
+ valueVec.getMutator().setSafe(valuesReadInCurrentPass + i,
valReader.readLong() / 1000);
+ }
+ } else {
+ int dataTypeLengthInBytes = (int) Math.ceil(dataTypeLengthInBits /
8.0);
+ for (int i = 0; i < recordsReadInThisIteration; i++) {
+ long value = pageReader.pageData.getLong((int) readStartInBytes + i
* dataTypeLengthInBytes);
+ valueVec.getMutator().setSafe(valuesReadInCurrentPass + i, value /
1000);
+ }
+ }
+ advanceWriterIndex(valueVec.getBuffer(),
BITS_COUNT_IN_BYTE_DOUBLE_VALUE);
+ }
+ }
+
static class DictionaryBinaryAsTimeStampReader extends
FixedByteAlignedReader<TimeStampVector> {
DictionaryBinaryAsTimeStampReader(ParquetRecordReader parentReader,
ColumnDescriptor descriptor,
ColumnChunkMetaData columnChunkMetaData, boolean
fixedLength, TimeStampVector v,
diff --git
a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/ParquetToDrillTypeConverter.java
b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/ParquetToDrillTypeConverter.java
index b571322..dbf3199 100644
---
a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/ParquetToDrillTypeConverter.java
+++
b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/ParquetToDrillTypeConverter.java
@@ -54,9 +54,7 @@ public class ParquetToDrillTypeConverter {
return (TypeProtos.MinorType.BIGINT);
}
switch(convertedType) {
- // DRILL-6670: handle TIMESTAMP_MICROS as INT64 with no logical type
case INT_64:
- case TIMESTAMP_MICROS:
return TypeProtos.MinorType.BIGINT;
case UINT_64:
return TypeProtos.MinorType.UINT8;
@@ -64,7 +62,10 @@ public class ParquetToDrillTypeConverter {
ParquetReaderUtility.checkDecimalTypeEnabled(options);
return TypeProtos.MinorType.VARDECIMAL;
case TIMESTAMP_MILLIS:
+ case TIMESTAMP_MICROS:
return TypeProtos.MinorType.TIMESTAMP;
+ case TIME_MICROS:
+ return TypeProtos.MinorType.TIME;
default:
throw new UnsupportedOperationException(String.format("unsupported
type: %s %s", primitiveTypeName, convertedType));
}
diff --git
a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet2/DrillParquetGroupConverter.java
b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet2/DrillParquetGroupConverter.java
index fdb6e79..d10aa46 100644
---
a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet2/DrillParquetGroupConverter.java
+++
b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet2/DrillParquetGroupConverter.java
@@ -25,6 +25,7 @@ import java.util.List;
import java.util.Optional;
import java.util.function.BiFunction;
import java.util.function.Function;
+import java.util.function.Supplier;
import io.netty.buffer.DrillBuf;
import org.apache.drill.common.exceptions.DrillRuntimeException;
@@ -255,11 +256,18 @@ public class DrillParquetGroupConverter extends
GroupConverter {
return getBigIntConverter(name, type);
}
switch(type.getOriginalType()) {
- // DRILL-6670: handle TIMESTAMP_MICROS as INT64 with no logical type
case UINT_64:
- case INT_64 :
- case TIMESTAMP_MICROS: {
+ case INT_64:
return getBigIntConverter(name, type);
+ case TIMESTAMP_MICROS: {
+ TimeStampWriter writer = getTimeStampWriter(name, type);
+ return new DrillTimeStampMicrosConverter(writer);
+ }
+ case TIME_MICROS: {
+ TimeWriter writer = type.isRepetition(Repetition.REPEATED)
+ ? getWriter(name, (m, f) -> m.list(f).time(), l ->
l.list().time())
+ : getWriter(name, MapWriter::time, ListWriter::time);
+ return new DrillTimeMicrosConverter(writer);
}
case DECIMAL: {
ParquetReaderUtility.checkDecimalTypeEnabled(options);
@@ -308,26 +316,33 @@ public class DrillParquetGroupConverter extends
GroupConverter {
return new DrillBoolConverter(writer);
}
case BINARY: {
- if (type.getOriginalType() == null) {
- VarBinaryWriter writer = type.isRepetition(Repetition.REPEATED)
- ? getWriter(name, (m, f) -> m.list(f).varBinary(), l ->
l.list().varBinary())
- : getWriter(name, (m, f) -> m.varBinary(f), l -> l.varBinary());
- return new DrillVarBinaryConverter(writer,
mutator.getManagedBuffer());
- }
- switch(type.getOriginalType()) {
- case UTF8:
- case ENUM: {
- return getVarCharConverter(name, type);
- }
- // See DRILL-4184 and DRILL-4834. Support for this is added using
new VarDecimal type.
- case DECIMAL: {
+ LogicalTypeAnnotation.LogicalTypeAnnotationVisitor<PrimitiveConverter>
typeAnnotationVisitor = new
LogicalTypeAnnotation.LogicalTypeAnnotationVisitor<PrimitiveConverter>() {
+ @Override
+ public Optional<PrimitiveConverter>
visit(LogicalTypeAnnotation.DecimalLogicalTypeAnnotation decimalLogicalType) {
ParquetReaderUtility.checkDecimalTypeEnabled(options);
- return getVarDecimalConverter(name, type);
+ return Optional.of(getVarDecimalConverter(name, type));
}
- default: {
- throw new UnsupportedOperationException("Unsupported type " +
type.getOriginalType());
+
+ @Override
+ public Optional<PrimitiveConverter>
visit(LogicalTypeAnnotation.StringLogicalTypeAnnotation stringLogicalType) {
+ return Optional.of(getVarCharConverter(name, type));
}
- }
+
+ @Override
+ public Optional<PrimitiveConverter>
visit(LogicalTypeAnnotation.EnumLogicalTypeAnnotation stringLogicalType) {
+ return Optional.of(getVarCharConverter(name, type));
+ }
+ };
+ Supplier<PrimitiveConverter> converterSupplier = () -> {
+ VarBinaryWriter writer = type.isRepetition(Repetition.REPEATED)
+ ? getWriter(name, (m, f) -> m.list(f).varBinary(), l ->
l.list().varBinary())
+ : getWriter(name, MapWriter::varBinary, ListWriter::varBinary);
+ return new DrillVarBinaryConverter(writer,
mutator.getManagedBuffer());
+ };
+ return Optional.ofNullable(type.getLogicalTypeAnnotation())
+ .map(typeAnnotation -> typeAnnotation.accept(typeAnnotationVisitor))
+ .flatMap(Function.identity())
+ .orElseGet(converterSupplier);
}
case FIXED_LEN_BYTE_ARRAY:
LogicalTypeAnnotation.LogicalTypeAnnotationVisitor<PrimitiveConverter>
typeAnnotationVisitor = new
LogicalTypeAnnotation.LogicalTypeAnnotationVisitor<PrimitiveConverter>() {
@@ -346,15 +361,16 @@ public class DrillParquetGroupConverter extends
GroupConverter {
}
};
- LogicalTypeAnnotation logicalTypeAnnotation =
type.getLogicalTypeAnnotation();
- if (logicalTypeAnnotation != null) {
- return
logicalTypeAnnotation.accept(typeAnnotationVisitor).orElseGet(() -> {
- VarBinaryWriter writer = type.isRepetition(Repetition.REPEATED)
- ? getWriter(name, (m, f) -> m.list(f).varBinary(), l ->
l.list().varBinary())
- : getWriter(name, MapWriter::varBinary, ListWriter::varBinary);
- return new DrillFixedBinaryToVarbinaryConverter(writer,
type.getTypeLength(), mutator.getManagedBuffer());
- });
- }
+ Supplier<PrimitiveConverter> converterSupplier = () -> {
+ VarBinaryWriter writer = type.isRepetition(Repetition.REPEATED)
+ ? getWriter(name, (m, f) -> m.list(f).varBinary(), l ->
l.list().varBinary())
+ : getWriter(name, MapWriter::varBinary, ListWriter::varBinary);
+ return new DrillFixedBinaryToVarbinaryConverter(writer,
type.getTypeLength(), mutator.getManagedBuffer());
+ };
+ return Optional.ofNullable(type.getLogicalTypeAnnotation())
+ .map(typeAnnotation -> typeAnnotation.accept(typeAnnotationVisitor))
+ .flatMap(Function.identity())
+ .orElseGet(converterSupplier);
default:
throw new UnsupportedOperationException("Unsupported type: " +
type.getPrimitiveTypeName());
}
@@ -519,6 +535,21 @@ public class DrillParquetGroupConverter extends
GroupConverter {
}
}
+ public static class DrillTimeMicrosConverter extends PrimitiveConverter {
+ private final TimeWriter writer;
+ private final TimeHolder holder = new TimeHolder();
+
+ public DrillTimeMicrosConverter(TimeWriter writer) {
+ this.writer = writer;
+ }
+
+ @Override
+ public void addLong(long value) {
+ holder.value = (int) (value / 1000);
+ writer.write(holder);
+ }
+ }
+
public static class DrillBigIntConverter extends PrimitiveConverter {
private BigIntWriter writer;
private BigIntHolder holder = new BigIntHolder();
@@ -549,6 +580,21 @@ public class DrillParquetGroupConverter extends
GroupConverter {
}
}
+ public static class DrillTimeStampMicrosConverter extends PrimitiveConverter
{
+ private final TimeStampWriter writer;
+ private final TimeStampHolder holder = new TimeStampHolder();
+
+ public DrillTimeStampMicrosConverter(TimeStampWriter writer) {
+ this.writer = writer;
+ }
+
+ @Override
+ public void addLong(long value) {
+ holder.value = value / 1000;
+ writer.write(holder);
+ }
+ }
+
public static class DrillFloat4Converter extends PrimitiveConverter {
private Float4Writer writer;
private Float4Holder holder = new Float4Holder();
diff --git
a/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/TestParquetLogicalTypes.java
b/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/TestParquetLogicalTypes.java
index 6c039e9..4a3de23 100644
---
a/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/TestParquetLogicalTypes.java
+++
b/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/TestParquetLogicalTypes.java
@@ -120,19 +120,19 @@ public class TestParquetLogicalTypes extends
BaseTestQuery {
.baselineValues(1, "UTF8 string1", "RANDOM_VALUE", 1234567, 123,
12345, 1234567, 123, 1234, 1234567,
1234567890123456L, 1234567890123456L, 1234567890123456L,
LocalDate.parse("5350-02-17"),
LocalDateTime.ofInstant(Instant.ofEpochMilli(1234567),
ZoneOffset.UTC).toLocalTime(),
- LocalDateTime.parse("1973-11-29T21:33:09.012"), 123456789012L,
+ LocalDateTime.parse("1973-11-29T21:33:09.012"),
LocalDateTime.of(1970, 1, 2, 10, 17, 36, 789_000_000),
new
Period().plusMonths(875770417).plusDays(943142453).plusMillis(1650536505),
bytes12)
.baselineValues(2, "UTF8 string2", "MAX_VALUE", 2147483647, 127,
32767, 2147483647, 255, 65535, -1,
9223372036854775807L, 9223372036854775807L, -1L,
LocalDate.parse("1969-12-31"),
LocalDateTime.ofInstant(Instant.ofEpochMilli(0xFFFFFFFF),
ZoneOffset.UTC).toLocalTime(),
- LocalDateTime.parse("2038-01-19T03:14:07.999"),
9223372036854775807L,
+ LocalDateTime.parse("2038-01-19T03:14:07.999"),
LocalDateTime.of(294247, 1, 10, 4, 0, 54, 775_000_000),
new
Period().plusMonths(16843009).plusDays(16843009).plusMillis(16843009),
bytesOnes)
.baselineValues(3, "UTF8 string3", "MIN_VALUE", -2147483648, -128,
-32768, -2147483648, 0, 0, 0,
-9223372036854775808L, -9223372036854775808L, 0L,
LocalDate.parse("1970-01-01"),
LocalDateTime.ofInstant(Instant.ofEpochMilli(0),
ZoneOffset.UTC).toLocalTime(),
- LocalDateTime.parse("1970-01-01T00:00:00.0"), 0L, new
Period("PT0S"), bytesZeros)
+ LocalDateTime.parse("1970-01-01T00:00:00.0"),
LocalDateTime.of(1970, 1, 1, 0, 0, 0), new Period("PT0S"), bytesZeros)
.build()
.run();
}
@@ -195,19 +195,19 @@ public class TestParquetLogicalTypes extends
BaseTestQuery {
.baselineValues(1, "UTF8 string1", "RANDOM_VALUE", 1234567, 123,
12345, 1234567, 123, 1234, 1234567,
1234567890123456L, 1234567890123456L, 1234567890123456L,
LocalDate.parse("5350-02-17"),
LocalDateTime.ofInstant(Instant.ofEpochMilli(1234567),
ZoneOffset.UTC).toLocalTime(),
- LocalDateTime.parse("1973-11-29T21:33:09.012"), 123456789012L,
+ LocalDateTime.parse("1973-11-29T21:33:09.012"),
LocalDateTime.of(1970, 1, 2, 10, 17, 36, 789_000_000),
new
Period().plusMonths(875770417).plusDays(943142453).plusMillis(1650536505),
bytes12)
.baselineValues(2, "UTF8 string2", "MAX_VALUE", 2147483647, 127,
32767, 2147483647, 255, 65535, -1,
9223372036854775807L, 9223372036854775807L, -1L,
LocalDate.parse("1969-12-31"),
LocalDateTime.ofInstant(Instant.ofEpochMilli(0xFFFFFFFF),
ZoneOffset.UTC).toLocalTime(),
- LocalDateTime.parse("2038-01-19T03:14:07.999"),
9223372036854775807L,
+ LocalDateTime.parse("2038-01-19T03:14:07.999"),
LocalDateTime.of(294247, 1, 10, 4, 0, 54, 775_000_000),
new
Period().plusMonths(16843009).plusDays(16843009).plusMillis(16843009),
bytesOnes)
.baselineValues(3, "UTF8 string3", "MIN_VALUE", -2147483648, -128,
-32768, -2147483648, 0, 0, 0,
-9223372036854775808L, -9223372036854775808L, 0L,
LocalDate.parse("1970-01-01"),
LocalDateTime.ofInstant(Instant.ofEpochMilli(0),
ZoneOffset.UTC).toLocalTime(),
- LocalDateTime.parse("1970-01-01T00:00:00.0"), 0L, new
Period("PT0S"), bytesZeros)
+ LocalDateTime.parse("1970-01-01T00:00:00.0"),
LocalDateTime.of(1970, 1, 1, 0, 0, 0), new Period("PT0S"), bytesZeros)
.baselineValues(4, null, null, null, null, null, null, null, null,
null, null, null, null, null,
null, null, null, null, null)
.build().run();
@@ -271,19 +271,19 @@ public class TestParquetLogicalTypes extends
BaseTestQuery {
.baselineValues(1, "UTF8 string1", "RANDOM_VALUE", 1234567, 123,
12345, 1234567, 123, 1234, 1234567,
1234567890123456L, 1234567890123456L, 1234567890123456L,
LocalDate.parse("5350-02-17"),
LocalDateTime.ofInstant(Instant.ofEpochMilli(1234567),
ZoneOffset.UTC).toLocalTime(),
- LocalDateTime.parse("1973-11-29T21:33:09.012"), 123456789012L,
+ LocalDateTime.parse("1973-11-29T21:33:09.012"),
LocalDateTime.of(1970, 1, 2, 10, 17, 36, 789_000_000),
new
Period().plusMonths(875770417).plusDays(943142453).plusMillis(1650536505),
bytes12)
.baselineValues(2, "UTF8 string2", "MAX_VALUE", 2147483647, 127,
32767, 2147483647, 255, 65535, -1,
9223372036854775807L, 9223372036854775807L, -1L,
LocalDate.parse("1969-12-31"),
LocalDateTime.ofInstant(Instant.ofEpochMilli(0xFFFFFFFF),
ZoneOffset.UTC).toLocalTime(),
- LocalDateTime.parse("2038-01-19T03:14:07.999"),
9223372036854775807L,
+ LocalDateTime.parse("2038-01-19T03:14:07.999"),
LocalDateTime.of(294247, 1, 10, 4, 0, 54, 775_000_000),
new
Period().plusMonths(16843009).plusDays(16843009).plusMillis(16843009),
bytesOnes)
.baselineValues(3, "UTF8 string3", "MIN_VALUE", -2147483648, -128,
-32768, -2147483648, 0, 0, 0,
-9223372036854775808L, -9223372036854775808L, 0L,
LocalDate.parse("1970-01-01"),
LocalDateTime.ofInstant(Instant.ofEpochMilli(0),
ZoneOffset.UTC).toLocalTime(),
- LocalDateTime.parse("1970-01-01T00:00:00.0"), 0L, new
Period("PT0S"), bytesZeros)
+ LocalDateTime.parse("1970-01-01T00:00:00.0"),
LocalDateTime.of(1970, 1, 1, 0, 0, 0), new Period("PT0S"), bytesZeros)
.build()
.run();
}
@@ -347,19 +347,19 @@ public class TestParquetLogicalTypes extends
BaseTestQuery {
.baselineValues(1, "UTF8 string1", "RANDOM_VALUE", 1234567, 123,
12345, 1234567, 123, 1234, 1234567,
1234567890123456L, 1234567890123456L, 1234567890123456L,
LocalDate.parse("5350-02-17"),
LocalDateTime.ofInstant(Instant.ofEpochMilli(1234567),
ZoneOffset.UTC).toLocalTime(),
- LocalDateTime.parse("1973-11-29T21:33:09.012"), 123456789012L,
+ LocalDateTime.parse("1973-11-29T21:33:09.012"),
LocalDateTime.of(1970, 1, 2, 10, 17, 36, 789_000_000),
new
Period().plusMonths(875770417).plusDays(943142453).plusMillis(1650536505),
bytes12)
.baselineValues(2, "UTF8 string2", "MAX_VALUE", 2147483647, 127,
32767, 2147483647, 255, 65535, -1,
9223372036854775807L, 9223372036854775807L, -1L,
LocalDate.parse("1969-12-31"),
LocalDateTime.ofInstant(Instant.ofEpochMilli(0xFFFFFFFF),
ZoneOffset.UTC).toLocalTime(),
- LocalDateTime.parse("2038-01-19T03:14:07.999"),
9223372036854775807L,
+ LocalDateTime.parse("2038-01-19T03:14:07.999"),
LocalDateTime.of(294247, 1, 10, 4, 0, 54, 775_000_000),
new
Period().plusMonths(16843009).plusDays(16843009).plusMillis(16843009),
bytesOnes)
.baselineValues(3, "UTF8 string3", "MIN_VALUE", -2147483648, -128,
-32768, -2147483648, 0, 0, 0,
-9223372036854775808L, -9223372036854775808L, 0L,
LocalDate.parse("1970-01-01"),
LocalDateTime.ofInstant(Instant.ofEpochMilli(0),
ZoneOffset.UTC).toLocalTime(),
- LocalDateTime.parse("1970-01-01T00:00:00.0"), 0L, new
Period("PT0S"), bytesZeros)
+ LocalDateTime.parse("1970-01-01T00:00:00.0"),
LocalDateTime.of(1970, 1, 1, 0, 0, 0), new Period("PT0S"), bytesZeros)
.baselineValues(4, null, null, null, null, null, null, null, null,
null, null, null, null, null,
null, null, null, null, null)
.build().run();
@@ -423,19 +423,19 @@ public class TestParquetLogicalTypes extends
BaseTestQuery {
.baselineValues(1, "UTF8 string1", "RANDOM_VALUE", 1234567, 123, 12345,
1234567, 123, 1234, 1234567,
1234567890123456L, 1234567890123456L, 1234567890123456L,
LocalDate.parse("5350-02-17"),
LocalDateTime.ofInstant(Instant.ofEpochMilli(1234567),
ZoneOffset.UTC).toLocalTime(),
- LocalDateTime.parse("1973-11-29T21:33:09.012"), 123456789012L,
+ LocalDateTime.parse("1973-11-29T21:33:09.012"), LocalDateTime.of(1970,
1, 2, 10, 17, 36, 789_000_000),
new
Period().plusMonths(875770417).plusDays(943142453).plusMillis(1650536505),
bytes12)
.baselineValues(2, "UTF8 string2", "MAX_VALUE", 2147483647, 127, 32767,
2147483647, 255, 65535, -1,
9223372036854775807L, 9223372036854775807L, -1L,
LocalDate.parse("1969-12-31"),
LocalDateTime.ofInstant(Instant.ofEpochMilli(0xFFFFFFFF),
ZoneOffset.UTC).toLocalTime(),
- LocalDateTime.parse("2038-01-19T03:14:07.999"), 9223372036854775807L,
+ LocalDateTime.parse("2038-01-19T03:14:07.999"),
LocalDateTime.of(294247, 1, 10, 4, 0, 54, 775_000_000),
new
Period().plusMonths(16843009).plusDays(16843009).plusMillis(16843009),
bytesOnes)
.baselineValues(3, "UTF8 string3", "MIN_VALUE", -2147483648, -128,
-32768, -2147483648, 0, 0, 0,
-9223372036854775808L, -9223372036854775808L, 0L,
LocalDate.parse("1970-01-01"),
LocalDateTime.ofInstant(Instant.ofEpochMilli(0),
ZoneOffset.UTC).toLocalTime(),
- LocalDateTime.parse("1970-01-01T00:00:00.0"), 0L, new Period("PT0S"),
bytesZeros)
+ LocalDateTime.parse("1970-01-01T00:00:00.0"), LocalDateTime.of(1970,
1, 1, 0, 0, 0), new Period("PT0S"), bytesZeros)
.build()
.run();
}
@@ -498,19 +498,19 @@ public class TestParquetLogicalTypes extends
BaseTestQuery {
.baselineValues(1, "UTF8 string1", "RANDOM_VALUE", 1234567, 123, 12345,
1234567, 123, 1234, 1234567,
1234567890123456L, 1234567890123456L, 1234567890123456L,
LocalDate.parse("5350-02-17"),
LocalDateTime.ofInstant(Instant.ofEpochMilli(1234567),
ZoneOffset.UTC).toLocalTime(),
- LocalDateTime.parse("1973-11-29T21:33:09.012"), 123456789012L,
+ LocalDateTime.parse("1973-11-29T21:33:09.012"), LocalDateTime.of(1970,
1, 2, 10, 17, 36, 789_000_000),
new
Period().plusMonths(875770417).plusDays(943142453).plusMillis(1650536505),
bytes12)
.baselineValues(2, "UTF8 string2", "MAX_VALUE", 2147483647, 127, 32767,
2147483647, 255, 65535, -1,
9223372036854775807L, 9223372036854775807L, -1L,
LocalDate.parse("1969-12-31"),
LocalDateTime.ofInstant(Instant.ofEpochMilli(0xFFFFFFFF),
ZoneOffset.UTC).toLocalTime(),
- LocalDateTime.parse("2038-01-19T03:14:07.999"), 9223372036854775807L,
+ LocalDateTime.parse("2038-01-19T03:14:07.999"),
LocalDateTime.of(294247, 1, 10, 4, 0, 54, 775_000_000),
new
Period().plusMonths(16843009).plusDays(16843009).plusMillis(16843009),
bytesOnes)
.baselineValues(3, "UTF8 string3", "MIN_VALUE", -2147483648, -128,
-32768, -2147483648, 0, 0, 0,
-9223372036854775808L, -9223372036854775808L, 0L,
LocalDate.parse("1970-01-01"),
LocalDateTime.ofInstant(Instant.ofEpochMilli(0),
ZoneOffset.UTC).toLocalTime(),
- LocalDateTime.parse("1970-01-01T00:00:00.0"), 0L, new Period("PT0S"),
bytesZeros)
+ LocalDateTime.parse("1970-01-01T00:00:00.0"), LocalDateTime.of(1970,
1, 1, 0, 0, 0), new Period("PT0S"), bytesZeros)
.baselineValues(4, null, null, null, null, null, null, null, null, null,
null, null, null, null,
null, null, null, null, null)
.build().run();
@@ -575,19 +575,19 @@ public class TestParquetLogicalTypes extends
BaseTestQuery {
.baselineValues(1, "UTF8 string1", "RANDOM_VALUE", 1234567, 123, 12345,
1234567, 123, 1234, 1234567,
1234567890123456L, 1234567890123456L, 1234567890123456L,
LocalDate.parse("5350-02-17"),
LocalDateTime.ofInstant(Instant.ofEpochMilli(1234567),
ZoneOffset.UTC).toLocalTime(),
- LocalDateTime.parse("1973-11-29T21:33:09.012"), 123456789012L,
+ LocalDateTime.parse("1973-11-29T21:33:09.012"), LocalDateTime.of(1970,
1, 2, 10, 17, 36, 789_000_000),
new
Period().plusMonths(875770417).plusDays(943142453).plusMillis(1650536505),
bytes12)
.baselineValues(2, "UTF8 string2", "MAX_VALUE", 2147483647, 127, 32767,
2147483647, 255, 65535, -1,
9223372036854775807L, 9223372036854775807L, -1L,
LocalDate.parse("1969-12-31"),
LocalDateTime.ofInstant(Instant.ofEpochMilli(0xFFFFFFFF),
ZoneOffset.UTC).toLocalTime(),
- LocalDateTime.parse("2038-01-19T03:14:07.999"), 9223372036854775807L,
+ LocalDateTime.parse("2038-01-19T03:14:07.999"),
LocalDateTime.of(294247, 1, 10, 4, 0, 54, 775_000_000),
new
Period().plusMonths(16843009).plusDays(16843009).plusMillis(16843009),
bytesOnes)
.baselineValues(3, "UTF8 string3", "MIN_VALUE", -2147483648, -128,
-32768, -2147483648, 0, 0, 0,
-9223372036854775808L, -9223372036854775808L, 0L,
LocalDate.parse("1970-01-01"),
LocalDateTime.ofInstant(Instant.ofEpochMilli(0),
ZoneOffset.UTC).toLocalTime(),
- LocalDateTime.parse("1970-01-01T00:00:00.0"), 0L, new Period("PT0S"),
bytesZeros)
+ LocalDateTime.parse("1970-01-01T00:00:00.0"), LocalDateTime.of(1970,
1, 1, 0, 0, 0), new Period("PT0S"), bytesZeros)
.build()
.run();
}
@@ -650,19 +650,19 @@ public class TestParquetLogicalTypes extends
BaseTestQuery {
.baselineValues(1, "UTF8 string1", "RANDOM_VALUE", 1234567, 123, 12345,
1234567, 123, 1234, 1234567,
1234567890123456L, 1234567890123456L, 1234567890123456L,
LocalDate.parse("5350-02-17"),
LocalDateTime.ofInstant(Instant.ofEpochMilli(1234567),
ZoneOffset.UTC).toLocalTime(),
- LocalDateTime.parse("1973-11-29T21:33:09.012"), 123456789012L,
+ LocalDateTime.parse("1973-11-29T21:33:09.012"), LocalDateTime.of(1970,
1, 2, 10, 17, 36, 789_000_000),
new
Period().plusMonths(875770417).plusDays(943142453).plusMillis(1650536505),
bytes12)
.baselineValues(2, "UTF8 string2", "MAX_VALUE", 2147483647, 127, 32767,
2147483647, 255, 65535, -1,
9223372036854775807L, 9223372036854775807L, -1L,
LocalDate.parse("1969-12-31"),
LocalDateTime.ofInstant(Instant.ofEpochMilli(0xFFFFFFFF),
ZoneOffset.UTC).toLocalTime(),
- LocalDateTime.parse("2038-01-19T03:14:07.999"), 9223372036854775807L,
+ LocalDateTime.parse("2038-01-19T03:14:07.999"),
LocalDateTime.of(294247, 1, 10, 4, 0, 54, 775_000_000),
new
Period().plusMonths(16843009).plusDays(16843009).plusMillis(16843009),
bytesOnes)
.baselineValues(3, "UTF8 string3", "MIN_VALUE", -2147483648, -128,
-32768, -2147483648, 0, 0, 0,
-9223372036854775808L, -9223372036854775808L, 0L,
LocalDate.parse("1970-01-01"),
LocalDateTime.ofInstant(Instant.ofEpochMilli(0),
ZoneOffset.UTC).toLocalTime(),
- LocalDateTime.parse("1970-01-01T00:00:00.0"), 0L, new Period("PT0S"),
bytesZeros)
+ LocalDateTime.parse("1970-01-01T00:00:00.0"), LocalDateTime.of(1970,
1, 1, 0, 0, 0), new Period("PT0S"), bytesZeros)
.baselineValues(4, null, null, null, null, null, null, null, null, null,
null, null, null, null,
null, null, null, null, null)
.build().run();
@@ -726,19 +726,19 @@ public class TestParquetLogicalTypes extends
BaseTestQuery {
.baselineValues(1, "UTF8 string1", "RANDOM_VALUE", 1234567, 123,
12345, 1234567, 123, 1234, 1234567,
1234567890123456L, 1234567890123456L, 1234567890123456L,
LocalDate.parse("5350-02-17"),
LocalDateTime.ofInstant(Instant.ofEpochMilli(1234567),
ZoneOffset.UTC).toLocalTime(),
- LocalDateTime.parse("1973-11-29T21:33:09.012"), 123456789012L,
+ LocalDateTime.parse("1973-11-29T21:33:09.012"),
LocalDateTime.of(1970, 1, 2, 10, 17, 36, 789_000_000),
new
Period().plusMonths(875770417).plusDays(943142453).plusMillis(1650536505),
bytes12)
.baselineValues(2, "UTF8 string2", "MAX_VALUE", 2147483647, 127, 32767,
2147483647, 255, 65535, -1,
9223372036854775807L, 9223372036854775807L, -1L,
LocalDate.parse("1969-12-31"),
LocalDateTime.ofInstant(Instant.ofEpochMilli(0xFFFFFFFF),
ZoneOffset.UTC).toLocalTime(),
- LocalDateTime.parse("2038-01-19T03:14:07.999"),
9223372036854775807L,
+ LocalDateTime.parse("2038-01-19T03:14:07.999"),
LocalDateTime.of(294247, 1, 10, 4, 0, 54, 775_000_000),
new
Period().plusMonths(16843009).plusDays(16843009).plusMillis(16843009),
bytesOnes)
.baselineValues(3, "UTF8 string3", "MIN_VALUE", -2147483648, -128,
-32768, -2147483648, 0, 0, 0,
-9223372036854775808L, -9223372036854775808L, 0L,
LocalDate.parse("1970-01-01"),
LocalDateTime.ofInstant(Instant.ofEpochMilli(0),
ZoneOffset.UTC).toLocalTime(),
- LocalDateTime.parse("1970-01-01T00:00:00.0"), 0L, new
Period("PT0S"), bytesZeros)
+ LocalDateTime.parse("1970-01-01T00:00:00.0"),
LocalDateTime.of(1970, 1, 1, 0, 0, 0), new Period("PT0S"), bytesZeros)
.build()
.run();
}
@@ -801,19 +801,19 @@ public class TestParquetLogicalTypes extends
BaseTestQuery {
.baselineValues(1, "UTF8 string1", "RANDOM_VALUE", 1234567, 123,
12345, 1234567, 123, 1234, 1234567,
1234567890123456L, 1234567890123456L, 1234567890123456L,
LocalDate.parse("5350-02-17"),
LocalDateTime.ofInstant(Instant.ofEpochMilli(1234567),
ZoneOffset.UTC).toLocalTime(),
- LocalDateTime.parse("1973-11-29T21:33:09.012"), 123456789012L,
+ LocalDateTime.parse("1973-11-29T21:33:09.012"),
LocalDateTime.of(1970, 1, 2, 10, 17, 36, 789_000_000),
new
Period().plusMonths(875770417).plusDays(943142453).plusMillis(1650536505),
bytes12)
.baselineValues(2, "UTF8 string2", "MAX_VALUE", 2147483647, 127,
32767, 2147483647, 255, 65535, -1,
9223372036854775807L, 9223372036854775807L, -1L,
LocalDate.parse("1969-12-31"),
LocalDateTime.ofInstant(Instant.ofEpochMilli(0xFFFFFFFF),
ZoneOffset.UTC).toLocalTime(),
- LocalDateTime.parse("2038-01-19T03:14:07.999"),
9223372036854775807L,
+ LocalDateTime.parse("2038-01-19T03:14:07.999"),
LocalDateTime.of(294247, 1, 10, 4, 0, 54, 775_000_000),
new
Period().plusMonths(16843009).plusDays(16843009).plusMillis(16843009),
bytesOnes)
.baselineValues(3, "UTF8 string3", "MIN_VALUE", -2147483648, -128,
-32768, -2147483648, 0, 0, 0,
-9223372036854775808L, -9223372036854775808L, 0L,
LocalDate.parse("1970-01-01"),
LocalDateTime.ofInstant(Instant.ofEpochMilli(0),
ZoneOffset.UTC).toLocalTime(),
- LocalDateTime.parse("1970-01-01T00:00:00.0"), 0L, new
Period("PT0S"), bytesZeros)
+ LocalDateTime.parse("1970-01-01T00:00:00.0"),
LocalDateTime.of(1970, 1, 1, 0, 0, 0), new Period("PT0S"), bytesZeros)
.baselineValues(4, null, null, null, null, null, null, null, null,
null, null, null, null, null,
null, null, null, null, null)
.build().run();
@@ -878,19 +878,19 @@ public class TestParquetLogicalTypes extends
BaseTestQuery {
.baselineValues(1, "UTF8 string1", "RANDOM_VALUE", 1234567, 123,
12345, 1234567, 123, 1234, 1234567,
1234567890123456L, 1234567890123456L, 1234567890123456L,
LocalDate.parse("5350-02-17"),
LocalDateTime.ofInstant(Instant.ofEpochMilli(1234567),
ZoneOffset.UTC).toLocalTime(),
- LocalDateTime.parse("1973-11-29T21:33:09.012"), 123456789012L,
+ LocalDateTime.parse("1973-11-29T21:33:09.012"),
LocalDateTime.of(1970, 1, 2, 10, 17, 36, 789_000_000),
new
Period().plusMonths(875770417).plusDays(943142453).plusMillis(1650536505),
bytes12)
.baselineValues(2, "UTF8 string2", "MAX_VALUE", 2147483647, 127, 32767,
2147483647, 255, 65535, -1,
9223372036854775807L, 9223372036854775807L, -1L,
LocalDate.parse("1969-12-31"),
LocalDateTime.ofInstant(Instant.ofEpochMilli(0xFFFFFFFF),
ZoneOffset.UTC).toLocalTime(),
- LocalDateTime.parse("2038-01-19T03:14:07.999"),
9223372036854775807L,
+ LocalDateTime.parse("2038-01-19T03:14:07.999"),
LocalDateTime.of(294247, 1, 10, 4, 0, 54, 775_000_000),
new
Period().plusMonths(16843009).plusDays(16843009).plusMillis(16843009),
bytesOnes)
.baselineValues(3, "UTF8 string3", "MIN_VALUE", -2147483648, -128,
-32768, -2147483648, 0, 0, 0,
-9223372036854775808L, -9223372036854775808L, 0L,
LocalDate.parse("1970-01-01"),
LocalDateTime.ofInstant(Instant.ofEpochMilli(0),
ZoneOffset.UTC).toLocalTime(),
- LocalDateTime.parse("1970-01-01T00:00:00.0"), 0L, new
Period("PT0S"), bytesZeros)
+ LocalDateTime.parse("1970-01-01T00:00:00.0"),
LocalDateTime.of(1970, 1, 1, 0, 0, 0), new Period("PT0S"), bytesZeros)
.build()
.run();
}
@@ -953,19 +953,19 @@ public class TestParquetLogicalTypes extends
BaseTestQuery {
.baselineValues(1, "UTF8 string1", "RANDOM_VALUE", 1234567, 123,
12345, 1234567, 123, 1234, 1234567,
1234567890123456L, 1234567890123456L, 1234567890123456L,
LocalDate.parse("5350-02-17"),
LocalDateTime.ofInstant(Instant.ofEpochMilli(1234567),
ZoneOffset.UTC).toLocalTime(),
- LocalDateTime.parse("1973-11-29T21:33:09.012"), 123456789012L,
+ LocalDateTime.parse("1973-11-29T21:33:09.012"),
LocalDateTime.of(1970, 1, 2, 10, 17, 36, 789_000_000),
new
Period().plusMonths(875770417).plusDays(943142453).plusMillis(1650536505),
bytes12)
.baselineValues(2, "UTF8 string2", "MAX_VALUE", 2147483647, 127,
32767, 2147483647, 255, 65535, -1,
9223372036854775807L, 9223372036854775807L, -1L,
LocalDate.parse("1969-12-31"),
LocalDateTime.ofInstant(Instant.ofEpochMilli(0xFFFFFFFF),
ZoneOffset.UTC).toLocalTime(),
- LocalDateTime.parse("2038-01-19T03:14:07.999"),
9223372036854775807L,
+ LocalDateTime.parse("2038-01-19T03:14:07.999"),
LocalDateTime.of(294247, 1, 10, 4, 0, 54, 775_000_000),
new
Period().plusMonths(16843009).plusDays(16843009).plusMillis(16843009),
bytesOnes)
.baselineValues(3, "UTF8 string3", "MIN_VALUE", -2147483648, -128,
-32768, -2147483648, 0, 0, 0,
-9223372036854775808L, -9223372036854775808L, 0L,
LocalDate.parse("1970-01-01"),
LocalDateTime.ofInstant(Instant.ofEpochMilli(0),
ZoneOffset.UTC).toLocalTime(),
- LocalDateTime.parse("1970-01-01T00:00:00.0"), 0L, new
Period("PT0S"), bytesZeros)
+ LocalDateTime.parse("1970-01-01T00:00:00.0"),
LocalDateTime.of(1970, 1, 1, 0, 0, 0), new Period("PT0S"), bytesZeros)
.baselineValues(4, null, null, null, null, null, null, null, null,
null, null, null, null, null,
null, null, null, null, null)
.build().run();
diff --git
a/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet2/TestDrillParquetReader.java
b/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet2/TestDrillParquetReader.java
index 0281ad7..b5c1924 100644
---
a/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet2/TestDrillParquetReader.java
+++
b/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet2/TestDrillParquetReader.java
@@ -18,9 +18,11 @@
package org.apache.drill.exec.store.parquet2;
import java.math.BigDecimal;
+import java.nio.charset.StandardCharsets;
import java.time.Instant;
import java.time.LocalDate;
import java.time.LocalDateTime;
+import java.time.LocalTime;
import java.time.ZoneOffset;
import java.time.ZonedDateTime;
import java.util.Arrays;
@@ -30,6 +32,7 @@ import org.apache.drill.categories.ParquetTest;
import org.apache.drill.categories.UnlikelyTest;
import org.apache.drill.exec.ExecConstants;
import org.apache.drill.exec.planner.physical.PlannerSettings;
+import org.apache.drill.exec.proto.UserBitShared;
import org.apache.drill.exec.util.Text;
import org.apache.drill.test.BaseTestQuery;
import org.joda.time.Period;
@@ -41,6 +44,9 @@ import org.junit.experimental.categories.Category;
import static java.util.Arrays.asList;
import static java.util.Collections.emptyList;
import static org.apache.drill.exec.expr.fn.impl.DateUtility.parseLocalDate;
+import static org.apache.drill.test.TestBuilder.listOf;
+import static org.apache.drill.test.TestBuilder.mapOf;
+import static org.apache.drill.test.TestBuilder.mapOfObject;
@Category({ParquetTest.class, UnlikelyTest.class})
public class TestDrillParquetReader extends BaseTestQuery {
@@ -194,19 +200,22 @@ public class TestDrillParquetReader extends BaseTestQuery
{
.baselineValues(1, "UTF8 string1", "RANDOM_VALUE", 1234567, 123,
12345, 1234567, 123, 1234, 1234567,
1234567890123456L, 1234567890123456L, 1234567890123456L,
LocalDate.parse("5350-02-17"),
ZonedDateTime.ofInstant(Instant.ofEpochMilli(1234567),
ZoneOffset.UTC).toLocalTime(),
- LocalDateTime.parse("1973-11-29T21:33:09.012"), 123456789012L,
+ LocalDateTime.parse("1973-11-29T21:33:09.012"),
+ LocalDateTime.of(1970, 1, 2, 10, 17, 36, 789_000_000),
new
Period().plusMonths(875770417).plusDays(943142453).plusMillis(1650536505),
bytes12)
.baselineValues(2, "UTF8 string2", "MAX_VALUE", 2147483647, 127,
32767, 2147483647, 255, 65535, -1,
9223372036854775807L, 9223372036854775807L, -1L,
LocalDate.parse("1969-12-31"),
ZonedDateTime.ofInstant(Instant.ofEpochMilli(0xFFFFFFFF),
ZoneOffset.UTC).toLocalTime(),
- LocalDateTime.parse("2038-01-19T03:14:07.999"),
9223372036854775807L,
+ LocalDateTime.parse("2038-01-19T03:14:07.999"),
+ LocalDateTime.of(294247, 1, 10, 4, 0, 54, 775_000_000),
new
Period().plusMonths(16843009).plusDays(16843009).plusMillis(16843009),
bytesOnes)
.baselineValues(3, "UTF8 string3", "MIN_VALUE", -2147483648, -128,
-32768, -2147483648, 0, 0, 0,
-9223372036854775808L, -9223372036854775808L, 0L,
LocalDate.parse("1970-01-01"),
ZonedDateTime.ofInstant(Instant.ofEpochMilli(0),
ZoneOffset.UTC).toLocalTime(),
- LocalDateTime.parse("1970-01-01T00:00:00.0"), 0L, new
Period("PT0S"), bytesZeros)
+ LocalDateTime.parse("1970-01-01T00:00:00.0"),
+ LocalDateTime.of(1970, 1, 1, 0, 0, 0),new Period("PT0S"),
bytesZeros)
.build()
.run();
}
@@ -271,19 +280,20 @@ public class TestDrillParquetReader extends BaseTestQuery
{
.baselineValues(1, "UTF8 string1", "RANDOM_VALUE", 1234567, 123,
12345, 1234567, 123, 1234, 1234567,
1234567890123456L, 1234567890123456L, 1234567890123456L,
LocalDate.parse("5350-02-17"),
ZonedDateTime.ofInstant(Instant.ofEpochMilli(1234567),
ZoneOffset.UTC).toLocalTime(),
- LocalDateTime.parse("1973-11-29T21:33:09.012"), 123456789012L,
+ LocalDateTime.parse("1973-11-29T21:33:09.012"),
LocalDateTime.of(1970, 1, 2, 10, 17, 36, 789_000_000),
new
Period().plusMonths(875770417).plusDays(943142453).plusMillis(1650536505),
bytes12)
.baselineValues(2, "UTF8 string2", "MAX_VALUE", 2147483647, 127,
32767, 2147483647, 255, 65535, -1,
9223372036854775807L, 9223372036854775807L, -1L,
LocalDate.parse("1969-12-31"),
ZonedDateTime.ofInstant(Instant.ofEpochMilli(0xFFFFFFFF),
ZoneOffset.UTC).toLocalTime(),
- LocalDateTime.parse("2038-01-19T03:14:07.999"),
9223372036854775807L,
+ LocalDateTime.parse("2038-01-19T03:14:07.999"),
LocalDateTime.of(294247, 1, 10, 4, 0, 54, 775_000_000),
new
Period().plusMonths(16843009).plusDays(16843009).plusMillis(16843009),
bytesOnes)
.baselineValues(3, "UTF8 string3", "MIN_VALUE", -2147483648, -128,
-32768, -2147483648, 0, 0, 0,
-9223372036854775808L, -9223372036854775808L, 0L,
LocalDate.parse("1970-01-01"),
ZonedDateTime.ofInstant(Instant.ofEpochMilli(0),
ZoneOffset.UTC).toLocalTime(),
- LocalDateTime.parse("1970-01-01T00:00:00.0"), 0L, new
Period("PT0S"), bytesZeros)
+ LocalDateTime.parse("1970-01-01T00:00:00.0"),
LocalDateTime.of(1970, 1, 1, 0, 0, 0),
+ new Period("PT0S"), bytesZeros)
.baselineValues(4, null, null, null, null, null, null, null, null,
null, null, null, null, null,
null, null, null, null, null)
.build().run();
@@ -349,19 +359,19 @@ public class TestDrillParquetReader extends BaseTestQuery
{
.baselineValues(1, "UTF8 string1", "RANDOM_VALUE", 1234567, 123,
12345, 1234567, 123, 1234, 1234567,
1234567890123456L, 1234567890123456L, 1234567890123456L,
LocalDate.parse("5350-02-17"),
ZonedDateTime.ofInstant(Instant.ofEpochMilli(1234567),
ZoneOffset.UTC).toLocalTime(),
- LocalDateTime.parse("1973-11-29T21:33:09.012"), 123456789012L,
+ LocalDateTime.parse("1973-11-29T21:33:09.012"),
LocalDateTime.of(1970, 1, 2, 10, 17, 36, 789_000_000),
new
Period().plusMonths(875770417).plusDays(943142453).plusMillis(1650536505),
bytes12)
.baselineValues(2, "UTF8 string2", "MAX_VALUE", 2147483647, 127,
32767, 2147483647, 255, 65535, -1,
9223372036854775807L, 9223372036854775807L, -1L,
LocalDate.parse("1969-12-31"),
ZonedDateTime.ofInstant(Instant.ofEpochMilli(0xFFFFFFFF),
ZoneOffset.UTC).toLocalTime(),
- LocalDateTime.parse("2038-01-19T03:14:07.999"),
9223372036854775807L,
+ LocalDateTime.parse("2038-01-19T03:14:07.999"),
LocalDateTime.of(294247, 1, 10, 4, 0, 54, 775_000_000),
new
Period().plusMonths(16843009).plusDays(16843009).plusMillis(16843009),
bytesOnes)
.baselineValues(3, "UTF8 string3", "MIN_VALUE", -2147483648, -128,
-32768, -2147483648, 0, 0, 0,
-9223372036854775808L, -9223372036854775808L, 0L,
LocalDate.parse("1970-01-01"),
ZonedDateTime.ofInstant(Instant.ofEpochMilli(0),
ZoneOffset.UTC).toLocalTime(),
- LocalDateTime.parse("1970-01-01T00:00:00.0"), 0L, new
Period("PT0S"), bytesZeros)
+ LocalDateTime.parse("1970-01-01T00:00:00.0"),
LocalDateTime.of(1970, 1, 1, 0, 0, 0), new Period("PT0S"), bytesZeros)
.build()
.run();
}
@@ -426,19 +436,19 @@ public class TestDrillParquetReader extends BaseTestQuery
{
.baselineValues(1, "UTF8 string1", "RANDOM_VALUE", 1234567, 123,
12345, 1234567, 123, 1234, 1234567,
1234567890123456L, 1234567890123456L, 1234567890123456L,
LocalDate.parse("5350-02-17"),
ZonedDateTime.ofInstant(Instant.ofEpochMilli(1234567),
ZoneOffset.UTC).toLocalTime(),
- LocalDateTime.parse("1973-11-29T21:33:09.012"), 123456789012L,
+ LocalDateTime.parse("1973-11-29T21:33:09.012"),
LocalDateTime.of(1970, 1, 2, 10, 17, 36, 789_000_000),
new
Period().plusMonths(875770417).plusDays(943142453).plusMillis(1650536505),
bytes12)
.baselineValues(2, "UTF8 string2", "MAX_VALUE", 2147483647, 127,
32767, 2147483647, 255, 65535, -1,
9223372036854775807L, 9223372036854775807L, -1L,
LocalDate.parse("1969-12-31"),
ZonedDateTime.ofInstant(Instant.ofEpochMilli(0xFFFFFFFF),
ZoneOffset.UTC).toLocalTime(),
- LocalDateTime.parse("2038-01-19T03:14:07.999"),
9223372036854775807L,
+ LocalDateTime.parse("2038-01-19T03:14:07.999"),
LocalDateTime.of(294247, 1, 10, 4, 0, 54, 775_000_000),
new
Period().plusMonths(16843009).plusDays(16843009).plusMillis(16843009),
bytesOnes)
.baselineValues(3, "UTF8 string3", "MIN_VALUE", -2147483648, -128,
-32768, -2147483648, 0, 0, 0,
-9223372036854775808L, -9223372036854775808L, 0L,
LocalDate.parse("1970-01-01"),
ZonedDateTime.ofInstant(Instant.ofEpochMilli(0),
ZoneOffset.UTC).toLocalTime(),
- LocalDateTime.parse("1970-01-01T00:00:00.0"), 0L, new
Period("PT0S"), bytesZeros)
+ LocalDateTime.parse("1970-01-01T00:00:00.0"),
LocalDateTime.of(1970, 1, 1, 0, 0, 0), new Period("PT0S"), bytesZeros)
.baselineValues(4, null, null, null, null, null, null, null, null,
null, null, null, null, null,
null, null, null, null, null)
.build().run();
@@ -1306,4 +1316,46 @@ public class TestDrillParquetReader extends
BaseTestQuery {
.go();
}
+ @Test
+ public void testTimeMicros() throws Exception {
+ testBuilder()
+ .unOrdered()
+ .sqlQuery("select * from cp.`parquet2/allTypes.parquet`")
+ .baselineColumns("int_field", "long_field", "float_field",
"double_field", "string_field",
+ "boolean_field", "time_field", "timestamp_field", "date_field",
"decimal_field", "uuid_field",
+ "fixed_field", "binary_field", "list_field", "map_field",
"struct_field", "repeated_struct_field",
+ "repeated_list_field", "repeated_map_field")
+ .baselineValues(1, 100L, 0.5F, 1.5D, "abc", true, LocalTime.of(2, 42,
42),
+ LocalDateTime.of(1994, 4, 18, 11, 0, 0), LocalDate.of(1994, 4, 18),
+ new BigDecimal("12.34"), new byte[16], new byte[10],
"hello".getBytes(StandardCharsets.UTF_8),
+ listOf("a", "b", "c"),
+ mapOfObject(
+ new Text("a"), 0.1F,
+ new Text("b"), 0.2F),
+ mapOf(
+ "struct_int_field", 123,
+ "struct_string_field", "abc"),
+ listOf(
+ mapOf(
+ "struct_int_field", 123,
+ "struct_string_field", "abc"),
+ mapOf(
+ "struct_int_field", 123,
+ "struct_string_field", "abc")),
+ listOf(listOf("a", "b", "c"), listOf("a", "b", "c")),
+ listOf(
+ mapOfObject(
+ new Text("a"), 0.1F,
+ new Text("b"), 0.2F),
+ mapOfObject(
+ new Text("a"), 0.1F,
+ new Text("b"), 0.2F))
+ )
+ .baselineValues(null, null, null, null, null, null, null, null, null,
null, null, null, null,
+ listOf(), mapOfObject(), mapOf(), listOf(), listOf(), listOf())
+ .go();
+
+ testRunAndPrint(UserBitShared.QueryType.SQL, "select * from
cp.`parquet2/allTypes.parquet`");
+ }
+
}
diff --git a/exec/java-exec/src/test/resources/parquet2/allTypes.parquet
b/exec/java-exec/src/test/resources/parquet2/allTypes.parquet
new file mode 100644
index 0000000..38c091f
Binary files /dev/null and
b/exec/java-exec/src/test/resources/parquet2/allTypes.parquet differ