This is an automated email from the ASF dual-hosted git repository.

volodymyr pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/drill.git

commit 062927aceb4279e6128c99586ff193090ff7518b
Author: Oleksandr Kalinin <alexk...@gmail.com>
AuthorDate: Thu Aug 9 00:13:06 2018 +0200

    DRILL-6670: Align Parquet TIMESTAMP_MICROS logical type handling with 
earlier versions + minor fixes
    
    closes #1428
---
 .../parquet/AbstractParquetScanBatchCreator.java   |  36 +-
 .../parquet/columnreaders/ColumnReaderFactory.java |   9 +
 .../columnreaders/ParquetToDrillTypeConverter.java |   5 +-
 .../store/parquet2/DrillParquetGroupConverter.java |   4 +-
 .../parquet/ParquetSimpleTestFileGenerator.java    |  39 +-
 .../exec/store/parquet/TestParquetComplex.java     | 177 ------
 .../store/parquet/TestParquetLogicalTypes.java     | 665 +++++++++++++++++++++
 .../store/parquet2/TestDrillParquetReader.java     | 190 +++++-
 .../parquet/parquet_logical_types_simple.parquet   | Bin 3366 -> 4098 bytes
 .../parquet_logical_types_simple_nodict.parquet    | Bin 0 -> 4070 bytes
 .../parquet_logical_types_simple_nullable.parquet  | Bin 3518 -> 4265 bytes
 ...et_logical_types_simple_nullable_nodict.parquet | Bin 0 -> 4237 bytes
 .../complex/parquet_logical_types_complex.parquet  | Bin 3908 -> 4667 bytes
 .../parquet_logical_types_complex_nodict.parquet   | Bin 0 -> 4639 bytes
 .../parquet_logical_types_complex_nullable.parquet | Bin 4081 -> 4863 bytes
 ...t_logical_types_complex_nullable_nodict.parquet | Bin 0 -> 4836 bytes
 16 files changed, 903 insertions(+), 222 deletions(-)

diff --git 
a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/AbstractParquetScanBatchCreator.java
 
b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/AbstractParquetScanBatchCreator.java
index 47f2e18..e0ff85d 100644
--- 
a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/AbstractParquetScanBatchCreator.java
+++ 
b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/AbstractParquetScanBatchCreator.java
@@ -106,11 +106,21 @@ public abstract class AbstractParquetScanBatchCreator {
           ParquetReaderUtility.detectCorruptDates(footer, 
rowGroupScan.getColumns(), autoCorrectCorruptDates);
         logger.debug("Contains corrupt dates: {}", containsCorruptDates);
 
-        if 
(!context.getOptions().getBoolean(ExecConstants.PARQUET_NEW_RECORD_READER)
-            && !ParquetReaderUtility.containsComplexColumn(footer, 
rowGroupScan.getColumns())) {
-          logger.debug("Query {} qualifies for new Parquet reader",
-              
QueryIdHelper.getQueryId(oContext.getFragmentContext().getHandle().getQueryId()));
-          readers.add(new ParquetRecordReader(context,
+        boolean useNewReader = 
context.getOptions().getBoolean(ExecConstants.PARQUET_NEW_RECORD_READER);
+        boolean containsComplexColumn = 
ParquetReaderUtility.containsComplexColumn(footer, rowGroupScan.getColumns());
+        logger.debug("PARQUET_NEW_RECORD_READER is {}. Complex columns {}.", 
useNewReader ? "enabled" : "disabled",
+            containsComplexColumn ? "found." : "not found.");
+        RecordReader reader;
+
+        if (useNewReader || containsComplexColumn) {
+          reader = new DrillParquetReader(context,
+              footer,
+              rowGroup,
+              columnExplorer.getTableColumns(),
+              fs,
+              containsCorruptDates);
+        } else {
+          reader = new ParquetRecordReader(context,
               rowGroup.getPath(),
               rowGroup.getRowGroupIndex(),
               rowGroup.getNumRecordsToRead(),
@@ -118,18 +128,14 @@ public abstract class AbstractParquetScanBatchCreator {
               CodecFactory.createDirectCodecFactory(fs.getConf(), new 
ParquetDirectByteBufferAllocator(oContext.getAllocator()), 0),
               footer,
               rowGroupScan.getColumns(),
-              containsCorruptDates));
-        } else {
-          logger.debug("Query {} doesn't qualify for new reader, using old 
one",
-              
QueryIdHelper.getQueryId(oContext.getFragmentContext().getHandle().getQueryId()));
-          readers.add(new DrillParquetReader(context,
-              footer,
-              rowGroup,
-              columnExplorer.getTableColumns(),
-              fs,
-              containsCorruptDates));
+              containsCorruptDates);
         }
 
+        logger.debug("Query {} uses {}",
+            
QueryIdHelper.getQueryId(oContext.getFragmentContext().getHandle().getQueryId()),
+            reader.getClass().getSimpleName());
+        readers.add(reader);
+
         List<String> partitionValues = 
rowGroupScan.getPartitionValues(rowGroup);
         Map<String, String> implicitValues = 
columnExplorer.populateImplicitColumns(rowGroup.getPath(), partitionValues, 
rowGroupScan.supportsFileImplicitColumns());
         implicitColumns.add(implicitValues);
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 798d3c2..03d5382 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
@@ -77,6 +77,10 @@ public class ColumnReaderFactory {
       } else if 
(!columnChunkMetaData.getEncodings().contains(Encoding.PLAIN_DICTIONARY) && (
           columnChunkMetaData.getType() == 
PrimitiveType.PrimitiveTypeName.FIXED_LEN_BYTE_ARRAY
               || columnChunkMetaData.getType() == 
PrimitiveType.PrimitiveTypeName.INT96)) {
+        if (convertedType == null) {
+          return new FixedByteAlignedReader.FixedBinaryReader(recordReader, 
descriptor,
+              columnChunkMetaData, (VariableWidthVector) v, schemaElement);
+        }
         switch (convertedType) {
           case DECIMAL:
             return new FixedByteAlignedReader.VarDecimalReader(recordReader, 
descriptor,
@@ -131,7 +135,9 @@ public class ColumnReaderFactory {
                 return new 
ParquetFixedWidthDictionaryReaders.DictionaryBigIntReader(recordReader, 
descriptor, 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);
                 case UINT_64:
                   return new 
ParquetFixedWidthDictionaryReaders.DictionaryUInt8Reader(recordReader, 
descriptor, columnChunkMetaData, fixedLength, (UInt8Vector) v, schemaElement);
@@ -291,6 +297,9 @@ 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 TIMESTAMP_MICROS:
+              return new 
NullableFixedByteAlignedReaders.NullableDictionaryBigIntReader(parentReader, 
columnDescriptor, columnChunkMetaData, fixedLength, 
(NullableBigIntVector)valueVec, schemaElement);
             default:
               throw new ExecutionSetupException("Unsupported nullable 
converted type " + convertedType + " for primitive type INT64");
           }
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 cb688fd..3a8c254 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,16 +54,15 @@ 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;
           case DECIMAL:
             ParquetReaderUtility.checkDecimalTypeEnabled(options);
             return TypeProtos.MinorType.VARDECIMAL;
-          // TODO - add this back if it is decided to be added upstream, was 
removed form our pull request July 2014
-//              case TIME_MICROS:
-//                throw new UnsupportedOperationException();
           case TIMESTAMP_MILLIS:
             return TypeProtos.MinorType.TIMESTAMP;
           default:
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 59302d0..d1acd80 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
@@ -219,8 +219,10 @@ public class DrillParquetGroupConverter extends 
GroupConverter {
           return new DrillBigIntConverter(writer);
         }
         switch(type.getOriginalType()) {
+          // DRILL-6670: handle TIMESTAMP_MICROS as INT64 with no logical type
           case UINT_64:
-          case INT_64 : {
+          case INT_64 :
+          case TIMESTAMP_MICROS: {
             BigIntWriter writer = type.getRepetition() == Repetition.REPEATED 
? mapWriter.list(name).bigInt() : mapWriter.bigInt(name);
             return new DrillBigIntConverter(writer);
           }
diff --git 
a/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/ParquetSimpleTestFileGenerator.java
 
b/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/ParquetSimpleTestFileGenerator.java
index 2e6923a..e399ee7 100644
--- 
a/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/ParquetSimpleTestFileGenerator.java
+++ 
b/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/ParquetSimpleTestFileGenerator.java
@@ -84,7 +84,7 @@ public class ParquetSimpleTestFileGenerator {
           "  required int32 _TIME_MILLIS_int32  ( TIME_MILLIS ) ; \n" +
           //  "      required int64 _TIME_MICROS_int64  ( TIME_MICROS ) ; \n" +
           "  required int64 _TIMESTAMP_MILLIS_int64  ( TIMESTAMP_MILLIS ) ; 
\n" +
-          //  "      required int64 _TIMESTAMP_MICROS_int64  ( 
TIMESTAMP_MICROS ) ; \n" +
+          "  required int64 _TIMESTAMP_MICROS_int64  ( TIMESTAMP_MICROS ) ; 
\n" +
           "  required fixed_len_byte_array(12) 
_INTERVAL_fixed_len_byte_array_12  ( INTERVAL ) ; \n" +
           "  required int96  _INT96_RAW  ; \n" +
           "} \n";
@@ -112,7 +112,7 @@ public class ParquetSimpleTestFileGenerator {
           "  optional int32 _TIME_MILLIS_int32  ( TIME_MILLIS ) ; \n" +
           //  "      optional int64 _TIME_MICROS_int64  ( TIME_MICROS ) ; \n" +
           "  optional int64 _TIMESTAMP_MILLIS_int64  ( TIMESTAMP_MILLIS ) ; 
\n" +
-          //  "      optional int64 _TIMESTAMP_MICROS_int64  ( 
TIMESTAMP_MICROS ) ; \n" +
+          "  optional int64 _TIMESTAMP_MICROS_int64  ( TIMESTAMP_MICROS ) ; 
\n" +
           "  optional fixed_len_byte_array(12) 
_INTERVAL_fixed_len_byte_array_12  ( INTERVAL ) ; \n" +
           "  optional int96  _INT96_RAW  ; \n" +
           "} \n";
@@ -153,7 +153,7 @@ public class ParquetSimpleTestFileGenerator {
           "      required int32 _TIME_MILLIS_int32  ( TIME_MILLIS ) ; \n" +
           //      "      required int64 _TIME_MICROS_int64  ( TIME_MICROS ) ; 
\n" +
           "      required int64 _TIMESTAMP_MILLIS_int64  ( TIMESTAMP_MILLIS ) 
; \n" +
-          //      "      required int64 _TIMESTAMP_MICROS_int64  ( 
TIMESTAMP_MICROS ) ; \n" +
+          "      required int64 _TIMESTAMP_MICROS_int64  ( TIMESTAMP_MICROS ) 
; \n" +
           "      required fixed_len_byte_array(12) 
_INTERVAL_fixed_len_byte_array_12  ( INTERVAL ) ; \n" +
           "    } \n" +
           "    required group Int96 { \n" +
@@ -197,7 +197,7 @@ public class ParquetSimpleTestFileGenerator {
           "      optional int32 _TIME_MILLIS_int32  ( TIME_MILLIS ) ; \n" +
           //      "      optional int64 _TIME_MICROS_int64  ( TIME_MICROS ) ; 
\n" +
           "      optional int64 _TIMESTAMP_MILLIS_int64  ( TIMESTAMP_MILLIS ) 
; \n" +
-          //      "      optional int64 _TIMESTAMP_MICROS_int64  ( 
TIMESTAMP_MICROS ) ; \n" +
+          "      optional int64 _TIMESTAMP_MICROS_int64  ( TIMESTAMP_MICROS ) 
; \n" +
           "      optional fixed_len_byte_array(12) 
_INTERVAL_fixed_len_byte_array_12  ( INTERVAL ) ; \n" +
           "    } \n" +
           "    optional group Int96 { \n" +
@@ -216,7 +216,7 @@ public class ParquetSimpleTestFileGenerator {
     return new Path(root, fileName);
   }
 
-  public static ParquetWriter<Group> initWriter(MessageType schema, String 
fileName) throws IOException {
+  public static ParquetWriter<Group> initWriter(MessageType schema, String 
fileName, boolean dictEncoding) throws IOException {
 
     GroupWriteSupport.setSchema(schema, conf);
 
@@ -228,7 +228,7 @@ public class ParquetSimpleTestFileGenerator {
             1024,
             1024,
             512,
-            true, // enable dictionary encoding,
+            dictEncoding, // enable dictionary encoding,
             false,
             ParquetProperties.WriterVersion.PARQUET_1_0, conf
         );
@@ -267,6 +267,7 @@ public class ParquetSimpleTestFileGenerator {
           .append("_DATE_int32", 1234567)
           .append("_TIME_MILLIS_int32", 1234567)
           .append("_TIMESTAMP_MILLIS_int64", 123456789012L)
+          .append("_TIMESTAMP_MICROS_int64", 123456789012L)
           .append("_INTERVAL_fixed_len_byte_array_12", 
Binary.fromConstantByteArray(bytes12, 0, 12));
       numeric.addGroup("Int96").append("_INT96_RAW", 
Binary.fromConstantByteArray(bytes12, 0, 12));
       complexWriter.write(complexGroup);
@@ -300,6 +301,7 @@ public class ParquetSimpleTestFileGenerator {
           .append("_DATE_int32", 0xFFFFFFFF)
           .append("_TIME_MILLIS_int32", 0xFFFFFFFF)
           .append("_TIMESTAMP_MILLIS_int64", 0x1F3FFFFFFFFL)
+          .append("_TIMESTAMP_MICROS_int64", 0x7FFFFFFFFFFFFFFFL)
           .append("_INTERVAL_fixed_len_byte_array_12", 
Binary.fromConstantByteArray(bytes, 0, 12));
       numeric.addGroup("Int96").append("_INT96_RAW", 
Binary.fromConstantByteArray(bytes, 0, 12));
       complexWriter.write(complexGroup);
@@ -331,6 +333,7 @@ public class ParquetSimpleTestFileGenerator {
           .append("_DATE_int32", 0x0)
           .append("_TIME_MILLIS_int32", 0x0)
           .append("_TIMESTAMP_MILLIS_int64", 0x0L)
+          .append("_TIMESTAMP_MICROS_int64", 0x0L)
           .append("_INTERVAL_fixed_len_byte_array_12", 
Binary.fromConstantByteArray( new byte[12], 0, 12));
       numeric.addGroup("Int96").append("_INT96_RAW", 
Binary.fromConstantByteArray( new byte[12], 0, 12));
       complexWriter.write(complexGroup);
@@ -370,6 +373,7 @@ public class ParquetSimpleTestFileGenerator {
           .append("_DATE_int32", 1234567)
           .append("_TIME_MILLIS_int32", 1234567)
           .append("_TIMESTAMP_MILLIS_int64", 123456789012L)
+          .append("_TIMESTAMP_MICROS_int64", 123456789012L)
           .append("_INTERVAL_fixed_len_byte_array_12", 
Binary.fromConstantByteArray(bytes12, 0, 12))
           .append("_INT96_RAW", Binary.fromConstantByteArray(bytes12, 0, 12));
       simpleWriter.write(simpleGroup);
@@ -399,6 +403,7 @@ public class ParquetSimpleTestFileGenerator {
           .append("_DATE_int32", 0xFFFFFFFF)
           .append("_TIME_MILLIS_int32", 0xFFFFFFFF)
           .append("_TIMESTAMP_MILLIS_int64", 0x1F3FFFFFFFFL)
+          .append("_TIMESTAMP_MICROS_int64", 0x7FFFFFFFFFFFFFFFL)
           .append("_INTERVAL_fixed_len_byte_array_12", 
Binary.fromConstantByteArray(bytes, 0, 12))
           .append("_INT96_RAW", Binary.fromConstantByteArray(bytes, 0, 12));
       simpleWriter.write(simpleGroup);
@@ -425,6 +430,7 @@ public class ParquetSimpleTestFileGenerator {
           .append("_DATE_int32", 0x0)
           .append("_TIME_MILLIS_int32", 0x0)
           .append("_TIMESTAMP_MILLIS_int64", 0x0L)
+          .append("_TIMESTAMP_MICROS_int64", 0x0L)
           .append("_INTERVAL_fixed_len_byte_array_12", 
Binary.fromConstantByteArray( new byte[12], 0, 12))
           .append("_INT96_RAW", Binary.fromConstantByteArray( new byte[12], 0, 
12));
       simpleWriter.write(simpleGroup);
@@ -443,20 +449,33 @@ public class ParquetSimpleTestFileGenerator {
     SimpleGroupFactory sngf = new SimpleGroupFactory(simpleNullableSchema);
     GroupFactory ngf = new SimpleGroupFactory(complexNullableSchema);
 
-    ParquetWriter<Group> simpleWriter = initWriter(simpleSchema, 
"drill/parquet_test_file_simple");
-    ParquetWriter<Group> complexWriter = initWriter(complexSchema, 
"drill/parquet_test_file_complex");
-    ParquetWriter<Group> simpleNullableWriter = 
initWriter(simpleNullableSchema, "drill/parquet_test_file_simple_nullable");
-    ParquetWriter<Group> complexNullableWriter = 
initWriter(complexNullableSchema, "drill/parquet_test_file_complex_nullable");
+    // Generate files with dictionary encoding enabled and disabled
+    ParquetWriter<Group> simpleWriter = initWriter(simpleSchema, 
"drill/parquet_test_file_simple", true);
+    ParquetWriter<Group> complexWriter = initWriter(complexSchema, 
"drill/parquet_test_file_complex", true);
+    ParquetWriter<Group> simpleNullableWriter = 
initWriter(simpleNullableSchema, "drill/parquet_test_file_simple_nullable", 
true);
+    ParquetWriter<Group> complexNullableWriter = 
initWriter(complexNullableSchema, "drill/parquet_test_file_complex_nullable", 
true);
+    ParquetWriter<Group> simpleNoDictWriter = initWriter(simpleSchema, 
"drill/parquet_test_file_simple_nodict", false);
+    ParquetWriter<Group> complexNoDictWriter = initWriter(complexSchema, 
"drill/parquet_test_file_complex_nodict", false);
+    ParquetWriter<Group> simpleNullableNoDictWriter = 
initWriter(simpleNullableSchema, 
"drill/parquet_test_file_simple_nullable_nodict", false);
+    ParquetWriter<Group> complexNullableNoDictWriter = 
initWriter(complexNullableSchema, 
"drill/parquet_test_file_complex_nullable_nodict", false);
 
     ParquetSimpleTestFileGenerator.writeSimpleValues(sgf, simpleWriter, false);
     ParquetSimpleTestFileGenerator.writeSimpleValues(sngf, 
simpleNullableWriter, true);
     ParquetSimpleTestFileGenerator.writeComplexValues(gf, complexWriter, 
false);
     ParquetSimpleTestFileGenerator.writeComplexValues(ngf, 
complexNullableWriter, true);
+    ParquetSimpleTestFileGenerator.writeSimpleValues(sgf, simpleNoDictWriter, 
false);
+    ParquetSimpleTestFileGenerator.writeSimpleValues(sngf, 
simpleNullableNoDictWriter, true);
+    ParquetSimpleTestFileGenerator.writeComplexValues(gf, complexNoDictWriter, 
false);
+    ParquetSimpleTestFileGenerator.writeComplexValues(ngf, 
complexNullableNoDictWriter, true);
 
     simpleWriter.close();
     complexWriter.close();
     simpleNullableWriter.close();
     complexNullableWriter.close();
+    simpleNoDictWriter.close();
+    complexNoDictWriter.close();
+    simpleNullableNoDictWriter.close();
+    complexNullableNoDictWriter.close();
 
   }
 
diff --git 
a/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/TestParquetComplex.java
 
b/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/TestParquetComplex.java
index c309cf3..32ffb68 100644
--- 
a/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/TestParquetComplex.java
+++ 
b/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/TestParquetComplex.java
@@ -17,18 +17,10 @@
  */
 package org.apache.drill.exec.store.parquet;
 
-import static org.apache.drill.test.TestBuilder.mapOf;
-
 import java.math.BigDecimal;
-import java.time.Instant;
-import java.time.LocalDate;
-import java.time.LocalDateTime;
-import java.time.ZoneOffset;
-import java.util.Arrays;
 
 import org.apache.drill.exec.util.JsonStringArrayList;
 import org.apache.drill.test.BaseTestQuery;
-import org.joda.time.Period;
 import org.junit.Test;
 
 public class TestParquetComplex extends BaseTestQuery {
@@ -204,175 +196,6 @@ public class TestParquetComplex extends BaseTestQuery {
         .run();
   }
 
-  @Test //DRILL-5971
-  public void testComplexLogicalIntTypes() throws Exception {
-    String query = "select t.complextype as complextype,  " +
-            "t.uint_64 as uint_64, t.uint_32 as uint_32, t.uint_16 as uint_16, 
t.uint_8 as uint_8,  " +
-            "t.int_64 as int_64, t.int_32 as int_32, t.int_16 as int_16, 
t.int_8 as int_8  " +
-            "from cp.`store/parquet/complex/logical_int_complex.parquet` t";
-    String[] columns = {"complextype", "uint_64", "uint_32", "uint_16", 
"uint_8", "int_64", "int_32", "int_16", "int_8" };
-    testBuilder()
-        .sqlQuery(query)
-        .unOrdered()
-        .baselineColumns(columns)
-        .baselineValues(mapOf("a","a","b","b"), 0L, 0, 0, 0, 0L, 0, 0, 0)
-        .baselineValues(mapOf("a","a","b","b"), -1L, -1, -1, -1, -1L, -1, -1, 
-1)
-        .baselineValues(mapOf("a","a","b","b"), 1L, 1, 1, 1, 
-9223372036854775808L, 1, 1, 1)
-        .baselineValues(mapOf("a","a","b","b"), 9223372036854775807L, 
2147483647, 65535, 255, 9223372036854775807L, -2147483648, -32768, -128)
-        .build()
-        .run();
-  }
-
-  @Test //DRILL-5971
-  public void testComplexLogicalIntTypes2() throws Exception {
-    byte[] bytes12 = {'1', '2', '3', '4', '5', '6', '7', '8', '9', '0', 'a', 
'b' };
-    byte[] bytesOnes = new byte[12];
-    byte[] bytesZeros = new byte[12];
-    Arrays.fill(bytesOnes, (byte) 1);
-    String query =
-        " select " +
-        " t.rowKey as rowKey, " +
-        " t.StringTypes._UTF8 as _UTF8, " +
-        " t.StringTypes._Enum as _Enum, " +
-        " t.NumericTypes.Int32._INT32_RAW as _INT32_RAW, " +
-        " t.NumericTypes.Int32._INT_8 as _INT_8, " +
-        " t.NumericTypes.Int32._INT_16 as _INT_16, " +
-        " t.NumericTypes.Int32._INT_32 as _INT_32, " +
-        " t.NumericTypes.Int32._UINT_8 as _UINT_8, " +
-        " t.NumericTypes.Int32._UINT_16 as _UINT_16, " +
-        " t.NumericTypes.Int32._UINT_32 as _UINT_32, " +
-        " t.NumericTypes.Int64._INT64_RAW as _INT64_RAW, " +
-        " t.NumericTypes.Int64._INT_64 as _INT_64, " +
-        " t.NumericTypes.Int64._UINT_64 as _UINT_64, " +
-        " t.NumericTypes.DateTimeTypes._DATE_int32 as _DATE_int32, " +
-        " t.NumericTypes.DateTimeTypes._TIME_MILLIS_int32 as 
_TIME_MILLIS_int32, " +
-        " t.NumericTypes.DateTimeTypes._TIMESTAMP_MILLIS_int64 as 
_TIMESTAMP_MILLIS_int64, " +
-        " t.NumericTypes.DateTimeTypes._INTERVAL_fixed_len_byte_array_12 as 
_INTERVAL_fixed_len_byte_array_12, " +
-        " t.NumericTypes.Int96._INT96_RAW as _INT96_RAW " +
-        " from " +
-        " cp.`store/parquet/complex/parquet_logical_types_complex.parquet` t " 
+
-        " order by t.rowKey ";
-    String[] columns = {
-        "rowKey ",
-        "_UTF8",
-        "_Enum",
-        "_INT32_RAW",
-        "_INT_8",
-        "_INT_16",
-        "_INT_32",
-        "_UINT_8",
-        "_UINT_16",
-        "_UINT_32",
-        "_INT64_RAW",
-        "_INT_64",
-        "_UINT_64",
-        "_DATE_int32",
-        "_TIME_MILLIS_int32",
-        "_TIMESTAMP_MILLIS_int64",
-        "_INTERVAL_fixed_len_byte_array_12",
-        "_INT96_RAW"
-
-    };
-    testBuilder()
-        .sqlQuery(query)
-        .ordered()
-        .baselineColumns(columns)
-        .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"),
-            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"),
-            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"), new Period("PT0S"), 
bytesZeros)
-        .build()
-        .run();
-  }
-
-  @Test //DRILL-5971
-  public void testComplexLogicalIntTypes3() throws Exception {
-    byte[] bytes12 = {'1', '2', '3', '4', '5', '6', '7', '8', '9', '0', 'a', 
'b' };
-    byte[] bytesOnes = new byte[12];
-    byte[] bytesZeros = new byte[12];
-    Arrays.fill(bytesOnes, (byte) 1);
-    String query =
-        " select " +
-            " t.rowKey as rowKey, " +
-            " t.StringTypes._UTF8 as _UTF8, " +
-            " t.StringTypes._Enum as _Enum, " +
-            " t.NumericTypes.Int32._INT32_RAW as _INT32_RAW, " +
-            " t.NumericTypes.Int32._INT_8 as _INT_8, " +
-            " t.NumericTypes.Int32._INT_16 as _INT_16, " +
-            " t.NumericTypes.Int32._INT_32 as _INT_32, " +
-            " t.NumericTypes.Int32._UINT_8 as _UINT_8, " +
-            " t.NumericTypes.Int32._UINT_16 as _UINT_16, " +
-            " t.NumericTypes.Int32._UINT_32 as _UINT_32, " +
-            " t.NumericTypes.Int64._INT64_RAW as _INT64_RAW, " +
-            " t.NumericTypes.Int64._INT_64 as _INT_64, " +
-            " t.NumericTypes.Int64._UINT_64 as _UINT_64, " +
-            " t.NumericTypes.DateTimeTypes._DATE_int32 as _DATE_int32, " +
-            " t.NumericTypes.DateTimeTypes._TIME_MILLIS_int32 as 
_TIME_MILLIS_int32, " +
-            " t.NumericTypes.DateTimeTypes._TIMESTAMP_MILLIS_int64 as 
_TIMESTAMP_MILLIS_int64, " +
-            " t.NumericTypes.DateTimeTypes._INTERVAL_fixed_len_byte_array_12 
as _INTERVAL_fixed_len_byte_array_12, " +
-            " t.NumericTypes.Int96._INT96_RAW as _INT96_RAW " +
-            " from " +
-            " 
cp.`store/parquet/complex/parquet_logical_types_complex_nullable.parquet` t " +
-            " order by t.rowKey ";
-    String[] columns = {
-        "rowKey ",
-        "_UTF8",
-        "_Enum",
-        "_INT32_RAW",
-        "_INT_8",
-        "_INT_16",
-        "_INT_32",
-        "_UINT_8",
-        "_UINT_16",
-        "_UINT_32",
-        "_INT64_RAW",
-        "_INT_64",
-        "_UINT_64",
-        "_DATE_int32",
-        "_TIME_MILLIS_int32",
-        "_TIMESTAMP_MILLIS_int64",
-        "_INTERVAL_fixed_len_byte_array_12",
-        "_INT96_RAW"
-
-    };
-    testBuilder()
-        .sqlQuery(query)
-        .ordered()
-        .baselineColumns(columns)
-        .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"),
-            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"),
-            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"), new Period("PT0S"), 
bytesZeros)
-        .baselineValues(4, null, null, null, null, null, null, null, null, 
null, null, null, null, null,
-            null, null, null, null)
-        .build().run();
-  }
-
-  @Test
   public void testReadRepeatedDecimals() throws Exception {
 
     JsonStringArrayList<BigDecimal> ints = new JsonStringArrayList<>();
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
new file mode 100644
index 0000000..74cec6e
--- /dev/null
+++ 
b/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/TestParquetLogicalTypes.java
@@ -0,0 +1,665 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.exec.store.parquet;
+
+import static org.apache.drill.test.TestBuilder.mapOf;
+
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.ZoneOffset;
+import java.util.Arrays;
+
+import org.apache.drill.test.BaseTestQuery;
+import org.joda.time.Period;
+import org.junit.Test;
+
+/**
+ * Test Parquet logical type handling with default Parquet reader.
+ * Tests are executed on data files with dictionary encoding enabled and 
disabled.
+ * parquet_logical_types* files are generated with 
ParquetSimpleTestFileGenerator class.
+ */
+public class TestParquetLogicalTypes extends BaseTestQuery {
+
+  @Test //DRILL-5971
+  public void testComplexLogicalIntTypes() throws Exception {
+    String query = "select t.complextype as complextype,  " +
+            "t.uint_64 as uint_64, t.uint_32 as uint_32, t.uint_16 as uint_16, 
t.uint_8 as uint_8,  " +
+            "t.int_64 as int_64, t.int_32 as int_32, t.int_16 as int_16, 
t.int_8 as int_8  " +
+            "from cp.`store/parquet/complex/logical_int_complex.parquet` t";
+    String[] columns = {"complextype", "uint_64", "uint_32", "uint_16", 
"uint_8", "int_64", "int_32", "int_16", "int_8" };
+    testBuilder()
+        .sqlQuery(query)
+        .unOrdered()
+        .baselineColumns(columns)
+        .baselineValues(mapOf("a","a","b","b"), 0L, 0, 0, 0, 0L, 0, 0, 0)
+        .baselineValues(mapOf("a","a","b","b"), -1L, -1, -1, -1, -1L, -1, -1, 
-1)
+        .baselineValues(mapOf("a","a","b","b"), 1L, 1, 1, 1, 
-9223372036854775808L, 1, 1, 1)
+        .baselineValues(mapOf("a","a","b","b"), 9223372036854775807L, 
2147483647, 65535, 255, 9223372036854775807L, -2147483648, -32768, -128)
+        .build()
+        .run();
+  }
+
+  @Test //DRILL-5971
+  public void testComplexLogicalIntTypes2() throws Exception {
+    byte[] bytes12 = {'1', '2', '3', '4', '5', '6', '7', '8', '9', '0', 'a', 
'b' };
+    byte[] bytesOnes = new byte[12];
+    byte[] bytesZeros = new byte[12];
+    Arrays.fill(bytesOnes, (byte) 1);
+    String query =
+        " select " +
+        " t.rowKey as rowKey, " +
+        " t.StringTypes._UTF8 as _UTF8, " +
+        " t.StringTypes._Enum as _Enum, " +
+        " t.NumericTypes.Int32._INT32_RAW as _INT32_RAW, " +
+        " t.NumericTypes.Int32._INT_8 as _INT_8, " +
+        " t.NumericTypes.Int32._INT_16 as _INT_16, " +
+        " t.NumericTypes.Int32._INT_32 as _INT_32, " +
+        " t.NumericTypes.Int32._UINT_8 as _UINT_8, " +
+        " t.NumericTypes.Int32._UINT_16 as _UINT_16, " +
+        " t.NumericTypes.Int32._UINT_32 as _UINT_32, " +
+        " t.NumericTypes.Int64._INT64_RAW as _INT64_RAW, " +
+        " t.NumericTypes.Int64._INT_64 as _INT_64, " +
+        " t.NumericTypes.Int64._UINT_64 as _UINT_64, " +
+        " t.NumericTypes.DateTimeTypes._DATE_int32 as _DATE_int32, " +
+        " t.NumericTypes.DateTimeTypes._TIME_MILLIS_int32 as 
_TIME_MILLIS_int32, " +
+        " t.NumericTypes.DateTimeTypes._TIMESTAMP_MILLIS_int64 as 
_TIMESTAMP_MILLIS_int64, " +
+        " t.NumericTypes.DateTimeTypes._TIMESTAMP_MICROS_int64 as 
_TIMESTAMP_MICROS_int64, " +
+        " t.NumericTypes.DateTimeTypes._INTERVAL_fixed_len_byte_array_12 as 
_INTERVAL_fixed_len_byte_array_12, " +
+        " t.NumericTypes.Int96._INT96_RAW as _INT96_RAW " +
+        " from " +
+        " cp.`store/parquet/complex/parquet_logical_types_complex.parquet` t " 
+
+        " order by t.rowKey ";
+    String[] columns = {
+        "rowKey ",
+        "_UTF8",
+        "_Enum",
+        "_INT32_RAW",
+        "_INT_8",
+        "_INT_16",
+        "_INT_32",
+        "_UINT_8",
+        "_UINT_16",
+        "_UINT_32",
+        "_INT64_RAW",
+        "_INT_64",
+        "_UINT_64",
+        "_DATE_int32",
+        "_TIME_MILLIS_int32",
+        "_TIMESTAMP_MILLIS_int64",
+        "_TIMESTAMP_MICROS_int64",
+        "_INTERVAL_fixed_len_byte_array_12",
+        "_INT96_RAW"
+
+    };
+    testBuilder()
+        .sqlQuery(query)
+        .ordered()
+        .baselineColumns(columns)
+        .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,
+            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,
+            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)
+        .build()
+        .run();
+  }
+
+  @Test //DRILL-5971
+  public void testComplexLogicalIntTypes3() throws Exception {
+    byte[] bytes12 = {'1', '2', '3', '4', '5', '6', '7', '8', '9', '0', 'a', 
'b' };
+    byte[] bytesOnes = new byte[12];
+    byte[] bytesZeros = new byte[12];
+    Arrays.fill(bytesOnes, (byte) 1);
+    String query =
+        " select " +
+            " t.rowKey as rowKey, " +
+            " t.StringTypes._UTF8 as _UTF8, " +
+            " t.StringTypes._Enum as _Enum, " +
+            " t.NumericTypes.Int32._INT32_RAW as _INT32_RAW, " +
+            " t.NumericTypes.Int32._INT_8 as _INT_8, " +
+            " t.NumericTypes.Int32._INT_16 as _INT_16, " +
+            " t.NumericTypes.Int32._INT_32 as _INT_32, " +
+            " t.NumericTypes.Int32._UINT_8 as _UINT_8, " +
+            " t.NumericTypes.Int32._UINT_16 as _UINT_16, " +
+            " t.NumericTypes.Int32._UINT_32 as _UINT_32, " +
+            " t.NumericTypes.Int64._INT64_RAW as _INT64_RAW, " +
+            " t.NumericTypes.Int64._INT_64 as _INT_64, " +
+            " t.NumericTypes.Int64._UINT_64 as _UINT_64, " +
+            " t.NumericTypes.DateTimeTypes._DATE_int32 as _DATE_int32, " +
+            " t.NumericTypes.DateTimeTypes._TIME_MILLIS_int32 as 
_TIME_MILLIS_int32, " +
+            " t.NumericTypes.DateTimeTypes._TIMESTAMP_MILLIS_int64 as 
_TIMESTAMP_MILLIS_int64, " +
+            " t.NumericTypes.DateTimeTypes._TIMESTAMP_MICROS_int64 as 
_TIMESTAMP_MICROS_int64, " +
+            " t.NumericTypes.DateTimeTypes._INTERVAL_fixed_len_byte_array_12 
as _INTERVAL_fixed_len_byte_array_12, " +
+            " t.NumericTypes.Int96._INT96_RAW as _INT96_RAW " +
+            " from " +
+            " 
cp.`store/parquet/complex/parquet_logical_types_complex_nullable.parquet` t " +
+            " order by t.rowKey ";
+    String[] columns = {
+        "rowKey ",
+        "_UTF8",
+        "_Enum",
+        "_INT32_RAW",
+        "_INT_8",
+        "_INT_16",
+        "_INT_32",
+        "_UINT_8",
+        "_UINT_16",
+        "_UINT_32",
+        "_INT64_RAW",
+        "_INT_64",
+        "_UINT_64",
+        "_DATE_int32",
+        "_TIME_MILLIS_int32",
+        "_TIMESTAMP_MILLIS_int64",
+        "_TIMESTAMP_MICROS_int64",
+        "_INTERVAL_fixed_len_byte_array_12",
+        "_INT96_RAW"
+    };
+    testBuilder()
+        .sqlQuery(query)
+        .ordered()
+        .baselineColumns(columns)
+        .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,
+            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,
+            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)
+        .baselineValues(4, null, null, null, null, null, null, null, null, 
null, null, null, null, null,
+            null, null, null, null, null)
+        .build().run();
+  }
+
+  @Test //DRILL-6670: include tests on data with dictionary encoding disabled
+  public void testComplexLogicalIntTypes4() throws Exception {
+    byte[] bytes12 = {'1', '2', '3', '4', '5', '6', '7', '8', '9', '0', 'a', 
'b' };
+    byte[] bytesOnes = new byte[12];
+    byte[] bytesZeros = new byte[12];
+    Arrays.fill(bytesOnes, (byte) 1);
+    String query =
+        " select " +
+        " t.rowKey as rowKey, " +
+        " t.StringTypes._UTF8 as _UTF8, " +
+        " t.StringTypes._Enum as _Enum, " +
+        " t.NumericTypes.Int32._INT32_RAW as _INT32_RAW, " +
+        " t.NumericTypes.Int32._INT_8 as _INT_8, " +
+        " t.NumericTypes.Int32._INT_16 as _INT_16, " +
+        " t.NumericTypes.Int32._INT_32 as _INT_32, " +
+        " t.NumericTypes.Int32._UINT_8 as _UINT_8, " +
+        " t.NumericTypes.Int32._UINT_16 as _UINT_16, " +
+        " t.NumericTypes.Int32._UINT_32 as _UINT_32, " +
+        " t.NumericTypes.Int64._INT64_RAW as _INT64_RAW, " +
+        " t.NumericTypes.Int64._INT_64 as _INT_64, " +
+        " t.NumericTypes.Int64._UINT_64 as _UINT_64, " +
+        " t.NumericTypes.DateTimeTypes._DATE_int32 as _DATE_int32, " +
+        " t.NumericTypes.DateTimeTypes._TIME_MILLIS_int32 as 
_TIME_MILLIS_int32, " +
+        " t.NumericTypes.DateTimeTypes._TIMESTAMP_MILLIS_int64 as 
_TIMESTAMP_MILLIS_int64, " +
+        " t.NumericTypes.DateTimeTypes._TIMESTAMP_MICROS_int64 as 
_TIMESTAMP_MICROS_int64, " +
+        " t.NumericTypes.DateTimeTypes._INTERVAL_fixed_len_byte_array_12 as 
_INTERVAL_fixed_len_byte_array_12, " +
+        " t.NumericTypes.Int96._INT96_RAW as _INT96_RAW " +
+        " from " +
+        " 
cp.`store/parquet/complex/parquet_logical_types_complex_nodict.parquet` t " +
+        " order by t.rowKey ";
+    String[] columns = {
+        "rowKey ",
+        "_UTF8",
+        "_Enum",
+        "_INT32_RAW",
+        "_INT_8",
+        "_INT_16",
+        "_INT_32",
+        "_UINT_8",
+        "_UINT_16",
+        "_UINT_32",
+        "_INT64_RAW",
+        "_INT_64",
+        "_UINT_64",
+        "_DATE_int32",
+        "_TIME_MILLIS_int32",
+        "_TIMESTAMP_MILLIS_int64",
+        "_TIMESTAMP_MICROS_int64",
+        "_INTERVAL_fixed_len_byte_array_12",
+        "_INT96_RAW"
+    };
+    testBuilder()
+        .sqlQuery(query)
+        .ordered()
+        .baselineColumns(columns)
+        .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,
+            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,
+            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)
+        .build()
+        .run();
+  }
+
+  @Test //DRILL-6670: include tests on data with dictionary encoding disabled
+  public void testComplexLogicalIntTypes5() throws Exception {
+    byte[] bytes12 = {'1', '2', '3', '4', '5', '6', '7', '8', '9', '0', 'a', 
'b' };
+    byte[] bytesOnes = new byte[12];
+    byte[] bytesZeros = new byte[12];
+    Arrays.fill(bytesOnes, (byte) 1);
+    String query =
+        " select " +
+            " t.rowKey as rowKey, " +
+            " t.StringTypes._UTF8 as _UTF8, " +
+            " t.StringTypes._Enum as _Enum, " +
+            " t.NumericTypes.Int32._INT32_RAW as _INT32_RAW, " +
+            " t.NumericTypes.Int32._INT_8 as _INT_8, " +
+            " t.NumericTypes.Int32._INT_16 as _INT_16, " +
+            " t.NumericTypes.Int32._INT_32 as _INT_32, " +
+            " t.NumericTypes.Int32._UINT_8 as _UINT_8, " +
+            " t.NumericTypes.Int32._UINT_16 as _UINT_16, " +
+            " t.NumericTypes.Int32._UINT_32 as _UINT_32, " +
+            " t.NumericTypes.Int64._INT64_RAW as _INT64_RAW, " +
+            " t.NumericTypes.Int64._INT_64 as _INT_64, " +
+            " t.NumericTypes.Int64._UINT_64 as _UINT_64, " +
+            " t.NumericTypes.DateTimeTypes._DATE_int32 as _DATE_int32, " +
+            " t.NumericTypes.DateTimeTypes._TIME_MILLIS_int32 as 
_TIME_MILLIS_int32, " +
+            " t.NumericTypes.DateTimeTypes._TIMESTAMP_MILLIS_int64 as 
_TIMESTAMP_MILLIS_int64, " +
+            " t.NumericTypes.DateTimeTypes._TIMESTAMP_MICROS_int64 as 
_TIMESTAMP_MICROS_int64, " +
+            " t.NumericTypes.DateTimeTypes._INTERVAL_fixed_len_byte_array_12 
as _INTERVAL_fixed_len_byte_array_12, " +
+            " t.NumericTypes.Int96._INT96_RAW as _INT96_RAW " +
+            " from " +
+            " 
cp.`store/parquet/complex/parquet_logical_types_complex_nullable.parquet` t " +
+            " order by t.rowKey ";
+    String[] columns = {
+        "rowKey ",
+        "_UTF8",
+        "_Enum",
+        "_INT32_RAW",
+        "_INT_8",
+        "_INT_16",
+        "_INT_32",
+        "_UINT_8",
+        "_UINT_16",
+        "_UINT_32",
+        "_INT64_RAW",
+        "_INT_64",
+        "_UINT_64",
+        "_DATE_int32",
+        "_TIME_MILLIS_int32",
+        "_TIMESTAMP_MILLIS_int64",
+        "_TIMESTAMP_MICROS_int64",
+        "_INTERVAL_fixed_len_byte_array_12",
+        "_INT96_RAW"
+
+    };
+    testBuilder()
+        .sqlQuery(query)
+        .ordered()
+        .baselineColumns(columns)
+        .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,
+            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,
+            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)
+        .baselineValues(4, null, null, null, null, null, null, null, null, 
null, null, null, null, null,
+            null, null, null, null, null)
+        .build().run();
+  }
+
+  @Test //DRILL-6670
+  public void testSimpleLogicalIntTypes1() throws Exception {
+    byte[] bytes12 = {'1', '2', '3', '4', '5', '6', '7', '8', '9', '0', 'a', 
'b' };
+    byte[] bytesOnes = new byte[12];
+    byte[] bytesZeros = new byte[12];
+    Arrays.fill(bytesOnes, (byte) 1);
+    String query =
+        " select " +
+        " rowKey, " +
+        " _UTF8, " +
+        " _Enum, " +
+        " _INT32_RAW, " +
+        " _INT_8, " +
+        " _INT_16, " +
+        " _INT_32, " +
+        " _UINT_8, " +
+        " _UINT_16, " +
+        " _UINT_32, " +
+        " _INT64_RAW, " +
+        " _INT_64, " +
+        " _UINT_64, " +
+        " _DATE_int32, " +
+        " _TIME_MILLIS_int32, " +
+        " _TIMESTAMP_MILLIS_int64, " +
+        " _TIMESTAMP_MICROS_int64, " +
+        " _INTERVAL_fixed_len_byte_array_12, " +
+        " _INT96_RAW " +
+        " from " +
+        " cp.`parquet/parquet_logical_types_simple.parquet` t " +
+        " order by t.rowKey ";
+  String[] columns = {
+        "rowKey ",
+        "_UTF8",
+        "_Enum",
+        "_INT32_RAW",
+        "_INT_8",
+        "_INT_16",
+        "_INT_32",
+        "_UINT_8",
+        "_UINT_16",
+        "_UINT_32",
+        "_INT64_RAW",
+        "_INT_64",
+        "_UINT_64",
+        "_DATE_int32",
+        "_TIME_MILLIS_int32",
+        "_TIMESTAMP_MILLIS_int64",
+        "_TIMESTAMP_MICROS_int64",
+        "_INTERVAL_fixed_len_byte_array_12",
+        "_INT96_RAW"
+    };
+    testBuilder()
+        .sqlQuery(query)
+        .ordered()
+        .baselineColumns(columns)
+        .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,
+            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,
+            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)
+        .build()
+        .run();
+  }
+
+  @Test //DRILL-6670
+  public void testSimpleLogicalIntTypes2() throws Exception {
+    byte[] bytes12 = {'1', '2', '3', '4', '5', '6', '7', '8', '9', '0', 'a', 
'b' };
+    byte[] bytesOnes = new byte[12];
+    byte[] bytesZeros = new byte[12];
+    Arrays.fill(bytesOnes, (byte) 1);
+    String query =
+        " select " +
+        " rowKey, " +
+        " _UTF8, " +
+        " _Enum, " +
+        " _INT32_RAW, " +
+        " _INT_8, " +
+        " _INT_16, " +
+        " _INT_32, " +
+        " _UINT_8, " +
+        " _UINT_16, " +
+        " _UINT_32, " +
+        " _INT64_RAW, " +
+        " _INT_64, " +
+        " _UINT_64, " +
+        " _DATE_int32, " +
+        " _TIME_MILLIS_int32, " +
+        " _TIMESTAMP_MILLIS_int64, " +
+        " _TIMESTAMP_MICROS_int64, " +
+        " _INTERVAL_fixed_len_byte_array_12, " +
+        " _INT96_RAW " +
+        " from " +
+        " cp.`parquet/parquet_logical_types_simple_nullable.parquet` t " +
+        " order by t.rowKey ";
+    String[] columns = {
+        "rowKey ",
+        "_UTF8",
+        "_Enum",
+        "_INT32_RAW",
+        "_INT_8",
+        "_INT_16",
+        "_INT_32",
+        "_UINT_8",
+        "_UINT_16",
+        "_UINT_32",
+        "_INT64_RAW",
+        "_INT_64",
+        "_UINT_64",
+        "_DATE_int32",
+        "_TIME_MILLIS_int32",
+        "_TIMESTAMP_MILLIS_int64",
+        "_TIMESTAMP_MICROS_int64",
+        "_INTERVAL_fixed_len_byte_array_12",
+        "_INT96_RAW"
+    };
+    testBuilder()
+        .sqlQuery(query)
+        .ordered()
+        .baselineColumns(columns)
+        .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,
+            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,
+            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)
+        .baselineValues(4, null, null, null, null, null, null, null, null, 
null, null, null, null, null,
+            null, null, null, null, null)
+        .build().run();
+  }
+
+  @Test //DRILL-6670
+  public void testSimpleLogicalIntTypes3() throws Exception {
+    byte[] bytes12 = {'1', '2', '3', '4', '5', '6', '7', '8', '9', '0', 'a', 
'b' };
+    byte[] bytesOnes = new byte[12];
+    byte[] bytesZeros = new byte[12];
+    Arrays.fill(bytesOnes, (byte) 1);
+    String query =
+        " select " +
+        " rowKey, " +
+        " _UTF8, " +
+        " _Enum, " +
+        " _INT32_RAW, " +
+        " _INT_8, " +
+        " _INT_16, " +
+        " _INT_32, " +
+        " _UINT_8, " +
+        " _UINT_16, " +
+        " _UINT_32, " +
+        " _INT64_RAW, " +
+        " _INT_64, " +
+        " _UINT_64, " +
+        " _DATE_int32, " +
+        " _TIME_MILLIS_int32, " +
+        " _TIMESTAMP_MILLIS_int64, " +
+        " _TIMESTAMP_MICROS_int64, " +
+        " _INTERVAL_fixed_len_byte_array_12, " +
+        " _INT96_RAW " +
+        " from " +
+        " cp.`parquet/parquet_logical_types_simple_nodict.parquet` t " +
+        " order by t.rowKey ";
+  String[] columns = {
+        "rowKey ",
+        "_UTF8",
+        "_Enum",
+        "_INT32_RAW",
+        "_INT_8",
+        "_INT_16",
+        "_INT_32",
+        "_UINT_8",
+        "_UINT_16",
+        "_UINT_32",
+        "_INT64_RAW",
+        "_INT_64",
+        "_UINT_64",
+        "_DATE_int32",
+        "_TIME_MILLIS_int32",
+        "_TIMESTAMP_MILLIS_int64",
+        "_TIMESTAMP_MICROS_int64",
+        "_INTERVAL_fixed_len_byte_array_12",
+        "_INT96_RAW"
+
+    };
+    testBuilder()
+        .sqlQuery(query)
+        .ordered()
+        .baselineColumns(columns)
+        .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,
+            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,
+            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)
+        .build()
+        .run();
+  }
+
+  @Test //DRILL-6670
+  public void testSimpleLogicalIntTypes4() throws Exception {
+    byte[] bytes12 = {'1', '2', '3', '4', '5', '6', '7', '8', '9', '0', 'a', 
'b' };
+    byte[] bytesOnes = new byte[12];
+    byte[] bytesZeros = new byte[12];
+    Arrays.fill(bytesOnes, (byte) 1);
+    String query =
+        " select " +
+        " rowKey, " +
+        " _UTF8, " +
+        " _Enum, " +
+        " _INT32_RAW, " +
+        " _INT_8, " +
+        " _INT_16, " +
+        " _INT_32, " +
+        " _UINT_8, " +
+        " _UINT_16, " +
+        " _UINT_32, " +
+        " _INT64_RAW, " +
+        " _INT_64, " +
+        " _UINT_64, " +
+        " _DATE_int32, " +
+        " _TIME_MILLIS_int32, " +
+        " _TIMESTAMP_MILLIS_int64, " +
+        " _TIMESTAMP_MICROS_int64, " +
+        " _INTERVAL_fixed_len_byte_array_12, " +
+        " _INT96_RAW " +
+        " from " +
+        " cp.`parquet/parquet_logical_types_simple_nullable_nodict.parquet` t 
" +
+        " order by t.rowKey ";
+    String[] columns = {
+        "rowKey ",
+        "_UTF8",
+        "_Enum",
+        "_INT32_RAW",
+        "_INT_8",
+        "_INT_16",
+        "_INT_32",
+        "_UINT_8",
+        "_UINT_16",
+        "_UINT_32",
+        "_INT64_RAW",
+        "_INT_64",
+        "_UINT_64",
+        "_DATE_int32",
+        "_TIME_MILLIS_int32",
+        "_TIMESTAMP_MILLIS_int64",
+        "_TIMESTAMP_MICROS_int64",
+        "_INTERVAL_fixed_len_byte_array_12",
+        "_INT96_RAW"
+    };
+    testBuilder()
+        .sqlQuery(query)
+        .ordered()
+        .baselineColumns(columns)
+        .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,
+            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,
+            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)
+        .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 941f50f..80fb495 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
@@ -25,6 +25,7 @@ import java.time.ZoneOffset;
 import java.time.ZonedDateTime;
 import java.util.Arrays;
 
+import org.apache.drill.exec.ExecConstants;
 import org.apache.drill.exec.planner.physical.PlannerSettings;
 import org.apache.drill.test.BaseTestQuery;
 import org.joda.time.Period;
@@ -33,20 +34,20 @@ import org.junit.BeforeClass;
 import org.junit.Test;
 
 public class TestDrillParquetReader extends BaseTestQuery {
-  // enable decimal data type
+  // enable decimal data type and make sure DrillParquetReader is used to 
handle test queries
   @BeforeClass
-  public static void enableDecimalDataType() throws Exception {
-    test(String.format("alter session set `%s` = true", 
PlannerSettings.ENABLE_DECIMAL_DATA_TYPE_KEY));
+  public static void setup() throws Exception {
+    alterSession(PlannerSettings.ENABLE_DECIMAL_DATA_TYPE_KEY, true);
+    alterSession(ExecConstants.PARQUET_NEW_RECORD_READER, true);
   }
 
   @AfterClass
-  public static void disableDecimalDataType() throws Exception {
-    test(String.format("alter session set `%s` = false", 
PlannerSettings.ENABLE_DECIMAL_DATA_TYPE_KEY));
+  public static void cleanup() throws Exception {
+    resetSessionOption(PlannerSettings.ENABLE_DECIMAL_DATA_TYPE_KEY);
+    resetSessionOption(ExecConstants.PARQUET_NEW_RECORD_READER);
   }
 
   private void testColumn(String columnName) throws Exception {
-    testNoResult("alter session set `store.parquet.use_new_reader` = true");
-
     BigDecimal result = new BigDecimal("1.20000000");
 
     testBuilder()
@@ -55,8 +56,6 @@ public class TestDrillParquetReader extends BaseTestQuery {
       .baselineColumns(columnName)
       .baselineValues(result)
       .go();
-
-    testNoResult("alter session set `store.parquet.use_new_reader` = false");
   }
 
   @Test
@@ -149,6 +148,7 @@ public class TestDrillParquetReader extends BaseTestQuery {
             " t._DATE_int32 as _DATE_int32, " +
             " t._TIME_MILLIS_int32 as _TIME_MILLIS_int32, " +
             " t._TIMESTAMP_MILLIS_int64 as _TIMESTAMP_MILLIS_int64, " +
+            " t._TIMESTAMP_MICROS_int64 as _TIMESTAMP_MICROS_int64, " +
             " t._INTERVAL_fixed_len_byte_array_12 as 
_INTERVAL_fixed_len_byte_array_12, " +
             " t._INT96_RAW as _INT96_RAW " +
             " from " +
@@ -172,6 +172,7 @@ public class TestDrillParquetReader extends BaseTestQuery {
         "_DATE_int32",
         "_TIME_MILLIS_int32",
         "_TIMESTAMP_MILLIS_int64",
+        "_TIMESTAMP_MICROS_int64",
         "_INTERVAL_fixed_len_byte_array_12",
         "_INT96_RAW"
 
@@ -183,19 +184,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"),
+            LocalDateTime.parse("1973-11-29T21:33:09.012"), 123456789012L,
             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"),
+            LocalDateTime.parse("2038-01-19T03:14:07.999"), 
9223372036854775807L,
             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"), new Period("PT0S"), 
bytesZeros)
+            LocalDateTime.parse("1970-01-01T00:00:00.0"), 0L, new 
Period("PT0S"), bytesZeros)
         .build()
         .run();
   }
@@ -224,6 +225,7 @@ public class TestDrillParquetReader extends BaseTestQuery {
             " t._DATE_int32 as _DATE_int32, " +
             " t._TIME_MILLIS_int32 as _TIME_MILLIS_int32, " +
             " t._TIMESTAMP_MILLIS_int64 as _TIMESTAMP_MILLIS_int64, " +
+            " t._TIMESTAMP_MICROS_int64 as _TIMESTAMP_MICROS_int64, " +
             " t._INTERVAL_fixed_len_byte_array_12 as 
_INTERVAL_fixed_len_byte_array_12, " +
             " t._INT96_RAW as _INT96_RAW " +
             " from " +
@@ -247,6 +249,162 @@ public class TestDrillParquetReader extends BaseTestQuery 
{
         "_DATE_int32",
         "_TIME_MILLIS_int32",
         "_TIMESTAMP_MILLIS_int64",
+        "_TIMESTAMP_MICROS_int64",
+        "_INTERVAL_fixed_len_byte_array_12",
+        "_INT96_RAW"
+
+    };
+    testBuilder()
+        .sqlQuery(query)
+        .ordered()
+        .baselineColumns(columns)
+        .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,
+            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,
+            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)
+        .baselineValues(4, null, null, null, null, null, null, null, null, 
null, null, null, null, null,
+            null, null, null, null, null)
+        .build().run();
+  }
+
+  @Test //DRILL-6670: include tests on data with dictionary encoding disabled
+  public void testLogicalIntTypes4() throws Exception {
+    byte[] bytes12 = {'1', '2', '3', '4', '5', '6', '7', '8', '9', '0', 'a', 
'b' };
+    byte[] bytesOnes = new byte[12];
+    Arrays.fill(bytesOnes, (byte)1);
+    byte[] bytesZeros = new byte[12];
+    String query = String.format(
+        " select " +
+            " t.rowKey as rowKey, " +
+            " t._UTF8 as _UTF8, " +
+            " t._Enum as _Enum, " +
+            " t._INT32_RAW as _INT32_RAW, " +
+            " t._INT_8 as _INT_8, " +
+            " t._INT_16 as _INT_16, " +
+            " t._INT_32 as _INT_32, " +
+            " t._UINT_8 as _UINT_8, " +
+            " t._UINT_16 as _UINT_16, " +
+            " t._UINT_32 as _UINT_32, " +
+            " t._INT64_RAW as _INT64_RAW, " +
+            " t._INT_64 as _INT_64, " +
+            " t._UINT_64 as _UINT_64, " +
+            " t._DATE_int32 as _DATE_int32, " +
+            " t._TIME_MILLIS_int32 as _TIME_MILLIS_int32, " +
+            " t._TIMESTAMP_MILLIS_int64 as _TIMESTAMP_MILLIS_int64, " +
+            " t._TIMESTAMP_MICROS_int64 as _TIMESTAMP_MICROS_int64, " +
+            " t._INTERVAL_fixed_len_byte_array_12 as 
_INTERVAL_fixed_len_byte_array_12, " +
+            " t._INT96_RAW as _INT96_RAW " +
+            " from " +
+            " cp.`parquet/parquet_logical_types_simple_nodict.parquet` t " +
+            " order by t.rowKey "
+    );
+    String[] columns = {
+        "rowKey ",
+        "_UTF8",
+        "_Enum",
+        "_INT32_RAW",
+        "_INT_8",
+        "_INT_16",
+        "_INT_32",
+        "_UINT_8",
+        "_UINT_16",
+        "_UINT_32",
+        "_INT64_RAW",
+        "_INT_64",
+        "_UINT_64",
+        "_DATE_int32",
+        "_TIME_MILLIS_int32",
+        "_TIMESTAMP_MILLIS_int64",
+        "_TIMESTAMP_MICROS_int64",
+        "_INTERVAL_fixed_len_byte_array_12",
+        "_INT96_RAW"
+
+    };
+    testBuilder()
+        .sqlQuery(query)
+        .ordered()
+        .baselineColumns(columns)
+        .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,
+            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,
+            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)
+        .build()
+        .run();
+  }
+
+  @Test //DRILL-6670: include tests on data with dictionary encoding disabled
+  public void testLogicalIntTypes5() throws Exception {
+    byte[] bytes12 = {'1', '2', '3', '4', '5', '6', '7', '8', '9', '0', 'a', 
'b' };
+    byte[] bytesOnes = new byte[12];
+    Arrays.fill(bytesOnes, (byte)1);
+    byte[] bytesZeros = new byte[12];
+    String query = String.format(
+        " select " +
+            " t.rowKey as rowKey, " +
+            " t._UTF8 as _UTF8, " +
+            " t._Enum as _Enum, " +
+            " t._INT32_RAW as _INT32_RAW, " +
+            " t._INT_8 as _INT_8, " +
+            " t._INT_16 as _INT_16, " +
+            " t._INT_32 as _INT_32, " +
+            " t._UINT_8 as _UINT_8, " +
+            " t._UINT_16 as _UINT_16, " +
+            " t._UINT_32 as _UINT_32, " +
+            " t._INT64_RAW as _INT64_RAW, " +
+            " t._INT_64 as _INT_64, " +
+            " t._UINT_64 as _UINT_64, " +
+            " t._DATE_int32 as _DATE_int32, " +
+            " t._TIME_MILLIS_int32 as _TIME_MILLIS_int32, " +
+            " t._TIMESTAMP_MILLIS_int64 as _TIMESTAMP_MILLIS_int64, " +
+            " t._TIMESTAMP_MICROS_int64 as _TIMESTAMP_MICROS_int64, " +
+            " t._INTERVAL_fixed_len_byte_array_12 as 
_INTERVAL_fixed_len_byte_array_12, " +
+            " t._INT96_RAW as _INT96_RAW " +
+            " from " +
+            " 
cp.`parquet/parquet_logical_types_simple_nullable_nodict.parquet` t " +
+            " order by t.rowKey "
+    );
+    String[] columns = {
+        "rowKey ",
+        "_UTF8",
+        "_Enum",
+        "_INT32_RAW",
+        "_INT_8",
+        "_INT_16",
+        "_INT_32",
+        "_UINT_8",
+        "_UINT_16",
+        "_UINT_32",
+        "_INT64_RAW",
+        "_INT_64",
+        "_UINT_64",
+        "_DATE_int32",
+        "_TIME_MILLIS_int32",
+        "_TIMESTAMP_MILLIS_int64",
+        "_TIMESTAMP_MICROS_int64",
         "_INTERVAL_fixed_len_byte_array_12",
         "_INT96_RAW"
 
@@ -258,21 +416,21 @@ 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"),
+            LocalDateTime.parse("1973-11-29T21:33:09.012"), 123456789012L,
             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"),
+            LocalDateTime.parse("2038-01-19T03:14:07.999"), 
9223372036854775807L,
             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"), new Period("PT0S"), 
bytesZeros)
+            LocalDateTime.parse("1970-01-01T00:00:00.0"), 0L, new 
Period("PT0S"), bytesZeros)
         .baselineValues(4, null, null, null, null, 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/resources/parquet/parquet_logical_types_simple.parquet
 
b/exec/java-exec/src/test/resources/parquet/parquet_logical_types_simple.parquet
index edae0e5..c24b96b 100644
Binary files 
a/exec/java-exec/src/test/resources/parquet/parquet_logical_types_simple.parquet
 and 
b/exec/java-exec/src/test/resources/parquet/parquet_logical_types_simple.parquet
 differ
diff --git 
a/exec/java-exec/src/test/resources/parquet/parquet_logical_types_simple_nodict.parquet
 
b/exec/java-exec/src/test/resources/parquet/parquet_logical_types_simple_nodict.parquet
new file mode 100644
index 0000000..7255e8e
Binary files /dev/null and 
b/exec/java-exec/src/test/resources/parquet/parquet_logical_types_simple_nodict.parquet
 differ
diff --git 
a/exec/java-exec/src/test/resources/parquet/parquet_logical_types_simple_nullable.parquet
 
b/exec/java-exec/src/test/resources/parquet/parquet_logical_types_simple_nullable.parquet
index 4c97da9..6db4527 100644
Binary files 
a/exec/java-exec/src/test/resources/parquet/parquet_logical_types_simple_nullable.parquet
 and 
b/exec/java-exec/src/test/resources/parquet/parquet_logical_types_simple_nullable.parquet
 differ
diff --git 
a/exec/java-exec/src/test/resources/parquet/parquet_logical_types_simple_nullable_nodict.parquet
 
b/exec/java-exec/src/test/resources/parquet/parquet_logical_types_simple_nullable_nodict.parquet
new file mode 100644
index 0000000..3ac6070
Binary files /dev/null and 
b/exec/java-exec/src/test/resources/parquet/parquet_logical_types_simple_nullable_nodict.parquet
 differ
diff --git 
a/exec/java-exec/src/test/resources/store/parquet/complex/parquet_logical_types_complex.parquet
 
b/exec/java-exec/src/test/resources/store/parquet/complex/parquet_logical_types_complex.parquet
index ec797e0..4600694 100644
Binary files 
a/exec/java-exec/src/test/resources/store/parquet/complex/parquet_logical_types_complex.parquet
 and 
b/exec/java-exec/src/test/resources/store/parquet/complex/parquet_logical_types_complex.parquet
 differ
diff --git 
a/exec/java-exec/src/test/resources/store/parquet/complex/parquet_logical_types_complex_nodict.parquet
 
b/exec/java-exec/src/test/resources/store/parquet/complex/parquet_logical_types_complex_nodict.parquet
new file mode 100644
index 0000000..1b023b2
Binary files /dev/null and 
b/exec/java-exec/src/test/resources/store/parquet/complex/parquet_logical_types_complex_nodict.parquet
 differ
diff --git 
a/exec/java-exec/src/test/resources/store/parquet/complex/parquet_logical_types_complex_nullable.parquet
 
b/exec/java-exec/src/test/resources/store/parquet/complex/parquet_logical_types_complex_nullable.parquet
index 59b7a75..bc1b59b 100644
Binary files 
a/exec/java-exec/src/test/resources/store/parquet/complex/parquet_logical_types_complex_nullable.parquet
 and 
b/exec/java-exec/src/test/resources/store/parquet/complex/parquet_logical_types_complex_nullable.parquet
 differ
diff --git 
a/exec/java-exec/src/test/resources/store/parquet/complex/parquet_logical_types_complex_nullable_nodict.parquet
 
b/exec/java-exec/src/test/resources/store/parquet/complex/parquet_logical_types_complex_nullable_nodict.parquet
new file mode 100644
index 0000000..3d3377d
Binary files /dev/null and 
b/exec/java-exec/src/test/resources/store/parquet/complex/parquet_logical_types_complex_nullable_nodict.parquet
 differ

Reply via email to