This reverts commit 1bb0532563.
This commit is contained in:
committed by
GitHub
parent
4b5512e685
commit
e3fc74668f
@@ -73,11 +73,6 @@ 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 {
|
||||
FSDataInputStream fsDataInputStream = fs.open(logFile.getPath(), bufferSize);
|
||||
@@ -87,11 +82,16 @@ public class HoodieLogFileReader implements HoodieLogFormat.Reader {
|
||||
this.readBlockLazily = readBlockLazily;
|
||||
this.reverseReader = reverseReader;
|
||||
if (this.reverseReader) {
|
||||
this.reverseLogFilePosition = this.lastReverseLogFilePosition = logFile.getFileSize();
|
||||
this.reverseLogFilePosition = this.lastReverseLogFilePosition = fs.getFileStatus(logFile.getPath()).getLen();
|
||||
}
|
||||
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);
|
||||
}
|
||||
|
||||
@@ -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);
|
||||
return new HoodieLogFileReader(fs, logFile, readerSchema, HoodieLogFileReader.DEFAULT_BUFFER_SIZE, false, false);
|
||||
}
|
||||
|
||||
static HoodieLogFormat.Reader newReader(FileSystem fs, HoodieLogFile logFile, Schema readerSchema,
|
||||
|
||||
@@ -59,7 +59,7 @@ public class HoodieLogFormatReader implements HoodieLogFormat.Reader {
|
||||
this.prevReadersInOpenState = new ArrayList<>();
|
||||
if (logFiles.size() > 0) {
|
||||
HoodieLogFile nextLogFile = logFiles.remove(0);
|
||||
this.currentReader = new HoodieLogFileReader(fs, nextLogFile, readerSchema, bufferSize, readBlocksLazily);
|
||||
this.currentReader = new HoodieLogFileReader(fs, nextLogFile, readerSchema, bufferSize, readBlocksLazily, false);
|
||||
}
|
||||
}
|
||||
|
||||
@@ -99,7 +99,7 @@ public class HoodieLogFormatReader implements HoodieLogFormat.Reader {
|
||||
this.prevReadersInOpenState.add(currentReader);
|
||||
}
|
||||
this.currentReader =
|
||||
new HoodieLogFileReader(fs, nextLogFile, readerSchema, bufferSize, readBlocksLazily);
|
||||
new HoodieLogFileReader(fs, nextLogFile, readerSchema, bufferSize, readBlocksLazily, false);
|
||||
} catch (IOException io) {
|
||||
throw new HoodieIOException("unable to initialize read with log file ", io);
|
||||
}
|
||||
|
||||
@@ -27,16 +27,14 @@ 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.FileStatus;
|
||||
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;
|
||||
|
||||
/**
|
||||
@@ -44,10 +42,9 @@ import java.util.stream.Collectors;
|
||||
*/
|
||||
public class LogReaderUtils {
|
||||
|
||||
private static Schema readSchemaFromLogFileInReverse(FileSystem fs, HoodieActiveTimeline activeTimeline, FileStatus logPathFileStatus)
|
||||
private static Schema readSchemaFromLogFileInReverse(FileSystem fs, HoodieActiveTimeline activeTimeline, Path path)
|
||||
throws IOException {
|
||||
// set length for the HoodieLogFile as it will be leveraged by HoodieLogFormat.Reader with reverseReading enabled
|
||||
Reader reader = HoodieLogFormat.newReader(fs, new HoodieLogFile(logPathFileStatus.getPath(), logPathFileStatus.getLen()), null, true, true);
|
||||
Reader reader = HoodieLogFormat.newReader(fs, new HoodieLogFile(path), null, true, true);
|
||||
Schema writerSchema = null;
|
||||
HoodieTimeline completedTimeline = activeTimeline.getCommitsTimeline().filterCompletedInstants();
|
||||
while (reader.hasPrev()) {
|
||||
@@ -65,19 +62,17 @@ public class LogReaderUtils {
|
||||
return writerSchema;
|
||||
}
|
||||
|
||||
public static Schema readLatestSchemaFromLogFiles(String basePath, List<FileStatus> deltaFileStatus, Configuration config)
|
||||
public static Schema readLatestSchemaFromLogFiles(String basePath, List<String> deltaFilePaths, Configuration config)
|
||||
throws IOException {
|
||||
HoodieTableMetaClient metaClient = HoodieTableMetaClient.builder().setConf(config).setBasePath(basePath).build();
|
||||
List<String> deltaPaths = deltaFileStatus.stream().map(s -> new HoodieLogFile(s.getPath()))
|
||||
List<String> deltaPaths = deltaFilePaths.stream().map(s -> new HoodieLogFile(new Path(s)))
|
||||
.sorted(HoodieLogFile.getReverseLogFileComparator()).map(s -> s.getPath().toString())
|
||||
.collect(Collectors.toList());
|
||||
Map<String, FileStatus> deltaFilePathToFileStatus = deltaFileStatus.stream().map(entry -> Pair.of(entry.getPath().toString(), entry))
|
||||
.collect(Collectors.toMap(Pair::getKey, Pair::getValue));
|
||||
if (deltaPaths.size() > 0) {
|
||||
for (String logPath : deltaPaths) {
|
||||
FileSystem fs = FSUtils.getFs(logPath, config);
|
||||
Schema schemaFromLogFile =
|
||||
readSchemaFromLogFileInReverse(fs, metaClient.getActiveTimeline(), deltaFilePathToFileStatus.get(logPath));
|
||||
readSchemaFromLogFileInReverse(fs, metaClient.getActiveTimeline(), new Path(logPath));
|
||||
if (schemaFromLogFile != null) {
|
||||
return schemaFromLogFile;
|
||||
}
|
||||
@@ -85,4 +80,5 @@ public class LogReaderUtils {
|
||||
}
|
||||
return null;
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
@@ -61,7 +61,7 @@ public class HoodieMetadataFileSystemView extends HoodieTableFileSystemView {
|
||||
* @throws IOException
|
||||
*/
|
||||
@Override
|
||||
public FileStatus[] listPartition(Path partitionPath) throws IOException {
|
||||
protected FileStatus[] listPartition(Path partitionPath) throws IOException {
|
||||
return tableMetadata.getAllFilesInPartition(partitionPath);
|
||||
}
|
||||
|
||||
|
||||
Reference in New Issue
Block a user