1
0

[HUDI-2548] Flink streaming reader misses the rolling over file handles (#3787)

This commit is contained in:
Danny Chan
2021-10-14 10:36:18 +08:00
committed by GitHub
parent cff384d23f
commit abf3e3fe71
16 changed files with 225 additions and 126 deletions

View File

@@ -27,6 +27,7 @@ import com.fasterxml.jackson.annotation.JsonIgnoreProperties;
import com.fasterxml.jackson.annotation.PropertyAccessor;
import com.fasterxml.jackson.databind.DeserializationFeature;
import com.fasterxml.jackson.databind.ObjectMapper;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.Path;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
@@ -104,8 +105,8 @@ public class HoodieCommitMetadata implements Serializable {
public HashMap<String, String> getFileIdAndRelativePaths() {
HashMap<String, String> filePaths = new HashMap<>();
// list all partitions paths
for (Map.Entry<String, List<HoodieWriteStat>> entry : getPartitionToWriteStats().entrySet()) {
for (HoodieWriteStat stat : entry.getValue()) {
for (List<HoodieWriteStat> stats : getPartitionToWriteStats().values()) {
for (HoodieWriteStat stat : stats) {
filePaths.put(stat.getFileId(), stat.getPath());
}
}
@@ -142,6 +143,60 @@ public class HoodieCommitMetadata implements Serializable {
return fileGroupIdToFullPaths;
}
/**
* Extract the file status of all affected files from the commit metadata. If a file has
* been touched multiple times in the given commits, the return value will keep the one
* from the latest commit.
*
* @param basePath The base path
* @return the file full path to file status mapping
*/
public Map<String, FileStatus> getFullPathToFileStatus(String basePath) {
Map<String, FileStatus> fullPathToFileStatus = new HashMap<>();
for (List<HoodieWriteStat> stats : getPartitionToWriteStats().values()) {
// Iterate through all the written files.
for (HoodieWriteStat stat : stats) {
String relativeFilePath = stat.getPath();
Path fullPath = relativeFilePath != null ? FSUtils.getPartitionPath(basePath, relativeFilePath) : null;
if (fullPath != null) {
FileStatus fileStatus = new FileStatus(stat.getFileSizeInBytes(), false, 0, 0,
0, fullPath);
fullPathToFileStatus.put(fullPath.getName(), fileStatus);
}
}
}
return fullPathToFileStatus;
}
/**
* Extract the file status of all affected files from the commit metadata. If a file has
* been touched multiple times in the given commits, the return value will keep the one
* from the latest commit by file group ID.
*
* <p>Note: different with {@link #getFullPathToFileStatus(String)},
* only the latest commit file for a file group is returned,
* this is an optimization for COPY_ON_WRITE table to eliminate legacy files for filesystem view.
*
* @param basePath The base path
* @return the file ID to file status mapping
*/
public Map<String, FileStatus> getFileIdToFileStatus(String basePath) {
Map<String, FileStatus> fileIdToFileStatus = new HashMap<>();
for (List<HoodieWriteStat> stats : getPartitionToWriteStats().values()) {
// Iterate through all the written files.
for (HoodieWriteStat stat : stats) {
String relativeFilePath = stat.getPath();
Path fullPath = relativeFilePath != null ? FSUtils.getPartitionPath(basePath, relativeFilePath) : null;
if (fullPath != null) {
FileStatus fileStatus = new FileStatus(stat.getFileSizeInBytes(), false, 0, 0,
0, fullPath);
fileIdToFileStatus.put(stat.getFileId(), fileStatus);
}
}
}
return fileIdToFileStatus;
}
public String toJsonString() throws IOException {
if (partitionToWriteStats.containsKey(null)) {
LOG.info("partition path is null for " + partitionToWriteStats.get(null));

View File

@@ -41,6 +41,7 @@ import java.io.Serializable;
import java.nio.charset.StandardCharsets;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.Comparator;
import java.util.HashMap;
import java.util.List;
@@ -208,9 +209,8 @@ public class HoodieArchivedTimeline extends HoodieDefaultTimeline {
List<HoodieInstant> instantsInRange = new ArrayList<>();
for (FileStatus fs : fsStatuses) {
//read the archived file
HoodieLogFormat.Reader reader = HoodieLogFormat.newReader(metaClient.getFs(),
new HoodieLogFile(fs.getPath()), HoodieArchivedMetaEntry.getClassSchema());
try {
try (HoodieLogFormat.Reader reader = HoodieLogFormat.newReader(metaClient.getFs(),
new HoodieLogFile(fs.getPath()), HoodieArchivedMetaEntry.getClassSchema())) {
int instantsInPreviousFile = instantsInRange.size();
//read the avro blocks
while (reader.hasNext()) {
@@ -220,8 +220,8 @@ public class HoodieArchivedTimeline extends HoodieDefaultTimeline {
List<IndexedRecord> records = blk.getRecords();
// filter blocks in desired time window
Stream<HoodieInstant> instantsInBlkStream = records.stream()
.filter(r -> commitsFilter.apply((GenericRecord) r))
.map(r -> readCommit((GenericRecord) r, loadInstantDetails));
.filter(r -> commitsFilter.apply((GenericRecord) r))
.map(r -> readCommit((GenericRecord) r, loadInstantDetails));
if (filter != null) {
instantsInBlkStream = instantsInBlkStream.filter(filter::isInRange);
@@ -238,11 +238,10 @@ public class HoodieArchivedTimeline extends HoodieDefaultTimeline {
break;
}
}
} finally {
reader.close();
}
}
Collections.sort(instantsInRange);
return instantsInRange;
} catch (IOException e) {
throw new HoodieIOException(