aboutsummaryrefslogtreecommitdiff
path: root/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet2/TestDrillParquetReader.java
blob: 4b5ed7ae02f84ea9ac79fd8b979f0a4d9f5cb542 (plain)
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
/*
 * 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.parquet2;

import java.math.BigDecimal;
import java.time.Instant;
import java.time.LocalDate;
import java.time.LocalDateTime;
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;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;

public class TestDrillParquetReader extends BaseTestQuery {
  // enable decimal data type and make sure DrillParquetReader is used to handle test queries
  @BeforeClass
  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 cleanup() throws Exception {
    resetSessionOption(PlannerSettings.ENABLE_DECIMAL_DATA_TYPE_KEY);
    resetSessionOption(ExecConstants.PARQUET_NEW_RECORD_READER);
  }

  private void testColumn(String columnName) throws Exception {
    BigDecimal result = new BigDecimal("1.20000000");

    testBuilder()
      .ordered()
      .sqlQuery("select %s from cp.`parquet2/decimal28_38.parquet`", columnName)
      .baselineColumns(columnName)
      .baselineValues(result)
      .go();
  }

  @Test
  public void testRequiredDecimal28() throws Exception {
    testColumn("d28_req");
  }

  @Test
  public void testRequiredDecimal38() throws Exception {
    testColumn("d38_req");
  }

  @Test
  public void testOptionalDecimal28() throws Exception {
    testColumn("d28_opt");
  }

  @Test
  public void testOptionalDecimal38() throws Exception {
    testColumn("d38_opt");
  }

  @Test
  public void test4349() throws Exception {
    // start by creating a parquet file from the input csv file
    runSQL("CREATE TABLE dfs.tmp.`4349` AS SELECT columns[0] id, CAST(NULLIF(columns[1], '') AS DOUBLE) val FROM cp.`parquet2/4349.csv.gz`");

    // querying the parquet file should return the same results found in the csv file
    testBuilder()
      .unOrdered()
      .sqlQuery("SELECT * FROM dfs.tmp.`4349` WHERE id = 'b'")
      .sqlBaselineQuery("SELECT columns[0] id, CAST(NULLIF(columns[1], '') AS DOUBLE) val FROM cp.`parquet2/4349.csv.gz` WHERE columns[0] = 'b'")
      .go();
  }

  @Test
  public void testUnsignedAndSignedIntTypes() throws Exception {
    testBuilder()
      .unOrdered()
      .sqlQuery("select * from cp.`parquet/uint_types.parquet`")
      .baselineColumns("uint8_field", "uint16_field", "uint32_field", "uint64_field", "int8_field", "int16_field",
        "required_uint8_field", "required_uint16_field", "required_uint32_field", "required_uint64_field",
        "required_int8_field", "required_int16_field")
      .baselineValues(255, 65535, 2147483647, 9223372036854775807L, 255, 65535, -1, -1, -1, -1L, -2147483648, -2147483648)
      .baselineValues(-1, -1, -1, -1L, -2147483648, -2147483648, 255, 65535, 2147483647, 9223372036854775807L, 255, 65535)
      .baselineValues(null, null, null, null, null, null, 0, 0, 0, 0L, 0, 0)
      .go();
  }

  @Test
  public void testLogicalIntTypes() throws Exception {
    String query = String.format("select " +
        "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.`parquet/logical_int.parquet` t" );
    String[] columns = {"uint_64", "uint_32", "uint_16", "uint_8", "int_64", "int_32", "int_16", "int_8" };
    testBuilder()
        .sqlQuery(query)
        .unOrdered()
        .baselineColumns(columns)
        .baselineValues(0L, 0, 0, 0, 0L, 0, 0, 0)
        .baselineValues(-1L, -1, -1, -1, -1L, -1, -1, -1)
        .baselineValues(1L, 1, 1, 1, -9223372036854775808L, 1, 1, 1)
        .baselineValues(9223372036854775807L, 2147483647, 65535, 255, 9223372036854775807L, -2147483648, -32768, -128)
        .build()
        .run();
  }

  @Test //DRILL-5971
  public void testLogicalIntTypes2() 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.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-5971
  public void testLogicalIntTypes3() 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.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)
        .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"

    };
    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-6856
  public void testIsTrueOrNullCondition() throws Exception {
    testBuilder()
        .sqlQuery("SELECT col_bln " +
            "FROM cp.`parquetFilterPush/blnTbl/0_0_2.parquet` " +
            "WHERE col_bln IS true OR col_bln IS null " +
            "ORDER BY col_bln")
        .ordered()
        .baselineColumns("col_bln")
        .baselineValuesForSingleColumn(true, null)
        .go();
  }

}