1
0

[HUDI-4171] Fixing Non partitioned with virtual keys in read path (#5747)

- When Non partitioned key gen is used with virtual keys, read path could break since partition path may not exist.
This commit is contained in:
Sivabalan Narayanan
2022-06-06 15:48:21 -04:00
committed by GitHub
parent 21b903fddb
commit 7da97c8096
7 changed files with 87 additions and 22 deletions

View File

@@ -27,12 +27,15 @@ import org.apache.hadoop.mapred.InputSplit;
import org.apache.hadoop.mapred.JobConf;
import org.apache.hadoop.mapred.RecordReader;
import org.apache.hadoop.mapreduce.Job;
import org.apache.hudi.avro.HoodieAvroUtils;
import org.apache.hudi.avro.model.HoodieCompactionPlan;
import org.apache.hudi.common.fs.FSUtils;
import org.apache.hudi.common.model.HoodieCommitMetadata;
import org.apache.hudi.common.model.HoodieFileFormat;
import org.apache.hudi.common.model.HoodieTableType;
import org.apache.hudi.common.model.HoodieWriteStat;
import org.apache.hudi.common.model.WriteOperationType;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
import org.apache.hudi.common.table.timeline.HoodieInstant;
@@ -40,6 +43,9 @@ import org.apache.hudi.common.table.timeline.HoodieTimeline;
import org.apache.hudi.common.table.timeline.TimelineMetadataUtils;
import org.apache.hudi.common.testutils.FileCreateUtils;
import org.apache.hudi.common.testutils.HoodieTestUtils;
import org.apache.hudi.common.testutils.SchemaTestUtil;
import org.apache.hudi.common.util.CommitUtils;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.exception.HoodieIOException;
import org.apache.hudi.hadoop.testutils.InputFormatTestUtil;
import org.apache.hudi.hadoop.utils.HoodieHiveUtils;
@@ -55,6 +61,7 @@ import java.io.IOException;
import java.nio.charset.StandardCharsets;
import java.nio.file.Paths;
import java.util.ArrayList;
import java.util.Collections;
import java.util.List;
import static org.apache.hudi.common.testutils.SchemaTestUtil.getSchemaFromResource;
@@ -167,6 +174,26 @@ public class TestHoodieParquetInputFormat {
assertEquals(10, files.length);
}
@Test
public void testInputFormatLoadForNonPartitionedAndVirtualKeyedTable() throws IOException {
// initial commit
Schema schema = HoodieAvroUtils.addMetadataFields(SchemaTestUtil.getSimpleSchema());
File partitionDir = InputFormatTestUtil.prepareCustomizedTable(basePath, baseFileFormat, 10, "100", true, false,
true, schema);
HoodieCommitMetadata commitMetadata = CommitUtils.buildMetadata(Collections.emptyList(), Collections.emptyMap(), Option.empty(), WriteOperationType.UPSERT,
schema.toString(), HoodieTimeline.COMMIT_ACTION);
FileCreateUtils.createCommit(basePath.toString(), "100", Option.of(commitMetadata));
// Add the paths
FileInputFormat.setInputPaths(jobConf, partitionDir.getPath());
InputSplit[] inputSplits = inputFormat.getSplits(jobConf, 10);
assertEquals(10, inputSplits.length);
FileStatus[] files = inputFormat.listStatus(jobConf);
assertEquals(10, files.length);
}
@Test
public void testInputFormatLoadWithEmptyTable() throws IOException {
// initial hoodie table

View File

@@ -71,16 +71,35 @@ public class InputFormatTestUtil {
private static String TEST_WRITE_TOKEN = "1-0-1";
public static File prepareTable(java.nio.file.Path basePath, HoodieFileFormat baseFileFormat, int numberOfFiles,
String commitNumber)
String commitNumber) throws IOException {
return prepareCustomizedTable(basePath, baseFileFormat, numberOfFiles, commitNumber, false, true, false, null);
}
public static File prepareCustomizedTable(java.nio.file.Path basePath, HoodieFileFormat baseFileFormat, int numberOfFiles,
String commitNumber, boolean useNonPartitionedKeyGen, boolean populateMetaFields, boolean injectData, Schema schema)
throws IOException {
HoodieTestUtils.init(HoodieTestUtils.getDefaultHadoopConf(), basePath.toString(), HoodieTableType.COPY_ON_WRITE,
baseFileFormat);
if (useNonPartitionedKeyGen) {
HoodieTestUtils.init(HoodieTestUtils.getDefaultHadoopConf(), basePath.toString(), HoodieTableType.COPY_ON_WRITE,
baseFileFormat, true, "org.apache.hudi.keygen.NonpartitionedKeyGenerator", populateMetaFields);
} else {
HoodieTestUtils.init(HoodieTestUtils.getDefaultHadoopConf(), basePath.toString(), HoodieTableType.COPY_ON_WRITE,
baseFileFormat);
}
java.nio.file.Path partitionPath = basePath.resolve(Paths.get("2016", "05", "01"));
setupPartition(basePath, partitionPath);
return simulateInserts(partitionPath.toFile(), baseFileFormat.getFileExtension(), "fileId1", numberOfFiles,
commitNumber);
if (injectData) {
try {
createSimpleData(schema, partitionPath, numberOfFiles, 100, commitNumber);
return partitionPath.toFile();
} catch (Exception e) {
throw new IOException("Excpetion thrown while writing data ", e);
}
} else {
return simulateInserts(partitionPath.toFile(), baseFileFormat.getFileExtension(), "fileId1", numberOfFiles,
commitNumber);
}
}
public static File prepareMultiPartitionTable(java.nio.file.Path basePath, HoodieFileFormat baseFileFormat, int numberOfFiles,