drill-issues mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "ASF GitHub Bot (JIRA)" <j...@apache.org>
Subject [jira] [Commented] (DRILL-4996) Parquet Date auto-correction is not working in auto-partitioned parquet files generated by drill-1.6
Date Fri, 16 Dec 2016 20:46:58 GMT

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

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

Github user chunhui-shi commented on a diff in the pull request:

    https://github.com/apache/drill/pull/687#discussion_r92882778
  
    --- Diff: exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/writer/TestCorruptParquetDateCorrection.java
---
    @@ -201,68 +203,34 @@ public void testCorrectDatesAndExceptionWhileParsingCreatedBy()
throws Exception
             .go();
       }
     
    -  /**
    -   * Test reading a directory full of partitioned parquet files with dates, these files
have a drill version
    -   * number of 1.4.0 in their footers, so we can be certain they are corrupt. The option
to disable the
    -   * correction is passed, but it will not change the result in the case where we are
certain correction
    -   * is needed. For more info see DRILL-4203.
    -   */
    -  @Test
    -  public void testReadPartitionedOnCorruptedDates() throws Exception {
    -    try {
    -      for (String selection : new String[]{"*", "date_col"}) {
    -        // for sanity, try reading all partitions without a filter
    -        TestBuilder builder = testBuilder()
    -            .sqlQuery("select " + selection + " from table(dfs.`" + CORRUPTED_PARTITIONED_DATES_1_4_0_PATH
+ "`" +
    -                "(type => 'parquet', autoCorrectCorruptDates => false))")
    -            .unOrdered()
    -            .baselineColumns("date_col");
    -        addDateBaselineVals(builder);
    -        builder.go();
    -
    -        String query = "select " + selection + " from table(dfs.`" + CORRUPTED_PARTITIONED_DATES_1_4_0_PATH
+ "` " +
    -            "(type => 'parquet', autoCorrectCorruptDates => false))" + " where
date_col = date '1970-01-01'";
    -        // verify that pruning is actually taking place
    -        testPlanMatchingPatterns(query, new String[]{"numFiles=1"}, null);
    -
    -        // read with a filter on the partition column
    -        testBuilder()
    -            .sqlQuery(query)
    -            .unOrdered()
    -            .baselineColumns("date_col")
    -            .baselineValues(new DateTime(1970, 1, 1, 0, 0))
    -            .go();
    -      }
    -    } finally {
    -      test("alter session reset all");
    -    }
    -  }
     
       @Test
       public void testReadPartitionedOnCorruptedDates_UserDisabledCorrection() throws Exception
{
         try {
           for (String selection : new String[]{"*", "date_col"}) {
    -        // for sanity, try reading all partitions without a filter
    -        TestBuilder builder = testBuilder()
    -            .sqlQuery("select " + selection + " from table(dfs.`" + CORRUPTED_PARTITIONED_DATES_1_2_PATH
+ "`" +
    -                "(type => 'parquet', autoCorrectCorruptDates => false))")
    -            .unOrdered()
    -            .baselineColumns("date_col");
    -        addCorruptedDateBaselineVals(builder);
    -        builder.go();
    -
    -        String query = "select " + selection + " from table(dfs.`" + CORRUPTED_PARTITIONED_DATES_1_2_PATH
+ "` " +
    -            "(type => 'parquet', autoCorrectCorruptDates => false))" + " where
date_col = cast('15334-03-17' as date)";
    -        // verify that pruning is actually taking place
    -        testPlanMatchingPatterns(query, new String[]{"numFiles=1"}, null);
    -
    -        // read with a filter on the partition column
    -        testBuilder()
    -            .sqlQuery(query)
    -            .unOrdered()
    -            .baselineColumns("date_col")
    -            .baselineValues(new DateTime(15334, 03, 17, 0, 0))
    -            .go();
    +        for (String table : new String[]{CORRUPTED_PARTITIONED_DATES_1_2_PATH, CORRUPTED_PARTITIONED_DATES_1_4_0_PATH})
{
    --- End diff --
    
    Could you keep for block the same indent as original text?


> Parquet Date auto-correction is not working in auto-partitioned parquet files generated
by drill-1.6
> ----------------------------------------------------------------------------------------------------
>
>                 Key: DRILL-4996
>                 URL: https://issues.apache.org/jira/browse/DRILL-4996
>             Project: Apache Drill
>          Issue Type: Bug
>          Components: Storage - Parquet
>            Reporter: Rahul Challapalli
>            Assignee: Vitalii Diravka
>            Priority: Critical
>         Attachments: item.tgz
>
>
> git.commit.id.abbrev=4ee1d4c
> Below are the steps I followed to generate the data :
> {code}
> 1. Generate a parquet file with date column using hive1.2
> 2. Use drill 1.6 to create auto-partitioned parquet files partitioned on the date column
> {code}
> Now the below query returns wrong results :
> {code}
> select i_rec_start_date, i_size from dfs.`/drill/testdata/parquet_date/auto_partition/item_multipart_autorefresh`
 group by i_rec_start_date, i_size;
> +-------------------+--------------+
> | i_rec_start_date  |    i_size    |
> +-------------------+--------------+
> | null              | large        |
> | 366-11-08        | extra large  |
> | 366-11-08        | medium       |
> | null              | medium       |
> | 366-11-08        | petite       |
> | 364-11-07        | medium       |
> | null              | petite       |
> | 365-11-07        | medium       |
> | 368-11-07        | economy      |
> | 365-11-07        | large        |
> | 365-11-07        | small        |
> | 366-11-08        | small        |
> | 365-11-07        | extra large  |
> | 364-11-07        | N/A          |
> | 366-11-08        | economy      |
> | 366-11-08        | large        |
> | 364-11-07        | small        |
> | null              | small        |
> | 364-11-07        | large        |
> | 364-11-07        | extra large  |
> | 368-11-07        | N/A          |
> | 368-11-07        | extra large  |
> | 368-11-07        | large        |
> | 365-11-07        | petite       |
> | null              | N/A          |
> | 365-11-07        | economy      |
> | 364-11-07        | economy      |
> | 364-11-07        | petite       |
> | 365-11-07        | N/A          |
> | 368-11-07        | medium       |
> | null              | extra large  |
> | 368-11-07        | small        |
> | 368-11-07        | petite       |
> | 366-11-08        | N/A          |
> +-------------------+--------------+
> 34 rows selected (0.691 seconds)
> {code}
> However I tried generating the auto-partitioned parquet files using Drill 1.2 and then
the above query returned the right results.
> I attached the required data sets.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

Mime
View raw message