diff --git a/docs/ops/state/state_backends.md b/docs/ops/state/state_backends.md index 4cda94fad35a543c20247388129fec8dafe11312..00e3c4fa5a13999dde9298679797f667b3f760af 100644 --- a/docs/ops/state/state_backends.md +++ b/docs/ops/state/state_backends.md @@ -26,7 +26,7 @@ Programs written in the [Data Stream API]({{ site.baseurl }}/dev/datastream_api. - Windows gather elements or aggregates until they are triggered - Transformation functions may use the key/value state interface to store values -- Transformation functions may implement the `Checkpointed` interface to make their local variables fault tolerant +- Transformation functions may implement the `CheckpointedFunction` interface to make their local variables fault tolerant See also [state section]({{ site.baseurl }}/dev/stream/state/index.html) in the streaming API guide. diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/source/SourceFunction.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/source/SourceFunction.java index 4665cc6005cff30e7df6fdfd8694ce01a6e9e2fe..cb2e15fd48a239ad0806ab85e69c80c8417bc796 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/source/SourceFunction.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/source/SourceFunction.java @@ -34,9 +34,9 @@ import java.io.Serializable; * The run method can run for as long as necessary. The source must, however, react to an * invocation of {@link #cancel()} by breaking out of its main loop. * - *

Checkpointed Sources

+ *

CheckpointedFunction Sources

* - *

Sources that also implement the {@link org.apache.flink.streaming.api.checkpoint.Checkpointed} + *

Sources that also implement the {@link org.apache.flink.streaming.api.checkpoint.CheckpointedFunction} * interface must ensure that state checkpointing, updating of internal state and emission of * elements are not done concurrently. This is achieved by using the provided checkpointing lock * object to protect update of state and emission of elements in a synchronized block. @@ -44,7 +44,7 @@ import java.io.Serializable; *

This is the basic pattern one should follow when implementing a (checkpointed) source: * *

{@code
- *  public class ExampleSource implements SourceFunction, Checkpointed {
+ *  public class ExampleSource implements SourceFunction, CheckpointedFunction {
  *      private long count = 0L;
  *      private volatile boolean isRunning = true;
  *
@@ -61,9 +61,9 @@ import java.io.Serializable;
  *          isRunning = false;
  *      }
  *
- *      public Long snapshotState(long checkpointId, long checkpointTimestamp) { return count; }
+ *      public void snapshotState(FunctionSnapshotContext context) {  }
  *
- *      public void restoreState(Long state) { this.count = state; }
+ *      public void initializeState(FunctionInitializationContext context) {  }
  * }
  * }
* @@ -96,12 +96,12 @@ public interface SourceFunction extends Function, Serializable { * Starts the source. Implementations can use the {@link SourceContext} emit * elements. * - *

Sources that implement {@link org.apache.flink.streaming.api.checkpoint.Checkpointed} + *

Sources that implement {@link org.apache.flink.streaming.api.checkpoint.CheckpointedFunction} * must lock on the checkpoint lock (using a synchronized block) before updating internal * state and emitting elements, to make both an atomic operation: * *

{@code
-	 *  public class ExampleSource implements SourceFunction, Checkpointed {
+	 *  public class ExampleSource implements SourceFunction, CheckpointedFunction {
 	 *      private long count = 0L;
 	 *      private volatile boolean isRunning = true;
 	 *
diff --git a/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/StreamExecutionEnvironment.scala b/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/StreamExecutionEnvironment.scala
index 9fd03c3d8c290c0db194dd5885b49fa2bd1611b5..3bba5055445f50f836a2a5a7c346c0717b8ff222 100644
--- a/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/StreamExecutionEnvironment.scala
+++ b/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/StreamExecutionEnvironment.scala
@@ -229,7 +229,7 @@ class StreamExecutionEnvironment(javaEnv: JavaEnv) {
    * [[KeyedStream]] is maintained (heap, managed memory, externally), and where state
    * snapshots/checkpoints are stored, both for the key/value state, and for checkpointed
    * functions (implementing the interface 
-   * [[org.apache.flink.streaming.api.checkpoint.Checkpointed]].
+   * [[org.apache.flink.streaming.api.checkpoint.CheckpointedFunction]].
    *
    * 

The [[org.apache.flink.runtime.state.memory.MemoryStateBackend]] for example * maintains the state in heap memory, as objects. It is lightweight without extra