1
0

[HUDI-2005] Removing direct fs call in HoodieLogFileReader (#3865)

This commit is contained in:
Sivabalan Narayanan
2021-11-25 18:51:38 -05:00
committed by GitHub
parent 6f5d8d04cd
commit 8340ccb503
17 changed files with 116 additions and 80 deletions

View File

@@ -76,6 +76,11 @@ public class HoodieLogFileReader implements HoodieLogFormat.Reader {
private boolean closed = false;
private transient Thread shutdownThread = null;
public HoodieLogFileReader(FileSystem fs, HoodieLogFile logFile, Schema readerSchema, int bufferSize,
boolean readBlockLazily) throws IOException {
this(fs, logFile, readerSchema, bufferSize, readBlockLazily, false);
}
public HoodieLogFileReader(FileSystem fs, HoodieLogFile logFile, Schema readerSchema, int bufferSize,
boolean readBlockLazily, boolean reverseReader) throws IOException {
this(fs, logFile, readerSchema, bufferSize, readBlockLazily, reverseReader, false,
@@ -94,16 +99,11 @@ public class HoodieLogFileReader implements HoodieLogFormat.Reader {
this.enableInlineReading = enableInlineReading;
this.keyField = keyField;
if (this.reverseReader) {
this.reverseLogFilePosition = this.lastReverseLogFilePosition = fs.getFileStatus(logFile.getPath()).getLen();
this.reverseLogFilePosition = this.lastReverseLogFilePosition = logFile.getFileSize();
}
addShutDownHook();
}
public HoodieLogFileReader(FileSystem fs, HoodieLogFile logFile, Schema readerSchema, boolean readBlockLazily,
boolean reverseReader) throws IOException {
this(fs, logFile, readerSchema, DEFAULT_BUFFER_SIZE, readBlockLazily, reverseReader);
}
public HoodieLogFileReader(FileSystem fs, HoodieLogFile logFile, Schema readerSchema) throws IOException {
this(fs, logFile, readerSchema, DEFAULT_BUFFER_SIZE, false, false);
}

View File

@@ -274,7 +274,7 @@ public interface HoodieLogFormat {
static HoodieLogFormat.Reader newReader(FileSystem fs, HoodieLogFile logFile, Schema readerSchema)
throws IOException {
return new HoodieLogFileReader(fs, logFile, readerSchema, HoodieLogFileReader.DEFAULT_BUFFER_SIZE, false, false);
return new HoodieLogFileReader(fs, logFile, readerSchema, HoodieLogFileReader.DEFAULT_BUFFER_SIZE, false);
}
static HoodieLogFormat.Reader newReader(FileSystem fs, HoodieLogFile logFile, Schema readerSchema,

View File

@@ -104,9 +104,8 @@ public class HoodieLogFormatReader implements HoodieLogFormat.Reader {
} else {
this.prevReadersInOpenState.add(currentReader);
}
this.currentReader =
new HoodieLogFileReader(fs, nextLogFile, readerSchema, bufferSize, readBlocksLazily, false,
enableInlineReading, recordKeyField);
this.currentReader = new HoodieLogFileReader(fs, nextLogFile, readerSchema, bufferSize, readBlocksLazily, false,
enableInlineReading, recordKeyField);
} catch (IOException io) {
throw new HoodieIOException("unable to initialize read with log file ", io);
}

View File

@@ -27,14 +27,15 @@ import org.apache.hudi.common.table.log.block.HoodieLogBlock;
import org.apache.hudi.common.table.log.block.HoodieLogBlock.HeaderMetadataType;
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
import org.apache.hudi.common.table.timeline.HoodieTimeline;
import org.apache.hudi.common.util.collection.Pair;
import org.apache.avro.Schema;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import java.io.IOException;
import java.util.List;
import java.util.Map;
import java.util.stream.Collectors;
/**
@@ -42,9 +43,10 @@ import java.util.stream.Collectors;
*/
public class LogReaderUtils {
private static Schema readSchemaFromLogFileInReverse(FileSystem fs, HoodieActiveTimeline activeTimeline, Path path)
private static Schema readSchemaFromLogFileInReverse(FileSystem fs, HoodieActiveTimeline activeTimeline, HoodieLogFile hoodieLogFile)
throws IOException {
Reader reader = HoodieLogFormat.newReader(fs, new HoodieLogFile(path), null, true, true);
// set length for the HoodieLogFile as it will be leveraged by HoodieLogFormat.Reader with reverseReading enabled
Reader reader = HoodieLogFormat.newReader(fs, hoodieLogFile, null, true, true);
Schema writerSchema = null;
HoodieTimeline completedTimeline = activeTimeline.getCommitsTimeline().filterCompletedInstants();
while (reader.hasPrev()) {
@@ -62,17 +64,17 @@ public class LogReaderUtils {
return writerSchema;
}
public static Schema readLatestSchemaFromLogFiles(String basePath, List<String> deltaFilePaths, Configuration config)
public static Schema readLatestSchemaFromLogFiles(String basePath, List<HoodieLogFile> logFiles, Configuration config)
throws IOException {
HoodieTableMetaClient metaClient = HoodieTableMetaClient.builder().setConf(config).setBasePath(basePath).build();
List<String> deltaPaths = deltaFilePaths.stream().map(s -> new HoodieLogFile(new Path(s)))
.sorted(HoodieLogFile.getReverseLogFileComparator()).map(s -> s.getPath().toString())
List<String> deltaPaths = logFiles.stream().sorted(HoodieLogFile.getReverseLogFileComparator()).map(s -> s.getPath().toString())
.collect(Collectors.toList());
if (deltaPaths.size() > 0) {
Map<String, HoodieLogFile> deltaFilePathToFileStatus = logFiles.stream().map(entry -> Pair.of(entry.getPath().toString(), entry))
.collect(Collectors.toMap(Pair::getKey, Pair::getValue));
for (String logPath : deltaPaths) {
FileSystem fs = FSUtils.getFs(logPath, config);
Schema schemaFromLogFile =
readSchemaFromLogFileInReverse(fs, metaClient.getActiveTimeline(), new Path(logPath));
Schema schemaFromLogFile = readSchemaFromLogFileInReverse(fs, metaClient.getActiveTimeline(), deltaFilePathToFileStatus.get(logPath));
if (schemaFromLogFile != null) {
return schemaFromLogFile;
}
@@ -80,5 +82,4 @@ public class LogReaderUtils {
}
return null;
}
}