fapaul commented on a change in pull request #16701:
URL: https://github.com/apache/flink/pull/16701#discussion_r682543690



##########
File path: 
flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/sink/FileSink.java
##########
@@ -161,6 +163,12 @@ private FileSink(BucketsBuilder<IN, ? extends 
BucketsBuilder<IN, ?>> bucketsBuil
         return Optional.empty();
     }
 
+    @Override
+    public Collection<String> getCompatibleStateNames() {

Review comment:
       Personally, I am not a big fan of the `Collection` abstraction. Is this 
commonly favored in Flink over exposing some more information and using a 
`List`?

##########
File path: 
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/sink/BatchCommitterHandler.java
##########
@@ -33,44 +30,52 @@
  * Runtime {@link org.apache.flink.streaming.api.operators.StreamOperator} for 
executing {@link
  * Committer} in the batch execution mode.
  *
- * @param <CommT> The committable type of the {@link Committer}.
+ * @param <InputT> The committable type of the {@link Committer}.
  */
-final class BatchCommitterOperator<CommT> extends AbstractStreamOperator<CommT>
-        implements OneInputStreamOperator<CommT, CommT>, BoundedOneInput {
+final class BatchCommitterHandler<InputT, OutputT>
+        extends AbstractCommitterHandler<InputT, OutputT> {
 
     /** Responsible for committing the committable to the external system. */
-    private final Committer<CommT> committer;
+    private final Committer<InputT> committer;
 
-    /** Record all the committables until the end of the input. */
-    private final List<CommT> allCommittables;
+    /**
+     * The committer that is chained to this committer. It's either {@link
+     * GlobalBatchCommitterHandler} or {@link NoopCommitterHandler}.

Review comment:
       Wouldn't is be easier to just not instantiate a `GlobalBatchCommitter` 
if not necessary and not introduce the `NoopCommitterHandler`.

##########
File path: 
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/sink/SinkOperatorFactory.java
##########
@@ -0,0 +1,113 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.runtime.operators.sink;
+
+import org.apache.flink.api.connector.sink.Committer;
+import org.apache.flink.api.connector.sink.Sink;
+import org.apache.flink.api.connector.sink.SinkWriter;
+import org.apache.flink.core.io.SimpleVersionedSerializer;
+import org.apache.flink.streaming.api.operators.AbstractStreamOperatorFactory;
+import org.apache.flink.streaming.api.operators.OneInputStreamOperatorFactory;
+import org.apache.flink.streaming.api.operators.StreamOperator;
+import org.apache.flink.streaming.api.operators.StreamOperatorParameters;
+
+import java.util.Optional;
+
+/**
+ * A {@link org.apache.flink.streaming.api.operators.StreamOperatorFactory} 
for {@link
+ * SinkOperator}.
+ *
+ * @param <InputT> The input type of the {@link SinkWriter}.
+ * @param <CommT> The committable type of the {@link SinkWriter}.
+ * @param <WriterStateT> The type of the {@link SinkWriter Writer's} state.
+ */
+public final class SinkOperatorFactory<InputT, CommT, WriterStateT>

Review comment:
       I wonder if this class should be part of the previous commit IMO it 
belongs to the newly introduced operator model.

##########
File path: 
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/sink/CommitterHandler.java
##########
@@ -0,0 +1,88 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.runtime.operators.sink;
+
+import org.apache.flink.runtime.state.StateInitializationContext;
+import org.apache.flink.runtime.state.StateSnapshotContext;
+import org.apache.flink.util.function.SupplierWithException;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+
+/**
+ * A wrapper around multiple {@link 
org.apache.flink.api.connector.sink.Committer} or {@link
+ * org.apache.flink.api.connector.sink.GlobalCommitter} that manages states.
+ *
+ * @param <InputT>
+ * @param <OutputT>
+ */
+interface CommitterHandler<InputT, OutputT> extends AutoCloseable, 
Serializable {
+
+    /** Initializes the state of the committer and this handler. */
+    default void initializeState(StateInitializationContext context) throws 
Exception {}
+
+    /** Snapshots the state of the committer and this handler. */
+    default void snapshotState(StateSnapshotContext context) throws Exception 
{}
+
+    /**
+     * Processes the committables be either directly transforming them or by 
adding them to the

Review comment:
       `by either directly ...`
   
   FYI I seems Github suggestions are broken ...

##########
File path: 
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/sink/CommitterOperatorFactory.java
##########
@@ -0,0 +1,117 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.runtime.operators.sink;
+
+import org.apache.flink.api.connector.sink.Committer;
+import org.apache.flink.api.connector.sink.GlobalCommitter;
+import org.apache.flink.api.connector.sink.Sink;
+import org.apache.flink.core.io.SimpleVersionedSerializer;
+import org.apache.flink.streaming.api.operators.AbstractStreamOperatorFactory;
+import org.apache.flink.streaming.api.operators.OneInputStreamOperatorFactory;
+import org.apache.flink.streaming.api.operators.StreamOperator;
+import org.apache.flink.streaming.api.operators.StreamOperatorParameters;
+
+import java.io.IOException;
+import java.util.Optional;
+
+import static org.apache.flink.util.Preconditions.checkState;
+
+/**
+ * A {@link org.apache.flink.streaming.api.operators.StreamOperatorFactory} 
for {@link
+ * CommitterOperator}.
+ *
+ * @param <CommT> the type of the committable
+ * @param <GlobalCommT> the type of the global committable
+ */
+public final class CommitterOperatorFactory<CommT, GlobalCommT>
+        extends AbstractStreamOperatorFactory<byte[]>
+        implements OneInputStreamOperatorFactory<byte[], byte[]> {
+
+    private final Sink<?, CommT, ?, GlobalCommT> sink;
+    private final boolean batch;
+
+    public CommitterOperatorFactory(Sink<?, CommT, ?, GlobalCommT> sink, 
boolean batch) {
+        this.sink = sink;
+        this.batch = batch;
+    }
+
+    @Override
+    @SuppressWarnings("unchecked")
+    public <T extends StreamOperator<byte[]>> T createStreamOperator(
+            StreamOperatorParameters<byte[]> parameters) {
+
+        SimpleVersionedSerializer<CommT> committableSerializer =
+                
sink.getCommittableSerializer().orElseThrow(this::noSerializerFound);
+        try {
+            CommitterHandler<CommT, GlobalCommT> committerHandler = 
getGlobalCommitterHandler();
+            if (batch) {
+                Optional<Committer<CommT>> committer = sink.createCommitter();
+                if (committer.isPresent()) {
+                    committerHandler =
+                            new BatchCommitterHandler<>(committer.get(), 
committerHandler);
+                }
+            }
+
+            checkState(
+                    !(committerHandler instanceof NoopCommitterHandler),
+                    "committer operator without commmitter");
+            final CommitterOperator<CommT, GlobalCommT> committerOperator =
+                    new CommitterOperator<>(committableSerializer, 
committerHandler, null);

Review comment:
       I prefer having a separate constructor when fields are nullable.

##########
File path: 
flink-core/src/main/java/org/apache/flink/api/connector/sink/Sink.java
##########
@@ -81,9 +83,17 @@
     /** Return the serializer of the writer's state type. */
     Optional<SimpleVersionedSerializer<WriterStateT>> 
getWriterStateSerializer();
 
+    /**
+     * A list of state names of sinks from which the state can be restored. 
For example, the new
+     * file sink can resume from the state of an old streaming file sink as a 
drop-in replacement
+     * when resuming from a checkpoint/savepoint.
+     */
+    default Collection<String> getCompatibleStateNames() {

Review comment:
       Nit: Do we need a FLIP for this change?

##########
File path: 
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/sink/SinkOperator.java
##########
@@ -0,0 +1,265 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.runtime.operators.sink;
+
+import org.apache.flink.api.connector.sink.Sink;
+import org.apache.flink.api.connector.sink.SinkWriter;
+import org.apache.flink.core.io.SimpleVersionedSerialization;
+import org.apache.flink.core.io.SimpleVersionedSerializer;
+import org.apache.flink.metrics.MetricGroup;
+import org.apache.flink.runtime.state.StateInitializationContext;
+import org.apache.flink.runtime.state.StateSnapshotContext;
+import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
+import org.apache.flink.streaming.api.operators.BoundedOneInput;
+import org.apache.flink.streaming.api.operators.InternalTimerService;
+import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
+import org.apache.flink.streaming.api.watermark.Watermark;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.streaming.runtime.tasks.ProcessingTimeService;
+import org.apache.flink.util.function.BiFunctionWithException;
+
+import javax.annotation.Nullable;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.List;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * An operator that processes records to be written into a {@link
+ * org.apache.flink.api.connector.sink.Sink}. It also has a way to process 
committables with the
+ * same parallelism or send them downstream to a {@link CommitterOperator} 
with a different
+ * parallelism.
+ *
+ * <p>The operator may be part of a sink pipeline and is the first operator. 
There are currently two
+ * ways this operator is used:
+ *
+ * <ul>
+ *   <li>In streaming mode, there is a this operator with parallelism p 
containing {@link

Review comment:
       Remove `a`

##########
File path: 
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/sink/CommitterHandler.java
##########
@@ -0,0 +1,88 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.runtime.operators.sink;
+
+import org.apache.flink.runtime.state.StateInitializationContext;
+import org.apache.flink.runtime.state.StateSnapshotContext;
+import org.apache.flink.util.function.SupplierWithException;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+
+/**
+ * A wrapper around multiple {@link 
org.apache.flink.api.connector.sink.Committer} or {@link
+ * org.apache.flink.api.connector.sink.GlobalCommitter} that manages states.
+ *
+ * @param <InputT>
+ * @param <OutputT>
+ */
+interface CommitterHandler<InputT, OutputT> extends AutoCloseable, 
Serializable {
+
+    /** Initializes the state of the committer and this handler. */
+    default void initializeState(StateInitializationContext context) throws 
Exception {}
+
+    /** Snapshots the state of the committer and this handler. */
+    default void snapshotState(StateSnapshotContext context) throws Exception 
{}
+
+    /**
+     * Processes the committables be either directly transforming them or by 
adding them to the
+     * internal state of this handler. The supplier should only be queried 
once.
+     *
+     * @return a list of output committables that is send downstream.
+     */
+    List<OutputT> processCommittables(
+            SupplierWithException<List<InputT>, IOException> 
committableSupplier)
+            throws IOException;
+
+    /**
+     * Called when no more committables are going to be added through {@link
+     * #processCommittables(SupplierWithException)}.
+     *
+     * @return a list of output committables that is send downstream.
+     */
+    default List<OutputT> endOfInput() throws IOException {
+        return Collections.emptyList();
+    }
+
+    default void close() throws Exception {}
+
+    /** Called when a checkpoint is completed and returns a list of output to 
be sent downstream. */
+    default Collection<OutputT> notifyCheckpointCompleted(long checkpointId) 
throws IOException {
+        return Collections.emptyList();
+    }
+
+    /** Swallows all committables and emits nothing. */
+    @SuppressWarnings("unchecked")
+    static <InputT, OutputT> CommitterHandler<InputT, OutputT> noop() {

Review comment:
       Why is this method needed?

##########
File path: 
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/sink/CommitterOperatorFactory.java
##########
@@ -0,0 +1,117 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.runtime.operators.sink;
+
+import org.apache.flink.api.connector.sink.Committer;
+import org.apache.flink.api.connector.sink.GlobalCommitter;
+import org.apache.flink.api.connector.sink.Sink;
+import org.apache.flink.core.io.SimpleVersionedSerializer;
+import org.apache.flink.streaming.api.operators.AbstractStreamOperatorFactory;
+import org.apache.flink.streaming.api.operators.OneInputStreamOperatorFactory;
+import org.apache.flink.streaming.api.operators.StreamOperator;
+import org.apache.flink.streaming.api.operators.StreamOperatorParameters;
+
+import java.io.IOException;
+import java.util.Optional;
+
+import static org.apache.flink.util.Preconditions.checkState;
+
+/**
+ * A {@link org.apache.flink.streaming.api.operators.StreamOperatorFactory} 
for {@link
+ * CommitterOperator}.
+ *
+ * @param <CommT> the type of the committable
+ * @param <GlobalCommT> the type of the global committable
+ */
+public final class CommitterOperatorFactory<CommT, GlobalCommT>
+        extends AbstractStreamOperatorFactory<byte[]>
+        implements OneInputStreamOperatorFactory<byte[], byte[]> {
+
+    private final Sink<?, CommT, ?, GlobalCommT> sink;
+    private final boolean batch;
+
+    public CommitterOperatorFactory(Sink<?, CommT, ?, GlobalCommT> sink, 
boolean batch) {
+        this.sink = sink;
+        this.batch = batch;
+    }
+
+    @Override
+    @SuppressWarnings("unchecked")
+    public <T extends StreamOperator<byte[]>> T createStreamOperator(
+            StreamOperatorParameters<byte[]> parameters) {
+
+        SimpleVersionedSerializer<CommT> committableSerializer =
+                
sink.getCommittableSerializer().orElseThrow(this::noSerializerFound);
+        try {
+            CommitterHandler<CommT, GlobalCommT> committerHandler = 
getGlobalCommitterHandler();
+            if (batch) {
+                Optional<Committer<CommT>> committer = sink.createCommitter();
+                if (committer.isPresent()) {
+                    committerHandler =
+                            new BatchCommitterHandler<>(committer.get(), 
committerHandler);
+                }
+            }
+
+            checkState(

Review comment:
       Isn't this always failing for batch mode because the batch mode set the 
committerHandler to `BatchCommitterHandler`?

##########
File path: 
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/sink/GlobalBatchCommitterHandler.java
##########
@@ -72,11 +58,12 @@ public void endInput() throws Exception {
             }
         }
         globalCommitter.endOfInput();
+        return Collections.emptyList();
     }
 
     @Override
     public void close() throws Exception {
-        super.close();
         globalCommitter.close();
+        super.close();

Review comment:
       Why do we need this change? I am not fully familiar with the exact 
semantics but first closing the object itself and next the `globalCommitter` 
looks safer.

##########
File path: 
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/sink/BatchCommitterHandler.java
##########
@@ -33,44 +30,52 @@
  * Runtime {@link org.apache.flink.streaming.api.operators.StreamOperator} for 
executing {@link
  * Committer} in the batch execution mode.
  *
- * @param <CommT> The committable type of the {@link Committer}.
+ * @param <InputT> The committable type of the {@link Committer}.
  */
-final class BatchCommitterOperator<CommT> extends AbstractStreamOperator<CommT>
-        implements OneInputStreamOperator<CommT, CommT>, BoundedOneInput {
+final class BatchCommitterHandler<InputT, OutputT>
+        extends AbstractCommitterHandler<InputT, OutputT> {
 
     /** Responsible for committing the committable to the external system. */
-    private final Committer<CommT> committer;
+    private final Committer<InputT> committer;
 
-    /** Record all the committables until the end of the input. */
-    private final List<CommT> allCommittables;
+    /**
+     * The committer that is chained to this committer. It's either {@link
+     * GlobalBatchCommitterHandler} or {@link NoopCommitterHandler}.

Review comment:
       I do not necessarily see the relation to OOP here ;) but isn't it also 
some kind of optimization to not instantiate this Noop object?

##########
File path: 
flink-core/src/main/java/org/apache/flink/api/connector/sink/Sink.java
##########
@@ -48,41 +54,70 @@
 public interface Sink<InputT, CommT, WriterStateT, GlobalCommT> extends 
Serializable {
 
     /**
-     * Create a {@link SinkWriter}.
+     * Create a {@link SinkWriter}. If the application is resumed from a 
checkpoint or savepoint and
+     * the sink is stateful, it will receive the corresponding state obtained 
with {@link
+     * SinkWriter#snapshotState()} and serialized with {@link 
#getWriterStateSerializer()}. If no
+     * state exists, the first existing, compatible state specified in {@link
+     * #getCompatibleStateNames()} will be loaded and passed.
      *
      * @param context the runtime context.
-     * @param states the writer's state.
+     * @param states the writer's previous state.
      * @return A sink writer.
-     * @throws IOException if fail to create a writer.
+     * @throws IOException for any failure during creation.
+     * @see SinkWriter#snapshotState()
+     * @see #getWriterStateSerializer()
+     * @see #getCompatibleStateNames()
      */
     SinkWriter<InputT, CommT, WriterStateT> createWriter(
             InitContext context, List<WriterStateT> states) throws IOException;
 
     /**
-     * Creates a {@link Committer}.
+     * Any stateful sink needs to provide this state serializer and implement 
{@link
+     * SinkWriter#snapshotState()} properly. The respective state is used in 
{@link
+     * #createWriter(InitContext, List)} on recovery.
+     *
+     * @return the serializer of the writer's state type.
+     */
+    Optional<SimpleVersionedSerializer<WriterStateT>> 
getWriterStateSerializer();
+
+    /**
+     * Creates a {@link Committer} which is part of a 2-phase-commit protocol. 
The {@link
+     * SinkWriter} creates committables through {@link 
SinkWriter#prepareCommit(boolean)} in the
+     * first phase. The committables are then passed to this committer and 
persisted with {@link
+     * Committer#commit(List)}. If a committer is returned, the sink must also 
return a {@link
+     * #getCommittableSerializer()}.
      *
-     * @return A committer.
-     * @throws IOException if fail to create a committer.
+     * @return A committer for the 2-phase-commit protocol.
+     * @throws IOException for any failure during creation.
      */
     Optional<Committer<CommT>> createCommitter() throws IOException;
 
     /**
-     * Creates a {@link GlobalCommitter}.
+     * Creates a {@link GlobalCommitter} which is part of a 2-phase-commit 
protocol. The {@link
+     * SinkWriter} creates committables through {@link 
SinkWriter#prepareCommit(boolean)} in the
+     * first phase. The committables are then passed to the Committer and 
persisted with {@link
+     * Committer#commit(List)} which also can return an aggregated 
committable. This aggregated

Review comment:
       Is this correct? AFAIK the return of `Committer#commit()` is only used 
to retry these committables later. Unfortunately, this is not implemented yet, 
so it will throw once the list is not empty.

##########
File path: 
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/sink/CommitterHandler.java
##########
@@ -0,0 +1,71 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.runtime.operators.sink;
+
+import org.apache.flink.runtime.state.StateInitializationContext;
+import org.apache.flink.runtime.state.StateSnapshotContext;
+import org.apache.flink.util.function.SupplierWithException;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+
+/**
+ * A wrapper around multiple {@link 
org.apache.flink.api.connector.sink.Committer} or {@link
+ * org.apache.flink.api.connector.sink.GlobalCommitter} that manages states.
+ *
+ * @param <InputT>
+ * @param <OutputT>
+ */
+interface CommitterHandler<InputT, OutputT> extends AutoCloseable, 
Serializable {
+
+    /** Initializes the state of the committer and this handler. */
+    default void initializeState(StateInitializationContext context) throws 
Exception {}
+
+    /** Snapshots the state of the committer and this handler. */
+    default void snapshotState(StateSnapshotContext context) throws Exception 
{}

Review comment:
       We can also move these to methods to `AbstractStreamingCommitterHandler` 
because AFAIS they are not necessary for the batch committers.

##########
File path: 
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/sink/AbstractCommitterHandler.java
##########
@@ -0,0 +1,53 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.runtime.operators.sink;
+
+import org.apache.flink.util.function.SupplierWithException;
+
+import java.io.IOException;
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Deque;
+import java.util.List;
+
+abstract class AbstractCommitterHandler<InputT, OutputT>
+        implements CommitterHandler<InputT, OutputT> {
+
+    /** Record all the committables until commit. */
+    private final Deque<InputT> committables = new ArrayDeque<>();
+
+    @Override
+    public List<OutputT> processCommittables(
+            SupplierWithException<List<InputT>, IOException> 
committableSupplier)
+            throws IOException {
+        this.committables.addAll(committableSupplier.get());
+        return Collections.emptyList();
+    }
+
+    public List<InputT> pollCommittables() {

Review comment:
       ```suggestion
       protected List<InputT> pollCommittables() {
   ```
   ?

##########
File path: 
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/sink/CommitterHandler.java
##########
@@ -0,0 +1,71 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.runtime.operators.sink;
+
+import org.apache.flink.runtime.state.StateInitializationContext;
+import org.apache.flink.runtime.state.StateSnapshotContext;
+import org.apache.flink.util.function.SupplierWithException;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+
+/**
+ * A wrapper around multiple {@link 
org.apache.flink.api.connector.sink.Committer} or {@link
+ * org.apache.flink.api.connector.sink.GlobalCommitter} that manages states.
+ *
+ * @param <InputT>
+ * @param <OutputT>
+ */
+interface CommitterHandler<InputT, OutputT> extends AutoCloseable, 
Serializable {
+
+    /** Initializes the state of the committer and this handler. */
+    default void initializeState(StateInitializationContext context) throws 
Exception {}
+
+    /** Snapshots the state of the committer and this handler. */
+    default void snapshotState(StateSnapshotContext context) throws Exception 
{}
+
+    /**
+     * Processes the committables by either directly transforming them or by 
adding them to the
+     * internal state of this handler. The supplier should only be queried 
once.
+     *
+     * @return a list of output committables that is send downstream.
+     */
+    List<OutputT> processCommittables(
+            SupplierWithException<List<InputT>, IOException> 
committableSupplier)
+            throws IOException;
+
+    /**
+     * Called when no more committables are going to be added through {@link
+     * #processCommittables(SupplierWithException)}.
+     *
+     * @return a list of output committables that is send downstream.
+     */
+    default List<OutputT> endOfInput() throws IOException {
+        return Collections.emptyList();
+    }
+
+    default void close() throws Exception {}

Review comment:
       Nit: If the default implementation is only for `NoopCommitterHandler` we 
can think of just implementing it there.

##########
File path: 
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/sink/CommitterHandler.java
##########
@@ -0,0 +1,71 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.runtime.operators.sink;
+
+import org.apache.flink.runtime.state.StateInitializationContext;
+import org.apache.flink.runtime.state.StateSnapshotContext;
+import org.apache.flink.util.function.SupplierWithException;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+
+/**
+ * A wrapper around multiple {@link 
org.apache.flink.api.connector.sink.Committer} or {@link
+ * org.apache.flink.api.connector.sink.GlobalCommitter} that manages states.
+ *
+ * @param <InputT>
+ * @param <OutputT>
+ */
+interface CommitterHandler<InputT, OutputT> extends AutoCloseable, 
Serializable {
+
+    /** Initializes the state of the committer and this handler. */
+    default void initializeState(StateInitializationContext context) throws 
Exception {}
+
+    /** Snapshots the state of the committer and this handler. */
+    default void snapshotState(StateSnapshotContext context) throws Exception 
{}
+
+    /**
+     * Processes the committables by either directly transforming them or by 
adding them to the
+     * internal state of this handler. The supplier should only be queried 
once.
+     *
+     * @return a list of output committables that is send downstream.
+     */
+    List<OutputT> processCommittables(
+            SupplierWithException<List<InputT>, IOException> 
committableSupplier)
+            throws IOException;
+
+    /**
+     * Called when no more committables are going to be added through {@link
+     * #processCommittables(SupplierWithException)}.
+     *
+     * @return a list of output committables that is send downstream.
+     */
+    default List<OutputT> endOfInput() throws IOException {

Review comment:
       Out of curiosity why does the `StreamingCommitterHandler` not implement 
`endOfInput`?

##########
File path: 
flink-streaming-java/src/test/java/org/apache/flink/streaming/util/TestHarnessUtil.java
##########
@@ -139,6 +139,7 @@ public static void assertNoLateRecords(Iterable<Object> 
elements) {
 
         testHarness.processElements(
                 
input.stream().map(StreamRecord::new).collect(Collectors.toList()));
+        testHarness.prepareSnapshotPreBarrier(1);

Review comment:
       Nit: Why was this changed needed?

##########
File path: 
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/sink/CommitterHandler.java
##########
@@ -0,0 +1,71 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.runtime.operators.sink;
+
+import org.apache.flink.runtime.state.StateInitializationContext;
+import org.apache.flink.runtime.state.StateSnapshotContext;
+import org.apache.flink.util.function.SupplierWithException;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+
+/**
+ * A wrapper around multiple {@link 
org.apache.flink.api.connector.sink.Committer} or {@link

Review comment:
       Nit: I find the word `multiple` a bit confusing here because afait it 
always exactly wraps one instance of a committer.

##########
File path: 
flink-core/src/main/java/org/apache/flink/api/connector/sink/Sink.java
##########
@@ -46,44 +54,81 @@
 public interface Sink<InputT, CommT, WriterStateT, GlobalCommT> extends 
Serializable {
 
     /**
-     * Create a {@link SinkWriter}.
+     * Create a {@link SinkWriter}. If the application is resumed from a 
checkpoint or savepoint and
+     * the sink is stateful, it will receive the corresponding state obtained 
with {@link
+     * SinkWriter#snapshotState()} and serialized with {@link 
#getWriterStateSerializer()}. If no
+     * state exists, the first existing, compatible state specified in {@link
+     * #getCompatibleStateNames()} will be loaded and passed.
      *
      * @param context the runtime context.
-     * @param states the writer's state.
+     * @param states the writer's previous state.
      * @return A sink writer.
-     * @throws IOException if fail to create a writer.
+     * @throws IOException for any failure during creation.
+     * @see SinkWriter#snapshotState()
+     * @see #getWriterStateSerializer()
+     * @see #getCompatibleStateNames()
      */
     SinkWriter<InputT, CommT, WriterStateT> createWriter(
             InitContext context, List<WriterStateT> states) throws IOException;
 
     /**
-     * Creates a {@link Committer}.
+     * Any stateful sink needs to provide this state serializer and implement 
{@link
+     * SinkWriter#snapshotState()} properly. The respective state is used in 
{@link
+     * #createWriter(InitContext, List)} on recovery.
+     *
+     * @return the serializer of the writer's state type.
+     */
+    Optional<SimpleVersionedSerializer<WriterStateT>> 
getWriterStateSerializer();
+
+    /**
+     * Creates a {@link Committer} which is part of a 2-phase-commit protocol. 
The {@link
+     * SinkWriter} creates committables through {@link 
SinkWriter#prepareCommit(boolean)} in the
+     * first phase. The committables are then passed to this committer and 
persisted with {@link
+     * Committer#commit(List)}. If a committer is returned, the sink must also 
return a {@link
+     * #getCommittableSerializer()}.
      *
-     * @return A committer.
-     * @throws IOException if fail to create a committer.
+     * @return A committer for the 2-phase-commit protocol.
+     * @throws IOException for any failure during creation.
      */
     Optional<Committer<CommT>> createCommitter() throws IOException;
 
     /**
-     * Creates a {@link GlobalCommitter}.
+     * Creates a {@link GlobalCommitter} which is part of a 2-phase-commit 
protocol. The {@link
+     * SinkWriter} creates committables through {@link 
SinkWriter#prepareCommit(boolean)} in the
+     * first phase. The committables are then passed to the Committer and 
persisted with {@link
+     * Committer#commit(List)} which also can return an aggregated 
committable. This aggregated
+     * committables are passed to this {@link GlobalCommitter} of which only a 
single instance
+     * exists. If a global committer is returned, the sink must also return a 
{@link
+     * #getCommittableSerializer()} and {@link 
#getGlobalCommittableSerializer()}.
      *
-     * @return A global committer.
-     * @throws IOException if fail to create a global committer.
+     * @return A global committer for the 2-phase-commit protocol.
+     * @throws IOException for any failure during creation.
      */
     Optional<GlobalCommitter<CommT, GlobalCommT>> createGlobalCommitter() 
throws IOException;
 
-    /** Returns the serializer of the committable type. */
+    /**
+     * Returns the serializer of the committable type. The serializer is 
required iff the sink has a
+     * {@link Committer} or {@link GlobalCommitter}.
+     */
     Optional<SimpleVersionedSerializer<CommT>> getCommittableSerializer();
 
-    /** Returns the serializer of the aggregated committable type. */
+    /**
+     * Returns the serializer of the aggregated committable type. The 
serializer is required iff the
+     * sink has a {@link GlobalCommitter}.
+     */
     Optional<SimpleVersionedSerializer<GlobalCommT>> 
getGlobalCommittableSerializer();
 
-    /** Return the serializer of the writer's state type. */
-    Optional<SimpleVersionedSerializer<WriterStateT>> 
getWriterStateSerializer();
+    /**
+     * A list of state names of sinks from which the state can be restored. 
For example, the new
+     * file sink can resume from the state of an old streaming file sink as a 
drop-in replacement

Review comment:
       Nit: Maybe use a docstring to reference the example file sinks. The 
terms old and new might be forgotten in the future.

##########
File path: 
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/sink/CommitterHandler.java
##########
@@ -0,0 +1,71 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.runtime.operators.sink;
+
+import org.apache.flink.runtime.state.StateInitializationContext;
+import org.apache.flink.runtime.state.StateSnapshotContext;
+import org.apache.flink.util.function.SupplierWithException;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+
+/**
+ * A wrapper around multiple {@link 
org.apache.flink.api.connector.sink.Committer} or {@link
+ * org.apache.flink.api.connector.sink.GlobalCommitter} that manages states.
+ *
+ * @param <InputT>
+ * @param <OutputT>
+ */
+interface CommitterHandler<InputT, OutputT> extends AutoCloseable, 
Serializable {
+
+    /** Initializes the state of the committer and this handler. */
+    default void initializeState(StateInitializationContext context) throws 
Exception {}
+
+    /** Snapshots the state of the committer and this handler. */
+    default void snapshotState(StateSnapshotContext context) throws Exception 
{}

Review comment:
       We can also move these two methods to 
`AbstractStreamingCommitterHandler` because AFAIS they are not necessary for 
the batch committers.

##########
File path: 
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/sink/CommitterHandler.java
##########
@@ -0,0 +1,71 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.runtime.operators.sink;
+
+import org.apache.flink.runtime.state.StateInitializationContext;
+import org.apache.flink.runtime.state.StateSnapshotContext;
+import org.apache.flink.util.function.SupplierWithException;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+
+/**
+ * A wrapper around multiple {@link 
org.apache.flink.api.connector.sink.Committer} or {@link
+ * org.apache.flink.api.connector.sink.GlobalCommitter} that manages states.
+ *
+ * @param <InputT>
+ * @param <OutputT>
+ */
+interface CommitterHandler<InputT, OutputT> extends AutoCloseable, 
Serializable {
+
+    /** Initializes the state of the committer and this handler. */
+    default void initializeState(StateInitializationContext context) throws 
Exception {}
+
+    /** Snapshots the state of the committer and this handler. */
+    default void snapshotState(StateSnapshotContext context) throws Exception 
{}

Review comment:
       I am just a bit unhappy with the overall structure of having this kind 
of deep inheritance but I guess refactoring this is probably out of scope. 




-- 
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.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


Reply via email to