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



##########
File path: 
flink-core/src/main/java/org/apache/flink/api/connector/sink2/StatefulSink.java
##########
@@ -0,0 +1,97 @@
+/*
+ * 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.api.connector.sink2;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.core.io.SimpleVersionedSerializer;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.List;
+
+/**
+ * A {@link Sink} with a stateful {@link SinkWriter}.
+ *
+ * <p>The {@link StatefulSink} needs to be serializable. All configuration 
should be validated
+ * eagerly. The respective sink writers are transient and will only be created 
in the subtasks on
+ * the taskmanagers.
+ *
+ * @param <InputT> The type of the sink's input
+ * @param <WriterStateT> The type of the sink writer's state
+ */
+@PublicEvolving
+public interface StatefulSink<InputT, WriterStateT> extends Sink<InputT> {
+
+    /**
+     * Create a {@link StatefulSinkWriter}.
+     *
+     * @param context the runtime context.
+     * @return A sink writer.
+     * @throws IOException for any failure during creation.
+     */
+    StatefulSinkWriter<InputT, WriterStateT> createWriter(InitContext context) 
throws IOException;
+
+    /**
+     * Create a {@link StatefulSinkWriter} from a recovered state.
+     *
+     * @param context the runtime context.
+     * @return A sink writer.
+     * @throws IOException for any failure during creation.
+     */
+    StatefulSinkWriter<InputT, WriterStateT> restoreWriter(
+            InitContext context, Collection<WriterStateT> recoveredState) 
throws IOException;
+
+    /**
+     * Any stateful sink needs to provide this state serializer and implement 
{@link
+     * StatefulSinkWriter#snapshotState(long)} properly. The respective state 
is used in {@link
+     * #restoreWriter(InitContext, Collection)} on recovery.
+     *
+     * @return the serializer of the writer's state type.
+     */
+    SimpleVersionedSerializer<WriterStateT> getWriterStateSerializer();

Review comment:
       Removing all these optionals was one of the intentions behind designing 
the new interfaces. Sink developers can now explicitly decide which 
functionality they want to support and implement the interfaces accordingly 
[1]. With the Sink V1 interfaces they basically always had to implement 
everything except that some of the methods have default implementations.
   
   [1] 
https://cwiki.apache.org/confluence/display/FLINK/FLIP-191%3A+Extend+unified+Sink+interface+to+support+small+file+compaction#FLIP191:ExtendunifiedSinkinterfacetosupportsmallfilecompaction-SimpleSink




-- 
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: issues-unsubscr...@flink.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


Reply via email to