You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@beam.apache.org by "Eugene Kirpichov (JIRA)" <ji...@apache.org> on 2017/11/10 18:21:00 UTC

[jira] [Commented] (BEAM-3169) WriteFiles data loss with some triggers

    [ https://issues.apache.org/jira/browse/BEAM-3169?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16247876#comment-16247876 ] 

Eugene Kirpichov commented on BEAM-3169:
----------------------------------------

User reads data from pubsub and windows it into 1-minute windows with trigger AfterProcessingTime.pastFirstElementInPane().plusDelayOf(1 minute), and writes it to TextIO.write().withWindowedWrites(). They observe that some files in the temp directory are never moved to final location, i.e. final output doesn't contain all the data it should.

My reconstruction of events that happens, exemplified by 1 window that experienced data loss:
Window [2017-11-10T13:38:00.000Z..2017-11-10T13:39:00.000Z):
Two temp files were written:
3e3fdee3-d928-485a-b3a5-1faa3c2a58d6 (opened at 2017-11-10 06:39:51.585 PDT, closed at 2017-11-10 06:39:52.090 PDT)
d12e91d2-39b3-43d4-b27f-3a340c25f888 (opened at 2017-11-10 06:40:00.624 PDT, closed at 2017-11-10 06:40:00.909 PDT)

The finalize operation for that window triggered at 2017-11-10 06:40:00.979 PDT (on a different worker) and moved only the first file. The second file was never mentioned in the logs again.

Explanation of why this happened - in the next comment.

> WriteFiles data loss with some triggers
> ---------------------------------------
>
>                 Key: BEAM-3169
>                 URL: https://issues.apache.org/jira/browse/BEAM-3169
>             Project: Beam
>          Issue Type: Bug
>          Components: sdk-java-core
>            Reporter: Eugene Kirpichov
>            Assignee: Eugene Kirpichov
>            Priority: Critical
>
> https://stackoverflow.com/questions/47113773/dataflow-2-1-0-streaming-application-is-not-cleaning-temp-folders/47142671?noredirect=1#comment81401472_47142671
> Details in comments



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)