openinx commented on a change in pull request #1793:
URL: https://github.com/apache/iceberg/pull/1793#discussion_r556249432



##########
File path: 
flink/src/main/java/org/apache/iceberg/flink/source/StreamingReaderOperator.java
##########
@@ -0,0 +1,233 @@
+/*
+ * 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.iceberg.flink.source;
+
+import java.io.IOException;
+import java.util.Queue;
+import org.apache.flink.api.common.state.ListState;
+import org.apache.flink.api.common.state.ListStateDescriptor;
+import org.apache.flink.runtime.state.JavaSerializer;
+import org.apache.flink.runtime.state.StateInitializationContext;
+import org.apache.flink.runtime.state.StateSnapshotContext;
+import org.apache.flink.streaming.api.functions.source.SourceFunction;
+import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
+import org.apache.flink.streaming.api.operators.AbstractStreamOperatorFactory;
+import org.apache.flink.streaming.api.operators.MailboxExecutor;
+import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
+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 org.apache.flink.streaming.api.operators.StreamSourceContexts;
+import org.apache.flink.streaming.api.operators.YieldingOperatorFactory;
+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.table.data.RowData;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * The operator that reads the {@link FlinkInputSplit splits} received from 
the preceding {@link
+ * StreamingMonitorFunction}. Contrary to the {@link StreamingMonitorFunction} 
which has a parallelism of 1,
+ * this operator can have multiple parallelism.
+ *
+ * <p>As soon as a split descriptor is received, it is put in a queue, and use 
{@link MailboxExecutor}
+ * read the actual data of the split. This architecture allows the separation 
of the reading thread from the one split
+ * processing the checkpoint barriers, thus removing any potential 
back-pressure.
+ */
+public class StreamingReaderOperator extends AbstractStreamOperator<RowData>
+    implements OneInputStreamOperator<FlinkInputSplit, RowData> {
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(StreamingReaderOperator.class);
+
+  private final MailboxExecutor executor;

Review comment:
       We cannot use a newly created thread to process the split asynchronously 
because it will break the checkpoint mechanism which depends on mail-box model 
in flink runtime,  Assume the asynchronously thread keep processing the records 
of the newly split (which is polled from splits queue),  now the flink 
checkpoint barrier come,  How should we coordinate the checkpoint barrier and 
the processing split so that the barrier could effect ( trigger to persist all 
states of this operator ) once the current split is finished ?  Will we go back 
to use the checkpoint lock to synchronize between checkpoint barrier event and 
the newly introduced async thread ? 
   
   In the current mail-box model,  both flink's internal control events and 
user-provided events will be processed in the same thread ( Which is 
[StreamTask](https://github.com/apache/flink/blob/master/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java#L151)
 in flink runtime).   The StreamTask will run in a endless loop to process the 
event from mail box queue ( see the code 
[here](https://github.com/apache/flink/blob/9313b01a73d5d05fab2fc8be8fac21a3ede039d6/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java#L572)).
  For each loop,  it will: 
   
   1.    Try to take events which has been enqueued in mail-box queue  ( 
[code](https://github.com/apache/flink/blob/9313b01a73d5d05fab2fc8be8fac21a3ede039d6/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java#L572)),
 those events are flink's control events, such as action to trigger flink's 
checkpoint , action to notify checkpoint complete etc.   If there's no events 
enqueued in mail-box queue,  then the `processMail` will do nothing. 
   
   2.   Then read one complete record from the flink's network queue and invoke 
the `processElement(record)` ,  that's the process about incremental compute.  
Take the `sum` as an example,   once a record come, the `processElement` will 
increment its counter.  
   
   So all the events (Regardless of flink's control events or user's events )  
are being processed in the same thread `StreamTask`.   In our flink streaming 
reader,  we only need to control that there's only one split that is being 
processing,  then the newly triggered checkpoint action could just enqueue the 
mail-box queue.  Once the processing splits is finished,  the `StreamTask` will 
execute the checkpoint action in step.1 .    We don't need any extra checkpoint 
lock or extra synchronization. 
   
   That's why we use Mailbox to enqueue the action  ( 
[processSplits](https://github.com/apache/iceberg/pull/1793/files#diff-22513db68a66d835391d1dad5b0b95ed776dbcc3df7c8df7df5450930f367893R135)
 )to process whole elements from a given split. 




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



---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to