1
0

[HUDI-2005] Avoiding direct fs calls in HoodieLogFileReader (#3757)

This commit is contained in:
Sivabalan Narayanan
2021-10-25 01:21:08 -04:00
committed by GitHub
parent d8560377c3
commit 1bb0532563
12 changed files with 62 additions and 33 deletions

View File

@@ -20,6 +20,7 @@ package org.apache.hudi.hadoop.realtime;
import org.apache.hudi.common.util.Option;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.io.Text;
import org.apache.hadoop.mapred.FileSplit;
@@ -56,6 +57,7 @@ public class TestHoodieRealtimeFileSplit {
private HoodieRealtimeFileSplit split;
private String basePath;
private List<FileStatus> deltaLogFileStatus;
private List<String> deltaLogPaths;
private String fileSplitName;
private FileSplit baseFileSplit;
@@ -64,12 +66,13 @@ public class TestHoodieRealtimeFileSplit {
@BeforeEach
public void setUp(@TempDir java.nio.file.Path tempDir) throws Exception {
basePath = tempDir.toAbsolutePath().toString();
deltaLogFileStatus = Collections.singletonList(new FileStatus(0L, false, 0, 0L, 0, new Path(basePath + "/1.log")));
deltaLogPaths = Collections.singletonList(basePath + "/1.log");
fileSplitName = basePath + "/test.file";
baseFileSplit = new FileSplit(new Path(fileSplitName), 0, 100, new String[] {});
maxCommitTime = "10001";
split = new HoodieRealtimeFileSplit(baseFileSplit, basePath, deltaLogPaths, maxCommitTime, Option.empty());
split = new HoodieRealtimeFileSplit(baseFileSplit, basePath, deltaLogFileStatus, maxCommitTime, Option.empty());
}
@Test

View File

@@ -41,6 +41,7 @@ import org.apache.hudi.hadoop.testutils.InputFormatTestUtil;
import org.apache.avro.Schema;
import org.apache.avro.Schema.Field;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants;
@@ -187,7 +188,7 @@ public class TestHoodieRealtimeRecordReader {
HoodieRealtimeFileSplit split = new HoodieRealtimeFileSplit(
new FileSplit(new Path(partitionDir + "/fileid0_1-0-1_" + baseInstant + ".parquet"), 0, 1, baseJobConf),
basePath.toUri().toString(), fileSlice.getLogFiles().sorted(HoodieLogFile.getLogFileComparator())
.map(h -> h.getPath().toString()).collect(Collectors.toList()),
.map(h -> new FileStatus(0L, false, 0, 0L, 0, h.getPath())).collect(Collectors.toList()),
instantTime, Option.empty());
// create a RecordReader to be used by HoodieRealtimeRecordReader
@@ -256,10 +257,10 @@ public class TestHoodieRealtimeRecordReader {
FileCreateUtils.createDeltaCommit(basePath.toString(), newCommitTime);
// create a split with baseFile (parquet file written earlier) and new log file(s)
String logFilePath = writer.getLogFile().getPath().toString();
FileStatus logFileFileStatus = new FileStatus(0L, false, 0, 0L, 0, writer.getLogFile().getPath());
HoodieRealtimeFileSplit split = new HoodieRealtimeFileSplit(
new FileSplit(new Path(partitionDir + "/fileid0_1-0-1_" + instantTime + ".parquet"), 0, 1, baseJobConf),
basePath.toUri().toString(), Collections.singletonList(logFilePath), newCommitTime, Option.empty());
basePath.toUri().toString(), Collections.singletonList(logFileFileStatus), newCommitTime, Option.empty());
// create a RecordReader to be used by HoodieRealtimeRecordReader
RecordReader<NullWritable, ArrayWritable> reader = new MapredParquetInputFormat().getRecordReader(
@@ -336,10 +337,10 @@ public class TestHoodieRealtimeRecordReader {
InputFormatTestUtil.deltaCommit(basePath, newCommitTime);
// create a split with baseFile (parquet file written earlier) and new log file(s)
String logFilePath = writer.getLogFile().getPath().toString();
FileStatus logFileStatus = new FileStatus(0L, false, 0, 0L, 0, writer.getLogFile().getPath());
HoodieRealtimeFileSplit split = new HoodieRealtimeFileSplit(
new FileSplit(new Path(partitionDir + "/fileid0_1-0-1_" + instantTime + ".parquet"), 0, 1, baseJobConf),
basePath.toUri().toString(), Collections.singletonList(logFilePath), newCommitTime, Option.empty());
basePath.toUri().toString(), Collections.singletonList(logFileStatus), newCommitTime, Option.empty());
// create a RecordReader to be used by HoodieRealtimeRecordReader
RecordReader<NullWritable, ArrayWritable> reader = new MapredParquetInputFormat().getRecordReader(
@@ -449,7 +450,7 @@ public class TestHoodieRealtimeRecordReader {
public void testSchemaEvolutionAndRollbackBlockInLastLogFile(ExternalSpillableMap.DiskMapType diskMapType,
boolean isCompressionEnabled) throws Exception {
// initial commit
List<String> logFilePaths = new ArrayList<>();
List<FileStatus> logFilePaths = new ArrayList<>();
Schema schema = HoodieAvroUtils.addMetadataFields(SchemaTestUtil.getSimpleSchema());
HoodieTestUtils.init(hadoopConf, basePath.toString(), HoodieTableType.MERGE_ON_READ);
String instantTime = "100";
@@ -470,7 +471,7 @@ public class TestHoodieRealtimeRecordReader {
InputFormatTestUtil.writeDataBlockToLogFile(partitionDir, fs, schema, "fileid0", instantTime, newCommitTime,
numberOfLogRecords, 0, 1);
long size = writer.getCurrentSize();
logFilePaths.add(writer.getLogFile().getPath().toString());
logFilePaths.add(new FileStatus(0L, false, 0, 0L, 0, writer.getLogFile().getPath()));
writer.close();
assertTrue(size > 0, "block - size should be > 0");
@@ -478,7 +479,7 @@ public class TestHoodieRealtimeRecordReader {
newCommitTime = "102";
writer = InputFormatTestUtil.writeRollbackBlockToLogFile(partitionDir, fs, schema, "fileid0", instantTime,
newCommitTime, "101", 1);
logFilePaths.add(writer.getLogFile().getPath().toString());
logFilePaths.add(new FileStatus(0L, false, 0, 0L, 0, writer.getLogFile().getPath()));
writer.close();
InputFormatTestUtil.deltaCommit(basePath, newCommitTime);