[HUDI-1900] Always close the file handle for a flink mini-batch write (#2943)
Close the file handle eagerly to avoid corrupted files as much as possible.
This commit is contained in:
@@ -142,22 +142,6 @@ public class StreamWriteFunction<K, I, O>
|
||||
*/
|
||||
private transient TotalSizeTracer tracer;
|
||||
|
||||
/**
|
||||
* Flag saying whether the write task is waiting for the checkpoint success notification
|
||||
* after it finished a checkpoint.
|
||||
*
|
||||
* <p>The flag is needed because the write task does not block during the waiting time interval,
|
||||
* some data buckets still flush out with old instant time. There are two cases that the flush may produce
|
||||
* corrupted files if the old instant is committed successfully:
|
||||
* 1) the write handle was writing data but interrupted, left a corrupted parquet file;
|
||||
* 2) the write handle finished the write but was not closed, left an empty parquet file.
|
||||
*
|
||||
* <p>To solve, when this flag was set to true, we flush the data buffer with a new instant time = old instant time + 1ms,
|
||||
* the new instant time would affect the write file name. The filesystem view does not recognize the file as committed because
|
||||
* it always filters the data files based on successful commit time.
|
||||
*/
|
||||
private volatile boolean confirming = false;
|
||||
|
||||
/**
|
||||
* Constructs a StreamingSinkFunction.
|
||||
*
|
||||
@@ -207,7 +191,7 @@ public class StreamWriteFunction<K, I, O>
|
||||
|
||||
@Override
|
||||
public void notifyCheckpointComplete(long checkpointId) {
|
||||
this.confirming = false;
|
||||
this.writeClient.cleanHandles();
|
||||
}
|
||||
|
||||
/**
|
||||
@@ -416,16 +400,12 @@ public class StreamWriteFunction<K, I, O>
|
||||
return;
|
||||
}
|
||||
|
||||
// if we are waiting for the checkpoint notification, shift the write instant time.
|
||||
boolean shift = confirming && StreamerUtil.equal(instant, this.currentInstant);
|
||||
final String flushInstant = shift ? StreamerUtil.instantTimePlus(instant, 1) : instant;
|
||||
|
||||
List<HoodieRecord> records = bucket.records;
|
||||
ValidationUtils.checkState(records.size() > 0, "Data bucket to flush has no buffering records");
|
||||
if (config.getBoolean(FlinkOptions.INSERT_DROP_DUPS)) {
|
||||
records = FlinkWriteHelper.newInstance().deduplicateRecords(records, (HoodieIndex) null, -1);
|
||||
}
|
||||
final List<WriteStatus> writeStatus = new ArrayList<>(writeFunction.apply(records, flushInstant));
|
||||
final List<WriteStatus> writeStatus = new ArrayList<>(writeFunction.apply(records, instant));
|
||||
final BatchWriteSuccessEvent event = BatchWriteSuccessEvent.builder()
|
||||
.taskID(taskID)
|
||||
.instantTime(instant) // the write instant may shift but the event still use the currentInstant.
|
||||
@@ -473,7 +453,5 @@ public class StreamWriteFunction<K, I, O>
|
||||
this.eventGateway.sendEventToCoordinator(event);
|
||||
this.buckets.clear();
|
||||
this.tracer.reset();
|
||||
this.writeClient.cleanHandles();
|
||||
this.confirming = true;
|
||||
}
|
||||
}
|
||||
|
||||
Reference in New Issue
Block a user