[
https://issues.apache.org/jira/browse/FLINK-5056?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15670082#comment-15670082
]
ASF GitHub Bot commented on FLINK-5056:
---------------------------------------
Github user zentol commented on a diff in the pull request:
https://github.com/apache/flink/pull/2797#discussion_r88207666
--- Diff:
flink-streaming-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/bucketing/BucketingSink.java
---
@@ -570,139 +563,165 @@ private void closeCurrentPartFile(BucketState<T>
bucketState) throws Exception {
/**
* Gets the truncate() call using reflection.
- *
* <p>
- * Note: This code comes from Flume
+ * <b>NOTE:</b> This code comes from Flume.
*/
private Method reflectTruncate(FileSystem fs) {
- Method m = null;
--- End diff --
we can heavily reduce the changes made here, by putting
```
if (this.refTruncated != null) {
return this.refTruncated;
}
```
at the start of the method. the remaining code will remain identical, aside
from the new assignment near the end.
> BucketingSink deletes valid data when checkpoint notification is slow.
> ----------------------------------------------------------------------
>
> Key: FLINK-5056
> URL: https://issues.apache.org/jira/browse/FLINK-5056
> Project: Flink
> Issue Type: Bug
> Components: filesystem-connector
> Affects Versions: 1.1.3
> Reporter: Kostas Kloudas
> Assignee: Kostas Kloudas
> Fix For: 1.2.0
>
>
> Currently if BucketingSink receives no data after a checkpoint and then a
> notification about a previous checkpoint arrives, it clears its state. This
> can
> lead to not committing valid data about intermediate checkpoints for whom
> a notification has not arrived yet. As a simple sequence that illustrates the
> problem:
> -> input data
> -> snapshot(0)
> -> input data
> -> snapshot(1)
> -> no data
> -> notifyCheckpointComplete(0)
> the last will clear the state of the Sink without committing as final the
> data
> that arrived for checkpoint 1.
>
--
This message was sent by Atlassian JIRA
(v6.3.4#6332)