PARQUET-1794: Random data generation may cause flaky tests (#758)
diff --git a/parquet-column/src/test/java/org/apache/parquet/FixedBinaryTestUtils.java b/parquet-column/src/test/java/org/apache/parquet/FixedBinaryTestUtils.java
new file mode 100644
index 0000000..2c45c81
--- /dev/null
+++ b/parquet-column/src/test/java/org/apache/parquet/FixedBinaryTestUtils.java
@@ -0,0 +1,96 @@
+/*
+ * 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.parquet;
+
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+
+import java.math.BigInteger;
+import java.util.Arrays;
+
+import org.apache.parquet.io.api.Binary;
+import org.apache.parquet.schema.PrimitiveType;
+import org.junit.Test;
+
+/**
+ * Some utility methods for generating Binary values that length if fixed (e.g. for FIXED_LEN_BYTE_ARRAY or INT96).
+ */
+public class FixedBinaryTestUtils {
+ public static Binary getFixedBinary(int length, BigInteger bigInt) {
+ byte[] array = bigInt.toByteArray();
+ if (array.length == length) {
+ return Binary.fromConstantByteArray(array);
+ } else if (array.length < length) {
+ byte[] padded = new byte[length];
+ int paddingLength = length - array.length;
+ if (bigInt.signum() < 0) {
+ Arrays.fill(padded, 0, paddingLength, (byte) 0xFF);
+ } else {
+ Arrays.fill(padded, 0, paddingLength, (byte) 0x00);
+ }
+ System.arraycopy(array, 0, padded, paddingLength, array.length);
+ return Binary.fromConstantByteArray(padded);
+ } else {
+ throw new IllegalArgumentException(
+ "Specified BigInteger (" + bigInt + ") is too long for fixed bytes (" + array.length + '>' + length + ')');
+ }
+ }
+
+ public static Binary getFixedBinary(PrimitiveType type, BigInteger bigInt) {
+ switch (type.getPrimitiveTypeName()) {
+ case FIXED_LEN_BYTE_ARRAY:
+ return getFixedBinary(type.getTypeLength(), bigInt);
+ case INT96:
+ return getFixedBinary(12, bigInt);
+ case BINARY:
+ return Binary.fromConstantByteArray(bigInt.toByteArray());
+ default:
+ throw new IllegalArgumentException("Type " + type + " cannot be represented by a Binary");
+ }
+ }
+
+ @Test
+ public void testGetFixedBinary() {
+ assertArrayEquals(b(0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0x80, 0x00, 0x00, 0x00),
+ getFixedBinary(10, BigInteger.valueOf(Integer.MIN_VALUE)).getBytes());
+ assertArrayEquals(b(0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF),
+ getFixedBinary(11, BigInteger.valueOf(-1)).getBytes());
+ assertArrayEquals(b(0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00),
+ getFixedBinary(12, BigInteger.valueOf(0)).getBytes());
+ assertArrayEquals(b(0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x01),
+ getFixedBinary(13, BigInteger.valueOf(1)).getBytes());
+ assertArrayEquals(b(0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x7F, 0xFF, 0xFF, 0xFF),
+ getFixedBinary(14, BigInteger.valueOf(Integer.MAX_VALUE)).getBytes());
+ }
+
+ public void assertCorrectBytes(byte[] expectedBytes, int length, BigInteger bigInt) {
+ byte[] actualBytes = getFixedBinary(length, bigInt).getBytes();
+ assertArrayEquals(expectedBytes, actualBytes);
+ assertEquals(bigInt, new BigInteger(actualBytes));
+ }
+
+ private static byte[] b(int... bytes) {
+ byte[] arr = new byte[bytes.length];
+ for (int i = 0, n = arr.length; i < n; ++i) {
+ arr[i] = (byte) bytes[i];
+ }
+ return arr;
+ }
+}
diff --git a/parquet-hadoop/pom.xml b/parquet-hadoop/pom.xml
index c493941..e047f03 100644
--- a/parquet-hadoop/pom.xml
+++ b/parquet-hadoop/pom.xml
@@ -47,6 +47,13 @@
<version>${project.version}</version>
</dependency>
<dependency>
+ <groupId>org.apache.parquet</groupId>
+ <artifactId>parquet-column</artifactId>
+ <version>${project.version}</version>
+ <type>test-jar</type>
+ <scope>test</scope>
+ </dependency>
+ <dependency>
<groupId>org.apache.hadoop</groupId>
<artifactId>hadoop-client</artifactId>
<version>${hadoop.version}</version>
diff --git a/parquet-hadoop/src/test/java/org/apache/parquet/filter2/dictionarylevel/DictionaryFilterTest.java b/parquet-hadoop/src/test/java/org/apache/parquet/filter2/dictionarylevel/DictionaryFilterTest.java
index 6af4437..3625ed0 100644
--- a/parquet-hadoop/src/test/java/org/apache/parquet/filter2/dictionarylevel/DictionaryFilterTest.java
+++ b/parquet-hadoop/src/test/java/org/apache/parquet/filter2/dictionarylevel/DictionaryFilterTest.java
@@ -25,6 +25,7 @@
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
+import org.apache.parquet.FixedBinaryTestUtils;
import org.apache.parquet.column.Encoding;
import org.apache.parquet.column.EncodingStats;
import org.apache.parquet.column.ParquetProperties.WriterVersion;
@@ -140,13 +141,7 @@
}
private static Binary toBinary(BigInteger decimalWithoutScale, int byteCount) {
- byte[] src = decimalWithoutScale.toByteArray();
- if (src.length > byteCount) {
- throw new IllegalArgumentException("Too large decimal value for byte count " + byteCount);
- }
- byte[] dest = new byte[byteCount];
- System.arraycopy(src, 0, dest, dest.length - src.length, src.length);
- return Binary.fromConstantByteArray(dest);
+ return FixedBinaryTestUtils.getFixedBinary(byteCount, decimalWithoutScale);
}
private static void writeData(SimpleGroupFactory f, ParquetWriter<Group> writer) throws IOException {
diff --git a/parquet-hadoop/src/test/java/org/apache/parquet/format/converter/TestParquetMetadataConverter.java b/parquet-hadoop/src/test/java/org/apache/parquet/format/converter/TestParquetMetadataConverter.java
index e510b60..256ea36 100644
--- a/parquet-hadoop/src/test/java/org/apache/parquet/format/converter/TestParquetMetadataConverter.java
+++ b/parquet-hadoop/src/test/java/org/apache/parquet/format/converter/TestParquetMetadataConverter.java
@@ -68,6 +68,7 @@
import com.google.common.collect.Sets;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.io.UTF8;
+import org.apache.parquet.FixedBinaryTestUtils;
import org.apache.parquet.Version;
import org.apache.parquet.bytes.BytesUtils;
import org.apache.parquet.column.ColumnDescriptor;
@@ -1043,8 +1044,8 @@
private static Statistics<?> createStatsTyped(PrimitiveType type, BigInteger min, BigInteger max) {
Statistics<?> stats = Statistics.createStats(type);
- Binary minBinary = Binary.fromConstantByteArray(min.toByteArray());
- Binary maxBinary = Binary.fromConstantByteArray(max.toByteArray());
+ Binary minBinary = FixedBinaryTestUtils.getFixedBinary(type, min);
+ Binary maxBinary = FixedBinaryTestUtils.getFixedBinary(type, max);
stats.updateStats(maxBinary);
stats.updateStats(minBinary);
assertEquals(minBinary, stats.genericGetMin());
diff --git a/parquet-hadoop/src/test/java/org/apache/parquet/statistics/RandomValues.java b/parquet-hadoop/src/test/java/org/apache/parquet/statistics/RandomValues.java
index 152f6ec..191e397 100644
--- a/parquet-hadoop/src/test/java/org/apache/parquet/statistics/RandomValues.java
+++ b/parquet-hadoop/src/test/java/org/apache/parquet/statistics/RandomValues.java
@@ -27,6 +27,7 @@
import java.util.Random;
import java.util.function.Supplier;
+import org.apache.parquet.FixedBinaryTestUtils;
import org.apache.parquet.io.api.Binary;
public class RandomValues {
@@ -213,7 +214,7 @@
@Override
public Binary nextBinaryValue() {
- return asReusedBinary(nextValue().toByteArray());
+ return FixedBinaryTestUtils.getFixedBinary(INT_96_LENGTH, nextValue());
}
}