[ 
https://issues.apache.org/jira/browse/BEAM-7551?focusedWorklogId=260333&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-260333
 ]

ASF GitHub Bot logged work on BEAM-7551:
----------------------------------------

                Author: ASF GitHub Bot
            Created on: 14/Jun/19 11:42
            Start Date: 14/Jun/19 11:42
    Worklog Time Spent: 10m 
      Work Description: JozoVilcek commented on pull request #8857: [BEAM-7551] 
Checkpoint Flink's ImpulseSourceFunction
URL: https://github.com/apache/beam/pull/8857#discussion_r293771387
 
 

 ##########
 File path: 
runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/ImpulseSourceFunction.java
 ##########
 @@ -18,30 +18,46 @@
 package org.apache.beam.runners.flink.translation.functions;
 
 import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.vendor.guava.v20_0.com.google.common.collect.Iterables;
+import org.apache.flink.api.common.state.ListState;
+import org.apache.flink.api.common.state.ListStateDescriptor;
+import org.apache.flink.api.common.typeutils.base.BooleanSerializer;
+import org.apache.flink.runtime.state.FunctionInitializationContext;
+import org.apache.flink.runtime.state.FunctionSnapshotContext;
+import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction;
 import org.apache.flink.streaming.api.functions.source.SourceFunction;
 
 /**
  * Source function which sends a single global impulse to a downstream 
operator. It may keep the
  * source alive although its work is already done. It will only shutdown when 
the streaming job is
  * cancelled.
  */
-public class ImpulseSourceFunction implements 
SourceFunction<WindowedValue<byte[]>> {
+public class ImpulseSourceFunction
+    implements SourceFunction<WindowedValue<byte[]>>, CheckpointedFunction {
 
   /** Keep source running even after it has done all the work. */
   private final boolean keepSourceAlive;
 
   /** Indicates the streaming job is running and the source can produce 
elements. */
   private volatile boolean running;
 
+  /** Checkpointed state which indicates whether the impulse has finished. */
+  private transient ListState<Boolean> impulseEmitted;
 
 Review comment:
   I guess ValueState can not be here because it is not re-scale friendly?
 
----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
[email protected]


Issue Time Tracking
-------------------

    Worklog Id:     (was: 260333)
    Time Spent: 50m  (was: 40m)

> ImpulseSourceFunction is not checkpointed
> -----------------------------------------
>
>                 Key: BEAM-7551
>                 URL: https://issues.apache.org/jira/browse/BEAM-7551
>             Project: Beam
>          Issue Type: Bug
>          Components: runner-flink
>            Reporter: Maximilian Michels
>            Assignee: Maximilian Michels
>            Priority: Major
>          Time Spent: 50m
>  Remaining Estimate: 0h
>
> The ImpulseSourceFunction should be checkpointed, i.e. only emit the impulse 
> once. Upon restore, the impulse should not be emitted again.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Reply via email to