[HUDI-1804] Continue to write when Flink write task restart because of container killing (#2843)
The `FlinkMergeHande` creates a marker file under the metadata path each time it initializes, when a write task restarts from killing, it tries to create the existing file and reports error. To solve this problem, skip the creation and use the original data file as base file to merge.
This commit is contained in:
@@ -189,16 +189,7 @@ public class MarkerFiles implements Serializable {
|
||||
* The marker path will be <base-path>/.hoodie/.temp/<instant_ts>/2019/04/25/filename.marker.writeIOType.
|
||||
*/
|
||||
public Path create(String partitionPath, String dataFileName, IOType type) {
|
||||
Path path = FSUtils.getPartitionPath(markerDirPath, partitionPath);
|
||||
try {
|
||||
if (!fs.exists(path)) {
|
||||
fs.mkdirs(path); // create a new partition as needed.
|
||||
}
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIOException("Failed to make dir " + path, e);
|
||||
}
|
||||
String markerFileName = String.format("%s%s.%s", dataFileName, HoodieTableMetaClient.MARKER_EXTN, type.name());
|
||||
Path markerPath = new Path(path, markerFileName);
|
||||
Path markerPath = getMarkerPath(partitionPath, dataFileName, type);
|
||||
try {
|
||||
LOG.info("Creating Marker Path=" + markerPath);
|
||||
fs.create(markerPath, false).close();
|
||||
@@ -208,4 +199,46 @@ public class MarkerFiles implements Serializable {
|
||||
return markerPath;
|
||||
}
|
||||
|
||||
/**
|
||||
* The marker path will be <base-path>/.hoodie/.temp/<instant_ts>/2019/04/25/filename.marker.writeIOType.
|
||||
*
|
||||
* @return true if the marker file creates successfully,
|
||||
* false if it already exists
|
||||
*/
|
||||
public boolean createIfNotExists(String partitionPath, String dataFileName, IOType type) {
|
||||
Path markerPath = getMarkerPath(partitionPath, dataFileName, type);
|
||||
try {
|
||||
if (fs.exists(markerPath)) {
|
||||
LOG.warn("Marker Path=" + markerPath + " already exists, cancel creation");
|
||||
return false;
|
||||
}
|
||||
LOG.info("Creating Marker Path=" + markerPath);
|
||||
fs.create(markerPath, false).close();
|
||||
} catch (IOException e) {
|
||||
throw new HoodieException("Failed to create marker file " + markerPath, e);
|
||||
}
|
||||
return true;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the marker path. Would create the partition path first if not exists.
|
||||
*
|
||||
* @param partitionPath The partition path
|
||||
* @param dataFileName The data file name
|
||||
* @param type The IO type
|
||||
* @return path of the marker file
|
||||
*/
|
||||
private Path getMarkerPath(String partitionPath, String dataFileName, IOType type) {
|
||||
Path path = FSUtils.getPartitionPath(markerDirPath, partitionPath);
|
||||
try {
|
||||
if (!fs.exists(path)) {
|
||||
fs.mkdirs(path); // create a new partition as needed.
|
||||
}
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIOException("Failed to make dir " + path, e);
|
||||
}
|
||||
String markerFileName = String.format("%s%s.%s", dataFileName, HoodieTableMetaClient.MARKER_EXTN, type.name());
|
||||
return new Path(path, markerFileName);
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
Reference in New Issue
Block a user