gaoyunhaii commented on a change in pull request #13617:
URL: https://github.com/apache/flink/pull/13617#discussion_r504512094



##########
File path: 
flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/sink/AbstractWriterOperator.java
##########
@@ -0,0 +1,147 @@
+/*
+ * 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.api.operators.sink;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.connector.sink.Sink;
+import org.apache.flink.api.connector.sink.Writer;
+import org.apache.flink.metrics.MetricGroup;
+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 java.util.List;
+
+/**
+ * This is the abstract class used for executing the sink's {@link Writer} 
which is proposed by FLIP-143.
+ * The sub-class of this class is responsible for creating the specific {@link 
Writer} by implementing the {@link #createWriter()}.
+ *
+ * @param <InputT>   The type of writer's input
+ * @param <CommT>    The type of information needed to commit data staged by 
the sink
+ */
+@Internal
+abstract class AbstractWriterOperator<InputT, CommT> extends 
AbstractStreamOperator<CommT>
+       implements OneInputStreamOperator<InputT, CommT>, BoundedOneInput {
+
+       private static final long serialVersionUID = 1L;
+
+       /** The runtime information of the input element. */
+       private final Context<InputT> context;
+
+       // ------------------------------- runtime fields 
---------------------------------------
+
+       /** We listen to this ourselves because we don't have an {@link 
InternalTimerService}. */
+       private Long currentWatermark;
+
+       /** The sink writer that does most of the work. */
+       private Writer<InputT, CommT, ?> writer;
+
+       AbstractWriterOperator() {
+               this.context = new Context<>();
+       }
+
+       @Override
+       public void open() throws Exception {
+               super.open();
+
+               this.currentWatermark = Long.MIN_VALUE;
+
+               writer = createWriter();
+       }
+
+       @Override
+       public void processElement(StreamRecord<InputT> element) throws 
Exception {
+               context.element = element;
+               writer.write(element.getValue(), context);
+       }
+
+       @Override
+       public void prepareSnapshotPreBarrier(long checkpointId) throws 
Exception {
+               super.prepareSnapshotPreBarrier(checkpointId);
+               sendCommittables(writer.prepareCommit(false));
+       }
+
+       @Override
+       public void processWatermark(Watermark mark) throws Exception {
+               super.processWatermark(mark);
+               this.currentWatermark = mark.getTimestamp();
+       }
+
+       @Override
+       public void endInput() throws Exception {
+               sendCommittables(writer.prepareCommit(true));
+       }
+
+       @Override
+       public void close() throws Exception {
+               super.close();
+               writer.close();
+       }
+
+       protected Sink.InitContext createInitContext() {
+               return new Sink.InitContext() {
+                       @Override
+                       public int getSubtaskId() {
+                               return 
getRuntimeContext().getIndexOfThisSubtask();
+                       }
+
+                       @Override
+                       public MetricGroup metricGroup() {
+                               return getMetricGroup();
+                       }
+               };
+       }
+
+       /**
+        * @return A sink {@link Writer}
+        * @throws Exception If creating {@link Writer} fail
+        */
+       abstract Writer<InputT, CommT, ?> createWriter() throws Exception;
+
+       protected Writer<InputT, CommT, ?> getWriter() {

Review comment:
       For me I would more like removing `getWriter` method and directly make 
`writer` to be `protected` since `getWriter` is a little confusing when 
occurring together with `createWriter`. 
   
   Currently the override of this method in `StatefulWriterOperator` is only 
used in `snapshotState`, I think it would be ok for us to do type casting 
there. 




----------------------------------------------------------------
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:
us...@infra.apache.org


Reply via email to