From 144980b3cdaaad00db72e9043ddbf7bb47e6e225 Mon Sep 17 00:00:00 2001 From: ges1227 <24355353+ges1227@users.noreply.github.com> Date: Wed, 6 Nov 2024 17:30:34 +1300 Subject: [PATCH 1/4] Replace random test values by discrete ones The use of Random() function in the parquet.batchreader.decoders tests may cause flakiness. Adds TestMode.java to parameterize tests with an arbitrary value and an upper-/lower-bounded value. Resolves: #23840 --- .../TestFlatDefinitionLevelDecoder.java | 11 +- .../batchreader/decoders/TestMode.java | 38 +++++ .../decoders/TestParquetUtils.java | 33 ++--- .../decoders/TestValuesDecoders.java | 135 +++++++++--------- 4 files changed, 123 insertions(+), 94 deletions(-) create mode 100644 presto-parquet/src/test/java/com/facebook/presto/parquet/batchreader/decoders/TestMode.java diff --git a/presto-parquet/src/test/java/com/facebook/presto/parquet/batchreader/decoders/TestFlatDefinitionLevelDecoder.java b/presto-parquet/src/test/java/com/facebook/presto/parquet/batchreader/decoders/TestFlatDefinitionLevelDecoder.java index d081d227480f..18d1bef22fe2 100644 --- a/presto-parquet/src/test/java/com/facebook/presto/parquet/batchreader/decoders/TestFlatDefinitionLevelDecoder.java +++ b/presto-parquet/src/test/java/com/facebook/presto/parquet/batchreader/decoders/TestFlatDefinitionLevelDecoder.java @@ -21,11 +21,10 @@ import java.io.IOException; import java.util.ArrayList; import java.util.List; -import java.util.Random; import static com.facebook.presto.parquet.batchreader.decoders.TestParquetUtils.addDLRLEBlock; import static com.facebook.presto.parquet.batchreader.decoders.TestParquetUtils.addDLValues; -import static com.facebook.presto.parquet.batchreader.decoders.TestParquetUtils.randomValues; +import static com.facebook.presto.parquet.batchreader.decoders.TestParquetUtils.fillValues; import static java.lang.Math.min; import static org.testng.Assert.assertEquals; import static org.testng.Assert.fail; @@ -35,19 +34,19 @@ public class TestFlatDefinitionLevelDecoder private static int valueCount; private static int nonNullCount; private static byte[] pageBytes; - private static List expectedValues = new ArrayList<>(); + private static final List expectedValues = new ArrayList<>(); @BeforeClass public void setup() throws IOException { - Random random = new Random(200); + TestMode testMode = TestMode.ARBITRARY; RunLengthBitPackingHybridEncoder encoder = TestParquetUtils.getSimpleDLEncoder(); addDLRLEBlock(1, 50, encoder, expectedValues); - addDLValues(randomValues(random, 457, 1), encoder, expectedValues); + addDLValues(fillValues(testMode, 457, 1), encoder, expectedValues); addDLRLEBlock(0, 37, encoder, expectedValues); - addDLValues(randomValues(random, 186, 1), encoder, expectedValues); + addDLValues(fillValues(testMode, 186, 1), encoder, expectedValues); valueCount = expectedValues.size(); for (Integer value : expectedValues) { diff --git a/presto-parquet/src/test/java/com/facebook/presto/parquet/batchreader/decoders/TestMode.java b/presto-parquet/src/test/java/com/facebook/presto/parquet/batchreader/decoders/TestMode.java new file mode 100644 index 000000000000..f8a4709598a5 --- /dev/null +++ b/presto-parquet/src/test/java/com/facebook/presto/parquet/batchreader/decoders/TestMode.java @@ -0,0 +1,38 @@ +package com.facebook.presto.parquet.batchreader.decoders; + +public enum TestMode +{ + UPPER_BOUND(Integer.MAX_VALUE, Long.MAX_VALUE), LOWER_BOUND(Integer.MIN_VALUE, 0), ARBITRARY(237, 237 * (1L << 31)); + + private final int testInt; + private final long testLong; + + TestMode(int i, long l) + { + this.testInt = i; + this.testLong = l; + } + + public int getInt() + { + return testInt; + } + + public long getLong() + { + return testLong; + } + + public int getPositiveUpperBoundedInt(int upper) + { + if (this.name().equals(LOWER_BOUND.name()) || upper <= 0) { + return 0; + } + + if (this.name().equals(UPPER_BOUND.name())) { + return upper; + } + + return ARBITRARY.testInt % upper; + } +} diff --git a/presto-parquet/src/test/java/com/facebook/presto/parquet/batchreader/decoders/TestParquetUtils.java b/presto-parquet/src/test/java/com/facebook/presto/parquet/batchreader/decoders/TestParquetUtils.java index 9168934f9213..458e559ce8f8 100644 --- a/presto-parquet/src/test/java/com/facebook/presto/parquet/batchreader/decoders/TestParquetUtils.java +++ b/presto-parquet/src/test/java/com/facebook/presto/parquet/batchreader/decoders/TestParquetUtils.java @@ -13,7 +13,6 @@ */ package com.facebook.presto.parquet.batchreader.decoders; -import org.apache.commons.lang3.RandomStringUtils; import org.apache.hadoop.hive.ql.io.parquet.timestamp.NanoTime; import org.apache.hadoop.hive.ql.io.parquet.timestamp.NanoTimeUtils; import org.apache.parquet.bytes.BytesUtils; @@ -31,7 +30,6 @@ import java.util.ArrayList; import java.util.Iterator; import java.util.List; -import java.util.Random; import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Preconditions.checkState; @@ -80,16 +78,16 @@ public static void addDLValues(Iterator values, RunLengthBitPackingHybr } } - public static Iterator randomValues(Random random, int numValues, int maxValue) + public static Iterator fillValues(TestMode testMode, int numValues, int maxValue) { List values = new ArrayList<>(); for (int i = 0; i < numValues; i++) { - values.add(random.nextInt(maxValue + 1)); + values.add(testMode.getPositiveUpperBoundedInt(maxValue)); } return values.iterator(); } - public static byte[] generatePlainValuesPage(int valueCount, int valueSizeBits, Random random, List addedValues) + public static byte[] generatePlainValuesPage(int valueCount, int valueSizeBits, TestMode testMode, List addedValues) { ValuesWriter writer; @@ -103,7 +101,7 @@ public static byte[] generatePlainValuesPage(int valueCount, int valueSizeBits, switch (valueSizeBits) { case 1: { for (int i = 0; i < valueCount; i++) { - int value = random.nextInt(2); + int value = testMode.getPositiveUpperBoundedInt(1); writer.writeInteger(value); addedValues.add(value); } @@ -111,7 +109,7 @@ public static byte[] generatePlainValuesPage(int valueCount, int valueSizeBits, } case -1: { for (int i = 0; i < valueCount; i++) { - String valueStr = RandomStringUtils.random(random.nextInt(10), 0, 0, true, true, null, random); + String valueStr = "4nY" + valueCount; byte[] valueUtf8 = valueStr.getBytes(StandardCharsets.UTF_8); writer.writeBytes(Binary.fromConstantByteArray(valueUtf8, 0, valueUtf8.length)); addedValues.add(valueStr); @@ -120,7 +118,7 @@ public static byte[] generatePlainValuesPage(int valueCount, int valueSizeBits, } case 32: { for (int i = 0; i < valueCount; i++) { - int value = random.nextInt(); + int value = testMode.getInt(); writer.writeInteger(value); addedValues.add(value); } @@ -128,7 +126,7 @@ public static byte[] generatePlainValuesPage(int valueCount, int valueSizeBits, } case 64: { for (int i = 0; i < valueCount; i++) { - long value = random.nextLong(); + long value = testMode.getLong(); writer.writeLong(value); addedValues.add(value); } @@ -136,7 +134,7 @@ public static byte[] generatePlainValuesPage(int valueCount, int valueSizeBits, } case 96: { for (int i = 0; i < valueCount; i++) { - long millisValue = Long.valueOf(random.nextInt(1572281176) * 1000); + long millisValue = testMode.getPositiveUpperBoundedInt(1572281175) * 1000L; NanoTime nanoTime = NanoTimeUtils.getNanoTime(new Timestamp(millisValue), false); writer.writeLong(nanoTime.getTimeOfDayNanos()); writer.writeInteger(nanoTime.getJulianDay()); @@ -146,10 +144,9 @@ public static byte[] generatePlainValuesPage(int valueCount, int valueSizeBits, } case 128: for (int i = 0; i < valueCount; i++) { - long value = random.nextLong(); + long value = testMode.getLong(); writer.writeLong(value); addedValues.add(value); - value = random.nextLong(); writer.writeLong(value); addedValues.add(value); } @@ -166,19 +163,19 @@ public static byte[] generatePlainValuesPage(int valueCount, int valueSizeBits, } } - public static byte[] generateDictionaryIdPage2048(int maxValue, Random random, List addedValues) + public static byte[] generateDictionaryIdPage2048(int maxValue, TestMode testMode, List addedValues) { RunLengthBitPackingHybridEncoder encoder = getDictionaryDataPageEncoder(maxValue); addDLRLEBlock(maxValue / 2, 50, encoder, addedValues); - addDLValues(randomValues(random, 457, maxValue), encoder, addedValues); + addDLValues(fillValues(testMode, 457, maxValue), encoder, addedValues); addDLRLEBlock(0, 37, encoder, addedValues); - addDLValues(randomValues(random, 186, maxValue), encoder, addedValues); - addDLValues(randomValues(random, 289, maxValue), encoder, addedValues); + addDLValues(fillValues(testMode, 186, maxValue), encoder, addedValues); + addDLValues(fillValues(testMode, 289, maxValue), encoder, addedValues); addDLRLEBlock(maxValue - 1, 76, encoder, addedValues); - addDLValues(randomValues(random, 789, maxValue), encoder, addedValues); + addDLValues(fillValues(testMode, 789, maxValue), encoder, addedValues); addDLRLEBlock(maxValue - 1, 137, encoder, addedValues); - addDLValues(randomValues(random, 27, maxValue), encoder, addedValues); + addDLValues(fillValues(testMode, 27, maxValue), encoder, addedValues); checkState(addedValues.size() == 2048); diff --git a/presto-parquet/src/test/java/com/facebook/presto/parquet/batchreader/decoders/TestValuesDecoders.java b/presto-parquet/src/test/java/com/facebook/presto/parquet/batchreader/decoders/TestValuesDecoders.java index c94a68e8d19e..321910287900 100644 --- a/presto-parquet/src/test/java/com/facebook/presto/parquet/batchreader/decoders/TestValuesDecoders.java +++ b/presto-parquet/src/test/java/com/facebook/presto/parquet/batchreader/decoders/TestValuesDecoders.java @@ -44,6 +44,7 @@ import com.facebook.presto.parquet.dictionary.IntegerDictionary; import com.facebook.presto.parquet.dictionary.LongDictionary; import io.airlift.slice.Slices; +import org.testng.annotations.DataProvider; import org.testng.annotations.Test; import java.io.ByteArrayInputStream; @@ -52,7 +53,6 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.List; -import java.util.Random; import java.util.stream.Collectors; import static com.facebook.presto.parquet.ParquetEncoding.PLAIN_DICTIONARY; @@ -60,7 +60,7 @@ import static com.facebook.presto.parquet.batchreader.decoders.TestParquetUtils.generatePlainValuesPage; import static com.google.common.collect.ImmutableList.toImmutableList; import static java.lang.Math.min; -import static org.apache.parquet.bytes.BytesUtils.UTF8; +import static java.nio.charset.StandardCharsets.UTF_8; import static org.apache.parquet.bytes.BytesUtils.getWidthFromMaxInt; import static org.testng.Assert.assertEquals; @@ -215,7 +215,7 @@ private static void binaryBatchReadWithSkipHelper(int batchSize, int skipSize, i decoder.readIntoBuffer(byteBuffer, 0, offsets, 0, valueBuffer); for (int i = 0; i < readBatchSize; i++) { - byte[] expected = ((String) expectedValues.get(inputOffset + i)).getBytes(UTF8); + byte[] expected = ((String) expectedValues.get(inputOffset + i)).getBytes(UTF_8); byte[] actual = Arrays.copyOfRange(byteBuffer, offsets[i], offsets[i + 1]); assertEquals(expected, actual); } @@ -365,14 +365,20 @@ private static void booleanBatchReadWithSkipHelper(int batchSize, int skipSize, } } - @Test - public void testInt32Plain() + @DataProvider(name = "testModes") + public TestMode[] getTestModes() + { + return TestMode.values(); + } + + @Test(dataProvider = "testModes") + public void testInt32Plain(TestMode testMode) throws IOException { int valueCount = 2048; List expectedValues = new ArrayList<>(); - byte[] pageBytes = generatePlainValuesPage(valueCount, 32, new Random(89), expectedValues); + byte[] pageBytes = generatePlainValuesPage(valueCount, 32, testMode, expectedValues); int32BatchReadWithSkipHelper(valueCount, 0, valueCount, int32Plain(pageBytes), expectedValues); // read all values in one batch int32BatchReadWithSkipHelper(29, 0, valueCount, int32Plain(pageBytes), expectedValues); @@ -384,18 +390,17 @@ public void testInt32Plain() int32BatchReadWithSkipHelper(1024, 1024, valueCount, int32Plain(pageBytes), expectedValues); } - @Test - public void testInt32RLEDictionary() + @Test(dataProvider = "testModes") + public void testInt32RLEDictionary(TestMode testMode) throws IOException { - Random random = new Random(83); int valueCount = 2048; int dictionarySize = 29; List dictionary = new ArrayList<>(); List dictionaryIds = new ArrayList<>(); - byte[] dictionaryPage = generatePlainValuesPage(dictionarySize, 32, random, dictionary); - byte[] dataPage = generateDictionaryIdPage2048(dictionarySize - 1, random, dictionaryIds); + byte[] dictionaryPage = generatePlainValuesPage(dictionarySize, 32, testMode, dictionary); + byte[] dataPage = generateDictionaryIdPage2048(dictionarySize - 1, testMode, dictionaryIds); List expectedValues = new ArrayList<>(); for (Integer dictionaryId : dictionaryIds) { @@ -414,14 +419,14 @@ public void testInt32RLEDictionary() int32BatchReadWithSkipHelper(1024, 1024, valueCount, int32Dictionary(dataPage, dictionarySize, integerDictionary), expectedValues); } - @Test - public void testBinaryPlain() + @Test(dataProvider = "testModes") + public void testBinaryPlain(TestMode testMode) throws IOException { int valueCount = 2048; List expectedValues = new ArrayList<>(); - byte[] pageBytes = generatePlainValuesPage(valueCount, -1, new Random(113), expectedValues); + byte[] pageBytes = generatePlainValuesPage(valueCount, -1, testMode, expectedValues); binaryBatchReadWithSkipHelper(valueCount, 0, valueCount, binaryPlain(pageBytes), expectedValues); // read all values in one batch binaryBatchReadWithSkipHelper(29, 0, valueCount, binaryPlain(pageBytes), expectedValues); @@ -433,18 +438,17 @@ public void testBinaryPlain() binaryBatchReadWithSkipHelper(1024, 1024, valueCount, binaryPlain(pageBytes), expectedValues); } - @Test - public void testBinaryRLEDictionary() + @Test(dataProvider = "testModes") + public void testBinaryRLEDictionary(TestMode testMode) throws IOException { - Random random = new Random(83); int valueCount = 2048; int dictionarySize = 29; List dictionary = new ArrayList<>(); List dictionaryIds = new ArrayList<>(); - byte[] dictionaryPage = TestParquetUtils.generatePlainValuesPage(dictionarySize, -1, random, dictionary); - byte[] dataPage = TestParquetUtils.generateDictionaryIdPage2048(dictionarySize - 1, random, dictionaryIds); + byte[] dictionaryPage = TestParquetUtils.generatePlainValuesPage(dictionarySize, -1, testMode, dictionary); + byte[] dataPage = TestParquetUtils.generateDictionaryIdPage2048(dictionarySize - 1, testMode, dictionaryIds); List expectedValues = new ArrayList<>(); for (Integer dictionaryId : dictionaryIds) { @@ -463,14 +467,14 @@ public void testBinaryRLEDictionary() binaryBatchReadWithSkipHelper(1024, 1024, valueCount, binaryDictionary(dataPage, dictionarySize, binaryDictionary), expectedValues); } - @Test - public void testInt64Plain() + @Test(dataProvider = "testModes") + public void testInt64Plain(TestMode testMode) throws IOException { int valueCount = 2048; List expectedValues = new ArrayList<>(); - byte[] pageBytes = generatePlainValuesPage(valueCount, 64, new Random(89), expectedValues); + byte[] pageBytes = generatePlainValuesPage(valueCount, 64, testMode, expectedValues); int64BatchReadWithSkipHelper(valueCount, 0, valueCount, int64Plain(pageBytes), expectedValues); // read all values in one batch int64BatchReadWithSkipHelper(29, 0, valueCount, int64Plain(pageBytes), expectedValues); @@ -492,18 +496,17 @@ public void testInt64Plain() int64BatchReadWithSkipHelper(1024, 1024, valueCount, int64TimestampMicrosPlain(pageBytes), expectedTimestampValues); } - @Test - public void testInt64RLEDictionary() + @Test(dataProvider = "testModes") + public void testInt64RLEDictionary(TestMode testMode) throws IOException { - Random random = new Random(83); int valueCount = 2048; int dictionarySize = 29; List dictionary = new ArrayList<>(); List dictionaryIds = new ArrayList<>(); - byte[] dictionaryPage = generatePlainValuesPage(dictionarySize, 64, random, dictionary); - byte[] dataPage = generateDictionaryIdPage2048(dictionarySize - 1, random, dictionaryIds); + byte[] dictionaryPage = generatePlainValuesPage(dictionarySize, 64, testMode, dictionary); + byte[] dataPage = generateDictionaryIdPage2048(dictionarySize - 1, testMode, dictionaryIds); List expectedValues = new ArrayList<>(); for (Integer dictionaryId : dictionaryIds) { @@ -532,14 +535,14 @@ public void testInt64RLEDictionary() int64BatchReadWithSkipHelper(1024, 1024, valueCount, int64TimestampMicrosDictionary(dataPage, dictionarySize, longDictionary), expectedTimestampValues); } - @Test - public void testTimestampPlain() + @Test(dataProvider = "testModes") + public void testTimestampPlain(TestMode testMode) throws IOException { int valueCount = 2048; List expectedValues = new ArrayList<>(); - byte[] pageBytes = generatePlainValuesPage(valueCount, 96, new Random(83), expectedValues); + byte[] pageBytes = generatePlainValuesPage(valueCount, 96, testMode, expectedValues); timestampBatchReadWithSkipHelper(valueCount, 0, valueCount, timestampPlain(pageBytes), expectedValues); // read all values in one batch timestampBatchReadWithSkipHelper(29, 0, valueCount, timestampPlain(pageBytes), expectedValues); @@ -551,18 +554,17 @@ public void testTimestampPlain() timestampBatchReadWithSkipHelper(1024, 1024, valueCount, timestampPlain(pageBytes), expectedValues); } - @Test - public void testTimestampRLEDictionary() + @Test(dataProvider = "testModes") + public void testTimestampRLEDictionary(TestMode testMode) throws IOException { - Random random = new Random(83); int valueCount = 2048; int dictionarySize = 29; List dictionary = new ArrayList<>(); List dictionaryIds = new ArrayList<>(); - byte[] dictionaryPage = generatePlainValuesPage(dictionarySize, 96, random, dictionary); - byte[] dataPage = generateDictionaryIdPage2048(dictionarySize - 1, random, dictionaryIds); + byte[] dictionaryPage = generatePlainValuesPage(dictionarySize, 96, testMode, dictionary); + byte[] dataPage = generateDictionaryIdPage2048(dictionarySize - 1, testMode, dictionaryIds); List expectedValues = new ArrayList<>(); for (Integer dictionaryId : dictionaryIds) { @@ -581,13 +583,13 @@ public void testTimestampRLEDictionary() timestampBatchReadWithSkipHelper(1024, 1024, valueCount, timestampDictionary(dataPage, dictionarySize, tsDictionary), expectedValues); } - @Test - public void testBooleanPlain() + @Test(dataProvider = "testModes") + public void testBooleanPlain(TestMode testMode) { int valueCount = 2048; List expectedValues = new ArrayList<>(); - byte[] pageBytes = generatePlainValuesPage(valueCount, 1, new Random(83), expectedValues); + byte[] pageBytes = generatePlainValuesPage(valueCount, 1, testMode, expectedValues); booleanBatchReadWithSkipHelper(valueCount, 0, valueCount, booleanPlain(pageBytes), expectedValues); // read all values in one batch booleanBatchReadWithSkipHelper(29, 0, valueCount, booleanPlain(pageBytes), expectedValues); @@ -599,19 +601,15 @@ public void testBooleanPlain() booleanBatchReadWithSkipHelper(1024, 1024, valueCount, booleanPlain(pageBytes), expectedValues); } - @Test - public void testBooleanRLE() + @Test(dataProvider = "testModes") + public void testBooleanRLE(TestMode testMode) { - Random random = new Random(111); int valueCount = 2048; List values = new ArrayList<>(); - byte[] dataPage = generateDictionaryIdPage2048(1, random, values); + byte[] dataPage = generateDictionaryIdPage2048(1, testMode, values); - List expectedValues = new ArrayList<>(); - for (Integer value : values) { - expectedValues.add(value.intValue()); - } + List expectedValues = new ArrayList<>(values); booleanBatchReadWithSkipHelper(valueCount, 0, valueCount, booleanRLE(dataPage), expectedValues); booleanBatchReadWithSkipHelper(29, 0, valueCount, booleanRLE(dataPage), expectedValues); @@ -623,14 +621,14 @@ public void testBooleanRLE() booleanBatchReadWithSkipHelper(1024, 1024, valueCount, booleanRLE(dataPage), expectedValues); } - @Test - public void testInt32ShortDecimalPlain() + @Test(dataProvider = "testModes") + public void testInt32ShortDecimalPlain(TestMode testMode) throws IOException { int valueCount = 2048; List expectedValues = new ArrayList<>(); - byte[] pageBytes = generatePlainValuesPage(valueCount, 32, new Random(83), expectedValues); + byte[] pageBytes = generatePlainValuesPage(valueCount, 32, testMode, expectedValues); int32ShortDecimalBatchReadWithSkipHelper(valueCount, 0, valueCount, int32ShortDecimalPlain(pageBytes), expectedValues); // read all values in one batch int32ShortDecimalBatchReadWithSkipHelper(29, 0, valueCount, int32ShortDecimalPlain(pageBytes), expectedValues); int32ShortDecimalBatchReadWithSkipHelper(89, 0, valueCount, int32ShortDecimalPlain(pageBytes), expectedValues); @@ -641,14 +639,14 @@ public void testInt32ShortDecimalPlain() int32ShortDecimalBatchReadWithSkipHelper(1024, 1024, valueCount, int32ShortDecimalPlain(pageBytes), expectedValues); } - @Test - public void testInt64ShortDecimalPlain() + @Test(dataProvider = "testModes") + public void testInt64ShortDecimalPlain(TestMode testMode) throws IOException { int valueCount = 2048; List expectedValues = new ArrayList<>(); - byte[] pageBytes = generatePlainValuesPage(valueCount, 64, new Random(83), expectedValues); + byte[] pageBytes = generatePlainValuesPage(valueCount, 64, testMode, expectedValues); int64ShortDecimalBatchReadWithSkipHelper(valueCount, 0, valueCount, int64ShortDecimalPlain(pageBytes), expectedValues); // read all values in one batch int64ShortDecimalBatchReadWithSkipHelper(29, 0, valueCount, int64ShortDecimalPlain(pageBytes), expectedValues); int64ShortDecimalBatchReadWithSkipHelper(89, 0, valueCount, int64ShortDecimalPlain(pageBytes), expectedValues); @@ -659,18 +657,17 @@ public void testInt64ShortDecimalPlain() int64ShortDecimalBatchReadWithSkipHelper(1024, 1024, valueCount, int64ShortDecimalPlain(pageBytes), expectedValues); } - @Test - public void testInt32ShortDecimalRLE() + @Test(dataProvider = "testModes") + public void testInt32ShortDecimalRLE(TestMode testMode) throws IOException { - Random random = new Random(83); int valueCount = 2048; int dictionarySize = 29; List dictionary = new ArrayList<>(); List dictionaryIds = new ArrayList<>(); - byte[] dictionaryPage = generatePlainValuesPage(dictionarySize, 32, random, dictionary); - byte[] dataPage = generateDictionaryIdPage2048(dictionarySize - 1, random, dictionaryIds); + byte[] dictionaryPage = generatePlainValuesPage(dictionarySize, 32, testMode, dictionary); + byte[] dataPage = generateDictionaryIdPage2048(dictionarySize - 1, testMode, dictionaryIds); List expectedValues = new ArrayList<>(); for (Integer dictionaryId : dictionaryIds) { @@ -689,18 +686,17 @@ public void testInt32ShortDecimalRLE() int32ShortDecimalBatchReadWithSkipHelper(1024, 1024, valueCount, int32ShortDecimalRLE(dataPage, dictionarySize, integerDictionary), expectedValues); } - @Test - public void testInt64ShortDecimalRLE() + @Test(dataProvider = "testModes") + public void testInt64ShortDecimalRLE(TestMode testMode) throws IOException { - Random random = new Random(83); int valueCount = 2048; int dictionarySize = 29; List dictionary = new ArrayList<>(); List dictionaryIds = new ArrayList<>(); - byte[] dictionaryPage = generatePlainValuesPage(dictionarySize, 64, random, dictionary); - byte[] dataPage = generateDictionaryIdPage2048(dictionarySize - 1, random, dictionaryIds); + byte[] dictionaryPage = generatePlainValuesPage(dictionarySize, 64, testMode, dictionary); + byte[] dataPage = generateDictionaryIdPage2048(dictionarySize - 1, testMode, dictionaryIds); List expectedValues = new ArrayList<>(); for (Integer dictionaryId : dictionaryIds) { @@ -719,14 +715,14 @@ public void testInt64ShortDecimalRLE() int64ShortDecimalBatchReadWithSkipHelper(1024, 1024, valueCount, int64ShortDecimalRLE(dataPage, dictionarySize, longDictionary), expectedValues); } - @Test - public void testUuidPlainPlain() + @Test(dataProvider = "testModes") + public void testUuidPlainPlain(TestMode testMode) throws IOException { int valueCount = 2048; List expectedValues = new ArrayList<>(); - byte[] pageBytes = generatePlainValuesPage(valueCount, 128, new Random(83), expectedValues); + byte[] pageBytes = generatePlainValuesPage(valueCount, 128, testMode, expectedValues); // page is read assuming in big endian, so we need to flip the bytes around when comparing read values expectedValues = expectedValues.stream() .map(Long.class::cast) @@ -744,18 +740,17 @@ public void testUuidPlainPlain() uuidBatchReadWithSkipHelper(1024, 1024, valueCount, uuidPlain(pageBytes), expectedValues); } - @Test - public void testUuidRLEDictionary() + @Test(dataProvider = "testModes") + public void testUuidRLEDictionary(TestMode testMode) throws IOException { int valueCount = 2048; - Random random = new Random(83); int dictionarySize = 29; List dictionary = new ArrayList<>(); List dictionaryIds = new ArrayList<>(); - byte[] dictionaryPage = generatePlainValuesPage(dictionarySize, 128, random, dictionary); - byte[] dataPage = generateDictionaryIdPage2048(dictionarySize - 1, random, dictionaryIds); + byte[] dictionaryPage = generatePlainValuesPage(dictionarySize, 128, testMode, dictionary); + byte[] dataPage = generateDictionaryIdPage2048(dictionarySize - 1, testMode, dictionaryIds); List expectedValues = new ArrayList<>(); for (Integer dictionaryId : dictionaryIds) { From 9d028cc25e82070982adcdabb3d8943ba3e17dea Mon Sep 17 00:00:00 2001 From: ges1227 <24355353+ges1227@users.noreply.github.com> Date: Wed, 6 Nov 2024 17:59:09 +1300 Subject: [PATCH 2/4] Add license header to TestMode.java --- .../parquet/batchreader/decoders/TestMode.java | 13 +++++++++++++ 1 file changed, 13 insertions(+) diff --git a/presto-parquet/src/test/java/com/facebook/presto/parquet/batchreader/decoders/TestMode.java b/presto-parquet/src/test/java/com/facebook/presto/parquet/batchreader/decoders/TestMode.java index f8a4709598a5..c817bad785ca 100644 --- a/presto-parquet/src/test/java/com/facebook/presto/parquet/batchreader/decoders/TestMode.java +++ b/presto-parquet/src/test/java/com/facebook/presto/parquet/batchreader/decoders/TestMode.java @@ -1,3 +1,16 @@ +/* + * Licensed 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 com.facebook.presto.parquet.batchreader.decoders; public enum TestMode From 013c642bab4ea04c39d2334378eccd82c2a7533e Mon Sep 17 00:00:00 2001 From: ges1227 <24355353+ges1227@users.noreply.github.com> Date: Sun, 24 Nov 2024 13:55:37 +0100 Subject: [PATCH 3/4] Replaced enum by delegate modes to methods --- .../TestFlatDefinitionLevelDecoder.java | 5 +- .../batchreader/decoders/TestMode.java | 51 ----- .../decoders/TestParquetUtils.java | 44 ++-- .../decoders/TestValuesDecoders.java | 209 ++++++++++++------ 4 files changed, 166 insertions(+), 143 deletions(-) delete mode 100644 presto-parquet/src/test/java/com/facebook/presto/parquet/batchreader/decoders/TestMode.java diff --git a/presto-parquet/src/test/java/com/facebook/presto/parquet/batchreader/decoders/TestFlatDefinitionLevelDecoder.java b/presto-parquet/src/test/java/com/facebook/presto/parquet/batchreader/decoders/TestFlatDefinitionLevelDecoder.java index 18d1bef22fe2..92fa0986523c 100644 --- a/presto-parquet/src/test/java/com/facebook/presto/parquet/batchreader/decoders/TestFlatDefinitionLevelDecoder.java +++ b/presto-parquet/src/test/java/com/facebook/presto/parquet/batchreader/decoders/TestFlatDefinitionLevelDecoder.java @@ -40,13 +40,12 @@ public class TestFlatDefinitionLevelDecoder public void setup() throws IOException { - TestMode testMode = TestMode.ARBITRARY; RunLengthBitPackingHybridEncoder encoder = TestParquetUtils.getSimpleDLEncoder(); addDLRLEBlock(1, 50, encoder, expectedValues); - addDLValues(fillValues(testMode, 457, 1), encoder, expectedValues); + addDLValues(fillValues(457, 1), encoder, expectedValues); addDLRLEBlock(0, 37, encoder, expectedValues); - addDLValues(fillValues(testMode, 186, 1), encoder, expectedValues); + addDLValues(fillValues(186, 1), encoder, expectedValues); valueCount = expectedValues.size(); for (Integer value : expectedValues) { diff --git a/presto-parquet/src/test/java/com/facebook/presto/parquet/batchreader/decoders/TestMode.java b/presto-parquet/src/test/java/com/facebook/presto/parquet/batchreader/decoders/TestMode.java deleted file mode 100644 index c817bad785ca..000000000000 --- a/presto-parquet/src/test/java/com/facebook/presto/parquet/batchreader/decoders/TestMode.java +++ /dev/null @@ -1,51 +0,0 @@ -/* - * Licensed 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 com.facebook.presto.parquet.batchreader.decoders; - -public enum TestMode -{ - UPPER_BOUND(Integer.MAX_VALUE, Long.MAX_VALUE), LOWER_BOUND(Integer.MIN_VALUE, 0), ARBITRARY(237, 237 * (1L << 31)); - - private final int testInt; - private final long testLong; - - TestMode(int i, long l) - { - this.testInt = i; - this.testLong = l; - } - - public int getInt() - { - return testInt; - } - - public long getLong() - { - return testLong; - } - - public int getPositiveUpperBoundedInt(int upper) - { - if (this.name().equals(LOWER_BOUND.name()) || upper <= 0) { - return 0; - } - - if (this.name().equals(UPPER_BOUND.name())) { - return upper; - } - - return ARBITRARY.testInt % upper; - } -} diff --git a/presto-parquet/src/test/java/com/facebook/presto/parquet/batchreader/decoders/TestParquetUtils.java b/presto-parquet/src/test/java/com/facebook/presto/parquet/batchreader/decoders/TestParquetUtils.java index 458e559ce8f8..9d8a6a2818c9 100644 --- a/presto-parquet/src/test/java/com/facebook/presto/parquet/batchreader/decoders/TestParquetUtils.java +++ b/presto-parquet/src/test/java/com/facebook/presto/parquet/batchreader/decoders/TestParquetUtils.java @@ -78,16 +78,16 @@ public static void addDLValues(Iterator values, RunLengthBitPackingHybr } } - public static Iterator fillValues(TestMode testMode, int numValues, int maxValue) + public static Iterator fillValues(int numValues, int maxValue) { List values = new ArrayList<>(); for (int i = 0; i < numValues; i++) { - values.add(testMode.getPositiveUpperBoundedInt(maxValue)); + values.add(maxValue); } return values.iterator(); } - public static byte[] generatePlainValuesPage(int valueCount, int valueSizeBits, TestMode testMode, List addedValues) + public static byte[] generatePlainValuesPage(int valueCount, int valueSizeBits, List addedValues, int valueInt, long valueLong, int positiveUpperBoundedInt) { ValuesWriter writer; @@ -101,9 +101,8 @@ public static byte[] generatePlainValuesPage(int valueCount, int valueSizeBits, switch (valueSizeBits) { case 1: { for (int i = 0; i < valueCount; i++) { - int value = testMode.getPositiveUpperBoundedInt(1); - writer.writeInteger(value); - addedValues.add(value); + writer.writeInteger(positiveUpperBoundedInt); + addedValues.add(positiveUpperBoundedInt); } break; } @@ -118,23 +117,21 @@ public static byte[] generatePlainValuesPage(int valueCount, int valueSizeBits, } case 32: { for (int i = 0; i < valueCount; i++) { - int value = testMode.getInt(); - writer.writeInteger(value); - addedValues.add(value); + writer.writeInteger(valueInt); + addedValues.add(valueInt); } break; } case 64: { for (int i = 0; i < valueCount; i++) { - long value = testMode.getLong(); - writer.writeLong(value); - addedValues.add(value); + writer.writeLong(valueLong); + addedValues.add(valueLong); } break; } case 96: { for (int i = 0; i < valueCount; i++) { - long millisValue = testMode.getPositiveUpperBoundedInt(1572281175) * 1000L; + long millisValue = positiveUpperBoundedInt * 1000L; NanoTime nanoTime = NanoTimeUtils.getNanoTime(new Timestamp(millisValue), false); writer.writeLong(nanoTime.getTimeOfDayNanos()); writer.writeInteger(nanoTime.getJulianDay()); @@ -144,11 +141,10 @@ public static byte[] generatePlainValuesPage(int valueCount, int valueSizeBits, } case 128: for (int i = 0; i < valueCount; i++) { - long value = testMode.getLong(); - writer.writeLong(value); - addedValues.add(value); - writer.writeLong(value); - addedValues.add(value); + writer.writeLong(valueLong); + addedValues.add(valueLong); + writer.writeLong(valueLong); + addedValues.add(valueLong); } break; default: @@ -163,19 +159,19 @@ public static byte[] generatePlainValuesPage(int valueCount, int valueSizeBits, } } - public static byte[] generateDictionaryIdPage2048(int maxValue, TestMode testMode, List addedValues) + public static byte[] generateDictionaryIdPage2048(int maxValue, List addedValues, int fillerValue) { RunLengthBitPackingHybridEncoder encoder = getDictionaryDataPageEncoder(maxValue); addDLRLEBlock(maxValue / 2, 50, encoder, addedValues); - addDLValues(fillValues(testMode, 457, maxValue), encoder, addedValues); + addDLValues(fillValues(457, fillerValue), encoder, addedValues); addDLRLEBlock(0, 37, encoder, addedValues); - addDLValues(fillValues(testMode, 186, maxValue), encoder, addedValues); - addDLValues(fillValues(testMode, 289, maxValue), encoder, addedValues); + addDLValues(fillValues(186, fillerValue), encoder, addedValues); + addDLValues(fillValues(289, fillerValue), encoder, addedValues); addDLRLEBlock(maxValue - 1, 76, encoder, addedValues); - addDLValues(fillValues(testMode, 789, maxValue), encoder, addedValues); + addDLValues(fillValues(789, fillerValue), encoder, addedValues); addDLRLEBlock(maxValue - 1, 137, encoder, addedValues); - addDLValues(fillValues(testMode, 27, maxValue), encoder, addedValues); + addDLValues(fillValues(27, fillerValue), encoder, addedValues); checkState(addedValues.size() == 2048); diff --git a/presto-parquet/src/test/java/com/facebook/presto/parquet/batchreader/decoders/TestValuesDecoders.java b/presto-parquet/src/test/java/com/facebook/presto/parquet/batchreader/decoders/TestValuesDecoders.java index 321910287900..15c7cbd2c80d 100644 --- a/presto-parquet/src/test/java/com/facebook/presto/parquet/batchreader/decoders/TestValuesDecoders.java +++ b/presto-parquet/src/test/java/com/facebook/presto/parquet/batchreader/decoders/TestValuesDecoders.java @@ -44,7 +44,6 @@ import com.facebook.presto.parquet.dictionary.IntegerDictionary; import com.facebook.presto.parquet.dictionary.LongDictionary; import io.airlift.slice.Slices; -import org.testng.annotations.DataProvider; import org.testng.annotations.Test; import java.io.ByteArrayInputStream; @@ -56,16 +55,19 @@ import java.util.stream.Collectors; import static com.facebook.presto.parquet.ParquetEncoding.PLAIN_DICTIONARY; -import static com.facebook.presto.parquet.batchreader.decoders.TestParquetUtils.generateDictionaryIdPage2048; -import static com.facebook.presto.parquet.batchreader.decoders.TestParquetUtils.generatePlainValuesPage; import static com.google.common.collect.ImmutableList.toImmutableList; import static java.lang.Math.min; import static java.nio.charset.StandardCharsets.UTF_8; import static org.apache.parquet.bytes.BytesUtils.getWidthFromMaxInt; import static org.testng.Assert.assertEquals; -public class TestValuesDecoders +public abstract class TestValuesDecoders { + + public abstract byte[] generatePlainValuesPage(int valueCount, int valueSizeBits, List addedValues); + + public abstract byte[] generateDictionaryIdPage2048(int maxValue, List addedValues); + private static Int32ValuesDecoder int32Plain(byte[] pageBytes) { return new Int32PlainValuesDecoder(pageBytes, 0, pageBytes.length); @@ -365,20 +367,14 @@ private static void booleanBatchReadWithSkipHelper(int batchSize, int skipSize, } } - @DataProvider(name = "testModes") - public TestMode[] getTestModes() - { - return TestMode.values(); - } - - @Test(dataProvider = "testModes") - public void testInt32Plain(TestMode testMode) + @Test + public void testInt32Plain() throws IOException { int valueCount = 2048; List expectedValues = new ArrayList<>(); - byte[] pageBytes = generatePlainValuesPage(valueCount, 32, testMode, expectedValues); + byte[] pageBytes = generatePlainValuesPage(valueCount, 32, expectedValues); int32BatchReadWithSkipHelper(valueCount, 0, valueCount, int32Plain(pageBytes), expectedValues); // read all values in one batch int32BatchReadWithSkipHelper(29, 0, valueCount, int32Plain(pageBytes), expectedValues); @@ -390,8 +386,8 @@ public void testInt32Plain(TestMode testMode) int32BatchReadWithSkipHelper(1024, 1024, valueCount, int32Plain(pageBytes), expectedValues); } - @Test(dataProvider = "testModes") - public void testInt32RLEDictionary(TestMode testMode) + @Test + public void testInt32RLEDictionary() throws IOException { int valueCount = 2048; @@ -399,8 +395,8 @@ public void testInt32RLEDictionary(TestMode testMode) List dictionary = new ArrayList<>(); List dictionaryIds = new ArrayList<>(); - byte[] dictionaryPage = generatePlainValuesPage(dictionarySize, 32, testMode, dictionary); - byte[] dataPage = generateDictionaryIdPage2048(dictionarySize - 1, testMode, dictionaryIds); + byte[] dictionaryPage = generatePlainValuesPage(dictionarySize, 32, dictionary); + byte[] dataPage = generateDictionaryIdPage2048(dictionarySize - 1, dictionaryIds); List expectedValues = new ArrayList<>(); for (Integer dictionaryId : dictionaryIds) { @@ -419,14 +415,14 @@ public void testInt32RLEDictionary(TestMode testMode) int32BatchReadWithSkipHelper(1024, 1024, valueCount, int32Dictionary(dataPage, dictionarySize, integerDictionary), expectedValues); } - @Test(dataProvider = "testModes") - public void testBinaryPlain(TestMode testMode) + @Test + public void testBinaryPlain() throws IOException { int valueCount = 2048; List expectedValues = new ArrayList<>(); - byte[] pageBytes = generatePlainValuesPage(valueCount, -1, testMode, expectedValues); + byte[] pageBytes = generatePlainValuesPage(valueCount, -1, expectedValues); binaryBatchReadWithSkipHelper(valueCount, 0, valueCount, binaryPlain(pageBytes), expectedValues); // read all values in one batch binaryBatchReadWithSkipHelper(29, 0, valueCount, binaryPlain(pageBytes), expectedValues); @@ -438,8 +434,8 @@ public void testBinaryPlain(TestMode testMode) binaryBatchReadWithSkipHelper(1024, 1024, valueCount, binaryPlain(pageBytes), expectedValues); } - @Test(dataProvider = "testModes") - public void testBinaryRLEDictionary(TestMode testMode) + @Test + public void testBinaryRLEDictionary() throws IOException { int valueCount = 2048; @@ -447,8 +443,8 @@ public void testBinaryRLEDictionary(TestMode testMode) List dictionary = new ArrayList<>(); List dictionaryIds = new ArrayList<>(); - byte[] dictionaryPage = TestParquetUtils.generatePlainValuesPage(dictionarySize, -1, testMode, dictionary); - byte[] dataPage = TestParquetUtils.generateDictionaryIdPage2048(dictionarySize - 1, testMode, dictionaryIds); + byte[] dictionaryPage = generatePlainValuesPage(dictionarySize, -1, dictionary); + byte[] dataPage = generateDictionaryIdPage2048(dictionarySize - 1, dictionaryIds); List expectedValues = new ArrayList<>(); for (Integer dictionaryId : dictionaryIds) { @@ -467,14 +463,14 @@ public void testBinaryRLEDictionary(TestMode testMode) binaryBatchReadWithSkipHelper(1024, 1024, valueCount, binaryDictionary(dataPage, dictionarySize, binaryDictionary), expectedValues); } - @Test(dataProvider = "testModes") - public void testInt64Plain(TestMode testMode) + @Test + public void testInt64Plain() throws IOException { int valueCount = 2048; List expectedValues = new ArrayList<>(); - byte[] pageBytes = generatePlainValuesPage(valueCount, 64, testMode, expectedValues); + byte[] pageBytes = generatePlainValuesPage(valueCount, 64, expectedValues); int64BatchReadWithSkipHelper(valueCount, 0, valueCount, int64Plain(pageBytes), expectedValues); // read all values in one batch int64BatchReadWithSkipHelper(29, 0, valueCount, int64Plain(pageBytes), expectedValues); @@ -496,8 +492,8 @@ public void testInt64Plain(TestMode testMode) int64BatchReadWithSkipHelper(1024, 1024, valueCount, int64TimestampMicrosPlain(pageBytes), expectedTimestampValues); } - @Test(dataProvider = "testModes") - public void testInt64RLEDictionary(TestMode testMode) + @Test + public void testInt64RLEDictionary() throws IOException { int valueCount = 2048; @@ -505,8 +501,8 @@ public void testInt64RLEDictionary(TestMode testMode) List dictionary = new ArrayList<>(); List dictionaryIds = new ArrayList<>(); - byte[] dictionaryPage = generatePlainValuesPage(dictionarySize, 64, testMode, dictionary); - byte[] dataPage = generateDictionaryIdPage2048(dictionarySize - 1, testMode, dictionaryIds); + byte[] dictionaryPage = generatePlainValuesPage(dictionarySize, 64, dictionary); + byte[] dataPage = generateDictionaryIdPage2048(dictionarySize - 1, dictionaryIds); List expectedValues = new ArrayList<>(); for (Integer dictionaryId : dictionaryIds) { @@ -535,14 +531,14 @@ public void testInt64RLEDictionary(TestMode testMode) int64BatchReadWithSkipHelper(1024, 1024, valueCount, int64TimestampMicrosDictionary(dataPage, dictionarySize, longDictionary), expectedTimestampValues); } - @Test(dataProvider = "testModes") - public void testTimestampPlain(TestMode testMode) + @Test + public void testTimestampPlain() throws IOException { int valueCount = 2048; List expectedValues = new ArrayList<>(); - byte[] pageBytes = generatePlainValuesPage(valueCount, 96, testMode, expectedValues); + byte[] pageBytes = generatePlainValuesPage(valueCount, 96, expectedValues); timestampBatchReadWithSkipHelper(valueCount, 0, valueCount, timestampPlain(pageBytes), expectedValues); // read all values in one batch timestampBatchReadWithSkipHelper(29, 0, valueCount, timestampPlain(pageBytes), expectedValues); @@ -554,8 +550,8 @@ public void testTimestampPlain(TestMode testMode) timestampBatchReadWithSkipHelper(1024, 1024, valueCount, timestampPlain(pageBytes), expectedValues); } - @Test(dataProvider = "testModes") - public void testTimestampRLEDictionary(TestMode testMode) + @Test + public void testTimestampRLEDictionary() throws IOException { int valueCount = 2048; @@ -563,8 +559,8 @@ public void testTimestampRLEDictionary(TestMode testMode) List dictionary = new ArrayList<>(); List dictionaryIds = new ArrayList<>(); - byte[] dictionaryPage = generatePlainValuesPage(dictionarySize, 96, testMode, dictionary); - byte[] dataPage = generateDictionaryIdPage2048(dictionarySize - 1, testMode, dictionaryIds); + byte[] dictionaryPage = generatePlainValuesPage(dictionarySize, 96, dictionary); + byte[] dataPage = generateDictionaryIdPage2048(dictionarySize - 1, dictionaryIds); List expectedValues = new ArrayList<>(); for (Integer dictionaryId : dictionaryIds) { @@ -583,13 +579,13 @@ public void testTimestampRLEDictionary(TestMode testMode) timestampBatchReadWithSkipHelper(1024, 1024, valueCount, timestampDictionary(dataPage, dictionarySize, tsDictionary), expectedValues); } - @Test(dataProvider = "testModes") - public void testBooleanPlain(TestMode testMode) + @Test + public void testBooleanPlain() { int valueCount = 2048; List expectedValues = new ArrayList<>(); - byte[] pageBytes = generatePlainValuesPage(valueCount, 1, testMode, expectedValues); + byte[] pageBytes = generatePlainValuesPage(valueCount, 1, expectedValues); booleanBatchReadWithSkipHelper(valueCount, 0, valueCount, booleanPlain(pageBytes), expectedValues); // read all values in one batch booleanBatchReadWithSkipHelper(29, 0, valueCount, booleanPlain(pageBytes), expectedValues); @@ -601,13 +597,13 @@ public void testBooleanPlain(TestMode testMode) booleanBatchReadWithSkipHelper(1024, 1024, valueCount, booleanPlain(pageBytes), expectedValues); } - @Test(dataProvider = "testModes") - public void testBooleanRLE(TestMode testMode) + @Test + public void testBooleanRLE() { int valueCount = 2048; List values = new ArrayList<>(); - byte[] dataPage = generateDictionaryIdPage2048(1, testMode, values); + byte[] dataPage = generateDictionaryIdPage2048(1, values); List expectedValues = new ArrayList<>(values); @@ -621,14 +617,14 @@ public void testBooleanRLE(TestMode testMode) booleanBatchReadWithSkipHelper(1024, 1024, valueCount, booleanRLE(dataPage), expectedValues); } - @Test(dataProvider = "testModes") - public void testInt32ShortDecimalPlain(TestMode testMode) + @Test + public void testInt32ShortDecimalPlain() throws IOException { int valueCount = 2048; List expectedValues = new ArrayList<>(); - byte[] pageBytes = generatePlainValuesPage(valueCount, 32, testMode, expectedValues); + byte[] pageBytes = generatePlainValuesPage(valueCount, 32, expectedValues); int32ShortDecimalBatchReadWithSkipHelper(valueCount, 0, valueCount, int32ShortDecimalPlain(pageBytes), expectedValues); // read all values in one batch int32ShortDecimalBatchReadWithSkipHelper(29, 0, valueCount, int32ShortDecimalPlain(pageBytes), expectedValues); int32ShortDecimalBatchReadWithSkipHelper(89, 0, valueCount, int32ShortDecimalPlain(pageBytes), expectedValues); @@ -639,14 +635,14 @@ public void testInt32ShortDecimalPlain(TestMode testMode) int32ShortDecimalBatchReadWithSkipHelper(1024, 1024, valueCount, int32ShortDecimalPlain(pageBytes), expectedValues); } - @Test(dataProvider = "testModes") - public void testInt64ShortDecimalPlain(TestMode testMode) + @Test + public void testInt64ShortDecimalPlain() throws IOException { int valueCount = 2048; List expectedValues = new ArrayList<>(); - byte[] pageBytes = generatePlainValuesPage(valueCount, 64, testMode, expectedValues); + byte[] pageBytes = generatePlainValuesPage(valueCount, 64, expectedValues); int64ShortDecimalBatchReadWithSkipHelper(valueCount, 0, valueCount, int64ShortDecimalPlain(pageBytes), expectedValues); // read all values in one batch int64ShortDecimalBatchReadWithSkipHelper(29, 0, valueCount, int64ShortDecimalPlain(pageBytes), expectedValues); int64ShortDecimalBatchReadWithSkipHelper(89, 0, valueCount, int64ShortDecimalPlain(pageBytes), expectedValues); @@ -657,8 +653,8 @@ public void testInt64ShortDecimalPlain(TestMode testMode) int64ShortDecimalBatchReadWithSkipHelper(1024, 1024, valueCount, int64ShortDecimalPlain(pageBytes), expectedValues); } - @Test(dataProvider = "testModes") - public void testInt32ShortDecimalRLE(TestMode testMode) + @Test + public void testInt32ShortDecimalRLE() throws IOException { int valueCount = 2048; @@ -666,8 +662,8 @@ public void testInt32ShortDecimalRLE(TestMode testMode) List dictionary = new ArrayList<>(); List dictionaryIds = new ArrayList<>(); - byte[] dictionaryPage = generatePlainValuesPage(dictionarySize, 32, testMode, dictionary); - byte[] dataPage = generateDictionaryIdPage2048(dictionarySize - 1, testMode, dictionaryIds); + byte[] dictionaryPage = generatePlainValuesPage(dictionarySize, 32, dictionary); + byte[] dataPage = generateDictionaryIdPage2048(dictionarySize - 1, dictionaryIds); List expectedValues = new ArrayList<>(); for (Integer dictionaryId : dictionaryIds) { @@ -686,8 +682,8 @@ public void testInt32ShortDecimalRLE(TestMode testMode) int32ShortDecimalBatchReadWithSkipHelper(1024, 1024, valueCount, int32ShortDecimalRLE(dataPage, dictionarySize, integerDictionary), expectedValues); } - @Test(dataProvider = "testModes") - public void testInt64ShortDecimalRLE(TestMode testMode) + @Test + public void testInt64ShortDecimalRLE() throws IOException { int valueCount = 2048; @@ -695,8 +691,8 @@ public void testInt64ShortDecimalRLE(TestMode testMode) List dictionary = new ArrayList<>(); List dictionaryIds = new ArrayList<>(); - byte[] dictionaryPage = generatePlainValuesPage(dictionarySize, 64, testMode, dictionary); - byte[] dataPage = generateDictionaryIdPage2048(dictionarySize - 1, testMode, dictionaryIds); + byte[] dictionaryPage = generatePlainValuesPage(dictionarySize, 64, dictionary); + byte[] dataPage = generateDictionaryIdPage2048(dictionarySize - 1, dictionaryIds); List expectedValues = new ArrayList<>(); for (Integer dictionaryId : dictionaryIds) { @@ -715,14 +711,14 @@ public void testInt64ShortDecimalRLE(TestMode testMode) int64ShortDecimalBatchReadWithSkipHelper(1024, 1024, valueCount, int64ShortDecimalRLE(dataPage, dictionarySize, longDictionary), expectedValues); } - @Test(dataProvider = "testModes") - public void testUuidPlainPlain(TestMode testMode) + @Test + public void testUuidPlainPlain() throws IOException { int valueCount = 2048; List expectedValues = new ArrayList<>(); - byte[] pageBytes = generatePlainValuesPage(valueCount, 128, testMode, expectedValues); + byte[] pageBytes = generatePlainValuesPage(valueCount, 128, expectedValues); // page is read assuming in big endian, so we need to flip the bytes around when comparing read values expectedValues = expectedValues.stream() .map(Long.class::cast) @@ -740,8 +736,8 @@ public void testUuidPlainPlain(TestMode testMode) uuidBatchReadWithSkipHelper(1024, 1024, valueCount, uuidPlain(pageBytes), expectedValues); } - @Test(dataProvider = "testModes") - public void testUuidRLEDictionary(TestMode testMode) + @Test + public void testUuidRLEDictionary() throws IOException { int valueCount = 2048; @@ -749,8 +745,8 @@ public void testUuidRLEDictionary(TestMode testMode) List dictionary = new ArrayList<>(); List dictionaryIds = new ArrayList<>(); - byte[] dictionaryPage = generatePlainValuesPage(dictionarySize, 128, testMode, dictionary); - byte[] dataPage = generateDictionaryIdPage2048(dictionarySize - 1, testMode, dictionaryIds); + byte[] dictionaryPage = generatePlainValuesPage(dictionarySize, 128, dictionary); + byte[] dataPage = generateDictionaryIdPage2048(dictionarySize - 1, dictionaryIds); List expectedValues = new ArrayList<>(); for (Integer dictionaryId : dictionaryIds) { @@ -775,4 +771,87 @@ public void testUuidRLEDictionary(TestMode testMode) uuidBatchReadWithSkipHelper(89, 29, valueCount, uuidRle(dataPage, dictionarySize, binaryDictionary), expectedValues); uuidBatchReadWithSkipHelper(1024, 1024, valueCount, uuidRle(dataPage, dictionarySize, binaryDictionary), expectedValues); } + + public static class TestValueDecodersArbitrary + extends TestValuesDecoders + { + + public static final int ARBITRARY_VALUE = 237; + + @Override + public byte[] generatePlainValuesPage(int valueCount, int valueSizeBits, List addedValues) + { + int positiveUpperBoundedInt = getPositiveUpperBoundedInt(valueSizeBits); + return TestParquetUtils.generatePlainValuesPage(valueCount, valueSizeBits, addedValues, ARBITRARY_VALUE, ARBITRARY_VALUE * (1L << 31), positiveUpperBoundedInt); + } + + @Override + public byte[] generateDictionaryIdPage2048(int maxValue, List addedValues) + { + return TestParquetUtils.generateDictionaryIdPage2048(maxValue, addedValues, ARBITRARY_VALUE % maxValue); + } + + private int getPositiveUpperBoundedInt(int valueSizeBits) + { + if (valueSizeBits == 1) { + return ARBITRARY_VALUE % 2; + } + if (valueSizeBits == 96) { + return ARBITRARY_VALUE % 1572281176; + } + return ARBITRARY_VALUE; + } + } + + public static class TestValueDecodersLowerBounded + extends TestValuesDecoders + { + + @Override + public byte[] generatePlainValuesPage(int valueCount, int valueSizeBits, List addedValues) + { + return TestParquetUtils.generatePlainValuesPage(valueCount, valueSizeBits, addedValues, Integer.MIN_VALUE, 0L, getPositiveUpperBoundedInt()); + } + + private int getPositiveUpperBoundedInt() + { + return 0; + } + + @Override + public byte[] generateDictionaryIdPage2048(int maxValue, List addedValues) + { + return TestParquetUtils.generateDictionaryIdPage2048(maxValue, addedValues, getPositiveUpperBoundedInt()); + } + } + + public static class TestValueDecodersUpperBounded + extends TestValuesDecoders + { + + @Override + public byte[] generatePlainValuesPage(int valueCount, int valueSizeBits, List addedValues) + { + int positiveUpperBoundedInt = getPositiveUpperBoundedInt(valueSizeBits); + return TestParquetUtils.generatePlainValuesPage(valueCount, valueSizeBits, addedValues, Integer.MAX_VALUE, Long.MAX_VALUE, positiveUpperBoundedInt); + } + + private static int getPositiveUpperBoundedInt(int valueSizeBits) + { + int positiveUpperBoundedInt = Integer.MAX_VALUE; + if (valueSizeBits == 1) { + positiveUpperBoundedInt = 1; + } + if (valueSizeBits == 96) { + positiveUpperBoundedInt = 1572281175; + } + return positiveUpperBoundedInt; + } + + @Override + public byte[] generateDictionaryIdPage2048(int maxValue, List addedValues) + { + return TestParquetUtils.generateDictionaryIdPage2048(maxValue, addedValues, Math.abs(maxValue)); + } + } } From 8cc6878529b925638c01c14d85b169b2e87bdc55 Mon Sep 17 00:00:00 2001 From: ges1227 <24355353+ges1227@users.noreply.github.com> Date: Sun, 24 Nov 2024 14:53:01 +0100 Subject: [PATCH 4/4] Comply with formatting style --- .../decoders/TestValuesDecoders.java | 26 ++++++++----------- 1 file changed, 11 insertions(+), 15 deletions(-) diff --git a/presto-parquet/src/test/java/com/facebook/presto/parquet/batchreader/decoders/TestValuesDecoders.java b/presto-parquet/src/test/java/com/facebook/presto/parquet/batchreader/decoders/TestValuesDecoders.java index 15c7cbd2c80d..e42e9af8987d 100644 --- a/presto-parquet/src/test/java/com/facebook/presto/parquet/batchreader/decoders/TestValuesDecoders.java +++ b/presto-parquet/src/test/java/com/facebook/presto/parquet/batchreader/decoders/TestValuesDecoders.java @@ -63,11 +63,6 @@ public abstract class TestValuesDecoders { - - public abstract byte[] generatePlainValuesPage(int valueCount, int valueSizeBits, List addedValues); - - public abstract byte[] generateDictionaryIdPage2048(int maxValue, List addedValues); - private static Int32ValuesDecoder int32Plain(byte[] pageBytes) { return new Int32PlainValuesDecoder(pageBytes, 0, pageBytes.length); @@ -367,6 +362,10 @@ private static void booleanBatchReadWithSkipHelper(int batchSize, int skipSize, } } + public abstract byte[] generatePlainValuesPage(int valueCount, int valueSizeBits, List addedValues); + + public abstract byte[] generateDictionaryIdPage2048(int maxValue, List addedValues); + @Test public void testInt32Plain() throws IOException @@ -775,7 +774,6 @@ public void testUuidRLEDictionary() public static class TestValueDecodersArbitrary extends TestValuesDecoders { - public static final int ARBITRARY_VALUE = 237; @Override @@ -806,7 +804,6 @@ private int getPositiveUpperBoundedInt(int valueSizeBits) public static class TestValueDecodersLowerBounded extends TestValuesDecoders { - @Override public byte[] generatePlainValuesPage(int valueCount, int valueSizeBits, List addedValues) { @@ -828,14 +825,6 @@ public byte[] generateDictionaryIdPage2048(int maxValue, List addedValu public static class TestValueDecodersUpperBounded extends TestValuesDecoders { - - @Override - public byte[] generatePlainValuesPage(int valueCount, int valueSizeBits, List addedValues) - { - int positiveUpperBoundedInt = getPositiveUpperBoundedInt(valueSizeBits); - return TestParquetUtils.generatePlainValuesPage(valueCount, valueSizeBits, addedValues, Integer.MAX_VALUE, Long.MAX_VALUE, positiveUpperBoundedInt); - } - private static int getPositiveUpperBoundedInt(int valueSizeBits) { int positiveUpperBoundedInt = Integer.MAX_VALUE; @@ -848,6 +837,13 @@ private static int getPositiveUpperBoundedInt(int valueSizeBits) return positiveUpperBoundedInt; } + @Override + public byte[] generatePlainValuesPage(int valueCount, int valueSizeBits, List addedValues) + { + int positiveUpperBoundedInt = getPositiveUpperBoundedInt(valueSizeBits); + return TestParquetUtils.generatePlainValuesPage(valueCount, valueSizeBits, addedValues, Integer.MAX_VALUE, Long.MAX_VALUE, positiveUpperBoundedInt); + } + @Override public byte[] generateDictionaryIdPage2048(int maxValue, List addedValues) {