1
0

[HUDI-1055] Remove hardcoded parquet in tests (#2740)

* Remove hardcoded parquet in tests
* Use DataFileUtils.getInstance
* Renaming DataFileUtils to BaseFileUtils

Co-authored-by: Vinoth Chandar <vinoth@apache.org>
This commit is contained in:
TeRS-K
2021-05-11 13:01:45 -04:00
committed by GitHub
parent ac72470e10
commit be9db2c4f5
42 changed files with 359 additions and 218 deletions

View File

@@ -172,7 +172,7 @@ public class InputFormatTestUtil {
public static File prepareParquetTable(java.nio.file.Path basePath, Schema schema, int numberOfFiles,
int numberOfRecords, String commitNumber, HoodieTableType tableType) throws IOException {
HoodieTestUtils.init(HoodieTestUtils.getDefaultHadoopConf(), basePath.toString(), tableType);
HoodieTestUtils.init(HoodieTestUtils.getDefaultHadoopConf(), basePath.toString(), tableType, HoodieFileFormat.PARQUET);
java.nio.file.Path partitionPath = basePath.resolve(Paths.get("2016", "05", "01"));
createData(schema, partitionPath, numberOfFiles, numberOfRecords, commitNumber);
return partitionPath.toFile();
@@ -185,7 +185,7 @@ public class InputFormatTestUtil {
public static File prepareSimpleParquetTable(java.nio.file.Path basePath, Schema schema, int numberOfFiles,
int numberOfRecords, String commitNumber, HoodieTableType tableType) throws Exception {
HoodieTestUtils.init(HoodieTestUtils.getDefaultHadoopConf(), basePath.toString(), tableType);
HoodieTestUtils.init(HoodieTestUtils.getDefaultHadoopConf(), basePath.toString(), tableType, HoodieFileFormat.PARQUET);
java.nio.file.Path partitionPath = basePath.resolve(Paths.get("2016", "05", "01"));
createSimpleData(schema, partitionPath, numberOfFiles, numberOfRecords, commitNumber);
return partitionPath.toFile();
@@ -198,7 +198,7 @@ public class InputFormatTestUtil {
public static File prepareNonPartitionedParquetTable(java.nio.file.Path basePath, Schema schema, int numberOfFiles,
int numberOfRecords, String commitNumber, HoodieTableType tableType) throws IOException {
HoodieTestUtils.init(HoodieTestUtils.getDefaultHadoopConf(), basePath.toString(), tableType);
HoodieTestUtils.init(HoodieTestUtils.getDefaultHadoopConf(), basePath.toString(), tableType, HoodieFileFormat.PARQUET);
createData(schema, basePath, numberOfFiles, numberOfRecords, commitNumber);
return basePath.toFile();
}
@@ -207,7 +207,7 @@ public class InputFormatTestUtil {
String commitNumber) throws IOException {
AvroParquetWriter parquetWriter;
for (int i = 0; i < numberOfFiles; i++) {
String fileId = FSUtils.makeDataFileName(commitNumber, TEST_WRITE_TOKEN, "fileid" + i);
String fileId = FSUtils.makeDataFileName(commitNumber, TEST_WRITE_TOKEN, "fileid" + i, HoodieFileFormat.PARQUET.getFileExtension());
parquetWriter = new AvroParquetWriter(new Path(partitionPath.resolve(fileId).toString()), schema);
try {
for (GenericRecord record : generateAvroRecords(schema, numberOfRecords, commitNumber, fileId)) {
@@ -223,7 +223,7 @@ public class InputFormatTestUtil {
String commitNumber) throws Exception {
AvroParquetWriter parquetWriter;
for (int i = 0; i < numberOfFiles; i++) {
String fileId = FSUtils.makeDataFileName(commitNumber, "1", "fileid" + i);
String fileId = FSUtils.makeDataFileName(commitNumber, "1", "fileid" + i, HoodieFileFormat.PARQUET.getFileExtension());
parquetWriter = new AvroParquetWriter(new Path(partitionPath.resolve(fileId).toString()), schema);
try {
List<IndexedRecord> records = SchemaTestUtil.generateTestRecords(0, numberOfRecords);
@@ -254,7 +254,8 @@ public class InputFormatTestUtil {
int totalNumberOfRecords, int numberOfRecordsToUpdate, String newCommit) throws IOException {
File fileToUpdate = Objects.requireNonNull(directory.listFiles((dir, name) -> name.endsWith("parquet")))[0];
String fileId = FSUtils.getFileId(fileToUpdate.getName());
File dataFile = new File(directory, FSUtils.makeDataFileName(newCommit, TEST_WRITE_TOKEN, fileId));
File dataFile = new File(directory,
FSUtils.makeDataFileName(newCommit, TEST_WRITE_TOKEN, fileId, HoodieFileFormat.PARQUET.getFileExtension()));
try (AvroParquetWriter parquetWriter = new AvroParquetWriter(new Path(dataFile.getAbsolutePath()), schema)) {
for (GenericRecord record : generateAvroRecords(schema, totalNumberOfRecords, originalCommit, fileId)) {
if (numberOfRecordsToUpdate > 0) {