From acaf6552afd4817dac0b6344f05c840f80bebabe Mon Sep 17 00:00:00 2001 From: yunfengzhou-hub Date: Mon, 18 Nov 2024 11:37:42 +0800 Subject: [PATCH] [flink] Replace legacy SourceFunction with v2 Source --- .../flink/source/operator/MonitorSource.java} | 8 +- .../flink/source/operator/MonitorSource.java | 277 ++++++++++++++++++ .../flink/source/operator/MonitorSource.java | 277 ++++++++++++++++++ .../flink/source/operator/MonitorSource.java | 277 ++++++++++++++++++ .../flink/source/operator/MonitorSource.java | 277 ++++++++++++++++++ paimon-flink/paimon-flink-1.20/pom.xml | 21 ++ .../flink/source/operator/MonitorSource.java | 277 ++++++++++++++++++ .../action/cdc/SynchronizationActionBase.java | 62 ++-- .../sink/cdc/FlinkCdcMultiTableSinkTest.java | 10 +- .../cdc/FlinkCdcSyncDatabaseSinkITCase.java | 6 +- .../sink/cdc/FlinkCdcSyncTableSinkITCase.java | 6 +- .../paimon/flink/sink/cdc/TestCdcSource.java | 123 ++++++++ .../flink/sink/cdc/TestCdcSourceFunction.java | 107 ------- .../compact/MultiAwareBucketTableScan.java | 12 +- .../flink/compact/MultiTableScanBase.java | 14 +- .../compact/MultiUnawareBucketTableScan.java | 12 +- .../UnawareBucketCompactionTopoBuilder.java | 2 +- .../flink/service/QueryFileMonitor.java | 95 +++--- .../source/AbstractNonCoordinatedSource.java | 50 ++++ .../AbstractNonCoordinatedSourceReader.java | 51 ++++ .../source/BucketUnawareCompactSource.java | 118 ++++---- .../CombinedTableCompactorSourceBuilder.java | 16 +- .../flink/source/FlinkSourceBuilder.java | 4 +- .../paimon/flink/source/NoOpEnumState.java | 22 ++ .../flink/source/NoOpEnumStateSerializer.java | 41 +++ .../paimon/flink/source/NoOpEnumerator.java | 54 ++++ .../flink/source/SimpleSourceSplit.java | 39 +++ .../source/SimpleSourceSplitSerializer.java | 45 +++ ...ion.java => CombinedAwareBatchSource.java} | 92 +++--- ...java => CombinedAwareStreamingSource.java} | 94 +++--- ...tion.java => CombinedCompactorSource.java} | 45 +-- ...n.java => CombinedUnawareBatchSource.java} | 99 ++++--- ...va => CombinedUnawareStreamingSource.java} | 98 +++---- .../flink/source/operator/MonitorSource.java | 248 ++++++++++++++++ .../operator/MultiTablesReadOperator.java | 5 +- .../MultiUnawareTablesReadOperator.java | 2 +- .../flink/source/operator/ReadOperator.java | 4 +- .../apache/paimon/flink/FileStoreITCase.java | 18 +- .../apache/paimon/flink/FiniteTestSource.java | 176 ++++++----- .../paimon/flink/SerializableRowData.java | 4 +- .../UnawareBucketAppendOnlyTableITCase.java | 90 +++--- .../source/operator/OperatorSourceTest.java | 113 ++++--- .../operator/TestingSourceOperator.java | 175 +++++++++++ 43 files changed, 2831 insertions(+), 735 deletions(-) rename paimon-flink/{paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/MonitorFunction.java => paimon-flink-1.15/src/main/java/org/apache/paimon/flink/source/operator/MonitorSource.java} (98%) create mode 100644 paimon-flink/paimon-flink-1.16/src/main/java/org/apache/paimon/flink/source/operator/MonitorSource.java create mode 100644 paimon-flink/paimon-flink-1.17/src/main/java/org/apache/paimon/flink/source/operator/MonitorSource.java create mode 100644 paimon-flink/paimon-flink-1.18/src/main/java/org/apache/paimon/flink/source/operator/MonitorSource.java create mode 100644 paimon-flink/paimon-flink-1.19/src/main/java/org/apache/paimon/flink/source/operator/MonitorSource.java create mode 100644 paimon-flink/paimon-flink-1.20/src/main/java/org/apache/paimon/flink/source/operator/MonitorSource.java create mode 100644 paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/TestCdcSource.java delete mode 100644 paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/TestCdcSourceFunction.java create mode 100644 paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/AbstractNonCoordinatedSource.java create mode 100644 paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/AbstractNonCoordinatedSourceReader.java create mode 100644 paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/NoOpEnumState.java create mode 100644 paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/NoOpEnumStateSerializer.java create mode 100644 paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/NoOpEnumerator.java create mode 100644 paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/SimpleSourceSplit.java create mode 100644 paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/SimpleSourceSplitSerializer.java rename paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/{CombinedAwareBatchSourceFunction.java => CombinedAwareBatchSource.java} (66%) rename paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/{CombinedAwareStreamingSourceFunction.java => CombinedAwareStreamingSource.java} (64%) rename paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/{CombinedCompactorSourceFunction.java => CombinedCompactorSource.java} (65%) rename paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/{CombinedUnawareBatchSourceFunction.java => CombinedUnawareBatchSource.java} (71%) rename paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/{CombinedUnawareStreamingSourceFunction.java => CombinedUnawareStreamingSource.java} (58%) create mode 100644 paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/MonitorSource.java create mode 100644 paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/source/operator/TestingSourceOperator.java diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/MonitorFunction.java b/paimon-flink/paimon-flink-1.15/src/main/java/org/apache/paimon/flink/source/operator/MonitorSource.java similarity index 98% rename from paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/MonitorFunction.java rename to paimon-flink/paimon-flink-1.15/src/main/java/org/apache/paimon/flink/source/operator/MonitorSource.java index 3805f6f8c536e..b38e32559c369 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/MonitorFunction.java +++ b/paimon-flink/paimon-flink-1.15/src/main/java/org/apache/paimon/flink/source/operator/MonitorSource.java @@ -77,12 +77,12 @@ * pass the watermark recorded in the snapshot. * */ -public class MonitorFunction extends RichSourceFunction +public class MonitorSource extends RichSourceFunction implements CheckpointedFunction, CheckpointListener { private static final long serialVersionUID = 1L; - private static final Logger LOG = LoggerFactory.getLogger(MonitorFunction.class); + private static final Logger LOG = LoggerFactory.getLogger(MonitorSource.class); private final ReadBuilder readBuilder; private final long monitorInterval; @@ -97,7 +97,7 @@ public class MonitorFunction extends RichSourceFunction private transient ListState> nextSnapshotState; private transient TreeMap nextSnapshotPerCheckpoint; - public MonitorFunction( + public MonitorSource( ReadBuilder readBuilder, long monitorInterval, boolean emitSnapshotWatermark) { this.readBuilder = readBuilder; this.monitorInterval = monitorInterval; @@ -238,7 +238,7 @@ public static DataStream buildSource( BucketMode bucketMode) { SingleOutputStreamOperator singleOutputStreamOperator = env.addSource( - new MonitorFunction( + new MonitorSource( readBuilder, monitorInterval, emitSnapshotWatermark), name + "-Monitor", new JavaTypeInfo<>(Split.class)) diff --git a/paimon-flink/paimon-flink-1.16/src/main/java/org/apache/paimon/flink/source/operator/MonitorSource.java b/paimon-flink/paimon-flink-1.16/src/main/java/org/apache/paimon/flink/source/operator/MonitorSource.java new file mode 100644 index 0000000000000..b38e32559c369 --- /dev/null +++ b/paimon-flink/paimon-flink-1.16/src/main/java/org/apache/paimon/flink/source/operator/MonitorSource.java @@ -0,0 +1,277 @@ +/* + * 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.BucketMode; +import org.apache.paimon.table.sink.ChannelComputer; +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.datastream.SingleOutputStreamOperator; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.functions.source.RichSourceFunction; +import org.apache.flink.streaming.api.watermark.Watermark; +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; + +import static org.apache.paimon.table.BucketMode.BUCKET_UNAWARE; + +/** + * This is the single (non-parallel) monitoring task, it is responsible for: + * + *
    + *
  1. Monitoring snapshots of the Paimon table. + *
  2. Creating the {@link Split splits} corresponding to the incremental files + *
  3. Assigning them to downstream tasks for further processing. + *
+ * + *

The splits to be read are forwarded to the downstream {@link ReadOperator} which can have + * parallelism greater than one. + * + *

Currently, there are two features that rely on this monitor: + * + *

    + *
  1. Consumer-id: rely on this function to do aligned snapshot consumption, and ensure that all + * data in a snapshot is consumed within each checkpoint. + *
  2. Snapshot-watermark: when there is no watermark definition, the default Paimon table will + * pass the watermark recorded in the snapshot. + *
+ */ +public class MonitorSource extends RichSourceFunction + implements CheckpointedFunction, CheckpointListener { + + private static final long serialVersionUID = 1L; + + private static final Logger LOG = LoggerFactory.getLogger(MonitorSource.class); + + private final ReadBuilder readBuilder; + private final long monitorInterval; + private final boolean emitSnapshotWatermark; + + private volatile boolean isRunning = true; + + private transient StreamTableScan scan; + private transient SourceContext ctx; + + private transient ListState checkpointState; + private transient ListState> nextSnapshotState; + private transient TreeMap nextSnapshotPerCheckpoint; + + public MonitorSource( + ReadBuilder readBuilder, long monitorInterval, boolean emitSnapshotWatermark) { + this.readBuilder = readBuilder; + this.monitorInterval = monitorInterval; + this.emitSnapshotWatermark = emitSnapshotWatermark; + } + + @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> typedTuple = + (Class>) (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 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)); + } + + for (Tuple2 tuple2 : nextSnapshotState.get()) { + nextSnapshotPerCheckpoint.put(tuple2.f0, tuple2.f1); + } + } else { + LOG.info("No state to restore for the {}.", getClass().getSimpleName()); + } + } + + @Override + public void snapshotState(FunctionSnapshotContext ctx) throws Exception { + this.checkpointState.clear(); + Long nextSnapshot = this.scan.checkpoint(); + if (nextSnapshot != null) { + this.checkpointState.add(nextSnapshot); + this.nextSnapshotPerCheckpoint.put(ctx.getCheckpointId(), nextSnapshot); + } + + List> nextSnapshots = new ArrayList<>(); + this.nextSnapshotPerCheckpoint.forEach((k, v) -> nextSnapshots.add(new Tuple2<>(k, v))); + this.nextSnapshotState.update(nextSnapshots); + + if (LOG.isDebugEnabled()) { + LOG.debug("{} checkpoint {}.", getClass().getSimpleName(), nextSnapshot); + } + } + + @SuppressWarnings("BusyWait") + @Override + public void run(SourceContext ctx) throws Exception { + this.ctx = ctx; + while (isRunning) { + boolean isEmpty; + synchronized (ctx.getCheckpointLock()) { + if (!isRunning) { + return; + } + try { + List splits = scan.plan().splits(); + isEmpty = splits.isEmpty(); + splits.forEach(ctx::collect); + + if (emitSnapshotWatermark) { + Long watermark = scan.watermark(); + if (watermark != null) { + ctx.emitWatermark(new Watermark(watermark)); + } + } + } catch (EndOfScanException esf) { + LOG.info("Catching EndOfStreamException, the stream is finished."); + return; + } + } + + if (isEmpty) { + Thread.sleep(monitorInterval); + } + } + } + + @Override + public void notifyCheckpointComplete(long checkpointId) { + NavigableMap nextSnapshots = + nextSnapshotPerCheckpoint.headMap(checkpointId, true); + OptionalLong max = nextSnapshots.values().stream().mapToLong(Long::longValue).max(); + max.ifPresent(scan::notifyCheckpointComplete); + nextSnapshots.clear(); + } + + @Override + public void cancel() { + // this is to cover the case where cancel() is called before the run() + if (ctx != null) { + synchronized (ctx.getCheckpointLock()) { + isRunning = false; + } + } else { + isRunning = false; + } + } + + public static DataStream buildSource( + StreamExecutionEnvironment env, + String name, + TypeInformation typeInfo, + ReadBuilder readBuilder, + long monitorInterval, + boolean emitSnapshotWatermark, + boolean shuffleBucketWithPartition, + BucketMode bucketMode) { + SingleOutputStreamOperator singleOutputStreamOperator = + env.addSource( + new MonitorSource( + readBuilder, monitorInterval, emitSnapshotWatermark), + name + "-Monitor", + new JavaTypeInfo<>(Split.class)) + .forceNonParallel(); + + DataStream sourceDataStream = + bucketMode == BUCKET_UNAWARE + ? shuffleUnwareBucket(singleOutputStreamOperator) + : shuffleNonUnwareBucket( + singleOutputStreamOperator, shuffleBucketWithPartition); + + return sourceDataStream.transform( + name + "-Reader", typeInfo, new ReadOperator(readBuilder)); + } + + private static DataStream shuffleUnwareBucket( + SingleOutputStreamOperator singleOutputStreamOperator) { + return singleOutputStreamOperator.rebalance(); + } + + private static DataStream shuffleNonUnwareBucket( + SingleOutputStreamOperator singleOutputStreamOperator, + boolean shuffleBucketWithPartition) { + return singleOutputStreamOperator.partitionCustom( + (key, numPartitions) -> { + if (shuffleBucketWithPartition) { + return ChannelComputer.select(key.f0, key.f1, numPartitions); + } + return ChannelComputer.select(key.f1, numPartitions); + }, + split -> { + DataSplit dataSplit = (DataSplit) split; + return Tuple2.of(dataSplit.partition(), dataSplit.bucket()); + }); + } +} diff --git a/paimon-flink/paimon-flink-1.17/src/main/java/org/apache/paimon/flink/source/operator/MonitorSource.java b/paimon-flink/paimon-flink-1.17/src/main/java/org/apache/paimon/flink/source/operator/MonitorSource.java new file mode 100644 index 0000000000000..b38e32559c369 --- /dev/null +++ b/paimon-flink/paimon-flink-1.17/src/main/java/org/apache/paimon/flink/source/operator/MonitorSource.java @@ -0,0 +1,277 @@ +/* + * 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.BucketMode; +import org.apache.paimon.table.sink.ChannelComputer; +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.datastream.SingleOutputStreamOperator; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.functions.source.RichSourceFunction; +import org.apache.flink.streaming.api.watermark.Watermark; +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; + +import static org.apache.paimon.table.BucketMode.BUCKET_UNAWARE; + +/** + * This is the single (non-parallel) monitoring task, it is responsible for: + * + *
    + *
  1. Monitoring snapshots of the Paimon table. + *
  2. Creating the {@link Split splits} corresponding to the incremental files + *
  3. Assigning them to downstream tasks for further processing. + *
+ * + *

The splits to be read are forwarded to the downstream {@link ReadOperator} which can have + * parallelism greater than one. + * + *

Currently, there are two features that rely on this monitor: + * + *

    + *
  1. Consumer-id: rely on this function to do aligned snapshot consumption, and ensure that all + * data in a snapshot is consumed within each checkpoint. + *
  2. Snapshot-watermark: when there is no watermark definition, the default Paimon table will + * pass the watermark recorded in the snapshot. + *
+ */ +public class MonitorSource extends RichSourceFunction + implements CheckpointedFunction, CheckpointListener { + + private static final long serialVersionUID = 1L; + + private static final Logger LOG = LoggerFactory.getLogger(MonitorSource.class); + + private final ReadBuilder readBuilder; + private final long monitorInterval; + private final boolean emitSnapshotWatermark; + + private volatile boolean isRunning = true; + + private transient StreamTableScan scan; + private transient SourceContext ctx; + + private transient ListState checkpointState; + private transient ListState> nextSnapshotState; + private transient TreeMap nextSnapshotPerCheckpoint; + + public MonitorSource( + ReadBuilder readBuilder, long monitorInterval, boolean emitSnapshotWatermark) { + this.readBuilder = readBuilder; + this.monitorInterval = monitorInterval; + this.emitSnapshotWatermark = emitSnapshotWatermark; + } + + @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> typedTuple = + (Class>) (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 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)); + } + + for (Tuple2 tuple2 : nextSnapshotState.get()) { + nextSnapshotPerCheckpoint.put(tuple2.f0, tuple2.f1); + } + } else { + LOG.info("No state to restore for the {}.", getClass().getSimpleName()); + } + } + + @Override + public void snapshotState(FunctionSnapshotContext ctx) throws Exception { + this.checkpointState.clear(); + Long nextSnapshot = this.scan.checkpoint(); + if (nextSnapshot != null) { + this.checkpointState.add(nextSnapshot); + this.nextSnapshotPerCheckpoint.put(ctx.getCheckpointId(), nextSnapshot); + } + + List> nextSnapshots = new ArrayList<>(); + this.nextSnapshotPerCheckpoint.forEach((k, v) -> nextSnapshots.add(new Tuple2<>(k, v))); + this.nextSnapshotState.update(nextSnapshots); + + if (LOG.isDebugEnabled()) { + LOG.debug("{} checkpoint {}.", getClass().getSimpleName(), nextSnapshot); + } + } + + @SuppressWarnings("BusyWait") + @Override + public void run(SourceContext ctx) throws Exception { + this.ctx = ctx; + while (isRunning) { + boolean isEmpty; + synchronized (ctx.getCheckpointLock()) { + if (!isRunning) { + return; + } + try { + List splits = scan.plan().splits(); + isEmpty = splits.isEmpty(); + splits.forEach(ctx::collect); + + if (emitSnapshotWatermark) { + Long watermark = scan.watermark(); + if (watermark != null) { + ctx.emitWatermark(new Watermark(watermark)); + } + } + } catch (EndOfScanException esf) { + LOG.info("Catching EndOfStreamException, the stream is finished."); + return; + } + } + + if (isEmpty) { + Thread.sleep(monitorInterval); + } + } + } + + @Override + public void notifyCheckpointComplete(long checkpointId) { + NavigableMap nextSnapshots = + nextSnapshotPerCheckpoint.headMap(checkpointId, true); + OptionalLong max = nextSnapshots.values().stream().mapToLong(Long::longValue).max(); + max.ifPresent(scan::notifyCheckpointComplete); + nextSnapshots.clear(); + } + + @Override + public void cancel() { + // this is to cover the case where cancel() is called before the run() + if (ctx != null) { + synchronized (ctx.getCheckpointLock()) { + isRunning = false; + } + } else { + isRunning = false; + } + } + + public static DataStream buildSource( + StreamExecutionEnvironment env, + String name, + TypeInformation typeInfo, + ReadBuilder readBuilder, + long monitorInterval, + boolean emitSnapshotWatermark, + boolean shuffleBucketWithPartition, + BucketMode bucketMode) { + SingleOutputStreamOperator singleOutputStreamOperator = + env.addSource( + new MonitorSource( + readBuilder, monitorInterval, emitSnapshotWatermark), + name + "-Monitor", + new JavaTypeInfo<>(Split.class)) + .forceNonParallel(); + + DataStream sourceDataStream = + bucketMode == BUCKET_UNAWARE + ? shuffleUnwareBucket(singleOutputStreamOperator) + : shuffleNonUnwareBucket( + singleOutputStreamOperator, shuffleBucketWithPartition); + + return sourceDataStream.transform( + name + "-Reader", typeInfo, new ReadOperator(readBuilder)); + } + + private static DataStream shuffleUnwareBucket( + SingleOutputStreamOperator singleOutputStreamOperator) { + return singleOutputStreamOperator.rebalance(); + } + + private static DataStream shuffleNonUnwareBucket( + SingleOutputStreamOperator singleOutputStreamOperator, + boolean shuffleBucketWithPartition) { + return singleOutputStreamOperator.partitionCustom( + (key, numPartitions) -> { + if (shuffleBucketWithPartition) { + return ChannelComputer.select(key.f0, key.f1, numPartitions); + } + return ChannelComputer.select(key.f1, numPartitions); + }, + split -> { + DataSplit dataSplit = (DataSplit) split; + return Tuple2.of(dataSplit.partition(), dataSplit.bucket()); + }); + } +} diff --git a/paimon-flink/paimon-flink-1.18/src/main/java/org/apache/paimon/flink/source/operator/MonitorSource.java b/paimon-flink/paimon-flink-1.18/src/main/java/org/apache/paimon/flink/source/operator/MonitorSource.java new file mode 100644 index 0000000000000..b38e32559c369 --- /dev/null +++ b/paimon-flink/paimon-flink-1.18/src/main/java/org/apache/paimon/flink/source/operator/MonitorSource.java @@ -0,0 +1,277 @@ +/* + * 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.BucketMode; +import org.apache.paimon.table.sink.ChannelComputer; +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.datastream.SingleOutputStreamOperator; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.functions.source.RichSourceFunction; +import org.apache.flink.streaming.api.watermark.Watermark; +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; + +import static org.apache.paimon.table.BucketMode.BUCKET_UNAWARE; + +/** + * This is the single (non-parallel) monitoring task, it is responsible for: + * + *
    + *
  1. Monitoring snapshots of the Paimon table. + *
  2. Creating the {@link Split splits} corresponding to the incremental files + *
  3. Assigning them to downstream tasks for further processing. + *
+ * + *

The splits to be read are forwarded to the downstream {@link ReadOperator} which can have + * parallelism greater than one. + * + *

Currently, there are two features that rely on this monitor: + * + *

    + *
  1. Consumer-id: rely on this function to do aligned snapshot consumption, and ensure that all + * data in a snapshot is consumed within each checkpoint. + *
  2. Snapshot-watermark: when there is no watermark definition, the default Paimon table will + * pass the watermark recorded in the snapshot. + *
+ */ +public class MonitorSource extends RichSourceFunction + implements CheckpointedFunction, CheckpointListener { + + private static final long serialVersionUID = 1L; + + private static final Logger LOG = LoggerFactory.getLogger(MonitorSource.class); + + private final ReadBuilder readBuilder; + private final long monitorInterval; + private final boolean emitSnapshotWatermark; + + private volatile boolean isRunning = true; + + private transient StreamTableScan scan; + private transient SourceContext ctx; + + private transient ListState checkpointState; + private transient ListState> nextSnapshotState; + private transient TreeMap nextSnapshotPerCheckpoint; + + public MonitorSource( + ReadBuilder readBuilder, long monitorInterval, boolean emitSnapshotWatermark) { + this.readBuilder = readBuilder; + this.monitorInterval = monitorInterval; + this.emitSnapshotWatermark = emitSnapshotWatermark; + } + + @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> typedTuple = + (Class>) (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 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)); + } + + for (Tuple2 tuple2 : nextSnapshotState.get()) { + nextSnapshotPerCheckpoint.put(tuple2.f0, tuple2.f1); + } + } else { + LOG.info("No state to restore for the {}.", getClass().getSimpleName()); + } + } + + @Override + public void snapshotState(FunctionSnapshotContext ctx) throws Exception { + this.checkpointState.clear(); + Long nextSnapshot = this.scan.checkpoint(); + if (nextSnapshot != null) { + this.checkpointState.add(nextSnapshot); + this.nextSnapshotPerCheckpoint.put(ctx.getCheckpointId(), nextSnapshot); + } + + List> nextSnapshots = new ArrayList<>(); + this.nextSnapshotPerCheckpoint.forEach((k, v) -> nextSnapshots.add(new Tuple2<>(k, v))); + this.nextSnapshotState.update(nextSnapshots); + + if (LOG.isDebugEnabled()) { + LOG.debug("{} checkpoint {}.", getClass().getSimpleName(), nextSnapshot); + } + } + + @SuppressWarnings("BusyWait") + @Override + public void run(SourceContext ctx) throws Exception { + this.ctx = ctx; + while (isRunning) { + boolean isEmpty; + synchronized (ctx.getCheckpointLock()) { + if (!isRunning) { + return; + } + try { + List splits = scan.plan().splits(); + isEmpty = splits.isEmpty(); + splits.forEach(ctx::collect); + + if (emitSnapshotWatermark) { + Long watermark = scan.watermark(); + if (watermark != null) { + ctx.emitWatermark(new Watermark(watermark)); + } + } + } catch (EndOfScanException esf) { + LOG.info("Catching EndOfStreamException, the stream is finished."); + return; + } + } + + if (isEmpty) { + Thread.sleep(monitorInterval); + } + } + } + + @Override + public void notifyCheckpointComplete(long checkpointId) { + NavigableMap nextSnapshots = + nextSnapshotPerCheckpoint.headMap(checkpointId, true); + OptionalLong max = nextSnapshots.values().stream().mapToLong(Long::longValue).max(); + max.ifPresent(scan::notifyCheckpointComplete); + nextSnapshots.clear(); + } + + @Override + public void cancel() { + // this is to cover the case where cancel() is called before the run() + if (ctx != null) { + synchronized (ctx.getCheckpointLock()) { + isRunning = false; + } + } else { + isRunning = false; + } + } + + public static DataStream buildSource( + StreamExecutionEnvironment env, + String name, + TypeInformation typeInfo, + ReadBuilder readBuilder, + long monitorInterval, + boolean emitSnapshotWatermark, + boolean shuffleBucketWithPartition, + BucketMode bucketMode) { + SingleOutputStreamOperator singleOutputStreamOperator = + env.addSource( + new MonitorSource( + readBuilder, monitorInterval, emitSnapshotWatermark), + name + "-Monitor", + new JavaTypeInfo<>(Split.class)) + .forceNonParallel(); + + DataStream sourceDataStream = + bucketMode == BUCKET_UNAWARE + ? shuffleUnwareBucket(singleOutputStreamOperator) + : shuffleNonUnwareBucket( + singleOutputStreamOperator, shuffleBucketWithPartition); + + return sourceDataStream.transform( + name + "-Reader", typeInfo, new ReadOperator(readBuilder)); + } + + private static DataStream shuffleUnwareBucket( + SingleOutputStreamOperator singleOutputStreamOperator) { + return singleOutputStreamOperator.rebalance(); + } + + private static DataStream shuffleNonUnwareBucket( + SingleOutputStreamOperator singleOutputStreamOperator, + boolean shuffleBucketWithPartition) { + return singleOutputStreamOperator.partitionCustom( + (key, numPartitions) -> { + if (shuffleBucketWithPartition) { + return ChannelComputer.select(key.f0, key.f1, numPartitions); + } + return ChannelComputer.select(key.f1, numPartitions); + }, + split -> { + DataSplit dataSplit = (DataSplit) split; + return Tuple2.of(dataSplit.partition(), dataSplit.bucket()); + }); + } +} diff --git a/paimon-flink/paimon-flink-1.19/src/main/java/org/apache/paimon/flink/source/operator/MonitorSource.java b/paimon-flink/paimon-flink-1.19/src/main/java/org/apache/paimon/flink/source/operator/MonitorSource.java new file mode 100644 index 0000000000000..b38e32559c369 --- /dev/null +++ b/paimon-flink/paimon-flink-1.19/src/main/java/org/apache/paimon/flink/source/operator/MonitorSource.java @@ -0,0 +1,277 @@ +/* + * 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.BucketMode; +import org.apache.paimon.table.sink.ChannelComputer; +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.datastream.SingleOutputStreamOperator; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.functions.source.RichSourceFunction; +import org.apache.flink.streaming.api.watermark.Watermark; +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; + +import static org.apache.paimon.table.BucketMode.BUCKET_UNAWARE; + +/** + * This is the single (non-parallel) monitoring task, it is responsible for: + * + *
    + *
  1. Monitoring snapshots of the Paimon table. + *
  2. Creating the {@link Split splits} corresponding to the incremental files + *
  3. Assigning them to downstream tasks for further processing. + *
+ * + *

The splits to be read are forwarded to the downstream {@link ReadOperator} which can have + * parallelism greater than one. + * + *

Currently, there are two features that rely on this monitor: + * + *

    + *
  1. Consumer-id: rely on this function to do aligned snapshot consumption, and ensure that all + * data in a snapshot is consumed within each checkpoint. + *
  2. Snapshot-watermark: when there is no watermark definition, the default Paimon table will + * pass the watermark recorded in the snapshot. + *
+ */ +public class MonitorSource extends RichSourceFunction + implements CheckpointedFunction, CheckpointListener { + + private static final long serialVersionUID = 1L; + + private static final Logger LOG = LoggerFactory.getLogger(MonitorSource.class); + + private final ReadBuilder readBuilder; + private final long monitorInterval; + private final boolean emitSnapshotWatermark; + + private volatile boolean isRunning = true; + + private transient StreamTableScan scan; + private transient SourceContext ctx; + + private transient ListState checkpointState; + private transient ListState> nextSnapshotState; + private transient TreeMap nextSnapshotPerCheckpoint; + + public MonitorSource( + ReadBuilder readBuilder, long monitorInterval, boolean emitSnapshotWatermark) { + this.readBuilder = readBuilder; + this.monitorInterval = monitorInterval; + this.emitSnapshotWatermark = emitSnapshotWatermark; + } + + @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> typedTuple = + (Class>) (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 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)); + } + + for (Tuple2 tuple2 : nextSnapshotState.get()) { + nextSnapshotPerCheckpoint.put(tuple2.f0, tuple2.f1); + } + } else { + LOG.info("No state to restore for the {}.", getClass().getSimpleName()); + } + } + + @Override + public void snapshotState(FunctionSnapshotContext ctx) throws Exception { + this.checkpointState.clear(); + Long nextSnapshot = this.scan.checkpoint(); + if (nextSnapshot != null) { + this.checkpointState.add(nextSnapshot); + this.nextSnapshotPerCheckpoint.put(ctx.getCheckpointId(), nextSnapshot); + } + + List> nextSnapshots = new ArrayList<>(); + this.nextSnapshotPerCheckpoint.forEach((k, v) -> nextSnapshots.add(new Tuple2<>(k, v))); + this.nextSnapshotState.update(nextSnapshots); + + if (LOG.isDebugEnabled()) { + LOG.debug("{} checkpoint {}.", getClass().getSimpleName(), nextSnapshot); + } + } + + @SuppressWarnings("BusyWait") + @Override + public void run(SourceContext ctx) throws Exception { + this.ctx = ctx; + while (isRunning) { + boolean isEmpty; + synchronized (ctx.getCheckpointLock()) { + if (!isRunning) { + return; + } + try { + List splits = scan.plan().splits(); + isEmpty = splits.isEmpty(); + splits.forEach(ctx::collect); + + if (emitSnapshotWatermark) { + Long watermark = scan.watermark(); + if (watermark != null) { + ctx.emitWatermark(new Watermark(watermark)); + } + } + } catch (EndOfScanException esf) { + LOG.info("Catching EndOfStreamException, the stream is finished."); + return; + } + } + + if (isEmpty) { + Thread.sleep(monitorInterval); + } + } + } + + @Override + public void notifyCheckpointComplete(long checkpointId) { + NavigableMap nextSnapshots = + nextSnapshotPerCheckpoint.headMap(checkpointId, true); + OptionalLong max = nextSnapshots.values().stream().mapToLong(Long::longValue).max(); + max.ifPresent(scan::notifyCheckpointComplete); + nextSnapshots.clear(); + } + + @Override + public void cancel() { + // this is to cover the case where cancel() is called before the run() + if (ctx != null) { + synchronized (ctx.getCheckpointLock()) { + isRunning = false; + } + } else { + isRunning = false; + } + } + + public static DataStream buildSource( + StreamExecutionEnvironment env, + String name, + TypeInformation typeInfo, + ReadBuilder readBuilder, + long monitorInterval, + boolean emitSnapshotWatermark, + boolean shuffleBucketWithPartition, + BucketMode bucketMode) { + SingleOutputStreamOperator singleOutputStreamOperator = + env.addSource( + new MonitorSource( + readBuilder, monitorInterval, emitSnapshotWatermark), + name + "-Monitor", + new JavaTypeInfo<>(Split.class)) + .forceNonParallel(); + + DataStream sourceDataStream = + bucketMode == BUCKET_UNAWARE + ? shuffleUnwareBucket(singleOutputStreamOperator) + : shuffleNonUnwareBucket( + singleOutputStreamOperator, shuffleBucketWithPartition); + + return sourceDataStream.transform( + name + "-Reader", typeInfo, new ReadOperator(readBuilder)); + } + + private static DataStream shuffleUnwareBucket( + SingleOutputStreamOperator singleOutputStreamOperator) { + return singleOutputStreamOperator.rebalance(); + } + + private static DataStream shuffleNonUnwareBucket( + SingleOutputStreamOperator singleOutputStreamOperator, + boolean shuffleBucketWithPartition) { + return singleOutputStreamOperator.partitionCustom( + (key, numPartitions) -> { + if (shuffleBucketWithPartition) { + return ChannelComputer.select(key.f0, key.f1, numPartitions); + } + return ChannelComputer.select(key.f1, numPartitions); + }, + split -> { + DataSplit dataSplit = (DataSplit) split; + return Tuple2.of(dataSplit.partition(), dataSplit.bucket()); + }); + } +} diff --git a/paimon-flink/paimon-flink-1.20/pom.xml b/paimon-flink/paimon-flink-1.20/pom.xml index 7cf1d8e98df71..50372614f9663 100644 --- a/paimon-flink/paimon-flink-1.20/pom.xml +++ b/paimon-flink/paimon-flink-1.20/pom.xml @@ -55,6 +55,27 @@ under the License. + + + org.apache.flink + flink-core + ${flink.version} + provided + + + + org.apache.flink + flink-streaming-java + ${flink.version} + provided + + + + org.apache.flink + flink-table-common + ${flink.version} + provided + diff --git a/paimon-flink/paimon-flink-1.20/src/main/java/org/apache/paimon/flink/source/operator/MonitorSource.java b/paimon-flink/paimon-flink-1.20/src/main/java/org/apache/paimon/flink/source/operator/MonitorSource.java new file mode 100644 index 0000000000000..b38e32559c369 --- /dev/null +++ b/paimon-flink/paimon-flink-1.20/src/main/java/org/apache/paimon/flink/source/operator/MonitorSource.java @@ -0,0 +1,277 @@ +/* + * 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.BucketMode; +import org.apache.paimon.table.sink.ChannelComputer; +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.datastream.SingleOutputStreamOperator; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.functions.source.RichSourceFunction; +import org.apache.flink.streaming.api.watermark.Watermark; +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; + +import static org.apache.paimon.table.BucketMode.BUCKET_UNAWARE; + +/** + * This is the single (non-parallel) monitoring task, it is responsible for: + * + *
    + *
  1. Monitoring snapshots of the Paimon table. + *
  2. Creating the {@link Split splits} corresponding to the incremental files + *
  3. Assigning them to downstream tasks for further processing. + *
+ * + *

The splits to be read are forwarded to the downstream {@link ReadOperator} which can have + * parallelism greater than one. + * + *

Currently, there are two features that rely on this monitor: + * + *

    + *
  1. Consumer-id: rely on this function to do aligned snapshot consumption, and ensure that all + * data in a snapshot is consumed within each checkpoint. + *
  2. Snapshot-watermark: when there is no watermark definition, the default Paimon table will + * pass the watermark recorded in the snapshot. + *
+ */ +public class MonitorSource extends RichSourceFunction + implements CheckpointedFunction, CheckpointListener { + + private static final long serialVersionUID = 1L; + + private static final Logger LOG = LoggerFactory.getLogger(MonitorSource.class); + + private final ReadBuilder readBuilder; + private final long monitorInterval; + private final boolean emitSnapshotWatermark; + + private volatile boolean isRunning = true; + + private transient StreamTableScan scan; + private transient SourceContext ctx; + + private transient ListState checkpointState; + private transient ListState> nextSnapshotState; + private transient TreeMap nextSnapshotPerCheckpoint; + + public MonitorSource( + ReadBuilder readBuilder, long monitorInterval, boolean emitSnapshotWatermark) { + this.readBuilder = readBuilder; + this.monitorInterval = monitorInterval; + this.emitSnapshotWatermark = emitSnapshotWatermark; + } + + @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> typedTuple = + (Class>) (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 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)); + } + + for (Tuple2 tuple2 : nextSnapshotState.get()) { + nextSnapshotPerCheckpoint.put(tuple2.f0, tuple2.f1); + } + } else { + LOG.info("No state to restore for the {}.", getClass().getSimpleName()); + } + } + + @Override + public void snapshotState(FunctionSnapshotContext ctx) throws Exception { + this.checkpointState.clear(); + Long nextSnapshot = this.scan.checkpoint(); + if (nextSnapshot != null) { + this.checkpointState.add(nextSnapshot); + this.nextSnapshotPerCheckpoint.put(ctx.getCheckpointId(), nextSnapshot); + } + + List> nextSnapshots = new ArrayList<>(); + this.nextSnapshotPerCheckpoint.forEach((k, v) -> nextSnapshots.add(new Tuple2<>(k, v))); + this.nextSnapshotState.update(nextSnapshots); + + if (LOG.isDebugEnabled()) { + LOG.debug("{} checkpoint {}.", getClass().getSimpleName(), nextSnapshot); + } + } + + @SuppressWarnings("BusyWait") + @Override + public void run(SourceContext ctx) throws Exception { + this.ctx = ctx; + while (isRunning) { + boolean isEmpty; + synchronized (ctx.getCheckpointLock()) { + if (!isRunning) { + return; + } + try { + List splits = scan.plan().splits(); + isEmpty = splits.isEmpty(); + splits.forEach(ctx::collect); + + if (emitSnapshotWatermark) { + Long watermark = scan.watermark(); + if (watermark != null) { + ctx.emitWatermark(new Watermark(watermark)); + } + } + } catch (EndOfScanException esf) { + LOG.info("Catching EndOfStreamException, the stream is finished."); + return; + } + } + + if (isEmpty) { + Thread.sleep(monitorInterval); + } + } + } + + @Override + public void notifyCheckpointComplete(long checkpointId) { + NavigableMap nextSnapshots = + nextSnapshotPerCheckpoint.headMap(checkpointId, true); + OptionalLong max = nextSnapshots.values().stream().mapToLong(Long::longValue).max(); + max.ifPresent(scan::notifyCheckpointComplete); + nextSnapshots.clear(); + } + + @Override + public void cancel() { + // this is to cover the case where cancel() is called before the run() + if (ctx != null) { + synchronized (ctx.getCheckpointLock()) { + isRunning = false; + } + } else { + isRunning = false; + } + } + + public static DataStream buildSource( + StreamExecutionEnvironment env, + String name, + TypeInformation typeInfo, + ReadBuilder readBuilder, + long monitorInterval, + boolean emitSnapshotWatermark, + boolean shuffleBucketWithPartition, + BucketMode bucketMode) { + SingleOutputStreamOperator singleOutputStreamOperator = + env.addSource( + new MonitorSource( + readBuilder, monitorInterval, emitSnapshotWatermark), + name + "-Monitor", + new JavaTypeInfo<>(Split.class)) + .forceNonParallel(); + + DataStream sourceDataStream = + bucketMode == BUCKET_UNAWARE + ? shuffleUnwareBucket(singleOutputStreamOperator) + : shuffleNonUnwareBucket( + singleOutputStreamOperator, shuffleBucketWithPartition); + + return sourceDataStream.transform( + name + "-Reader", typeInfo, new ReadOperator(readBuilder)); + } + + private static DataStream shuffleUnwareBucket( + SingleOutputStreamOperator singleOutputStreamOperator) { + return singleOutputStreamOperator.rebalance(); + } + + private static DataStream shuffleNonUnwareBucket( + SingleOutputStreamOperator singleOutputStreamOperator, + boolean shuffleBucketWithPartition) { + return singleOutputStreamOperator.partitionCustom( + (key, numPartitions) -> { + if (shuffleBucketWithPartition) { + return ChannelComputer.select(key.f0, key.f1, numPartitions); + } + return ChannelComputer.select(key.f1, numPartitions); + }, + split -> { + DataSplit dataSplit = (DataSplit) split; + return Tuple2.of(dataSplit.partition(), dataSplit.bucket()); + }); + } +} diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/SynchronizationActionBase.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/SynchronizationActionBase.java index f103396389e5e..a7c770347410a 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/SynchronizationActionBase.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/SynchronizationActionBase.java @@ -40,7 +40,6 @@ import org.apache.flink.configuration.ExecutionOptions; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.datastream.DataStreamSource; -import org.apache.flink.streaming.api.functions.source.SourceFunction; import java.time.Duration; import java.util.HashMap; @@ -131,7 +130,7 @@ public void build() throws Exception { protected void beforeBuildingSourceSink() throws Exception {} - protected Object buildSource() { + protected Source buildSource() { return syncJobHandler.provideSource(); } @@ -147,41 +146,32 @@ protected void validateRuntimeExecutionMode() { "It's only support STREAMING mode for flink-cdc sync table action."); } - private DataStreamSource buildDataStreamSource(Object source) { - if (source instanceof Source) { - boolean isAutomaticWatermarkCreationEnabled = - tableConfig.containsKey(CoreOptions.TAG_AUTOMATIC_CREATION.key()) - && Objects.equals( - tableConfig.get(CoreOptions.TAG_AUTOMATIC_CREATION.key()), - WATERMARK.toString()); - - Options options = Options.fromMap(tableConfig); - Duration idleTimeout = options.get(SCAN_WATERMARK_IDLE_TIMEOUT); - String watermarkAlignGroup = options.get(SCAN_WATERMARK_ALIGNMENT_GROUP); - WatermarkStrategy watermarkStrategy = - isAutomaticWatermarkCreationEnabled - ? watermarkAlignGroup != null - ? new CdcWatermarkStrategy(createCdcTimestampExtractor()) - .withWatermarkAlignment( - watermarkAlignGroup, - options.get(SCAN_WATERMARK_ALIGNMENT_MAX_DRIFT), - options.get( - SCAN_WATERMARK_ALIGNMENT_UPDATE_INTERVAL)) - : new CdcWatermarkStrategy(createCdcTimestampExtractor()) - : WatermarkStrategy.noWatermarks(); - if (idleTimeout != null) { - watermarkStrategy = watermarkStrategy.withIdleness(idleTimeout); - } - return env.fromSource( - (Source) source, - watermarkStrategy, - syncJobHandler.provideSourceName()); + private DataStreamSource buildDataStreamSource( + Source source) { + boolean isAutomaticWatermarkCreationEnabled = + tableConfig.containsKey(CoreOptions.TAG_AUTOMATIC_CREATION.key()) + && Objects.equals( + tableConfig.get(CoreOptions.TAG_AUTOMATIC_CREATION.key()), + WATERMARK.toString()); + + Options options = Options.fromMap(tableConfig); + Duration idleTimeout = options.get(SCAN_WATERMARK_IDLE_TIMEOUT); + String watermarkAlignGroup = options.get(SCAN_WATERMARK_ALIGNMENT_GROUP); + WatermarkStrategy watermarkStrategy = + isAutomaticWatermarkCreationEnabled + ? watermarkAlignGroup != null + ? new CdcWatermarkStrategy(createCdcTimestampExtractor()) + .withWatermarkAlignment( + watermarkAlignGroup, + options.get(SCAN_WATERMARK_ALIGNMENT_MAX_DRIFT), + options.get( + SCAN_WATERMARK_ALIGNMENT_UPDATE_INTERVAL)) + : new CdcWatermarkStrategy(createCdcTimestampExtractor()) + : WatermarkStrategy.noWatermarks(); + if (idleTimeout != null) { + watermarkStrategy = watermarkStrategy.withIdleness(idleTimeout); } - if (source instanceof SourceFunction) { - return env.addSource( - (SourceFunction) source, syncJobHandler.provideSourceName()); - } - throw new UnsupportedOperationException("Unrecognized source type"); + return env.fromSource(source, watermarkStrategy, syncJobHandler.provideSourceName()); } protected abstract FlatMapFunction recordParse(); diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/FlinkCdcMultiTableSinkTest.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/FlinkCdcMultiTableSinkTest.java index e50661e0c6552..3f58d0ca0fa9f 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/FlinkCdcMultiTableSinkTest.java +++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/FlinkCdcMultiTableSinkTest.java @@ -26,7 +26,6 @@ import org.apache.flink.streaming.api.datastream.DataStreamSink; import org.apache.flink.streaming.api.datastream.DataStreamSource; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.streaming.api.functions.source.ParallelSourceFunction; import org.apache.flink.streaming.api.transformations.OneInputTransformation; import org.apache.flink.streaming.api.transformations.PartitionTransformation; import org.junit.jupiter.api.Test; @@ -45,14 +44,7 @@ public void testTransformationParallelism() { env.setParallelism(8); int inputParallelism = ThreadLocalRandom.current().nextInt(8) + 1; DataStreamSource input = - env.addSource( - new ParallelSourceFunction() { - @Override - public void run(SourceContext ctx) {} - - @Override - public void cancel() {} - }) + env.fromData(CdcMultiplexRecord.class, new CdcMultiplexRecord("", "", null)) .setParallelism(inputParallelism); FlinkCdcMultiTableSink sink = diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/FlinkCdcSyncDatabaseSinkITCase.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/FlinkCdcSyncDatabaseSinkITCase.java index a7c6b2cb63238..4f00c53c328b1 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/FlinkCdcSyncDatabaseSinkITCase.java +++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/FlinkCdcSyncDatabaseSinkITCase.java @@ -42,6 +42,7 @@ import org.apache.paimon.utils.FailingFileIO; import org.apache.paimon.utils.TraceableFileIO; +import org.apache.flink.api.common.eventtime.WatermarkStrategy; import org.apache.flink.streaming.api.datastream.DataStreamSource; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.junit.jupiter.api.Test; @@ -154,8 +155,9 @@ private void innerTestRandomCdcEvents(Supplier bucket, boolean unawareB .allowRestart(enableFailure) .build(); - TestCdcSourceFunction sourceFunction = new TestCdcSourceFunction(events); - DataStreamSource source = env.addSource(sourceFunction); + TestCdcSource sourceFunction = new TestCdcSource(events); + DataStreamSource source = + env.fromSource(sourceFunction, WatermarkStrategy.noWatermarks(), "TestCdcSource"); source.setParallelism(2); Options catalogOptions = new Options(); diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/FlinkCdcSyncTableSinkITCase.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/FlinkCdcSyncTableSinkITCase.java index 081bd7d073d74..8b19391f3edab 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/FlinkCdcSyncTableSinkITCase.java +++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/FlinkCdcSyncTableSinkITCase.java @@ -43,6 +43,7 @@ import org.apache.paimon.utils.FailingFileIO; import org.apache.paimon.utils.TraceableFileIO; +import org.apache.flink.api.common.eventtime.WatermarkStrategy; import org.apache.flink.streaming.api.datastream.DataStreamSource; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.junit.jupiter.api.Disabled; @@ -151,8 +152,9 @@ private void innerTestRandomCdcEvents( .allowRestart(enableFailure) .build(); - TestCdcSourceFunction sourceFunction = new TestCdcSourceFunction(testTable.events()); - DataStreamSource source = env.addSource(sourceFunction); + TestCdcSource testCdcSource = new TestCdcSource(testTable.events()); + DataStreamSource source = + env.fromSource(testCdcSource, WatermarkStrategy.noWatermarks(), "TestCdcSource"); source.setParallelism(2); Options catalogOptions = new Options(); diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/TestCdcSource.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/TestCdcSource.java new file mode 100644 index 0000000000000..841a9b8515a8d --- /dev/null +++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/TestCdcSource.java @@ -0,0 +1,123 @@ +/* + * 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.sink.cdc; + +import org.apache.paimon.flink.source.AbstractNonCoordinatedSource; +import org.apache.paimon.flink.source.AbstractNonCoordinatedSourceReader; +import org.apache.paimon.flink.source.SimpleSourceSplit; + +import org.apache.flink.api.connector.source.Boundedness; +import org.apache.flink.api.connector.source.ReaderOutput; +import org.apache.flink.api.connector.source.SourceReader; +import org.apache.flink.api.connector.source.SourceReaderContext; +import org.apache.flink.core.io.InputStatus; + +import java.util.Collection; +import java.util.Collections; +import java.util.LinkedList; +import java.util.List; +import java.util.concurrent.ThreadLocalRandom; +import java.util.concurrent.atomic.AtomicInteger; + +/** + * Testing parallel {@link org.apache.flink.api.connector.source.Source} to produce {@link + * TestCdcEvent}. {@link TestCdcEvent}s with the same key will be produced by the same parallelism. + */ +public class TestCdcSource extends AbstractNonCoordinatedSource { + + private static final long serialVersionUID = 1L; + private final LinkedList events; + + public TestCdcSource(Collection events) { + this.events = new LinkedList<>(events); + } + + @Override + public Boundedness getBoundedness() { + return Boundedness.CONTINUOUS_UNBOUNDED; + } + + @Override + public SourceReader createReader(SourceReaderContext context) { + return new Reader( + context.getIndexOfSubtask(), + context.currentParallelism(), + new LinkedList<>(events)); + } + + private static class Reader extends AbstractNonCoordinatedSourceReader { + private final int subtaskId; + private final int totalSubtasks; + + private final LinkedList events; + + private final int numRecordsPerCheckpoint; + private final AtomicInteger recordsThisCheckpoint; + + private Reader(int subtaskId, int totalSubtasks, LinkedList events) { + this.subtaskId = subtaskId; + this.totalSubtasks = totalSubtasks; + this.events = events; + numRecordsPerCheckpoint = + events.size() / ThreadLocalRandom.current().nextInt(10, 20) + 1; + recordsThisCheckpoint = new AtomicInteger(0); + } + + @Override + public InputStatus pollNext(ReaderOutput readerOutput) throws Exception { + if (events.isEmpty()) { + return InputStatus.END_OF_INPUT; + } + + if (recordsThisCheckpoint.get() >= numRecordsPerCheckpoint) { + Thread.sleep(10); + return InputStatus.MORE_AVAILABLE; + } + + TestCdcEvent event = events.poll(); + if (event.records() != null) { + if (Math.abs(event.hashCode()) % totalSubtasks != subtaskId) { + return InputStatus.MORE_AVAILABLE; + } + } + readerOutput.collect(event); + recordsThisCheckpoint.incrementAndGet(); + return InputStatus.MORE_AVAILABLE; + } + + @Override + public List snapshotState(long l) { + recordsThisCheckpoint.set(0); + return Collections.singletonList( + new SimpleSourceSplit(Integer.toString(events.size()))); + } + + @Override + public void addSplits(List list) { + int count = + list.stream() + .map(x -> Integer.parseInt(x.splitId())) + .reduce(Integer::sum) + .orElse(0); + while (events.size() > count) { + events.poll(); + } + } + } +} diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/TestCdcSourceFunction.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/TestCdcSourceFunction.java deleted file mode 100644 index 4e03256a52532..0000000000000 --- a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/TestCdcSourceFunction.java +++ /dev/null @@ -1,107 +0,0 @@ -/* - * 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.sink.cdc; - -import org.apache.flink.api.common.state.ListState; -import org.apache.flink.api.common.state.ListStateDescriptor; -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.functions.source.RichParallelSourceFunction; - -import java.util.Collection; -import java.util.LinkedList; -import java.util.concurrent.ThreadLocalRandom; -import java.util.concurrent.atomic.AtomicInteger; - -/** - * Testing {@link RichParallelSourceFunction} to produce {@link TestCdcEvent}. {@link TestCdcEvent}s - * with the same key will be produced by the same parallelism. - */ -public class TestCdcSourceFunction extends RichParallelSourceFunction - implements CheckpointedFunction { - - private static final long serialVersionUID = 1L; - - private final LinkedList events; - - private volatile boolean isRunning = true; - private transient int numRecordsPerCheckpoint; - private transient AtomicInteger recordsThisCheckpoint; - private transient ListState remainingEventsCount; - - public TestCdcSourceFunction(Collection events) { - this.events = new LinkedList<>(events); - } - - @Override - public void initializeState(FunctionInitializationContext context) throws Exception { - numRecordsPerCheckpoint = events.size() / ThreadLocalRandom.current().nextInt(10, 20) + 1; - recordsThisCheckpoint = new AtomicInteger(0); - - remainingEventsCount = - context.getOperatorStateStore() - .getListState(new ListStateDescriptor<>("count", Integer.class)); - - if (context.isRestored()) { - int count = 0; - for (int c : remainingEventsCount.get()) { - count += c; - } - while (events.size() > count) { - events.poll(); - } - } - } - - @Override - public void snapshotState(FunctionSnapshotContext context) throws Exception { - recordsThisCheckpoint.set(0); - remainingEventsCount.clear(); - remainingEventsCount.add(events.size()); - } - - @Override - public void run(SourceContext ctx) throws Exception { - while (isRunning && !events.isEmpty()) { - if (recordsThisCheckpoint.get() >= numRecordsPerCheckpoint) { - Thread.sleep(10); - continue; - } - - synchronized (ctx.getCheckpointLock()) { - TestCdcEvent event = events.poll(); - if (event.records() != null) { - int subtaskId = getRuntimeContext().getIndexOfThisSubtask(); - int totalSubtasks = getRuntimeContext().getNumberOfParallelSubtasks(); - if (Math.abs(event.hashCode()) % totalSubtasks != subtaskId) { - continue; - } - } - ctx.collect(event); - recordsThisCheckpoint.incrementAndGet(); - } - } - } - - @Override - public void cancel() { - isRunning = false; - } -} diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/compact/MultiAwareBucketTableScan.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/compact/MultiAwareBucketTableScan.java index 747995d20d675..88730132ef68c 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/compact/MultiAwareBucketTableScan.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/compact/MultiAwareBucketTableScan.java @@ -32,7 +32,6 @@ import java.util.HashMap; import java.util.List; import java.util.Map; -import java.util.concurrent.atomic.AtomicBoolean; import java.util.regex.Pattern; import java.util.stream.Collectors; @@ -52,15 +51,8 @@ public MultiAwareBucketTableScan( Pattern includingPattern, Pattern excludingPattern, Pattern databasePattern, - boolean isStreaming, - AtomicBoolean isRunning) { - super( - catalogLoader, - includingPattern, - excludingPattern, - databasePattern, - isStreaming, - isRunning); + boolean isStreaming) { + super(catalogLoader, includingPattern, excludingPattern, databasePattern, isStreaming); tablesMap = new HashMap<>(); scansMap = new HashMap<>(); } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/compact/MultiTableScanBase.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/compact/MultiTableScanBase.java index bd4ffe83a4ca0..f5940740b691c 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/compact/MultiTableScanBase.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/compact/MultiTableScanBase.java @@ -26,12 +26,11 @@ import org.apache.paimon.table.source.EndOfScanException; import org.apache.paimon.table.source.Split; -import org.apache.flink.streaming.api.functions.source.SourceFunction; +import org.apache.flink.api.connector.source.ReaderOutput; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.util.List; -import java.util.concurrent.atomic.AtomicBoolean; import java.util.regex.Pattern; import static org.apache.paimon.flink.utils.MultiTablesCompactorUtil.shouldCompactTable; @@ -57,7 +56,6 @@ public abstract class MultiTableScanBase implements AutoCloseable { protected transient Catalog catalog; - protected AtomicBoolean isRunning; protected boolean isStreaming; public MultiTableScanBase( @@ -65,14 +63,12 @@ public MultiTableScanBase( Pattern includingPattern, Pattern excludingPattern, Pattern databasePattern, - boolean isStreaming, - AtomicBoolean isRunning) { + boolean isStreaming) { catalog = catalogLoader.load(); this.includingPattern = includingPattern; this.excludingPattern = excludingPattern; this.databasePattern = databasePattern; - this.isRunning = isRunning; this.isStreaming = isStreaming; } @@ -104,13 +100,9 @@ protected void updateTableMap() } } - public ScanResult scanTable(SourceFunction.SourceContext ctx) + public ScanResult scanTable(ReaderOutput ctx) throws Catalog.TableNotExistException, Catalog.DatabaseNotExistException { try { - if (!isRunning.get()) { - return ScanResult.FINISHED; - } - updateTableMap(); List tasks = doScan(); diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/compact/MultiUnawareBucketTableScan.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/compact/MultiUnawareBucketTableScan.java index 56bf971240e73..da86b93af5120 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/compact/MultiUnawareBucketTableScan.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/compact/MultiUnawareBucketTableScan.java @@ -29,7 +29,6 @@ import java.util.HashMap; import java.util.List; import java.util.Map; -import java.util.concurrent.atomic.AtomicBoolean; import java.util.regex.Pattern; /** @@ -46,15 +45,8 @@ public MultiUnawareBucketTableScan( Pattern includingPattern, Pattern excludingPattern, Pattern databasePattern, - boolean isStreaming, - AtomicBoolean isRunning) { - super( - catalogLoader, - includingPattern, - excludingPattern, - databasePattern, - isStreaming, - isRunning); + boolean isStreaming) { + super(catalogLoader, includingPattern, excludingPattern, databasePattern, isStreaming); tablesMap = new HashMap<>(); } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/compact/UnawareBucketCompactionTopoBuilder.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/compact/UnawareBucketCompactionTopoBuilder.java index 8c6ed4c9f59e1..a572354e89845 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/compact/UnawareBucketCompactionTopoBuilder.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/compact/UnawareBucketCompactionTopoBuilder.java @@ -126,7 +126,7 @@ private DataStreamSource buildSource() { new BucketUnawareCompactSource( table, isContinuous, scanInterval, partitionPredicate); - return BucketUnawareCompactSource.buildSource(env, source, isContinuous, tableIdentifier); + return BucketUnawareCompactSource.buildSource(env, source, tableIdentifier); } private void sinkFromSource(DataStreamSource input) { diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/service/QueryFileMonitor.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/service/QueryFileMonitor.java index 0b659d28bcc81..f0c17f85eceb2 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/service/QueryFileMonitor.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/service/QueryFileMonitor.java @@ -21,6 +21,9 @@ import org.apache.paimon.CoreOptions; import org.apache.paimon.data.BinaryRow; import org.apache.paimon.data.InternalRow; +import org.apache.paimon.flink.source.AbstractNonCoordinatedSource; +import org.apache.paimon.flink.source.AbstractNonCoordinatedSourceReader; +import org.apache.paimon.flink.source.SimpleSourceSplit; import org.apache.paimon.flink.utils.InternalTypeInfo; import org.apache.paimon.options.Options; import org.apache.paimon.table.FileStoreTable; @@ -31,11 +34,14 @@ import org.apache.paimon.table.source.TableRead; import org.apache.paimon.table.system.FileMonitorTable; -import org.apache.flink.api.common.functions.OpenContext; -import org.apache.flink.configuration.Configuration; +import org.apache.flink.api.common.eventtime.WatermarkStrategy; +import org.apache.flink.api.connector.source.Boundedness; +import org.apache.flink.api.connector.source.ReaderOutput; +import org.apache.flink.api.connector.source.SourceReader; +import org.apache.flink.api.connector.source.SourceReaderContext; +import org.apache.flink.core.io.InputStatus; 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 java.util.ArrayList; import java.util.List; @@ -50,19 +56,13 @@ *
  • Assigning them to downstream tasks for further processing. * */ -public class QueryFileMonitor extends RichSourceFunction { +public class QueryFileMonitor extends AbstractNonCoordinatedSource { private static final long serialVersionUID = 1L; private final Table table; private final long monitorInterval; - private transient SourceContext ctx; - private transient StreamTableScan scan; - private transient TableRead read; - - private volatile boolean isRunning = true; - public QueryFileMonitor(Table table) { this.table = table; this.monitorInterval = @@ -72,62 +72,53 @@ public QueryFileMonitor(Table table) { } @Override - public void open(OpenContext openContext) throws Exception { - open(new Configuration()); + public Boundedness getBoundedness() { + return Boundedness.CONTINUOUS_UNBOUNDED; } - /** - * Do not annotate with @override here to maintain compatibility with Flink 1.18-. - */ - public void open(Configuration parameters) throws Exception { - FileMonitorTable monitorTable = new FileMonitorTable((FileStoreTable) table); - ReadBuilder readBuilder = monitorTable.newReadBuilder(); - this.scan = readBuilder.newStreamScan(); - this.read = readBuilder.newRead(); + @Override + public SourceReader createReader( + SourceReaderContext sourceReaderContext) throws Exception { + return new Reader(); } - @Override - public void run(SourceContext ctx) throws Exception { - this.ctx = ctx; - while (isRunning) { - boolean isEmpty; - synchronized (ctx.getCheckpointLock()) { - if (!isRunning) { - return; - } - isEmpty = doScan(); - } + private class Reader extends AbstractNonCoordinatedSourceReader { + private transient StreamTableScan scan; + private transient TableRead read; + + @Override + public void start() { + FileMonitorTable monitorTable = new FileMonitorTable((FileStoreTable) table); + ReadBuilder readBuilder = monitorTable.newReadBuilder(); + this.scan = readBuilder.newStreamScan(); + this.read = readBuilder.newRead(); + } + + @Override + public InputStatus pollNext(ReaderOutput readerOutput) throws Exception { + boolean isEmpty = doScan(readerOutput); if (isEmpty) { Thread.sleep(monitorInterval); } + return InputStatus.MORE_AVAILABLE; } - } - private boolean doScan() throws Exception { - List records = new ArrayList<>(); - read.createReader(scan.plan()).forEachRemaining(records::add); - records.forEach(ctx::collect); - return records.isEmpty(); - } - - @Override - public void cancel() { - // this is to cover the case where cancel() is called before the run() - if (ctx != null) { - synchronized (ctx.getCheckpointLock()) { - isRunning = false; - } - } else { - isRunning = false; + private boolean doScan(ReaderOutput readerOutput) throws Exception { + List records = new ArrayList<>(); + read.createReader(scan.plan()).forEachRemaining(records::add); + records.forEach(readerOutput::collect); + return records.isEmpty(); } } public static DataStream build(StreamExecutionEnvironment env, Table table) { - return env.addSource( - new QueryFileMonitor(table), - "FileMonitor-" + table.name(), - InternalTypeInfo.fromRowType(FileMonitorTable.getRowType())); + return env.fromSource( + new QueryFileMonitor(table), + WatermarkStrategy.noWatermarks(), + "FileMonitor-" + table.name(), + InternalTypeInfo.fromRowType(FileMonitorTable.getRowType())) + .setParallelism(1); } public static ChannelComputer createChannelComputer() { diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/AbstractNonCoordinatedSource.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/AbstractNonCoordinatedSource.java new file mode 100644 index 0000000000000..a9a389e837a21 --- /dev/null +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/AbstractNonCoordinatedSource.java @@ -0,0 +1,50 @@ +/* + * 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; + +import org.apache.flink.api.connector.source.Source; +import org.apache.flink.api.connector.source.SplitEnumerator; +import org.apache.flink.api.connector.source.SplitEnumeratorContext; +import org.apache.flink.core.io.SimpleVersionedSerializer; + +/** {@link Source} that does not require coordination between JobManager and TaskManagers. */ +public abstract class AbstractNonCoordinatedSource + implements Source { + @Override + public SplitEnumerator createEnumerator( + SplitEnumeratorContext enumContext) { + return new NoOpEnumerator<>(); + } + + @Override + public SplitEnumerator restoreEnumerator( + SplitEnumeratorContext enumContext, NoOpEnumState checkpoint) { + return new NoOpEnumerator<>(); + } + + @Override + public SimpleVersionedSerializer getSplitSerializer() { + return new SimpleSourceSplitSerializer(); + } + + @Override + public SimpleVersionedSerializer getEnumeratorCheckpointSerializer() { + return new NoOpEnumStateSerializer(); + } +} diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/AbstractNonCoordinatedSourceReader.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/AbstractNonCoordinatedSourceReader.java new file mode 100644 index 0000000000000..18c278868ffae --- /dev/null +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/AbstractNonCoordinatedSourceReader.java @@ -0,0 +1,51 @@ +/* + * 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; + +import org.apache.flink.api.connector.source.SourceReader; + +import java.util.Collections; +import java.util.List; +import java.util.concurrent.CompletableFuture; + +/** Abstract {@link SourceReader} for {@link AbstractNonCoordinatedSource}. */ +public abstract class AbstractNonCoordinatedSourceReader + implements SourceReader { + @Override + public void start() {} + + @Override + public List snapshotState(long l) { + return Collections.emptyList(); + } + + @Override + public CompletableFuture isAvailable() { + return CompletableFuture.completedFuture(null); + } + + @Override + public void addSplits(List list) {} + + @Override + public void notifyNoMoreSplits() {} + + @Override + public void close() throws Exception {} +} diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/BucketUnawareCompactSource.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/BucketUnawareCompactSource.java index c2021d13ccf3a..e588fef0fb79f 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/BucketUnawareCompactSource.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/BucketUnawareCompactSource.java @@ -21,19 +21,19 @@ import org.apache.paimon.append.UnawareAppendCompactionTask; import org.apache.paimon.append.UnawareAppendTableCompactionCoordinator; import org.apache.paimon.flink.sink.CompactionTaskTypeInfo; -import org.apache.paimon.flink.utils.RuntimeContextUtils; import org.apache.paimon.predicate.Predicate; import org.apache.paimon.table.FileStoreTable; import org.apache.paimon.table.source.EndOfScanException; -import org.apache.paimon.utils.Preconditions; -import org.apache.flink.api.common.functions.OpenContext; +import org.apache.flink.api.common.eventtime.WatermarkStrategy; import org.apache.flink.api.connector.source.Boundedness; -import org.apache.flink.configuration.Configuration; +import org.apache.flink.api.connector.source.ReaderOutput; +import org.apache.flink.api.connector.source.SourceReader; +import org.apache.flink.api.connector.source.SourceReaderContext; +import org.apache.flink.core.io.InputStatus; import org.apache.flink.streaming.api.datastream.DataStreamSource; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.streaming.api.functions.source.RichSourceFunction; -import org.apache.flink.streaming.api.operators.StreamSource; +import org.apache.flink.util.Preconditions; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -42,15 +42,16 @@ import java.util.List; /** - * Source Function for unaware-bucket Compaction. + * Source for unaware-bucket Compaction. * - *

    Note: The function is the source function of unaware-bucket compactor coordinator. It will - * read the latest snapshot continuously by compactionCoordinator, and generate new compaction - * tasks. The source function is used in unaware-bucket compaction job (both stand-alone and - * write-combined). Besides, we don't need to save state in this function, it will invoke a full - * scan when starting up, and scan continuously for the following snapshot. + *

    Note: The function is the source of unaware-bucket compactor coordinator. It will read the + * latest snapshot continuously by compactionCoordinator, and generate new compaction tasks. The + * source is used in unaware-bucket compaction job (both stand-alone and write-combined). Besides, + * we don't need to save state in this source, it will invoke a full scan when starting up, and scan + * continuously for the following snapshot. */ -public class BucketUnawareCompactSource extends RichSourceFunction { +public class BucketUnawareCompactSource + extends AbstractNonCoordinatedSource { private static final Logger LOG = LoggerFactory.getLogger(BucketUnawareCompactSource.class); private static final String COMPACTION_COORDINATOR_NAME = "Compaction Coordinator"; @@ -59,9 +60,6 @@ public class BucketUnawareCompactSource extends RichSourceFunction ctx; - private volatile boolean isRunning = true; public BucketUnawareCompactSource( FileStoreTable table, @@ -75,73 +73,65 @@ public BucketUnawareCompactSource( } @Override - public void open(OpenContext openContext) throws Exception { - open(new Configuration()); + public Boundedness getBoundedness() { + return streaming ? Boundedness.CONTINUOUS_UNBOUNDED : Boundedness.BOUNDED; } - /** - * Do not annotate with @override here to maintain compatibility with Flink 1.18-. - */ - public void open(Configuration parameters) throws Exception { - compactionCoordinator = - new UnawareAppendTableCompactionCoordinator(table, streaming, filter); + @Override + public SourceReader createReader( + SourceReaderContext readerContext) throws Exception { Preconditions.checkArgument( - RuntimeContextUtils.getNumberOfParallelSubtasks(getRuntimeContext()) == 1, + readerContext.currentParallelism() == 1, "Compaction Operator parallelism in paimon MUST be one."); + return new BucketUnawareCompactSourceReader(table, streaming, filter, scanInterval); } - @Override - public void run(SourceContext sourceContext) throws Exception { - this.ctx = sourceContext; - while (isRunning) { - boolean isEmpty; - synchronized (ctx.getCheckpointLock()) { - if (!isRunning) { - return; - } - try { - // do scan and plan action, emit append-only compaction tasks. - List tasks = compactionCoordinator.run(); - isEmpty = tasks.isEmpty(); - tasks.forEach(ctx::collect); - } catch (EndOfScanException esf) { - LOG.info("Catching EndOfStreamException, the stream is finished."); - return; - } - } + /** BucketUnawareCompactSourceReader. */ + public static class BucketUnawareCompactSourceReader + extends AbstractNonCoordinatedSourceReader { + private final UnawareAppendTableCompactionCoordinator compactionCoordinator; + private final long scanInterval; + private long lastFetchTimeMillis = 0L; - if (isEmpty) { - Thread.sleep(scanInterval); - } + public BucketUnawareCompactSourceReader( + FileStoreTable table, boolean streaming, Predicate filter, long scanInterval) { + this.scanInterval = scanInterval; + compactionCoordinator = + new UnawareAppendTableCompactionCoordinator(table, streaming, filter); } - } - @Override - public void cancel() { - if (ctx != null) { - synchronized (ctx.getCheckpointLock()) { - isRunning = false; + @Override + public InputStatus pollNext(ReaderOutput readerOutput) + throws Exception { + long sleepTimeMillis = scanInterval - System.currentTimeMillis() + lastFetchTimeMillis; + if (sleepTimeMillis > 0) { + Thread.sleep(sleepTimeMillis); + } + + try { + // do scan and plan action, emit append-only compaction tasks. + List tasks = compactionCoordinator.run(); + tasks.forEach(readerOutput::collect); + return InputStatus.MORE_AVAILABLE; + } catch (EndOfScanException esf) { + LOG.info("Catching EndOfStreamException, the stream is finished."); + return InputStatus.END_OF_INPUT; + } finally { + lastFetchTimeMillis = System.currentTimeMillis(); } - } else { - isRunning = false; } } public static DataStreamSource buildSource( StreamExecutionEnvironment env, BucketUnawareCompactSource source, - boolean streaming, String tableIdentifier) { - final StreamSource sourceOperator = - new StreamSource<>(source); return (DataStreamSource) - new DataStreamSource<>( - env, - new CompactionTaskTypeInfo(), - sourceOperator, - false, + env.fromSource( + source, + WatermarkStrategy.noWatermarks(), COMPACTION_COORDINATOR_NAME + " : " + tableIdentifier, - streaming ? Boundedness.CONTINUOUS_UNBOUNDED : Boundedness.BOUNDED) + new CompactionTaskTypeInfo()) .setParallelism(1) .setMaxParallelism(1); } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/CombinedTableCompactorSourceBuilder.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/CombinedTableCompactorSourceBuilder.java index e5cbbe845ceb0..415eddb037dfe 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/CombinedTableCompactorSourceBuilder.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/CombinedTableCompactorSourceBuilder.java @@ -21,10 +21,10 @@ import org.apache.paimon.append.MultiTableUnawareAppendCompactionTask; import org.apache.paimon.catalog.Catalog; import org.apache.paimon.flink.LogicalTypeConversion; -import org.apache.paimon.flink.source.operator.CombinedAwareBatchSourceFunction; -import org.apache.paimon.flink.source.operator.CombinedAwareStreamingSourceFunction; -import org.apache.paimon.flink.source.operator.CombinedUnawareBatchSourceFunction; -import org.apache.paimon.flink.source.operator.CombinedUnawareStreamingSourceFunction; +import org.apache.paimon.flink.source.operator.CombinedAwareBatchSource; +import org.apache.paimon.flink.source.operator.CombinedAwareStreamingSource; +import org.apache.paimon.flink.source.operator.CombinedUnawareBatchSource; +import org.apache.paimon.flink.source.operator.CombinedUnawareStreamingSource; import org.apache.paimon.table.system.CompactBucketsTable; import org.apache.paimon.types.RowType; import org.apache.paimon.utils.Preconditions; @@ -87,7 +87,7 @@ public DataStream buildAwareBucketTableSource() { Preconditions.checkArgument(env != null, "StreamExecutionEnvironment should not be null."); RowType produceType = CompactBucketsTable.getRowType(); if (isContinuous) { - return CombinedAwareStreamingSourceFunction.buildSource( + return CombinedAwareStreamingSource.buildSource( env, "Combine-MultiBucketTables--StreamingCompactorSource", InternalTypeInfo.of(LogicalTypeConversion.toLogicalType(produceType)), @@ -97,7 +97,7 @@ public DataStream buildAwareBucketTableSource() { databasePattern, monitorInterval); } else { - return CombinedAwareBatchSourceFunction.buildSource( + return CombinedAwareBatchSource.buildSource( env, "Combine-MultiBucketTables-BatchCompactorSource", InternalTypeInfo.of(LogicalTypeConversion.toLogicalType(produceType)), @@ -112,7 +112,7 @@ public DataStream buildAwareBucketTableSource() { public DataStream buildForUnawareBucketsTableSource() { Preconditions.checkArgument(env != null, "StreamExecutionEnvironment should not be null."); if (isContinuous) { - return CombinedUnawareStreamingSourceFunction.buildSource( + return CombinedUnawareStreamingSource.buildSource( env, "Combined-UnawareBucketTables-StreamingCompactorSource", catalogLoader, @@ -121,7 +121,7 @@ public DataStream buildForUnawareBucketsT databasePattern, monitorInterval); } else { - return CombinedUnawareBatchSourceFunction.buildSource( + return CombinedUnawareBatchSource.buildSource( env, "Combined-UnawareBucketTables-BatchCompactorSource", catalogLoader, diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/FlinkSourceBuilder.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/FlinkSourceBuilder.java index 6933be18def67..895927f51e934 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/FlinkSourceBuilder.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/FlinkSourceBuilder.java @@ -26,7 +26,7 @@ import org.apache.paimon.flink.log.LogSourceProvider; import org.apache.paimon.flink.sink.FlinkSink; import org.apache.paimon.flink.source.align.AlignedContinuousFileStoreSource; -import org.apache.paimon.flink.source.operator.MonitorFunction; +import org.apache.paimon.flink.source.operator.MonitorSource; import org.apache.paimon.flink.utils.TableScanUtils; import org.apache.paimon.options.Options; import org.apache.paimon.predicate.Predicate; @@ -307,7 +307,7 @@ private DataStream buildContinuousStreamOperator() { "Cannot limit streaming source, please use batch execution mode."); } dataStream = - MonitorFunction.buildSource( + MonitorSource.buildSource( env, sourceName, produceTypeInfo(), diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/NoOpEnumState.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/NoOpEnumState.java new file mode 100644 index 0000000000000..f07317c155aa5 --- /dev/null +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/NoOpEnumState.java @@ -0,0 +1,22 @@ +/* + * 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; + +/** The enumerator state class for {@link NoOpEnumerator}. */ +public class NoOpEnumState {} diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/NoOpEnumStateSerializer.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/NoOpEnumStateSerializer.java new file mode 100644 index 0000000000000..89c0ad6ac1f10 --- /dev/null +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/NoOpEnumStateSerializer.java @@ -0,0 +1,41 @@ +/* + * 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; + +import org.apache.flink.core.io.SimpleVersionedSerializer; + +import java.io.IOException; + +/** {@link SimpleVersionedSerializer} for {@link NoOpEnumState}. */ +public class NoOpEnumStateSerializer implements SimpleVersionedSerializer { + @Override + public int getVersion() { + return 0; + } + + @Override + public byte[] serialize(NoOpEnumState obj) throws IOException { + return new byte[0]; + } + + @Override + public NoOpEnumState deserialize(int version, byte[] serialized) throws IOException { + return new NoOpEnumState(); + } +} diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/NoOpEnumerator.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/NoOpEnumerator.java new file mode 100644 index 0000000000000..f29c6d6db76dd --- /dev/null +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/NoOpEnumerator.java @@ -0,0 +1,54 @@ +/* + * 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; + +import org.apache.flink.api.connector.source.SourceSplit; +import org.apache.flink.api.connector.source.SplitEnumerator; + +import javax.annotation.Nullable; + +import java.io.IOException; +import java.util.List; + +/** + * A {@link SplitEnumerator} that provides no functionality. It is basically used for sources that + * does not require a coordinator. + */ +public class NoOpEnumerator + implements SplitEnumerator { + @Override + public void start() {} + + @Override + public void handleSplitRequest(int subtaskId, @Nullable String requesterHostname) {} + + @Override + public void addSplitsBack(List splits, int subtaskId) {} + + @Override + public void addReader(int subtaskId) {} + + @Override + public NoOpEnumState snapshotState(long checkpointId) throws Exception { + return new NoOpEnumState(); + } + + @Override + public void close() throws IOException {} +} diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/SimpleSourceSplit.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/SimpleSourceSplit.java new file mode 100644 index 0000000000000..92d5e4a436d5a --- /dev/null +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/SimpleSourceSplit.java @@ -0,0 +1,39 @@ +/* + * 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; + +import org.apache.flink.api.connector.source.SourceSplit; + +/** A {@link SourceSplit} that provides basic information through splitId. */ +public class SimpleSourceSplit implements SourceSplit { + private final String splitId; + + public SimpleSourceSplit() { + this(""); + } + + public SimpleSourceSplit(String splitId) { + this.splitId = splitId; + } + + @Override + public String splitId() { + return splitId; + } +} diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/SimpleSourceSplitSerializer.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/SimpleSourceSplitSerializer.java new file mode 100644 index 0000000000000..234c8fb3f82e6 --- /dev/null +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/SimpleSourceSplitSerializer.java @@ -0,0 +1,45 @@ +/* + * 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; + +import org.apache.flink.core.io.SimpleVersionedSerializer; + +import java.io.IOException; + +/** {@link SimpleVersionedSerializer} for {@link SimpleSourceSplit}. */ +public class SimpleSourceSplitSerializer implements SimpleVersionedSerializer { + + @Override + public int getVersion() { + return 0; + } + + @Override + public byte[] serialize(SimpleSourceSplit obj) throws IOException { + String splitId = obj.splitId(); + return splitId == null ? new byte[0] : splitId.getBytes(); + } + + @Override + public SimpleSourceSplit deserialize(int version, byte[] serialized) throws IOException { + return serialized.length == 0 + ? new SimpleSourceSplit() + : new SimpleSourceSplit(new String(serialized)); + } +} diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/CombinedAwareBatchSourceFunction.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/CombinedAwareBatchSource.java similarity index 66% rename from paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/CombinedAwareBatchSourceFunction.java rename to paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/CombinedAwareBatchSource.java index 5f11f49a70e02..7df2b37fcb00b 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/CombinedAwareBatchSourceFunction.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/CombinedAwareBatchSource.java @@ -21,21 +21,23 @@ import org.apache.paimon.catalog.Catalog; import org.apache.paimon.flink.compact.MultiAwareBucketTableScan; import org.apache.paimon.flink.compact.MultiTableScanBase; +import org.apache.paimon.flink.source.AbstractNonCoordinatedSourceReader; +import org.apache.paimon.flink.source.SimpleSourceSplit; import org.apache.paimon.flink.utils.JavaTypeInfo; import org.apache.paimon.table.source.DataSplit; import org.apache.paimon.table.source.Split; -import org.apache.flink.api.common.functions.OpenContext; +import org.apache.flink.api.common.eventtime.WatermarkStrategy; import org.apache.flink.api.common.typeinfo.BasicTypeInfo; import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.api.connector.source.Boundedness; +import org.apache.flink.api.connector.source.ReaderOutput; +import org.apache.flink.api.connector.source.SourceReader; +import org.apache.flink.api.connector.source.SourceReaderContext; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.api.java.typeutils.TupleTypeInfo; -import org.apache.flink.configuration.Configuration; +import org.apache.flink.core.io.InputStatus; import org.apache.flink.streaming.api.datastream.DataStream; -import org.apache.flink.streaming.api.datastream.DataStreamSource; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.streaming.api.operators.StreamSource; import org.apache.flink.table.data.RowData; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -47,15 +49,11 @@ import static org.apache.paimon.flink.compact.MultiTableScanBase.ScanResult.IS_EMPTY; /** It is responsible for monitoring compactor source of aware bucket table in batch mode. */ -public class CombinedAwareBatchSourceFunction - extends CombinedCompactorSourceFunction> { +public class CombinedAwareBatchSource extends CombinedCompactorSource> { - private static final Logger LOGGER = - LoggerFactory.getLogger(CombinedAwareBatchSourceFunction.class); + private static final Logger LOGGER = LoggerFactory.getLogger(CombinedAwareBatchSource.class); - private MultiTableScanBase> tableScan; - - public CombinedAwareBatchSourceFunction( + public CombinedAwareBatchSource( Catalog.Loader catalogLoader, Pattern includingPattern, Pattern excludingPattern, @@ -64,31 +62,33 @@ public CombinedAwareBatchSourceFunction( } @Override - public void open(OpenContext openContext) throws Exception { - open(new Configuration()); + public SourceReader, SimpleSourceSplit> createReader( + SourceReaderContext sourceReaderContext) throws Exception { + return new Reader(); } - /** - * Do not annotate with @override here to maintain compatibility with Flink 1.18-. - */ - public void open(Configuration parameters) throws Exception { - super.open(parameters); - tableScan = - new MultiAwareBucketTableScan( - catalogLoader, - includingPattern, - excludingPattern, - databasePattern, - isStreaming, - isRunning); - } + private class Reader extends AbstractNonCoordinatedSourceReader> { - @Override - void scanTable() throws Exception { - if (isRunning.get()) { - MultiTableScanBase.ScanResult scanResult = tableScan.scanTable(ctx); + private MultiTableScanBase> tableScan; + + @Override + public void start() { + super.start(); + tableScan = + new MultiAwareBucketTableScan( + catalogLoader, + includingPattern, + excludingPattern, + databasePattern, + isStreaming); + } + + @Override + public InputStatus pollNext(ReaderOutput> readerOutput) + throws Exception { + MultiTableScanBase.ScanResult scanResult = tableScan.scanTable(readerOutput); if (scanResult == FINISHED) { - return; + return InputStatus.END_OF_INPUT; } if (scanResult == IS_EMPTY) { // Currently, in the combined mode, there are two scan tasks for the table of two @@ -97,6 +97,15 @@ void scanTable() throws Exception { // should not be thrown exception here. LOGGER.info("No file were collected for the table of aware-bucket"); } + return InputStatus.END_OF_INPUT; + } + + @Override + public void close() throws Exception { + super.close(); + if (tableScan != null) { + tableScan.close(); + } } } @@ -109,15 +118,14 @@ public static DataStream buildSource( Pattern excludingPattern, Pattern databasePattern, Duration partitionIdleTime) { - CombinedAwareBatchSourceFunction function = - new CombinedAwareBatchSourceFunction( + CombinedAwareBatchSource source = + new CombinedAwareBatchSource( catalogLoader, includingPattern, excludingPattern, databasePattern); - StreamSource, ?> sourceOperator = new StreamSource<>(function); TupleTypeInfo> tupleTypeInfo = new TupleTypeInfo<>( new JavaTypeInfo<>(Split.class), BasicTypeInfo.STRING_TYPE_INFO); - return new DataStreamSource<>( - env, tupleTypeInfo, sourceOperator, false, name, Boundedness.BOUNDED) + + return env.fromSource(source, WatermarkStrategy.noWatermarks(), name, tupleTypeInfo) .forceNonParallel() .partitionCustom( (key, numPartitions) -> key % numPartitions, @@ -127,12 +135,4 @@ public static DataStream buildSource( typeInfo, new MultiTablesReadOperator(catalogLoader, false, partitionIdleTime)); } - - @Override - public void close() throws Exception { - super.close(); - if (tableScan != null) { - tableScan.close(); - } - } } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/CombinedAwareStreamingSourceFunction.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/CombinedAwareStreamingSource.java similarity index 64% rename from paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/CombinedAwareStreamingSourceFunction.java rename to paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/CombinedAwareStreamingSource.java index 2b8f3440941f2..9bd4a84f571c3 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/CombinedAwareStreamingSourceFunction.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/CombinedAwareStreamingSource.java @@ -21,21 +21,23 @@ import org.apache.paimon.catalog.Catalog; import org.apache.paimon.flink.compact.MultiAwareBucketTableScan; import org.apache.paimon.flink.compact.MultiTableScanBase; +import org.apache.paimon.flink.source.AbstractNonCoordinatedSourceReader; +import org.apache.paimon.flink.source.SimpleSourceSplit; import org.apache.paimon.flink.utils.JavaTypeInfo; import org.apache.paimon.table.source.DataSplit; import org.apache.paimon.table.source.Split; -import org.apache.flink.api.common.functions.OpenContext; +import org.apache.flink.api.common.eventtime.WatermarkStrategy; import org.apache.flink.api.common.typeinfo.BasicTypeInfo; import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.api.connector.source.Boundedness; +import org.apache.flink.api.connector.source.ReaderOutput; +import org.apache.flink.api.connector.source.SourceReader; +import org.apache.flink.api.connector.source.SourceReaderContext; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.api.java.typeutils.TupleTypeInfo; -import org.apache.flink.configuration.Configuration; +import org.apache.flink.core.io.InputStatus; import org.apache.flink.streaming.api.datastream.DataStream; -import org.apache.flink.streaming.api.datastream.DataStreamSource; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.streaming.api.operators.StreamSource; import org.apache.flink.table.data.RowData; import java.util.regex.Pattern; @@ -44,13 +46,11 @@ import static org.apache.paimon.flink.compact.MultiTableScanBase.ScanResult.IS_EMPTY; /** It is responsible for monitoring compactor source of multi bucket table in stream mode. */ -public class CombinedAwareStreamingSourceFunction - extends CombinedCompactorSourceFunction> { +public class CombinedAwareStreamingSource extends CombinedCompactorSource> { private final long monitorInterval; - private transient MultiTableScanBase> tableScan; - public CombinedAwareStreamingSourceFunction( + public CombinedAwareStreamingSource( Catalog.Loader catalogLoader, Pattern includingPattern, Pattern excludingPattern, @@ -61,36 +61,45 @@ public CombinedAwareStreamingSourceFunction( } @Override - public void open(OpenContext openContext) throws Exception { - open(new Configuration()); + public SourceReader, SimpleSourceSplit> createReader( + SourceReaderContext sourceReaderContext) throws Exception { + return new Reader(); } - /** - * Do not annotate with @override here to maintain compatibility with Flink 1.18-. - */ - public void open(Configuration parameters) throws Exception { - super.open(parameters); - tableScan = - new MultiAwareBucketTableScan( - catalogLoader, - includingPattern, - excludingPattern, - databasePattern, - isStreaming, - isRunning); - } + private class Reader extends AbstractNonCoordinatedSourceReader> { + private transient MultiTableScanBase> tableScan; - @SuppressWarnings("BusyWait") - @Override - void scanTable() throws Exception { - while (isRunning.get()) { - MultiTableScanBase.ScanResult scanResult = tableScan.scanTable(ctx); + @Override + public void start() { + super.start(); + tableScan = + new MultiAwareBucketTableScan( + catalogLoader, + includingPattern, + excludingPattern, + databasePattern, + isStreaming); + } + + @Override + public InputStatus pollNext(ReaderOutput> readerOutput) + throws Exception { + MultiTableScanBase.ScanResult scanResult = tableScan.scanTable(readerOutput); if (scanResult == FINISHED) { - return; + return InputStatus.END_OF_INPUT; } if (scanResult == IS_EMPTY) { Thread.sleep(monitorInterval); } + return InputStatus.MORE_AVAILABLE; + } + + @Override + public void close() throws Exception { + super.close(); + if (tableScan != null) { + tableScan.close(); + } } } @@ -104,37 +113,22 @@ public static DataStream buildSource( Pattern databasePattern, long monitorInterval) { - CombinedAwareStreamingSourceFunction function = - new CombinedAwareStreamingSourceFunction( + CombinedAwareStreamingSource source = + new CombinedAwareStreamingSource( catalogLoader, includingPattern, excludingPattern, databasePattern, monitorInterval); - StreamSource, ?> sourceOperator = new StreamSource<>(function); - boolean isParallel = false; TupleTypeInfo> tupleTypeInfo = new TupleTypeInfo<>( new JavaTypeInfo<>(Split.class), BasicTypeInfo.STRING_TYPE_INFO); - return new DataStreamSource<>( - env, - tupleTypeInfo, - sourceOperator, - isParallel, - name, - Boundedness.CONTINUOUS_UNBOUNDED) + + return env.fromSource(source, WatermarkStrategy.noWatermarks(), name, tupleTypeInfo) .forceNonParallel() .partitionCustom( (key, numPartitions) -> key % numPartitions, split -> ((DataSplit) split.f0).bucket()) .transform(name, typeInfo, new MultiTablesReadOperator(catalogLoader, true)); } - - @Override - public void close() throws Exception { - super.close(); - if (tableScan != null) { - tableScan.close(); - } - } } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/CombinedCompactorSourceFunction.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/CombinedCompactorSource.java similarity index 65% rename from paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/CombinedCompactorSourceFunction.java rename to paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/CombinedCompactorSource.java index 4614d30e486fc..f58d86cdd65eb 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/CombinedCompactorSourceFunction.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/CombinedCompactorSource.java @@ -20,13 +20,11 @@ import org.apache.paimon.append.UnawareAppendCompactionTask; import org.apache.paimon.catalog.Catalog; +import org.apache.paimon.flink.source.AbstractNonCoordinatedSource; import org.apache.paimon.table.source.Split; -import org.apache.flink.api.common.functions.OpenContext; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.streaming.api.functions.source.RichSourceFunction; +import org.apache.flink.api.connector.source.Boundedness; -import java.util.concurrent.atomic.AtomicBoolean; import java.util.regex.Pattern; /** @@ -45,8 +43,7 @@ *

    Currently, only dedicated compaction job for multi-tables rely on this monitor. This is the * single (non-parallel) monitoring task, it is responsible for the new Paimon table. */ -public abstract class CombinedCompactorSourceFunction extends RichSourceFunction { - +public abstract class CombinedCompactorSource extends AbstractNonCoordinatedSource { private static final long serialVersionUID = 2L; protected final Catalog.Loader catalogLoader; @@ -55,10 +52,7 @@ public abstract class CombinedCompactorSourceFunction extends RichSourceFunct protected final Pattern databasePattern; protected final boolean isStreaming; - protected transient AtomicBoolean isRunning; - protected transient SourceContext ctx; - - public CombinedCompactorSourceFunction( + public CombinedCompactorSource( Catalog.Loader catalogLoader, Pattern includingPattern, Pattern excludingPattern, @@ -72,34 +66,7 @@ public CombinedCompactorSourceFunction( } @Override - public void open(OpenContext openContext) throws Exception { - open(new Configuration()); - } - - /** - * Do not annotate with @override here to maintain compatibility with Flink 1.18-. - */ - public void open(Configuration parameters) throws Exception { - isRunning = new AtomicBoolean(true); + public Boundedness getBoundedness() { + return isStreaming ? Boundedness.CONTINUOUS_UNBOUNDED : Boundedness.BOUNDED; } - - @Override - public void run(SourceContext sourceContext) throws Exception { - this.ctx = sourceContext; - scanTable(); - } - - @Override - public void cancel() { - // this is to cover the case where cancel() is called before the run() - if (ctx != null) { - synchronized (ctx.getCheckpointLock()) { - isRunning.set(false); - } - } else { - isRunning.set(false); - } - } - - abstract void scanTable() throws Exception; } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/CombinedUnawareBatchSourceFunction.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/CombinedUnawareBatchSource.java similarity index 71% rename from paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/CombinedUnawareBatchSourceFunction.java rename to paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/CombinedUnawareBatchSource.java index b512a195e58ea..64f0c38f5a11d 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/CombinedUnawareBatchSourceFunction.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/CombinedUnawareBatchSource.java @@ -25,18 +25,20 @@ import org.apache.paimon.flink.compact.MultiTableScanBase; import org.apache.paimon.flink.compact.MultiUnawareBucketTableScan; import org.apache.paimon.flink.sink.MultiTableCompactionTaskTypeInfo; +import org.apache.paimon.flink.source.AbstractNonCoordinatedSourceReader; +import org.apache.paimon.flink.source.SimpleSourceSplit; import org.apache.paimon.manifest.PartitionEntry; import org.apache.paimon.table.FileStoreTable; import org.apache.paimon.table.Table; -import org.apache.flink.api.common.functions.OpenContext; -import org.apache.flink.api.connector.source.Boundedness; -import org.apache.flink.configuration.Configuration; +import org.apache.flink.api.common.eventtime.WatermarkStrategy; +import org.apache.flink.api.connector.source.ReaderOutput; +import org.apache.flink.api.connector.source.SourceReader; +import org.apache.flink.api.connector.source.SourceReaderContext; +import org.apache.flink.core.io.InputStatus; import org.apache.flink.streaming.api.datastream.DataStream; -import org.apache.flink.streaming.api.datastream.DataStreamSource; import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.streaming.api.operators.StreamSource; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -55,14 +57,12 @@ * It is responsible for the batch compactor source of the table with unaware bucket in combined * mode. */ -public class CombinedUnawareBatchSourceFunction - extends CombinedCompactorSourceFunction { +public class CombinedUnawareBatchSource + extends CombinedCompactorSource { - private static final Logger LOGGER = - LoggerFactory.getLogger(CombinedUnawareBatchSourceFunction.class); - private transient MultiTableScanBase tableScan; + private static final Logger LOGGER = LoggerFactory.getLogger(CombinedUnawareBatchSource.class); - public CombinedUnawareBatchSourceFunction( + public CombinedUnawareBatchSource( Catalog.Loader catalogLoader, Pattern includingPattern, Pattern excludingPattern, @@ -71,31 +71,33 @@ public CombinedUnawareBatchSourceFunction( } @Override - public void open(OpenContext openContext) throws Exception { - open(new Configuration()); + public SourceReader createReader( + SourceReaderContext sourceReaderContext) throws Exception { + return new Reader(); } - /** - * Do not annotate with @override here to maintain compatibility with Flink 1.18-. - */ - public void open(Configuration parameters) throws Exception { - super.open(parameters); - tableScan = - new MultiUnawareBucketTableScan( - catalogLoader, - includingPattern, - excludingPattern, - databasePattern, - isStreaming, - isRunning); - } + private class Reader + extends AbstractNonCoordinatedSourceReader { + private transient MultiTableScanBase tableScan; + + @Override + public void start() { + super.start(); + tableScan = + new MultiUnawareBucketTableScan( + catalogLoader, + includingPattern, + excludingPattern, + databasePattern, + isStreaming); + } - @Override - void scanTable() throws Exception { - if (isRunning.get()) { - MultiTableScanBase.ScanResult scanResult = tableScan.scanTable(ctx); + @Override + public InputStatus pollNext( + ReaderOutput readerOutput) throws Exception { + MultiTableScanBase.ScanResult scanResult = tableScan.scanTable(readerOutput); if (scanResult == FINISHED) { - return; + return InputStatus.END_OF_INPUT; } if (scanResult == IS_EMPTY) { // Currently, in the combined mode, there are two scan tasks for the table of two @@ -104,6 +106,15 @@ void scanTable() throws Exception { // should not be thrown exception here. LOGGER.info("No file were collected for the table of unaware-bucket"); } + return InputStatus.END_OF_INPUT; + } + + @Override + public void close() throws Exception { + super.close(); + if (tableScan != null) { + tableScan.close(); + } } } @@ -115,22 +126,18 @@ public static DataStream buildSource( Pattern excludingPattern, Pattern databasePattern, @Nullable Duration partitionIdleTime) { - CombinedUnawareBatchSourceFunction function = - new CombinedUnawareBatchSourceFunction( + CombinedUnawareBatchSource combinedUnawareBatchSource = + new CombinedUnawareBatchSource( catalogLoader, includingPattern, excludingPattern, databasePattern); - StreamSource - sourceOperator = new StreamSource<>(function); MultiTableCompactionTaskTypeInfo compactionTaskTypeInfo = new MultiTableCompactionTaskTypeInfo(); SingleOutputStreamOperator source = - new DataStreamSource<>( - env, - compactionTaskTypeInfo, - sourceOperator, - false, + env.fromSource( + combinedUnawareBatchSource, + WatermarkStrategy.noWatermarks(), name, - Boundedness.BOUNDED) + compactionTaskTypeInfo) .forceNonParallel(); if (partitionIdleTime != null) { @@ -175,12 +182,4 @@ private static Long getPartitionInfo( } return partitionInfo.get(partition); } - - @Override - public void close() throws Exception { - super.close(); - if (tableScan != null) { - tableScan.close(); - } - } } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/CombinedUnawareStreamingSourceFunction.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/CombinedUnawareStreamingSource.java similarity index 58% rename from paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/CombinedUnawareStreamingSourceFunction.java rename to paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/CombinedUnawareStreamingSource.java index 6a8e52cf12daf..6ea1ead4db301 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/CombinedUnawareStreamingSourceFunction.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/CombinedUnawareStreamingSource.java @@ -23,14 +23,16 @@ import org.apache.paimon.flink.compact.MultiTableScanBase; import org.apache.paimon.flink.compact.MultiUnawareBucketTableScan; import org.apache.paimon.flink.sink.MultiTableCompactionTaskTypeInfo; +import org.apache.paimon.flink.source.AbstractNonCoordinatedSourceReader; +import org.apache.paimon.flink.source.SimpleSourceSplit; -import org.apache.flink.api.common.functions.OpenContext; -import org.apache.flink.api.connector.source.Boundedness; -import org.apache.flink.configuration.Configuration; +import org.apache.flink.api.common.eventtime.WatermarkStrategy; +import org.apache.flink.api.connector.source.ReaderOutput; +import org.apache.flink.api.connector.source.SourceReader; +import org.apache.flink.api.connector.source.SourceReaderContext; +import org.apache.flink.core.io.InputStatus; import org.apache.flink.streaming.api.datastream.DataStream; -import org.apache.flink.streaming.api.datastream.DataStreamSource; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.streaming.api.operators.StreamSource; import java.util.regex.Pattern; @@ -40,13 +42,12 @@ /** * It is responsible for monitoring compactor source in stream mode for the table of unaware bucket. */ -public class CombinedUnawareStreamingSourceFunction - extends CombinedCompactorSourceFunction { +public class CombinedUnawareStreamingSource + extends CombinedCompactorSource { private final long monitorInterval; - private MultiTableScanBase tableScan; - public CombinedUnawareStreamingSourceFunction( + public CombinedUnawareStreamingSource( Catalog.Loader catalogLoader, Pattern includingPattern, Pattern excludingPattern, @@ -57,36 +58,46 @@ public CombinedUnawareStreamingSourceFunction( } @Override - public void open(OpenContext openContext) throws Exception { - open(new Configuration()); + public SourceReader createReader( + SourceReaderContext sourceReaderContext) throws Exception { + return new Reader(); } - /** - * Do not annotate with @override here to maintain compatibility with Flink 1.18-. - */ - public void open(Configuration parameters) throws Exception { - super.open(parameters); - tableScan = - new MultiUnawareBucketTableScan( - catalogLoader, - includingPattern, - excludingPattern, - databasePattern, - isStreaming, - isRunning); - } + private class Reader + extends AbstractNonCoordinatedSourceReader { + private MultiTableScanBase tableScan; - @SuppressWarnings("BusyWait") - @Override - void scanTable() throws Exception { - while (isRunning.get()) { - MultiTableScanBase.ScanResult scanResult = tableScan.scanTable(ctx); + @Override + public void start() { + super.start(); + tableScan = + new MultiUnawareBucketTableScan( + catalogLoader, + includingPattern, + excludingPattern, + databasePattern, + isStreaming); + } + + @Override + public InputStatus pollNext( + ReaderOutput readerOutput) throws Exception { + MultiTableScanBase.ScanResult scanResult = tableScan.scanTable(readerOutput); if (scanResult == FINISHED) { - return; + return InputStatus.END_OF_INPUT; } if (scanResult == IS_EMPTY) { Thread.sleep(monitorInterval); } + return InputStatus.MORE_AVAILABLE; + } + + @Override + public void close() throws Exception { + super.close(); + if (tableScan != null) { + tableScan.close(); + } } } @@ -99,33 +110,18 @@ public static DataStream buildSource( Pattern databasePattern, long monitorInterval) { - CombinedUnawareStreamingSourceFunction function = - new CombinedUnawareStreamingSourceFunction( + CombinedUnawareStreamingSource source = + new CombinedUnawareStreamingSource( catalogLoader, includingPattern, excludingPattern, databasePattern, monitorInterval); - StreamSource - sourceOperator = new StreamSource<>(function); - boolean isParallel = false; MultiTableCompactionTaskTypeInfo compactionTaskTypeInfo = new MultiTableCompactionTaskTypeInfo(); - return new DataStreamSource<>( - env, - compactionTaskTypeInfo, - sourceOperator, - isParallel, - name, - Boundedness.CONTINUOUS_UNBOUNDED) - .forceNonParallel(); - } - @Override - public void close() throws Exception { - super.close(); - if (tableScan != null) { - tableScan.close(); - } + return env.fromSource( + source, WatermarkStrategy.noWatermarks(), name, compactionTaskTypeInfo) + .forceNonParallel(); } } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/MonitorSource.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/MonitorSource.java new file mode 100644 index 0000000000000..4e33376930f8a --- /dev/null +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/MonitorSource.java @@ -0,0 +1,248 @@ +/* + * 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.source.AbstractNonCoordinatedSource; +import org.apache.paimon.flink.source.AbstractNonCoordinatedSourceReader; +import org.apache.paimon.flink.source.SimpleSourceSplit; +import org.apache.paimon.flink.utils.JavaTypeInfo; +import org.apache.paimon.table.BucketMode; +import org.apache.paimon.table.sink.ChannelComputer; +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.eventtime.Watermark; +import org.apache.flink.api.common.eventtime.WatermarkStrategy; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.connector.source.Boundedness; +import org.apache.flink.api.connector.source.ReaderOutput; +import org.apache.flink.api.connector.source.SourceReader; +import org.apache.flink.api.connector.source.SourceReaderContext; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.core.io.InputStatus; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +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; +import java.util.stream.Collectors; + +import static org.apache.paimon.table.BucketMode.BUCKET_UNAWARE; + +/** + * This is the single (non-parallel) monitoring task, it is responsible for: + * + *

      + *
    1. Monitoring snapshots of the Paimon table. + *
    2. Creating the {@link Split splits} corresponding to the incremental files + *
    3. Assigning them to downstream tasks for further processing. + *
    + * + *

    The splits to be read are forwarded to the downstream {@link ReadOperator} which can have + * parallelism greater than one. + * + *

    Currently, there are two features that rely on this monitor: + * + *

      + *
    1. Consumer-id: rely on this source to do aligned snapshot consumption, and ensure that all + * data in a snapshot is consumed within each checkpoint. + *
    2. Snapshot-watermark: when there is no watermark definition, the default Paimon table will + * pass the watermark recorded in the snapshot. + *
    + */ +public class MonitorSource extends AbstractNonCoordinatedSource { + + private static final long serialVersionUID = 1L; + + private static final Logger LOG = LoggerFactory.getLogger(MonitorSource.class); + + private final ReadBuilder readBuilder; + private final long monitorInterval; + private final boolean emitSnapshotWatermark; + + public MonitorSource( + ReadBuilder readBuilder, long monitorInterval, boolean emitSnapshotWatermark) { + this.readBuilder = readBuilder; + this.monitorInterval = monitorInterval; + this.emitSnapshotWatermark = emitSnapshotWatermark; + } + + @Override + public Boundedness getBoundedness() { + return Boundedness.CONTINUOUS_UNBOUNDED; + } + + @Override + public SourceReader createReader( + SourceReaderContext sourceReaderContext) throws Exception { + return new Reader(); + } + + private class Reader extends AbstractNonCoordinatedSourceReader { + private static final String CHECKPOINT_STATE = "CS"; + private static final String NEXT_SNAPSHOT_STATE = "NSS"; + + private final StreamTableScan scan = readBuilder.newStreamScan(); + private final TreeMap nextSnapshotPerCheckpoint = new TreeMap<>(); + + @Override + public void notifyCheckpointComplete(long checkpointId) { + NavigableMap nextSnapshots = + nextSnapshotPerCheckpoint.headMap(checkpointId, true); + OptionalLong max = nextSnapshots.values().stream().mapToLong(Long::longValue).max(); + max.ifPresent(scan::notifyCheckpointComplete); + nextSnapshots.clear(); + } + + @Override + public List snapshotState(long checkpointId) { + List results = new ArrayList<>(); + + Long nextSnapshot = this.scan.checkpoint(); + if (nextSnapshot != null) { + results.add(new SimpleSourceSplit(CHECKPOINT_STATE + nextSnapshot)); + this.nextSnapshotPerCheckpoint.put(checkpointId, nextSnapshot); + } + + this.nextSnapshotPerCheckpoint.forEach( + (k, v) -> + results.add(new SimpleSourceSplit(NEXT_SNAPSHOT_STATE + k + ":" + v))); + + if (LOG.isDebugEnabled()) { + LOG.debug("{} checkpoint {}.", getClass().getSimpleName(), nextSnapshot); + } + return results; + } + + @Override + public void addSplits(List list) { + LOG.info("Restoring state for the {}.", getClass().getSimpleName()); + + List retrievedStates = + list.stream() + .map(SimpleSourceSplit::splitId) + .filter(x -> x.startsWith(CHECKPOINT_STATE)) + .map(x -> Long.parseLong(x.substring(CHECKPOINT_STATE.length()))) + .collect(Collectors.toList()); + + // given that the parallelism of the source 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)); + } + + list.stream() + .map(SimpleSourceSplit::splitId) + .filter(x -> x.startsWith(NEXT_SNAPSHOT_STATE)) + .map(x -> x.substring(NEXT_SNAPSHOT_STATE.length()).split(":")) + .forEach( + x -> + nextSnapshotPerCheckpoint.put( + Long.parseLong(x[0]), Long.parseLong(x[1]))); + } + + @Override + public InputStatus pollNext(ReaderOutput readerOutput) throws Exception { + boolean isEmpty; + try { + List splits = scan.plan().splits(); + isEmpty = splits.isEmpty(); + splits.forEach(readerOutput::collect); + + if (emitSnapshotWatermark) { + Long watermark = scan.watermark(); + if (watermark != null) { + readerOutput.emitWatermark(new Watermark(watermark)); + } + } + } catch (EndOfScanException esf) { + LOG.info("Catching EndOfStreamException, the stream is finished."); + return InputStatus.END_OF_INPUT; + } + + if (isEmpty) { + Thread.sleep(monitorInterval); + } + return InputStatus.MORE_AVAILABLE; + } + } + + public static DataStream buildSource( + StreamExecutionEnvironment env, + String name, + TypeInformation typeInfo, + ReadBuilder readBuilder, + long monitorInterval, + boolean emitSnapshotWatermark, + boolean shuffleBucketWithPartition, + BucketMode bucketMode) { + SingleOutputStreamOperator singleOutputStreamOperator = + env.fromSource( + new MonitorSource( + readBuilder, monitorInterval, emitSnapshotWatermark), + WatermarkStrategy.noWatermarks(), + name + "-Monitor", + new JavaTypeInfo<>(Split.class)) + .forceNonParallel(); + + DataStream sourceDataStream = + bucketMode == BUCKET_UNAWARE + ? shuffleUnwareBucket(singleOutputStreamOperator) + : shuffleNonUnwareBucket( + singleOutputStreamOperator, shuffleBucketWithPartition); + + return sourceDataStream.transform( + name + "-Reader", typeInfo, new ReadOperator(readBuilder)); + } + + private static DataStream shuffleUnwareBucket( + SingleOutputStreamOperator singleOutputStreamOperator) { + return singleOutputStreamOperator.rebalance(); + } + + private static DataStream shuffleNonUnwareBucket( + SingleOutputStreamOperator singleOutputStreamOperator, + boolean shuffleBucketWithPartition) { + return singleOutputStreamOperator.partitionCustom( + (key, numPartitions) -> { + if (shuffleBucketWithPartition) { + return ChannelComputer.select(key.f0, key.f1, numPartitions); + } + return ChannelComputer.select(key.f1, numPartitions); + }, + split -> { + DataSplit dataSplit = (DataSplit) split; + return Tuple2.of(dataSplit.partition(), dataSplit.bucket()); + }); + } +} diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/MultiTablesReadOperator.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/MultiTablesReadOperator.java index 73d46ae1e3f19..fbc8bb9d756a3 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/MultiTablesReadOperator.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/MultiTablesReadOperator.java @@ -52,9 +52,8 @@ /** * The operator that reads the Tuple2<{@link Split}, String> received from the preceding {@link - * CombinedAwareBatchSourceFunction} or {@link CombinedAwareStreamingSourceFunction}. Contrary to - * the {@link CombinedCompactorSourceFunction} which has a parallelism of 1, this operator can have - * DOP > 1. + * CombinedAwareBatchSource} or {@link CombinedAwareStreamingSource}. Contrary to the {@link + * CombinedCompactorSource} which has a parallelism of 1, this operator can have DOP > 1. */ public class MultiTablesReadOperator extends AbstractStreamOperator implements OneInputStreamOperator, RowData> { diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/MultiUnawareTablesReadOperator.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/MultiUnawareTablesReadOperator.java index c501c2519b412..0864741a178f1 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/MultiUnawareTablesReadOperator.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/MultiUnawareTablesReadOperator.java @@ -44,7 +44,7 @@ /** * The operator is used for historical partition compaction. It reads {@link * MultiTableUnawareAppendCompactionTask} received from the preceding {@link - * CombinedUnawareBatchSourceFunction} and filter partitions which is not historical. + * CombinedUnawareBatchSource} and filter partitions which is not historical. */ public class MultiUnawareTablesReadOperator extends AbstractStreamOperator diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/ReadOperator.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/ReadOperator.java index 80c85f7cdb35a..6caf4544e514b 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/ReadOperator.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/ReadOperator.java @@ -38,8 +38,8 @@ /** * The operator that reads the {@link Split splits} received from the preceding {@link - * MonitorFunction}. Contrary to the {@link MonitorFunction} which has a parallelism of 1, this - * operator can have DOP > 1. + * MonitorSource}. Contrary to the {@link MonitorSource} which has a parallelism of 1, this operator + * can have DOP > 1. */ public class ReadOperator extends AbstractStreamOperator implements OneInputStreamOperator { diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FileStoreITCase.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FileStoreITCase.java index 6a2c7b071d2d1..5245114e80eed 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FileStoreITCase.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FileStoreITCase.java @@ -36,6 +36,7 @@ import org.apache.paimon.utils.BranchManager; import org.apache.paimon.utils.FailingFileIO; +import org.apache.flink.api.common.eventtime.WatermarkStrategy; import org.apache.flink.api.common.functions.MapFunction; import org.apache.flink.api.connector.source.Boundedness; import org.apache.flink.api.dag.Transformation; @@ -450,7 +451,12 @@ private void sinkAndValidate( throw new UnsupportedOperationException(); } DataStreamSource source = - env.addSource(new FiniteTestSource<>(src, true), InternalTypeInfo.of(TABLE_TYPE)); + env.fromSource( + new FiniteTestSource<>(src, true), + WatermarkStrategy.noWatermarks(), + "FiniteTestSource", + InternalTypeInfo.of(TABLE_TYPE)); + source.forceNonParallel(); new FlinkSinkBuilder(table).forRowData(source).build(); env.execute(); assertThat(iterator.collect(expected.length)).containsExactlyInAnyOrder(expected); @@ -521,9 +527,13 @@ public static DataStreamSource buildTestSource( StreamExecutionEnvironment env, boolean isBatch) { return isBatch ? env.fromCollection(SOURCE_DATA, InternalTypeInfo.of(TABLE_TYPE)) - : env.addSource( - new FiniteTestSource<>(SOURCE_DATA, false), - InternalTypeInfo.of(TABLE_TYPE)); + : (DataStreamSource) + env.fromSource( + new FiniteTestSource<>(SOURCE_DATA, false), + WatermarkStrategy.noWatermarks(), + "FiniteTestSource", + InternalTypeInfo.of(TABLE_TYPE)) + .forceNonParallel(); } public static List executeAndCollect(DataStream source) throws Exception { diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FiniteTestSource.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FiniteTestSource.java index 9c5254d6283b7..add93bb8b6649 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FiniteTestSource.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FiniteTestSource.java @@ -18,19 +18,21 @@ package org.apache.paimon.flink; +import org.apache.paimon.flink.source.AbstractNonCoordinatedSource; +import org.apache.paimon.flink.source.AbstractNonCoordinatedSourceReader; +import org.apache.paimon.flink.source.SimpleSourceSplit; import org.apache.paimon.utils.Preconditions; -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.typeutils.base.IntSerializer; -import org.apache.flink.runtime.state.FunctionInitializationContext; -import org.apache.flink.runtime.state.FunctionSnapshotContext; +import org.apache.flink.api.connector.source.Boundedness; +import org.apache.flink.api.connector.source.ReaderOutput; +import org.apache.flink.api.connector.source.SourceReader; +import org.apache.flink.api.connector.source.SourceReaderContext; +import org.apache.flink.core.io.InputStatus; import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction; -import org.apache.flink.streaming.api.functions.source.SourceFunction; -import java.util.ArrayList; +import java.util.Collections; import java.util.List; +import java.util.stream.Collectors; /** * A stream source that: 1) emits a list of elements without allowing checkpoints, 2) then waits for @@ -39,8 +41,7 @@ * *

    The reason this class is rewritten is to support {@link CheckpointedFunction}. */ -public class FiniteTestSource - implements SourceFunction, CheckpointedFunction, CheckpointListener { +public class FiniteTestSource extends AbstractNonCoordinatedSource { private static final long serialVersionUID = 1L; @@ -48,113 +49,110 @@ public class FiniteTestSource private final boolean emitOnce; - private volatile boolean running = true; - - private transient int numCheckpointsComplete; - - private transient ListState checkpointedState; - - private volatile int numTimesEmitted; - public FiniteTestSource(List elements, boolean emitOnce) { this.elements = elements; this.emitOnce = emitOnce; } @Override - public void initializeState(FunctionInitializationContext context) throws Exception { - this.checkpointedState = - context.getOperatorStateStore() - .getListState( - new ListStateDescriptor<>("emit-times", IntSerializer.INSTANCE)); - - if (context.isRestored()) { - List retrievedStates = new ArrayList<>(); - for (Integer entry : this.checkpointedState.get()) { - retrievedStates.add(entry); + public Boundedness getBoundedness() { + return Boundedness.BOUNDED; + } + + @Override + public SourceReader createReader(SourceReaderContext sourceReaderContext) + throws Exception { + return new Reader<>(elements, emitOnce); + } + + private static class Reader extends AbstractNonCoordinatedSourceReader { + + private final List elements; + + private final boolean emitOnce; + + private int numTimesEmitted = 0; + + private int numCheckpointsComplete; + + private Integer checkpointToAwait; + + private Reader(List elements, boolean emitOnce) { + this.elements = elements; + this.emitOnce = emitOnce; + this.numCheckpointsComplete = 0; + } + + @Override + public synchronized InputStatus pollNext(ReaderOutput readerOutput) { + if (checkpointToAwait == null) { + checkpointToAwait = numCheckpointsComplete + 2; } + switch (numTimesEmitted) { + case 0: + emitElements(readerOutput, false); + if (numCheckpointsComplete < checkpointToAwait) { + return InputStatus.MORE_AVAILABLE; + } + emitElements(readerOutput, true); + if (numCheckpointsComplete < checkpointToAwait + 2) { + return InputStatus.MORE_AVAILABLE; + } + break; + case 1: + emitElements(readerOutput, true); + if (numCheckpointsComplete < checkpointToAwait) { + return InputStatus.MORE_AVAILABLE; + } + break; + case 2: + // Maybe missed notifyCheckpointComplete, wait next notifyCheckpointComplete + if (numCheckpointsComplete < checkpointToAwait) { + return InputStatus.MORE_AVAILABLE; + } + break; + } + return InputStatus.END_OF_INPUT; + } + + @Override + public void addSplits(List list) { + List retrievedStates = + list.stream() + .map(x -> Integer.parseInt(x.splitId())) + .collect(Collectors.toList()); // given that the parallelism of the function is 1, we can only have 1 state Preconditions.checkArgument( retrievedStates.size() == 1, getClass().getSimpleName() + " retrieved invalid state."); - this.numTimesEmitted = retrievedStates.get(0); + numTimesEmitted = retrievedStates.get(0); Preconditions.checkArgument( numTimesEmitted <= 2, getClass().getSimpleName() + " retrieved invalid numTimesEmitted: " + numTimesEmitted); - } else { - this.numTimesEmitted = 0; } - } - @Override - public void run(SourceContext ctx) throws Exception { - switch (numTimesEmitted) { - case 0: - emitElementsAndWaitForCheckpoints(ctx, false); - emitElementsAndWaitForCheckpoints(ctx, true); - break; - case 1: - emitElementsAndWaitForCheckpoints(ctx, true); - break; - case 2: - // Maybe missed notifyCheckpointComplete, wait next notifyCheckpointComplete - final Object lock = ctx.getCheckpointLock(); - synchronized (lock) { - int checkpointToAwait = numCheckpointsComplete + 2; - while (running && numCheckpointsComplete < checkpointToAwait) { - lock.wait(1); - } - } - break; + @Override + public List snapshotState(long l) { + return Collections.singletonList( + new SimpleSourceSplit(Integer.toString(numTimesEmitted))); } - } - private void emitElementsAndWaitForCheckpoints(SourceContext ctx, boolean isSecond) - throws InterruptedException { - final Object lock = ctx.getCheckpointLock(); + @Override + public void notifyCheckpointComplete(long checkpointId) { + numCheckpointsComplete++; + } - final int checkpointToAwait; - synchronized (lock) { - checkpointToAwait = numCheckpointsComplete + 2; + private void emitElements(ReaderOutput readerOutput, boolean isSecond) { if (!isSecond || !emitOnce) { for (T t : elements) { - ctx.collect(t); + readerOutput.collect(t); } } numTimesEmitted++; } - - synchronized (lock) { - while (running && numCheckpointsComplete < checkpointToAwait) { - lock.wait(1); - } - } - } - - @Override - public void cancel() { - running = false; - } - - @Override - public void notifyCheckpointComplete(long checkpointId) { - numCheckpointsComplete++; - } - - @Override - public void notifyCheckpointAborted(long checkpointId) {} - - @Override - public void snapshotState(FunctionSnapshotContext context) throws Exception { - Preconditions.checkState( - this.checkpointedState != null, - "The " + getClass().getSimpleName() + " has not been properly initialized."); - - this.checkpointedState.clear(); - this.checkpointedState.add(this.numTimesEmitted); } } diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/SerializableRowData.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/SerializableRowData.java index 594affc124ebb..75b96cbe02eb3 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/SerializableRowData.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/SerializableRowData.java @@ -47,8 +47,10 @@ public SerializableRowData(RowData row, TypeSerializer serializer) { this.serializer = serializer; } - private void writeObject(ObjectOutputStream out) throws IOException { + private synchronized void writeObject(ObjectOutputStream out) throws IOException { out.defaultWriteObject(); + // This following invocation needs to be synchronized to avoid racing problems when the + // serializer is reused across multiple subtasks. serializer.serialize(row, new DataOutputViewStreamWrapper(out)); } diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/UnawareBucketAppendOnlyTableITCase.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/UnawareBucketAppendOnlyTableITCase.java index eeb99df75cb53..301849070d6db 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/UnawareBucketAppendOnlyTableITCase.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/UnawareBucketAppendOnlyTableITCase.java @@ -20,7 +20,9 @@ import org.apache.paimon.Snapshot; import org.apache.paimon.data.InternalRow; -import org.apache.paimon.flink.utils.RuntimeContextUtils; +import org.apache.paimon.flink.source.AbstractNonCoordinatedSource; +import org.apache.paimon.flink.source.AbstractNonCoordinatedSourceReader; +import org.apache.paimon.flink.source.SimpleSourceSplit; import org.apache.paimon.fs.Path; import org.apache.paimon.fs.local.LocalFileIO; import org.apache.paimon.reader.RecordReader; @@ -29,9 +31,14 @@ import org.apache.paimon.table.FileStoreTableFactory; import org.apache.paimon.utils.FailingFileIO; +import org.apache.flink.api.common.eventtime.WatermarkStrategy; +import org.apache.flink.api.connector.source.Boundedness; +import org.apache.flink.api.connector.source.ReaderOutput; +import org.apache.flink.api.connector.source.SourceReader; +import org.apache.flink.api.connector.source.SourceReaderContext; +import org.apache.flink.core.io.InputStatus; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.streaming.api.functions.source.RichParallelSourceFunction; import org.apache.flink.table.api.bridge.java.StreamTableEnvironment; import org.apache.flink.table.planner.factories.TestValuesTableFactory; import org.apache.flink.types.Row; @@ -372,7 +379,12 @@ public void testStatelessWriter() throws Exception { .checkpointIntervalMs(500) .build(); DataStream source = - env.addSource(new TestStatelessWriterSource(table)).setParallelism(2).forward(); + env.fromSource( + new TestStatelessWriterSource(table), + WatermarkStrategy.noWatermarks(), + "TestStatelessWriterSource") + .setParallelism(2) + .forward(); StreamTableEnvironment tEnv = StreamTableEnvironment.create(env); tEnv.registerCatalog("mycat", sEnv.getCatalog("PAIMON").get()); @@ -384,46 +396,59 @@ public void testStatelessWriter() throws Exception { .containsExactlyInAnyOrder(Row.of(1, "test"), Row.of(2, "test")); } - private static class TestStatelessWriterSource extends RichParallelSourceFunction { + private static class TestStatelessWriterSource extends AbstractNonCoordinatedSource { private final FileStoreTable table; - private volatile boolean isRunning = true; - private TestStatelessWriterSource(FileStoreTable table) { this.table = table; } @Override - public void run(SourceContext sourceContext) throws Exception { - int taskId = RuntimeContextUtils.getIndexOfThisSubtask(getRuntimeContext()); - // wait some time in parallelism #2, - // so that it does not commit in the same checkpoint with parallelism #1 - int waitCount = (taskId == 0 ? 0 : 10); - - while (isRunning) { - synchronized (sourceContext.getCheckpointLock()) { - if (taskId == 0) { + public Boundedness getBoundedness() { + return Boundedness.CONTINUOUS_UNBOUNDED; + } + + @Override + public SourceReader createReader( + SourceReaderContext sourceReaderContext) throws Exception { + return new Reader(sourceReaderContext.getIndexOfSubtask()); + } + + private class Reader extends AbstractNonCoordinatedSourceReader { + private final int taskId; + private int waitCount; + + private Reader(int taskId) { + this.taskId = taskId; + this.waitCount = (taskId == 0 ? 0 : 10); + } + + @Override + public InputStatus pollNext(ReaderOutput readerOutput) throws Exception { + if (taskId == 0) { + if (waitCount == 0) { + readerOutput.collect(1); + } else if (countNumRecords() >= 1) { + // wait for the record to commit before exiting + Thread.sleep(1000); + return InputStatus.END_OF_INPUT; + } + } else { + int numRecords = countNumRecords(); + if (numRecords >= 1) { if (waitCount == 0) { - sourceContext.collect(1); - } else if (countNumRecords() >= 1) { - // wait for the record to commit before exiting - break; - } - } else { - int numRecords = countNumRecords(); - if (numRecords >= 1) { - if (waitCount == 0) { - sourceContext.collect(2); - } else if (countNumRecords() >= 2) { - // make sure the next checkpoint is successful - break; - } + readerOutput.collect(2); + } else if (countNumRecords() >= 2) { + // make sure the next checkpoint is successful + Thread.sleep(1000); + return InputStatus.END_OF_INPUT; } } - waitCount--; } + waitCount--; Thread.sleep(1000); + return InputStatus.MORE_AVAILABLE; } } @@ -439,11 +464,6 @@ private int countNumRecords() throws Exception { } return ret; } - - @Override - public void cancel() { - isRunning = false; - } } @Override diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/source/operator/OperatorSourceTest.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/source/operator/OperatorSourceTest.java index 61a03a29a21bd..b1e0fb83610e2 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/source/operator/OperatorSourceTest.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/source/operator/OperatorSourceTest.java @@ -33,12 +33,17 @@ import org.apache.paimon.table.source.TableRead; import org.apache.paimon.types.DataTypes; +import org.apache.flink.api.common.eventtime.WatermarkStrategy; import org.apache.flink.metrics.MetricGroup; import org.apache.flink.runtime.checkpoint.OperatorSubtaskState; -import org.apache.flink.streaming.api.functions.source.SourceFunction; -import org.apache.flink.streaming.api.operators.StreamSource; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.operators.SourceOperator; import org.apache.flink.streaming.api.watermark.Watermark; +import org.apache.flink.streaming.runtime.io.PushingAsyncDataInput; +import org.apache.flink.streaming.runtime.streamrecord.LatencyMarker; +import org.apache.flink.streaming.runtime.streamrecord.RecordAttributes; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.apache.flink.streaming.runtime.watermarkstatus.WatermarkStatus; import org.apache.flink.streaming.util.AbstractStreamOperatorTestHarness; import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness; import org.apache.flink.table.data.GenericRowData; @@ -46,6 +51,7 @@ import org.apache.flink.table.runtime.typeutils.InternalSerializers; import org.apache.flink.table.types.logical.IntType; import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.util.CloseableIterator; import org.apache.flink.util.function.SupplierWithException; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; @@ -58,11 +64,13 @@ import java.util.List; import java.util.concurrent.ArrayBlockingQueue; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicReference; import static org.apache.paimon.CoreOptions.CONSUMER_ID; import static org.assertj.core.api.Assertions.assertThat; -/** Test for {@link MonitorFunction} and {@link ReadOperator}. */ +/** Test for {@link MonitorSource} and {@link ReadOperator}. */ public class OperatorSourceTest { @TempDir Path tempDir; @@ -114,28 +122,39 @@ private List> readSplit(Split split) throws IOException { } @Test - public void testMonitorFunction() throws Exception { + public void testMonitorSource() throws Exception { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); // 1. run first OperatorSubtaskState snapshot; { - MonitorFunction function = new MonitorFunction(table.newReadBuilder(), 10, false); - StreamSource src = new StreamSource<>(function); + MonitorSource source = new MonitorSource(table.newReadBuilder(), 10, false); + TestingSourceOperator operator = + (TestingSourceOperator) + TestingSourceOperator.createTestOperator( + source.createReader(null), + WatermarkStrategy.noWatermarks(), + false); AbstractStreamOperatorTestHarness testHarness = - new AbstractStreamOperatorTestHarness<>(src, 1, 1, 0); + new AbstractStreamOperatorTestHarness<>(operator, 1, 1, 0); testHarness.open(); - snapshot = testReadSplit(function, () -> testHarness.snapshot(0, 0), 1, 1, 1); + snapshot = testReadSplit(operator, () -> testHarness.snapshot(0, 0), 1, 1, 1); } // 2. restore from state { - MonitorFunction functionCopy1 = new MonitorFunction(table.newReadBuilder(), 10, false); - StreamSource srcCopy1 = new StreamSource<>(functionCopy1); + MonitorSource sourceCopy1 = new MonitorSource(table.newReadBuilder(), 10, false); + TestingSourceOperator operatorCopy1 = + (TestingSourceOperator) + TestingSourceOperator.createTestOperator( + sourceCopy1.createReader(null), + WatermarkStrategy.noWatermarks(), + false); AbstractStreamOperatorTestHarness testHarnessCopy1 = - new AbstractStreamOperatorTestHarness<>(srcCopy1, 1, 1, 0); + new AbstractStreamOperatorTestHarness<>(operatorCopy1, 1, 1, 0); testHarnessCopy1.initializeState(snapshot); testHarnessCopy1.open(); testReadSplit( - functionCopy1, + operatorCopy1, () -> { testHarnessCopy1.snapshot(1, 1); testHarnessCopy1.notifyOfCompletedCheckpoint(1); @@ -148,12 +167,17 @@ public void testMonitorFunction() throws Exception { // 3. restore from consumer id { - MonitorFunction functionCopy2 = new MonitorFunction(table.newReadBuilder(), 10, false); - StreamSource srcCopy2 = new StreamSource<>(functionCopy2); + MonitorSource sourceCopy2 = new MonitorSource(table.newReadBuilder(), 10, false); + TestingSourceOperator operatorCopy2 = + (TestingSourceOperator) + TestingSourceOperator.createTestOperator( + sourceCopy2.createReader(null), + WatermarkStrategy.noWatermarks(), + false); AbstractStreamOperatorTestHarness testHarnessCopy2 = - new AbstractStreamOperatorTestHarness<>(srcCopy2, 1, 1, 0); + new AbstractStreamOperatorTestHarness<>(operatorCopy2, 1, 1, 0); testHarnessCopy2.open(); - testReadSplit(functionCopy2, () -> null, 3, 3, 3); + testReadSplit(operatorCopy2, () -> null, 3, 3, 3); } } @@ -231,7 +255,7 @@ public void testReadOperatorMetricsRegisterAndUpdate() throws Exception { } private T testReadSplit( - MonitorFunction function, + SourceOperator operator, SupplierWithException beforeClose, int a, int b, @@ -239,20 +263,36 @@ private T testReadSplit( throws Exception { Throwable[] error = new Throwable[1]; ArrayBlockingQueue queue = new ArrayBlockingQueue<>(10); + AtomicReference> iteratorRef = new AtomicReference<>(); - DummySourceContext sourceContext = - new DummySourceContext() { + PushingAsyncDataInput.DataOutput output = + new PushingAsyncDataInput.DataOutput() { @Override - public void collect(Split element) { - queue.add(element); + public void emitRecord(StreamRecord streamRecord) { + queue.add(streamRecord.getValue()); } + + @Override + public void emitWatermark(Watermark watermark) {} + + @Override + public void emitWatermarkStatus(WatermarkStatus watermarkStatus) {} + + @Override + public void emitLatencyMarker(LatencyMarker latencyMarker) {} + + @Override + public void emitRecordAttributes(RecordAttributes recordAttributes) {} }; + AtomicBoolean isRunning = new AtomicBoolean(true); Thread runner = new Thread( () -> { try { - function.run(sourceContext); + while (isRunning.get()) { + operator.emitNext(output); + } } catch (Throwable t) { t.printStackTrace(); error[0] = t; @@ -266,34 +306,15 @@ public void collect(Split element) { assertThat(readSplit(split)).containsExactlyInAnyOrder(Arrays.asList(a, b, c)); T t = beforeClose.get(); - function.cancel(); + CloseableIterator iterator = iteratorRef.get(); + if (iterator != null) { + iterator.close(); + } + isRunning.set(false); runner.join(); assertThat(error[0]).isNull(); return t; } - - private abstract static class DummySourceContext - implements SourceFunction.SourceContext { - - private final Object lock = new Object(); - - @Override - public void collectWithTimestamp(Split element, long timestamp) {} - - @Override - public void emitWatermark(Watermark mark) {} - - @Override - public void markAsTemporarilyIdle() {} - - @Override - public Object getCheckpointLock() { - return lock; - } - - @Override - public void close() {} - } } diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/source/operator/TestingSourceOperator.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/source/operator/TestingSourceOperator.java new file mode 100644 index 0000000000000..8e24db54c5ec2 --- /dev/null +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/source/operator/TestingSourceOperator.java @@ -0,0 +1,175 @@ +/* + * 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.source.SimpleSourceSplit; +import org.apache.paimon.flink.source.SimpleSourceSplitSerializer; + +import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.eventtime.WatermarkStrategy; +import org.apache.flink.api.common.state.OperatorStateStore; +import org.apache.flink.api.connector.source.SourceReader; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.core.fs.CloseableRegistry; +import org.apache.flink.runtime.execution.Environment; +import org.apache.flink.runtime.metrics.groups.UnregisteredMetricGroups; +import org.apache.flink.runtime.operators.coordination.OperatorEvent; +import org.apache.flink.runtime.operators.coordination.OperatorEventGateway; +import org.apache.flink.runtime.operators.testutils.DummyEnvironment; +import org.apache.flink.runtime.operators.testutils.MockEnvironmentBuilder; +import org.apache.flink.runtime.state.AbstractStateBackend; +import org.apache.flink.runtime.state.OperatorStateBackendParametersImpl; +import org.apache.flink.runtime.state.StateInitializationContext; +import org.apache.flink.runtime.state.StateInitializationContextImpl; +import org.apache.flink.runtime.state.hashmap.HashMapStateBackend; +import org.apache.flink.streaming.api.operators.SourceOperator; +import org.apache.flink.streaming.api.operators.StreamOperatorParameters; +import org.apache.flink.streaming.api.operators.StreamingRuntimeContext; +import org.apache.flink.streaming.runtime.tasks.ProcessingTimeService; +import org.apache.flink.streaming.runtime.tasks.SourceOperatorStreamTask; +import org.apache.flink.streaming.runtime.tasks.TestProcessingTimeService; +import org.apache.flink.streaming.util.MockOutput; +import org.apache.flink.streaming.util.MockStreamConfig; +import org.apache.flink.streaming.util.MockStreamingRuntimeContext; + +import java.util.ArrayList; +import java.util.Collections; + +/** A SourceOperator extension to simplify test setup. */ +public class TestingSourceOperator extends SourceOperator { + + private static final long serialVersionUID = 1L; + + private final int subtaskIndex; + private final int parallelism; + + public TestingSourceOperator( + StreamOperatorParameters parameters, + SourceReader reader, + WatermarkStrategy watermarkStrategy, + ProcessingTimeService timeService, + boolean emitProgressiveWatermarks) { + + this( + parameters, + reader, + watermarkStrategy, + timeService, + new TestingOperatorEventGateway(), + 1, + 5, + emitProgressiveWatermarks); + } + + public TestingSourceOperator( + StreamOperatorParameters parameters, + SourceReader reader, + WatermarkStrategy watermarkStrategy, + ProcessingTimeService timeService, + OperatorEventGateway eventGateway, + int subtaskIndex, + int parallelism, + boolean emitProgressiveWatermarks) { + + super( + (context) -> reader, + eventGateway, + new SimpleSourceSplitSerializer(), + watermarkStrategy, + timeService, + new Configuration(), + "localhost", + emitProgressiveWatermarks, + () -> false); + + this.subtaskIndex = subtaskIndex; + this.parallelism = parallelism; + this.metrics = UnregisteredMetricGroups.createUnregisteredOperatorMetricGroup(); + initSourceMetricGroup(); + + // unchecked wrapping is okay to keep tests simpler + try { + initReader(); + } catch (Exception e) { + throw new RuntimeException(e); + } + + setup(parameters.getContainingTask(), parameters.getStreamConfig(), parameters.getOutput()); + } + + @Override + public StreamingRuntimeContext getRuntimeContext() { + return new MockStreamingRuntimeContext(false, parallelism, subtaskIndex); + } + + // this is overridden to avoid complex mock injection through the "containingTask" + @Override + public ExecutionConfig getExecutionConfig() { + ExecutionConfig cfg = new ExecutionConfig(); + cfg.setAutoWatermarkInterval(100); + return cfg; + } + + public static SourceOperator createTestOperator( + SourceReader reader, + WatermarkStrategy watermarkStrategy, + boolean emitProgressiveWatermarks) + throws Exception { + + AbstractStateBackend abstractStateBackend = new HashMapStateBackend(); + Environment env = new MockEnvironmentBuilder().build(); + CloseableRegistry cancelStreamRegistry = new CloseableRegistry(); + final OperatorStateStore operatorStateStore = + abstractStateBackend.createOperatorStateBackend( + new OperatorStateBackendParametersImpl( + env, + "test-operator", + Collections.emptyList(), + cancelStreamRegistry)); + + final StateInitializationContext stateContext = + new StateInitializationContextImpl(null, operatorStateStore, null, null, null); + + TestProcessingTimeService timeService = new TestProcessingTimeService(); + timeService.setCurrentTime(Integer.MAX_VALUE); // start somewhere that is not zero + + final SourceOperator sourceOperator = + new TestingSourceOperator<>( + new StreamOperatorParameters<>( + new SourceOperatorStreamTask(new DummyEnvironment()), + new MockStreamConfig(new Configuration(), 1), + new MockOutput<>(new ArrayList<>()), + null, + null, + null), + reader, + watermarkStrategy, + timeService, + emitProgressiveWatermarks); + sourceOperator.initializeState(stateContext); + sourceOperator.open(); + + return sourceOperator; + } + + private static class TestingOperatorEventGateway implements OperatorEventGateway { + @Override + public void sendEventToCoordinator(OperatorEvent event) {} + } +}