openinx commented on a change in pull request #1793: URL: https://github.com/apache/iceberg/pull/1793#discussion_r556252305
########## 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; + private FlinkInputFormat format; + + private transient SourceFunction.SourceContext<RowData> sourceContext; + + private transient ListState<FlinkInputSplit> inputSplitsState; + private transient Queue<FlinkInputSplit> splits; + + // This state is used to control that only one split is occupying the executor for record reading. If there're more + // splits coming, we will buffer them in flink's state. Once the executor is idle (the currentSplitState will be + // marked as IDLE), it will schedule one new split from buffered splits in flink's state to executor (the + // currentSplitState will be marked as RUNNING). After finished all records processing, the currentSplitState will + // be marked as IDLE again. + // NOTICE: all the reader and writer of this variable are the same thread, so we don't need extra synchronization. + private transient SplitState currentSplitState; Review comment: There's only one thread (the thread that running `StreamTask` ) to access this `currentSplitState`, so we don't have make it to be `volatile`. Hope I explained the mechanism clearly in above comment, if it's still no clear, Pls bring it up , I'd like to provide more background :-) ---------------------------------------------------------------- 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]
