FangYongs commented on code in PR #1022: URL: https://github.com/apache/incubator-paimon/pull/1022#discussion_r1177407410
########## paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/MonitorFunction.java: ########## @@ -0,0 +1,222 @@ +/* + * 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.paimon.flink.source.operator; + +import org.apache.paimon.flink.utils.JavaTypeInfo; +import org.apache.paimon.table.source.DataSplit; +import org.apache.paimon.table.source.EndOfScanException; +import org.apache.paimon.table.source.ReadBuilder; +import org.apache.paimon.table.source.Split; +import org.apache.paimon.table.source.StreamTableScan; + +import org.apache.flink.api.common.state.CheckpointListener; +import org.apache.flink.api.common.state.ListState; +import org.apache.flink.api.common.state.ListStateDescriptor; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.base.LongSerializer; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.api.java.typeutils.runtime.TupleSerializer; +import org.apache.flink.runtime.state.FunctionInitializationContext; +import org.apache.flink.runtime.state.FunctionSnapshotContext; +import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.functions.source.RichSourceFunction; +import org.apache.flink.table.data.RowData; +import org.apache.flink.util.Preconditions; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.ArrayList; +import java.util.List; +import java.util.NavigableMap; +import java.util.OptionalLong; +import java.util.TreeMap; + +/** + * This is the single (non-parallel) monitoring task, it is responsible for: + * + * <ol> + * <li>Monitoring snapshots of the Paimon table. + * <li>Creating the {@link Split splits} corresponding to the incremental files + * <li>Assigning them to downstream tasks for further processing. + * </ol> + * + * <p>The splits to be read are forwarded to the downstream {@link ReadOperator} which can have + * parallelism greater than one. + */ +public class MonitorFunction extends RichSourceFunction<Split> + implements CheckpointedFunction, CheckpointListener { + + private static final long serialVersionUID = 1L; + + private static final Logger LOG = LoggerFactory.getLogger(MonitorFunction.class); + + private final ReadBuilder readBuilder; + private final long monitorInterval; + + private volatile boolean isRunning = true; + + private transient StreamTableScan scan; + private transient SourceContext<Split> ctx; + + private transient ListState<Long> checkpointState; + private transient ListState<Tuple2<Long, Long>> nextSnapshotState; + private transient TreeMap<Long, Long> nextSnapshotPerCheckpoint; + + public MonitorFunction(ReadBuilder readBuilder, long monitorInterval) { + this.readBuilder = readBuilder; + this.monitorInterval = monitorInterval; + } + + @Override + public void initializeState(FunctionInitializationContext context) throws Exception { + this.scan = readBuilder.newStreamScan(); + + this.checkpointState = + context.getOperatorStateStore() + .getListState( + new ListStateDescriptor<>( + "next-snapshot", LongSerializer.INSTANCE)); + + @SuppressWarnings("unchecked") + final Class<Tuple2<Long, Long>> typedTuple = + (Class<Tuple2<Long, Long>>) (Class<?>) Tuple2.class; + this.nextSnapshotState = + context.getOperatorStateStore() + .getListState( + new ListStateDescriptor<>( + "next-snapshot-per-checkpoint", + new TupleSerializer<>( + typedTuple, + new TypeSerializer[] { + LongSerializer.INSTANCE, LongSerializer.INSTANCE + }))); + + this.nextSnapshotPerCheckpoint = new TreeMap<>(); + + if (context.isRestored()) { + LOG.info("Restoring state for the {}.", getClass().getSimpleName()); + + List<Long> retrievedStates = new ArrayList<>(); + for (Long entry : this.checkpointState.get()) { + retrievedStates.add(entry); + } + + // given that the parallelism of the function is 1, we can only have 1 retrieved items. + + Preconditions.checkArgument( + retrievedStates.size() <= 1, + getClass().getSimpleName() + " retrieved invalid state."); + + if (retrievedStates.size() == 1) { + this.scan.restore(retrievedStates.get(0)); Review Comment: Should we need to restore scan with the snapshot id in state or in consume-id file here? -- 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...@paimon.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org