[HUDI-1878] Add max memory option for flink writer task (#2920)
Also removes the rate limiter because it has the similar functionality, modify the create and merge handle cleans the retry files automatically.
This commit is contained in:
@@ -20,6 +20,7 @@ package org.apache.hudi.io;
|
||||
|
||||
import org.apache.hudi.client.WriteStatus;
|
||||
import org.apache.hudi.common.engine.TaskContextSupplier;
|
||||
import org.apache.hudi.common.fs.FSUtils;
|
||||
import org.apache.hudi.common.model.HoodieRecordPayload;
|
||||
import org.apache.hudi.common.util.HoodieTimer;
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
@@ -27,9 +28,9 @@ import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.exception.HoodieException;
|
||||
import org.apache.hudi.exception.HoodieInsertException;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
import org.apache.hudi.table.MarkerFiles;
|
||||
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
|
||||
@@ -66,27 +67,70 @@ public class FlinkCreateHandle<T extends HoodieRecordPayload, I, K, O>
|
||||
TaskContextSupplier taskContextSupplier) {
|
||||
super(config, instantTime, hoodieTable, partitionPath, fileId, writerSchemaIncludingAndExcludingMetadataPair,
|
||||
taskContextSupplier);
|
||||
// delete invalid data files generated by task retry.
|
||||
if (getAttemptId() > 0) {
|
||||
deleteInvalidDataFile(getAttemptId() - 1);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* The flink checkpoints start in sequence and asynchronously, when one write task finish the checkpoint(A)
|
||||
* (thus the fs view got the written data files some of which may be invalid),
|
||||
* it goes on with the next round checkpoint(B) write immediately,
|
||||
* if it tries to reuse the last small data bucket(small file) of an invalid data file,
|
||||
* finally, when the coordinator receives the checkpoint success event of checkpoint(A),
|
||||
* the invalid data file would be cleaned,
|
||||
* and this merger got a FileNotFoundException when it close the write file handle.
|
||||
*
|
||||
* <p> To solve, deletes the invalid data file eagerly
|
||||
* so that the invalid file small bucket would never be reused.
|
||||
*
|
||||
* @param lastAttemptId The last attempt ID
|
||||
*/
|
||||
private void deleteInvalidDataFile(long lastAttemptId) {
|
||||
final String lastWriteToken = FSUtils.makeWriteToken(getPartitionId(), getStageId(), lastAttemptId);
|
||||
final String lastDataFileName = FSUtils.makeDataFileName(instantTime,
|
||||
lastWriteToken, this.fileId, hoodieTable.getBaseFileExtension());
|
||||
final Path path = makeNewFilePath(partitionPath, lastDataFileName);
|
||||
try {
|
||||
if (fs.exists(path)) {
|
||||
LOG.info("Deleting invalid INSERT file due to task retry: " + lastDataFileName);
|
||||
fs.delete(path, false);
|
||||
}
|
||||
} catch (IOException e) {
|
||||
throw new HoodieException("Error while deleting the INSERT file due to task retry: " + lastDataFileName, e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void createMarkerFile(String partitionPath, String dataFileName) {
|
||||
MarkerFiles markerFiles = new MarkerFiles(hoodieTable, instantTime);
|
||||
boolean created = markerFiles.createIfNotExists(partitionPath, dataFileName, getIOType());
|
||||
if (!created) {
|
||||
// If the marker file already exists, that means the write task
|
||||
// was pulled up again with same data file name, removes the legacy
|
||||
// data file first.
|
||||
try {
|
||||
if (fs.exists(path)) {
|
||||
fs.delete(path, false);
|
||||
LOG.warn("Legacy data file: " + path + " delete success");
|
||||
}
|
||||
} catch (IOException e) {
|
||||
throw new HoodieException("Error while deleting legacy data file: " + path, e);
|
||||
public Path makeNewPath(String partitionPath) {
|
||||
Path path = super.makeNewPath(partitionPath);
|
||||
// If the data file already exists, it means the write task write new data bucket multiple times
|
||||
// in one hoodie commit, rolls over to a new name instead.
|
||||
|
||||
// Write to a new file which behaves like a different task write.
|
||||
try {
|
||||
int rollNumber = 0;
|
||||
while (fs.exists(path)) {
|
||||
Path existing = path;
|
||||
path = newFilePathWithRollover(rollNumber++);
|
||||
LOG.warn("Duplicate write for INSERT bucket with path: " + existing + ", rolls over to new path: " + path);
|
||||
}
|
||||
return path;
|
||||
} catch (IOException e) {
|
||||
throw new HoodieException("Checking existing path for create handle error: " + path, e);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Use the writeToken + "-" + rollNumber as the new writeToken of a mini-batch write.
|
||||
*/
|
||||
private Path newFilePathWithRollover(int rollNumber) {
|
||||
final String dataFileName = FSUtils.makeDataFileName(instantTime, writeToken + "-" + rollNumber, fileId,
|
||||
hoodieTable.getBaseFileExtension());
|
||||
return makeNewFilePath(partitionPath, dataFileName);
|
||||
}
|
||||
|
||||
/**
|
||||
* Get the incremental write status. In mini-batch write mode,
|
||||
* this handle would be reused for a checkpoint bucket(the bucket is appended as mini-batches),
|
||||
@@ -111,7 +155,7 @@ public class FlinkCreateHandle<T extends HoodieRecordPayload, I, K, O>
|
||||
}
|
||||
|
||||
@Override
|
||||
protected long computeTotalWriteBytes() throws IOException {
|
||||
protected long computeTotalWriteBytes() {
|
||||
long fileSizeInBytes = computeFileSizeInBytes();
|
||||
long incFileSizeInBytes = fileSizeInBytes - lastFileSize;
|
||||
this.lastFileSize = fileSizeInBytes;
|
||||
|
||||
@@ -25,6 +25,7 @@ import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieRecordPayload;
|
||||
import org.apache.hudi.common.util.HoodieTimer;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.exception.HoodieException;
|
||||
import org.apache.hudi.exception.HoodieIOException;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
import org.apache.hudi.table.MarkerFiles;
|
||||
@@ -78,18 +79,76 @@ public class FlinkMergeHandle<T extends HoodieRecordPayload, I, K, O>
|
||||
TaskContextSupplier taskContextSupplier) {
|
||||
super(config, instantTime, hoodieTable, recordItr, partitionPath, fileId, taskContextSupplier);
|
||||
if (rolloverPaths == null) {
|
||||
// #createMarkerFile may already initialize it already
|
||||
// #makeOldAndNewFilePaths may already initialize it already
|
||||
rolloverPaths = new ArrayList<>();
|
||||
}
|
||||
// delete invalid data files generated by task retry.
|
||||
if (getAttemptId() > 0) {
|
||||
deleteInvalidDataFile(getAttemptId() - 1);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Use the fileId + "-" + rollNumber as the new fileId of a mini-batch write.
|
||||
* The flink checkpoints start in sequence and asynchronously, when one write task finish the checkpoint(A)
|
||||
* (thus the fs view got the written data files some of which may be invalid),
|
||||
* it goes on with the next round checkpoint(B) write immediately,
|
||||
* if it tries to reuse the last small data bucket(small file) of an invalid data file,
|
||||
* finally, when the coordinator receives the checkpoint success event of checkpoint(A),
|
||||
* the invalid data file would be cleaned,
|
||||
* and this merger got a FileNotFoundException when it close the write file handle.
|
||||
*
|
||||
* <p> To solve, deletes the invalid data file eagerly
|
||||
* so that the invalid file small bucket would never be reused.
|
||||
*
|
||||
* @param lastAttemptId The last attempt ID
|
||||
*/
|
||||
protected String generatesDataFileNameWithRollover() {
|
||||
private void deleteInvalidDataFile(long lastAttemptId) {
|
||||
final String lastWriteToken = FSUtils.makeWriteToken(getPartitionId(), getStageId(), lastAttemptId);
|
||||
final String lastDataFileName = FSUtils.makeDataFileName(instantTime,
|
||||
lastWriteToken, this.fileId, hoodieTable.getBaseFileExtension());
|
||||
final Path path = makeNewFilePath(partitionPath, lastDataFileName);
|
||||
try {
|
||||
if (fs.exists(path)) {
|
||||
LOG.info("Deleting invalid MERGE base file due to task retry: " + lastDataFileName);
|
||||
fs.delete(path, false);
|
||||
}
|
||||
} catch (IOException e) {
|
||||
throw new HoodieException("Error while deleting the MERGE base file due to task retry: " + lastDataFileName, e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void makeOldAndNewFilePaths(String partitionPath, String oldFileName, String newFileName) {
|
||||
// If the data file already exists, it means the write task write merge data bucket multiple times
|
||||
// in one hoodie commit, rolls over to a new name instead.
|
||||
|
||||
// Use the existing file path as the base file path (file1),
|
||||
// and generates new file path with roll over number (file2).
|
||||
// the incremental data set would merge into the file2 instead of file1.
|
||||
//
|
||||
// When the task finalizes in #finishWrite, the intermediate files would be cleaned.
|
||||
super.makeOldAndNewFilePaths(partitionPath, oldFileName, newFileName);
|
||||
rolloverPaths = new ArrayList<>();
|
||||
try {
|
||||
while (fs.exists(newFilePath)) {
|
||||
oldFilePath = newFilePath; // override the old file name
|
||||
rolloverPaths.add(oldFilePath);
|
||||
newFileName = newFileNameWithRollover(rollNumber++);
|
||||
newFilePath = makeNewFilePath(partitionPath, newFileName);
|
||||
LOG.warn("Duplicate write for MERGE bucket with path: " + oldFilePath + ", rolls over to new path: " + newFilePath);
|
||||
}
|
||||
} catch (IOException e) {
|
||||
throw new HoodieException("Checking existing path for merge handle error: " + newFilePath, e);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Use the writeToken + "-" + rollNumber as the new writeToken of a mini-batch write.
|
||||
*/
|
||||
protected String newFileNameWithRollover(int rollNumber) {
|
||||
// make the intermediate file as hidden
|
||||
return FSUtils.makeDataFileName("." + instantTime,
|
||||
writeToken + "-" + rollNumber, this.fileId, hoodieTable.getBaseFileExtension());
|
||||
return FSUtils.makeDataFileName(instantTime, writeToken + "-" + rollNumber,
|
||||
this.fileId, hoodieTable.getBaseFileExtension());
|
||||
}
|
||||
|
||||
public boolean shouldRollover() {
|
||||
@@ -109,25 +168,6 @@ public class FlinkMergeHandle<T extends HoodieRecordPayload, I, K, O>
|
||||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void createMarkerFile(String partitionPath, String dataFileName) {
|
||||
MarkerFiles markerFiles = new MarkerFiles(hoodieTable, instantTime);
|
||||
boolean created = markerFiles.createIfNotExists(partitionPath, dataFileName, getIOType());
|
||||
if (!created) {
|
||||
// If the marker file already exists, that means the write task
|
||||
// was pulled up again with same data file name, performs rolling over action here:
|
||||
// use the new file path as the base file path (file1),
|
||||
// and generates new file path with roll over number (file2).
|
||||
// the incremental data set would merge into the file2 instead of file1.
|
||||
//
|
||||
// When the task do finalization in #finishWrite, the intermediate files would be cleaned.
|
||||
oldFilePath = newFilePath;
|
||||
rolloverPaths = new ArrayList<>();
|
||||
rolloverPaths.add(oldFilePath);
|
||||
newFilePath = makeNewFilePathWithRollover();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
*
|
||||
* Rolls over the write handle to prepare for the next batch write.
|
||||
@@ -156,7 +196,13 @@ public class FlinkMergeHandle<T extends HoodieRecordPayload, I, K, O>
|
||||
|
||||
rolloverPaths.add(newFilePath);
|
||||
oldFilePath = newFilePath;
|
||||
newFilePath = makeNewFilePathWithRollover();
|
||||
final String newFileName = newFileNameWithRollover(rollNumber);
|
||||
newFilePath = makeNewFilePath(partitionPath, newFileName);
|
||||
|
||||
// create the marker file so that the intermediate roll over files
|
||||
// of the retry task can be cleaned.
|
||||
MarkerFiles markerFiles = new MarkerFiles(hoodieTable, instantTime);
|
||||
markerFiles.createIfNotExists(partitionPath, newFileName, getIOType());
|
||||
|
||||
try {
|
||||
fileWriter = createNewFileWriter(instantTime, newFilePath, hoodieTable, config, writerSchemaWithMetafields, taskContextSupplier);
|
||||
@@ -168,16 +214,6 @@ public class FlinkMergeHandle<T extends HoodieRecordPayload, I, K, O>
|
||||
newFilePath.toString()));
|
||||
}
|
||||
|
||||
/**
|
||||
* Use the fileId + "-" + rollNumber as the new fileId of a mini-batch write.
|
||||
*/
|
||||
private Path makeNewFilePathWithRollover() {
|
||||
String newFileName = generatesDataFileNameWithRollover();
|
||||
String relativePath = new Path((partitionPath.isEmpty() ? "" : partitionPath + "/")
|
||||
+ newFileName).toString();
|
||||
return new Path(config.getBasePath(), relativePath);
|
||||
}
|
||||
|
||||
public void finishWrite() {
|
||||
// The file visibility should be kept by the configured ConsistencyGuard instance.
|
||||
rolloverPaths.add(newFilePath);
|
||||
|
||||
@@ -31,10 +31,12 @@ import org.apache.hudi.common.model.HoodieBaseFile;
|
||||
import org.apache.hudi.common.model.HoodieKey;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieRecordPayload;
|
||||
import org.apache.hudi.common.model.HoodieWriteStat;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.exception.HoodieIOException;
|
||||
import org.apache.hudi.exception.HoodieNotSupportedException;
|
||||
import org.apache.hudi.exception.HoodieUpsertException;
|
||||
import org.apache.hudi.io.HoodieCreateHandle;
|
||||
@@ -318,6 +320,12 @@ public class HoodieFlinkCopyOnWriteTable<T extends HoodieRecordPayload> extends
|
||||
throw new HoodieNotSupportedException("Savepoint and restore is not supported yet");
|
||||
}
|
||||
|
||||
@Override
|
||||
public void finalizeWrite(HoodieEngineContext context, String instantTs, List<HoodieWriteStat> stats) throws HoodieIOException {
|
||||
// do nothing because flink create and merge handles can clean the
|
||||
// retry files by themselves.
|
||||
}
|
||||
|
||||
// -------------------------------------------------------------------------
|
||||
// Used for compaction
|
||||
// -------------------------------------------------------------------------
|
||||
|
||||
Reference in New Issue
Block a user