[ 
https://issues.apache.org/jira/browse/DRILL-5971?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16303838#comment-16303838
 ] 

ASF GitHub Bot commented on DRILL-5971:
---------------------------------------

Github user vvysotskyi commented on a diff in the pull request:

    https://github.com/apache/drill/pull/1049#discussion_r158703477
  
    --- Diff: 
exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/TestParquetComplex.java
 ---
    @@ -193,4 +200,168 @@ public void notxistsField() throws Exception {
             .run();
       }
     
    +  @Test //DRILL-5971
    +  public void testComplexLogicalIntTypes() throws Exception {
    +    String query = String.format("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]; Arrays.fill(bytesOnes, (byte)1);
    +    byte[] bytesZeros = new byte[12];
    +    String query = String.format(
    +        " 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, new 
DateTime("5350-02-17"),
    +            new DateTime(1234567, 
DateTimeZone.UTC).withZoneRetainFields(DateTimeZone.getDefault()),
    +            new DateTime("1973-11-29T21:33:09.012"), new 
Period("P875770417M943142453DT1650536.505S"),
    +            bytes12)
    +        .baselineValues(2, "UTF8 string2", "MAX_VALUE", 2147483647, 127, 
32767, 2147483647, 255, 65535, -1,
    +            9223372036854775807L, 9223372036854775807L, -1L, new 
DateTime("1969-12-31"),
    +            new DateTime(0xFFFFFFFF, 
DateTimeZone.UTC).withZoneRetainFields(DateTimeZone.getDefault()),
    +            new DateTime("2038-01-19T03:14:07.999"), new 
Period("P16843009M16843009DT16843.009S"),
    +            bytesOnes)
    +        .baselineValues(3, "UTF8 string3", "MIN_VALUE", -2147483648, -128, 
-32768, -2147483648, 0, 0, 0,
    +            -9223372036854775808L, -9223372036854775808L, 0L, new 
DateTime("1970-01-01"),
    +            new DateTime(0, 
DateTimeZone.UTC).withZoneRetainFields(DateTimeZone.getDefault()),
    +            new DateTime("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]; Arrays.fill(bytesOnes, (byte)1);
    +    byte[] bytesZeros = new byte[12];
    +    String query = String.format(
    +        " 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()
    --- End diff --
    
    unOrdered


> Fix INT64, INT32 logical types in complex parquet reader
> --------------------------------------------------------
>
>                 Key: DRILL-5971
>                 URL: https://issues.apache.org/jira/browse/DRILL-5971
>             Project: Apache Drill
>          Issue Type: Bug
>    Affects Versions: 1.11.0
>            Reporter: Parth Chandra
>            Assignee: Parth Chandra
>             Fix For: 1.13.0
>
>
> The 'complex' Parquet reader does not recognize the Parquet logical types 
> INT64, and INT32. 
> Should be a simple change to add these logical types.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)

Reply via email to