This is an automated email from the ASF dual-hosted git repository.
shangxinli pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/parquet-mr.git
The following commit(s) were added to refs/heads/master by this push:
new 7398d9b PARQUET-2105: Refactor the test code of creating the test
file (#939)
7398d9b is described below
commit 7398d9b522733c669d497c25495c9efa1c860994
Author: Xinli Shang <[email protected]>
AuthorDate: Thu Dec 16 15:01:35 2021 -0800
PARQUET-2105: Refactor the test code of creating the test file (#939)
---
.../parquet/hadoop/util/ColumnEncryptorTest.java | 102 ++++++-----
.../parquet/hadoop/util/EncDecProperties.java | 27 ++-
.../parquet/hadoop/util/EncryptionTestFile.java | 38 ++++
.../parquet/hadoop/util/TestFileBuilder.java | 198 +++++++++++++++++++++
.../apache/parquet/hadoop/util/TestFileHelper.java | 153 ----------------
5 files changed, 309 insertions(+), 209 deletions(-)
diff --git
a/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/util/ColumnEncryptorTest.java
b/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/util/ColumnEncryptorTest.java
index 670f6e2..978a1d9 100644
---
a/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/util/ColumnEncryptorTest.java
+++
b/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/util/ColumnEncryptorTest.java
@@ -18,7 +18,6 @@
*/
package org.apache.parquet.hadoop.util;
-import com.google.common.collect.ImmutableMap;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
import org.apache.parquet.HadoopReadOptions;
@@ -37,19 +36,27 @@ import org.apache.parquet.hadoop.example.GroupReadSupport;
import org.apache.parquet.hadoop.metadata.ColumnChunkMetaData;
import org.apache.parquet.hadoop.metadata.ParquetMetadata;
import
org.apache.parquet.hadoop.util.CompressionConverter.TransParquetFileReader;
+import org.apache.parquet.schema.GroupType;
+import org.apache.parquet.schema.MessageType;
+import org.apache.parquet.schema.PrimitiveType;
+
import org.apache.parquet.internal.column.columnindex.OffsetIndex;
import org.apache.parquet.io.InputFile;
import org.apache.parquet.io.SeekableInputStream;
-import org.junit.Before;
import org.junit.Test;
import java.io.IOException;
+import java.nio.charset.StandardCharsets;
import java.util.Arrays;
import java.util.HashSet;
import java.util.List;
-import java.util.Map;
import java.util.Set;
+import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.BINARY;
+import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.INT64;
+import static org.apache.parquet.schema.Type.Repetition.OPTIONAL;
+import static org.apache.parquet.schema.Type.Repetition.REPEATED;
+import static org.apache.parquet.schema.Type.Repetition.REQUIRED;
import static org.junit.Assert.assertArrayEquals;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
@@ -58,63 +65,63 @@ import static org.junit.Assert.assertTrue;
public class ColumnEncryptorTest {
private Configuration conf = new Configuration();
- private Map<String, String> extraMeta = ImmutableMap.of("key1", "value1",
"key2", "value2");
private ColumnEncryptor columnEncryptor = null;
private final int numRecord = 100000;
- private String inputFile = null;
+ private EncryptionTestFile inputFile = null;
private String outputFile = null;
- private TestFileHelper.TestDocs testDocs = null;
private void testSetup(String compression) throws IOException {
+ MessageType schema = createSchema();
columnEncryptor = new ColumnEncryptor(conf);
- testDocs = new TestFileHelper.TestDocs(numRecord);
- inputFile = TestFileHelper.createParquetFile(conf, extraMeta, numRecord,
"input", compression,
- ParquetProperties.WriterVersion.PARQUET_1_0,
ParquetProperties.DEFAULT_PAGE_SIZE, testDocs);
- outputFile = TestFileHelper.createTempFile("test");
+ inputFile = new TestFileBuilder(conf, schema)
+ .withNumRecord(numRecord)
+ .withCodec(compression)
+ .withPageSize(ParquetProperties.DEFAULT_PAGE_SIZE)
+ .build();
+ outputFile = TestFileBuilder.createTempFile("test");
}
@Test
public void testFlatColumn() throws IOException {
- testSetup("GZIP");
String[] encryptColumns = {"DocId"};
- columnEncryptor.encryptColumns(inputFile, outputFile,
Arrays.asList(encryptColumns),
+ testSetup("GZIP");
+ columnEncryptor.encryptColumns(inputFile.getFileName(), outputFile,
Arrays.asList(encryptColumns),
EncDecProperties.getFileEncryptionProperties(encryptColumns,
ParquetCipher.AES_GCM_CTR_V1, false));
-
verifyResultDecryptionWithValidKey();
}
@Test
public void testNestedColumn() throws IOException {
- testSetup("GZIP");
String[] encryptColumns = {"Links.Forward"};
- columnEncryptor.encryptColumns(inputFile, outputFile,
Arrays.asList(encryptColumns),
+ testSetup("GZIP");
+ columnEncryptor.encryptColumns(inputFile.getFileName(), outputFile,
Arrays.asList(encryptColumns),
EncDecProperties.getFileEncryptionProperties(encryptColumns,
ParquetCipher.AES_GCM_CTR_V1, false));
verifyResultDecryptionWithValidKey();
}
@Test
public void testNoEncryption() throws IOException {
- testSetup("GZIP");
String[] encryptColumns = {};
- columnEncryptor.encryptColumns(inputFile, outputFile,
Arrays.asList(encryptColumns),
+ testSetup("GZIP");
+ columnEncryptor.encryptColumns(inputFile.getFileName(), outputFile,
Arrays.asList(encryptColumns),
EncDecProperties.getFileEncryptionProperties(encryptColumns,
ParquetCipher.AES_GCM_CTR_V1, false));
verifyResultDecryptionWithValidKey();
}
@Test
public void testEncryptAllColumns() throws IOException {
- testSetup("GZIP");
String[] encryptColumns = {"DocId", "Name", "Gender", "Links.Forward",
"Links.Backward"};
- columnEncryptor.encryptColumns(inputFile, outputFile,
Arrays.asList(encryptColumns),
+ testSetup("GZIP");
+ columnEncryptor.encryptColumns(inputFile.getFileName(), outputFile,
Arrays.asList(encryptColumns),
EncDecProperties.getFileEncryptionProperties(encryptColumns,
ParquetCipher.AES_GCM_CTR_V1, false));
verifyResultDecryptionWithValidKey();
}
@Test
public void testEncryptSomeColumns() throws IOException {
- testSetup("GZIP");
String[] encryptColumns = {"DocId", "Name", "Links.Forward"};
- columnEncryptor.encryptColumns(inputFile, outputFile,
Arrays.asList(encryptColumns),
+ testSetup("GZIP");
+ columnEncryptor.encryptColumns(inputFile.getFileName(), outputFile,
Arrays.asList(encryptColumns),
EncDecProperties.getFileEncryptionProperties(encryptColumns,
ParquetCipher.AES_GCM_CTR_V1, false));
ParquetMetadata metaData =
getParquetMetadata(EncDecProperties.getFileDecryptionProperties());
@@ -132,9 +139,9 @@ public class ColumnEncryptorTest {
@Test
public void testFooterEncryption() throws IOException {
- testSetup("GZIP");
String[] encryptColumns = {"DocId"};
- columnEncryptor.encryptColumns(inputFile, outputFile,
Arrays.asList(encryptColumns),
+ testSetup("GZIP");
+ columnEncryptor.encryptColumns(inputFile.getFileName(), outputFile,
Arrays.asList(encryptColumns),
EncDecProperties.getFileEncryptionProperties(encryptColumns,
ParquetCipher.AES_GCM_CTR_V1, true));
verifyResultDecryptionWithValidKey();
@@ -142,9 +149,9 @@ public class ColumnEncryptorTest {
@Test
public void testAesGcm() throws IOException {
- testSetup("GZIP");
String[] encryptColumns = {"DocId"};
- columnEncryptor.encryptColumns(inputFile, outputFile,
Arrays.asList(encryptColumns),
+ testSetup("GZIP");
+ columnEncryptor.encryptColumns(inputFile.getFileName(), outputFile,
Arrays.asList(encryptColumns),
EncDecProperties.getFileEncryptionProperties(encryptColumns,
ParquetCipher.AES_GCM_V1, true));
verifyResultDecryptionWithValidKey();
@@ -152,9 +159,9 @@ public class ColumnEncryptorTest {
@Test
public void testColumnIndex() throws IOException {
- testSetup("GZIP");
String[] encryptColumns = {"Name"};
- columnEncryptor.encryptColumns(inputFile, outputFile,
Arrays.asList(encryptColumns),
+ testSetup("GZIP");
+ columnEncryptor.encryptColumns(inputFile.getFileName(), outputFile,
Arrays.asList(encryptColumns),
EncDecProperties.getFileEncryptionProperties(encryptColumns,
ParquetCipher.AES_GCM_V1, false));
verifyResultDecryptionWithValidKey();
@@ -163,27 +170,32 @@ public class ColumnEncryptorTest {
@Test
public void testDifferentCompression() throws IOException {
+ String[] encryptColumns = {"Links.Forward"};
String[] compressions = {"GZIP", "ZSTD", "SNAPPY", "UNCOMPRESSED"};
for (String compression : compressions) {
testSetup(compression);
+ columnEncryptor.encryptColumns(inputFile.getFileName(), outputFile,
Arrays.asList(encryptColumns),
+ EncDecProperties.getFileEncryptionProperties(encryptColumns,
ParquetCipher.AES_GCM_CTR_V1, false));
+ verifyResultDecryptionWithValidKey();
}
- String[] encryptColumns = {"Links.Forward"};
- columnEncryptor.encryptColumns(inputFile, outputFile,
Arrays.asList(encryptColumns),
- EncDecProperties.getFileEncryptionProperties(encryptColumns,
ParquetCipher.AES_GCM_CTR_V1, false));
- verifyResultDecryptionWithValidKey();
-
}
private void verifyResultDecryptionWithValidKey() throws IOException {
ParquetReader<Group> reader = createReader(outputFile);
for (int i = 0; i < numRecord; i++) {
Group group = reader.read();
- assertTrue(group.getLong("DocId", 0) == testDocs.docId[i]);
- assertArrayEquals(group.getBinary("Name", 0).getBytes(),
testDocs.name[i].getBytes());
- assertArrayEquals(group.getBinary("Gender", 0).getBytes(),
testDocs.gender[i].getBytes());
- Group subGroup = group.getGroup("Links", 0);
- assertArrayEquals(subGroup.getBinary("Forward", 0).getBytes(),
testDocs.linkForward[i].getBytes());
- assertArrayEquals(subGroup.getBinary("Backward", 0).getBytes(),
testDocs.linkBackward[i].getBytes());
+ assertTrue(group.getLong("DocId", 0) ==
+ inputFile.getFileContent()[i].getLong("DocId", 0));
+ assertArrayEquals(group.getBinary("Name", 0).getBytes(),
+ inputFile.getFileContent()[i].getString("Name",
0).getBytes(StandardCharsets.UTF_8));
+ assertArrayEquals(group.getBinary("Gender", 0).getBytes(),
+ inputFile.getFileContent()[i].getString("Gender",
0).getBytes(StandardCharsets.UTF_8));
+ Group subGroupInRead = group.getGroup("Links", 0);
+ Group expectedSubGroup = inputFile.getFileContent()[i].getGroup("Links",
0);
+ assertArrayEquals(subGroupInRead.getBinary("Forward", 0).getBytes(),
+ expectedSubGroup.getBinary("Forward", 0).getBytes());
+ assertArrayEquals(subGroupInRead.getBinary("Backward", 0).getBytes(),
+ expectedSubGroup.getBinary("Backward", 0).getBytes());
}
reader.close();
}
@@ -193,9 +205,9 @@ public class ColumnEncryptorTest {
.withDecryption(EncDecProperties.getFileDecryptionProperties())
.build();
- try (TransParquetFileReader inReader = createFileReader(inputFile);
+ try (TransParquetFileReader inReader =
createFileReader(inputFile.getFileName());
TransParquetFileReader outReader = createFileReader(outputFile)) {
- ParquetMetadata inMetaData = getMetadata(readOptions, inputFile,
inReader);
+ ParquetMetadata inMetaData = getMetadata(readOptions,
inputFile.getFileName(), inReader);
ParquetMetadata outMetaData = getMetadata(readOptions, outputFile,
outReader);
compareOffsetIndexes(inReader, outReader, inMetaData, outMetaData);
}
@@ -290,4 +302,14 @@ public class ColumnEncryptorTest {
}
return metaData;
}
+
+ private MessageType createSchema() {
+ return new MessageType("schema",
+ new PrimitiveType(OPTIONAL, INT64, "DocId"),
+ new PrimitiveType(REQUIRED, BINARY, "Name"),
+ new PrimitiveType(OPTIONAL, BINARY, "Gender"),
+ new GroupType(OPTIONAL, "Links",
+ new PrimitiveType(REPEATED, BINARY, "Backward"),
+ new PrimitiveType(REPEATED, BINARY, "Forward")));
+ }
}
diff --git
a/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/util/EncDecProperties.java
b/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/util/EncDecProperties.java
index ac877a2..fc220c1 100644
---
a/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/util/EncDecProperties.java
+++
b/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/util/EncDecProperties.java
@@ -64,24 +64,19 @@ public class EncDecProperties {
return
FileDecryptionProperties.builder().withPlaintextFilesAllowed().withKeyRetriever(keyRetriever).build();
}
- public static FileEncryptionProperties getFileEncryptionProperties(String[]
encrCols, ParquetCipher cipher, Boolean encryptFooter) {
- if (encrCols.length == 0) {
+ public static FileEncryptionProperties getFileEncryptionProperties(String[]
encryptColumns, ParquetCipher cipher, Boolean encryptFooter) {
+ if (encryptColumns.length == 0) {
return null;
}
- Map<ColumnPath, ColumnEncryptionProperties> columnPropertyMap = new
HashMap<>();
- Set<String> paths = new HashSet<>(Arrays.asList(encrCols));
- for (ColumnPath columnPath : TestFileHelper.getPaths()) {
- ColumnEncryptionProperties colEncProp;
- if (paths.contains(columnPath.toDotString())) {
- colEncProp = ColumnEncryptionProperties.builder(columnPath, true)
- .withKey(COL_KEY)
- .withKeyMetaData(COL_KEY_METADATA)
- .build();
- } else {
- colEncProp = ColumnEncryptionProperties.builder(columnPath,
false).build();
- }
- columnPropertyMap.put(columnPath, colEncProp);
+ Map<ColumnPath, ColumnEncryptionProperties> columnPropertyMap = new
HashMap<>();
+ for (String encryptColumn : encryptColumns) {
+ ColumnPath columnPath = ColumnPath.fromDotString(encryptColumn);
+ ColumnEncryptionProperties columnEncryptionProperties =
ColumnEncryptionProperties.builder(columnPath)
+ .withKey(COL_KEY)
+ .withKeyMetaData(COL_KEY_METADATA)
+ .build();
+ columnPropertyMap.put(columnPath, columnEncryptionProperties);
}
FileEncryptionProperties.Builder encryptionPropertiesBuilder =
@@ -90,7 +85,7 @@ public class EncDecProperties {
.withAlgorithm(cipher)
.withEncryptedColumns(columnPropertyMap);
- if(!encryptFooter) {
+ if (!encryptFooter) {
encryptionPropertiesBuilder.withPlaintextFooter();
}
diff --git
a/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/util/EncryptionTestFile.java
b/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/util/EncryptionTestFile.java
new file mode 100644
index 0000000..c65090d
--- /dev/null
+++
b/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/util/EncryptionTestFile.java
@@ -0,0 +1,38 @@
+/*
+ * 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 org.apache.parquet.hadoop.util;
+
+import org.apache.parquet.example.data.simple.SimpleGroup;
+
+public class EncryptionTestFile
+{
+ private final String fileName;
+ private final SimpleGroup[] fileContent;
+
+ public EncryptionTestFile(String fileName, SimpleGroup[] fileContent)
+ {
+ this.fileName = fileName;
+ this.fileContent = fileContent;
+ }
+
+ public String getFileName()
+ {
+ return this.fileName;
+ }
+
+ public SimpleGroup[] getFileContent()
+ {
+ return this.fileContent;
+ }
+}
diff --git
a/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/util/TestFileBuilder.java
b/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/util/TestFileBuilder.java
new file mode 100644
index 0000000..5018beb
--- /dev/null
+++
b/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/util/TestFileBuilder.java
@@ -0,0 +1,198 @@
+/*
+ * 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 org.apache.parquet.hadoop.util;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.parquet.column.ParquetProperties;
+import org.apache.parquet.crypto.FileEncryptionProperties;
+import org.apache.parquet.crypto.ParquetCipher;
+import org.apache.parquet.example.data.Group;
+import org.apache.parquet.example.data.simple.SimpleGroup;
+import org.apache.parquet.hadoop.ParquetWriter;
+import org.apache.parquet.hadoop.example.ExampleParquetWriter;
+import org.apache.parquet.hadoop.example.GroupWriteSupport;
+import org.apache.parquet.hadoop.metadata.CompressionCodecName;
+import org.apache.parquet.schema.GroupType;
+import org.apache.parquet.schema.MessageType;
+import org.apache.parquet.schema.PrimitiveType;
+import org.apache.parquet.schema.Type;
+
+import java.io.IOException;
+import java.nio.file.Files;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.ThreadLocalRandom;
+
+import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.BINARY;
+import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.INT32;
+import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.INT64;
+
+public class TestFileBuilder
+{
+ private MessageType schema;
+ private Configuration conf;
+ private Map<String, String> extraMeta = new HashMap<>();
+ private int numRecord = 100000;
+ private ParquetProperties.WriterVersion writerVersion =
ParquetProperties.WriterVersion.PARQUET_1_0;
+ private int pageSize = ParquetProperties.DEFAULT_PAGE_SIZE;
+ private String codec = "ZSTD";
+ private String[] encryptColumns = {};
+ private ParquetCipher cipher = ParquetCipher.AES_GCM_V1;
+ private Boolean footerEncryption = false;
+
+ public TestFileBuilder(Configuration conf, MessageType schema)
+ {
+ this.conf = conf;
+ this.schema = schema;
+ conf.set(GroupWriteSupport.PARQUET_EXAMPLE_SCHEMA, schema.toString());
+ }
+
+ public TestFileBuilder withNumRecord(int numRecord)
+ {
+ this.numRecord = numRecord;
+ return this;
+ }
+
+ public TestFileBuilder withEncrytionAlgorithm(ParquetCipher cipher)
+ {
+ this.cipher = cipher;
+ return this;
+ }
+
+ public TestFileBuilder withExtraMeta(Map<String, String> extraMeta)
+ {
+ this.extraMeta = extraMeta;
+ return this;
+ }
+
+ public TestFileBuilder withWriterVersion(ParquetProperties.WriterVersion
writerVersion)
+ {
+ this.writerVersion = writerVersion;
+ return this;
+ }
+
+ public TestFileBuilder withPageSize(int pageSize)
+ {
+ this.pageSize = pageSize;
+ return this;
+ }
+
+ public TestFileBuilder withCodec(String codec)
+ {
+ this.codec = codec;
+ return this;
+ }
+
+ public TestFileBuilder withEncryptColumns(String[] encryptColumns)
+ {
+ this.encryptColumns = encryptColumns;
+ return this;
+ }
+
+ public TestFileBuilder withFooterEncryption()
+ {
+ this.footerEncryption = true;
+ return this;
+ }
+
+ public EncryptionTestFile build()
+ throws IOException
+ {
+ String fileName = createTempFile("test");
+ SimpleGroup[] fileContent = createFileContent(schema);
+ FileEncryptionProperties encryptionProperties =
EncDecProperties.getFileEncryptionProperties(encryptColumns, cipher,
footerEncryption);
+ ExampleParquetWriter.Builder builder =
ExampleParquetWriter.builder(new Path(fileName))
+ .withConf(conf)
+ .withWriterVersion(writerVersion)
+ .withExtraMetaData(extraMeta)
+ .withValidation(true)
+ .withPageSize(pageSize)
+ .withEncryption(encryptionProperties)
+ .withCompressionCodec(CompressionCodecName.valueOf(codec));
+ try (ParquetWriter writer = builder.build()) {
+ for (int i = 0; i < fileContent.length; i++) {
+ writer.write(fileContent[i]);
+ }
+ }
+ return new EncryptionTestFile(fileName, fileContent);
+ }
+
+ private SimpleGroup[] createFileContent(MessageType schema)
+ {
+ SimpleGroup[] simpleGroups = new SimpleGroup[numRecord];
+ for (int i = 0; i < simpleGroups.length; i++) {
+ SimpleGroup g = new SimpleGroup(schema);
+ for (Type type : schema.getFields()) {
+ addValueToSimpleGroup(g, type);
+ }
+ simpleGroups[i] = g;
+ }
+ return simpleGroups;
+ }
+
+ private void addValueToSimpleGroup(Group g, Type type)
+ {
+ if (type.isPrimitive()) {
+ PrimitiveType primitiveType = (PrimitiveType) type;
+ if (primitiveType.getPrimitiveTypeName().equals(INT32)) {
+ g.add(type.getName(), getInt());
+ }
+ else if (primitiveType.getPrimitiveTypeName().equals(INT64)) {
+ g.add(type.getName(), getLong());
+ }
+ else if (primitiveType.getPrimitiveTypeName().equals(BINARY)) {
+ g.add(type.getName(), getString());
+ }
+ // Only support 3 types now, more can be added later
+ }
+ else {
+ GroupType groupType = (GroupType) type;
+ Group parentGroup = g.addGroup(groupType.getName());
+ for (Type field : groupType.getFields()) {
+ addValueToSimpleGroup(parentGroup, field);
+ }
+ }
+ }
+
+ private static long getInt()
+ {
+ return ThreadLocalRandom.current().nextInt(10000);
+ }
+
+ private static long getLong()
+ {
+ return ThreadLocalRandom.current().nextLong(100000);
+ }
+
+ private static String getString()
+ {
+ char[] chars = {'a', 'b', 'c', 'd', 'e', 'f', 'g', 'x', 'z', 'y'};
+ StringBuilder sb = new StringBuilder();
+ for (int i = 0; i < 100; i++) {
+ sb.append(chars[ThreadLocalRandom.current().nextInt(10)]);
+ }
+ return sb.toString();
+ }
+
+ public static String createTempFile(String prefix)
+ {
+ try {
+ return
Files.createTempDirectory(prefix).toAbsolutePath().toString() + "/test.parquet";
+ }
+ catch (IOException e) {
+ throw new AssertionError("Unable to create temporary file", e);
+ }
+ }
+}
diff --git
a/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/util/TestFileHelper.java
b/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/util/TestFileHelper.java
deleted file mode 100644
index f661035..0000000
---
a/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/util/TestFileHelper.java
+++ /dev/null
@@ -1,153 +0,0 @@
-/*
- * 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.hadoop.util;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.Path;
-import org.apache.parquet.column.ParquetProperties;
-import org.apache.parquet.example.data.Group;
-import org.apache.parquet.example.data.simple.SimpleGroup;
-import org.apache.parquet.hadoop.ParquetWriter;
-import org.apache.parquet.hadoop.example.ExampleParquetWriter;
-import org.apache.parquet.hadoop.example.GroupWriteSupport;
-import org.apache.parquet.hadoop.metadata.ColumnPath;
-import org.apache.parquet.hadoop.metadata.CompressionCodecName;
-import org.apache.parquet.schema.GroupType;
-import org.apache.parquet.schema.MessageType;
-import org.apache.parquet.schema.PrimitiveType;
-
-import java.io.IOException;
-import java.nio.file.Files;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Map;
-import java.util.Random;
-import java.util.concurrent.ThreadLocalRandom;
-
-import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.BINARY;
-import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.INT64;
-import static org.apache.parquet.schema.Type.Repetition.OPTIONAL;
-import static org.apache.parquet.schema.Type.Repetition.REPEATED;
-import static org.apache.parquet.schema.Type.Repetition.REQUIRED;
-
-public class TestFileHelper {
-
- public static MessageType schema = new MessageType("schema",
- new PrimitiveType(OPTIONAL, INT64, "DocId"),
- new PrimitiveType(REQUIRED, BINARY, "Name"),
- new PrimitiveType(OPTIONAL, BINARY, "Gender"),
- new GroupType(OPTIONAL, "Links",
- new PrimitiveType(REPEATED, BINARY, "Backward"),
- new PrimitiveType(REPEATED, BINARY, "Forward")));
-
- private static Random rnd = new Random(5);
-
- public static String createParquetFile(Configuration conf, Map<String,
String> extraMeta, int numRecord, String prefix, String codec,
- ParquetProperties.WriterVersion
writerVersion, int pageSize, TestDocs testDocs) throws IOException {
- conf.set(GroupWriteSupport.PARQUET_EXAMPLE_SCHEMA, schema.toString());
-
- String file = createTempFile(prefix);
- ExampleParquetWriter.Builder builder = ExampleParquetWriter.builder(new
Path(file))
- .withConf(conf)
- .withWriterVersion(writerVersion)
- .withExtraMetaData(extraMeta)
- .withValidation(true)
- .withPageSize(pageSize)
- .withCompressionCodec(CompressionCodecName.valueOf(codec));
- try (ParquetWriter writer = builder.build()) {
- for (int i = 0; i < numRecord; i++) {
- SimpleGroup g = new SimpleGroup(schema);
- g.add("DocId", testDocs.docId[i]);
- g.add("Name", testDocs.name[i]);
- g.add("Gender", testDocs.gender[i]);
- Group links = g.addGroup("Links");
- links.add(0, testDocs.linkBackward[i]);
- links.add(1, testDocs.linkForward[i]);
- writer.write(g);
- }
- }
- return file;
- }
-
- private static long getLong() {
- return ThreadLocalRandom.current().nextLong(1000);
- }
-
- private static String getString() {
- char[] chars = {'a', 'b', 'c', 'd', 'e', 'f', 'g', 'x', 'z', 'y'};
- StringBuilder sb = new StringBuilder();
- for (int i = 0; i < 100; i++) {
- sb.append(chars[rnd.nextInt(10)]);
- }
- return sb.toString();
- }
-
- public static String createTempFile(String prefix) {
- try {
- return Files.createTempDirectory(prefix).toAbsolutePath().toString() +
"/test.parquet";
- } catch (IOException e) {
- throw new AssertionError("Unable to create temporary file", e);
- }
- }
-
- public static List<ColumnPath> getPaths() {
- List<ColumnPath> paths = new ArrayList<>();
- paths.add(ColumnPath.fromDotString("DocId"));
- paths.add(ColumnPath.fromDotString("Name"));
- paths.add(ColumnPath.fromDotString("Gender"));
- paths.add(ColumnPath.fromDotString("Links.Backward"));
- paths.add(ColumnPath.fromDotString("Links.Forward"));
- return paths;
- }
-
- public static class TestDocs {
- public long[] docId;
- public String[] name;
- public String[] gender;
- public String[] linkBackward;
- public String[] linkForward;
-
- public TestDocs(int numRecord) {
- docId = new long[numRecord];
- for (int i = 0; i < numRecord; i++) {
- docId[i] = getLong();
- }
-
- name = new String[numRecord];
- for (int i = 0; i < numRecord; i++) {
- name[i] = getString();
- }
-
- gender = new String[numRecord];
- for (int i = 0; i < numRecord; i++) {
- gender[i] = getString();
- }
-
- linkBackward = new String[numRecord];
- for (int i = 0; i < numRecord; i++) {
- linkBackward[i] = getString();
- }
-
- linkForward = new String[numRecord];
- for (int i = 0; i < numRecord; i++) {
- linkForward[i] = getString();
- }
- }
- }
-}