From a0d9c595bb91d2462f7a8e0e646da6e87d89fc24 Mon Sep 17 00:00:00 2001 From: Xianxun Ye Date: Fri, 15 Sep 2023 11:03:02 +0800 Subject: [PATCH] [AMORO-1422][Flink] Support MoR with Flink in SQL/Table API for Mix-format table (#1952) * [AMORO-1422][Flink] Support MoR with Flink in SQL/Table API for Mix-format table * [AMORO-1422][Flink] Support MoR with Flink in SQL/Table API for Mix-format table * [AMORO-1422][Flink] fixed jingsong's comments. --- docs/engines/flink/flink-dml.md | 3 +- .../arctic/flink/read/ArcticSource.java | 6 +- .../hybrid/assigner/ShuffleSplitAssigner.java | 2 +- .../arctic/flink/table/FlinkSource.java | 2 +- .../arctic/flink/read/ArcticSource.java | 22 +-- .../hybrid/assigner/ShuffleSplitAssigner.java | 29 +++- .../hybrid/assigner/StaticSplitAssigner.java | 130 +++++++++++++++++ .../ContinuousSplitPlannerImpl.java | 6 + .../enumerator/MergeOnReadPlannerImpl.java | 85 +++++++++++ .../StaticArcticSourceEnumerator.java | 11 +- .../reader/DataIteratorReaderFunction.java | 8 +- .../read/hybrid/reader/HybridSplitReader.java | 5 +- .../hybrid/reader/RowDataReaderFunction.java | 20 ++- .../hybrid/split/ArcticSplitSerializer.java | 21 ++- .../read/hybrid/split/ArcticSplitState.java | 12 +- .../read/hybrid/split/MergeOnReadSplit.java | 24 ++- .../flink/read/source/ArcticScanContext.java | 126 ++++++---------- .../flink/read/source/DataIterator.java | 15 +- .../read/source/MergeOnReadDataIterator.java | 133 +++++++++++++++++ .../arctic/flink/table/ArcticFileSource.java | 24 ++- .../flink/table/DynamicTableFactory.java | 17 +-- .../arctic/flink/table/FlinkSource.java | 15 +- .../arctic/flink/read/TestArcticSource.java | 17 ++- .../assigner/TestShuffleSplitAssigner.java | 4 +- .../assigner/TestStaticSplitAssigner.java | 85 +++++++++++ .../TestArcticSourceEnumStateSerializer.java | 39 ++--- .../reader/TestRowDataReaderFunction.java | 33 ++++- .../split/TestArcticSplitSerializer.java | 13 +- .../netease/arctic/flink/table/TestKeyed.java | 112 ++++++-------- .../arctic/flink/read/ArcticSource.java | 24 +-- .../hybrid/assigner/ShuffleSplitAssigner.java | 29 +++- .../hybrid/assigner/StaticSplitAssigner.java | 130 +++++++++++++++++ .../ContinuousSplitPlannerImpl.java | 6 + .../enumerator/MergeOnReadPlannerImpl.java | 85 +++++++++++ .../StaticArcticSourceEnumerator.java | 11 +- .../reader/DataIteratorReaderFunction.java | 8 +- .../read/hybrid/reader/HybridSplitReader.java | 5 +- .../hybrid/reader/RowDataReaderFunction.java | 20 ++- .../hybrid/split/ArcticSplitSerializer.java | 21 ++- .../read/hybrid/split/ArcticSplitState.java | 12 +- .../read/hybrid/split/MergeOnReadSplit.java | 24 ++- .../flink/read/source/ArcticScanContext.java | 128 ++++++---------- .../flink/read/source/DataIterator.java | 15 +- .../read/source/MergeOnReadDataIterator.java | 132 +++++++++++++++++ .../arctic/flink/table/ArcticFileSource.java | 24 ++- .../flink/table/DynamicTableFactory.java | 19 ++- .../arctic/flink/table/FlinkSource.java | 15 +- .../arctic/flink/read/TestArcticSource.java | 26 ++-- .../assigner/TestShuffleSplitAssigner.java | 4 +- .../assigner/TestStaticSplitAssigner.java | 85 +++++++++++ .../TestArcticSourceEnumStateSerializer.java | 39 ++--- .../reader/TestRowDataReaderFunction.java | 33 ++++- .../split/TestArcticSplitSerializer.java | 13 +- .../netease/arctic/flink/table/TestKeyed.java | 138 ++++++++---------- 54 files changed, 1539 insertions(+), 526 deletions(-) create mode 100644 flink/v1.14/flink/src/main/java/com/netease/arctic/flink/read/hybrid/assigner/StaticSplitAssigner.java create mode 100644 flink/v1.14/flink/src/main/java/com/netease/arctic/flink/read/hybrid/enumerator/MergeOnReadPlannerImpl.java create mode 100644 flink/v1.14/flink/src/main/java/com/netease/arctic/flink/read/source/MergeOnReadDataIterator.java create mode 100644 flink/v1.14/flink/src/test/java/com/netease/arctic/flink/read/hybrid/assigner/TestStaticSplitAssigner.java create mode 100644 flink/v1.15/flink/src/main/java/com/netease/arctic/flink/read/hybrid/assigner/StaticSplitAssigner.java create mode 100644 flink/v1.15/flink/src/main/java/com/netease/arctic/flink/read/hybrid/enumerator/MergeOnReadPlannerImpl.java create mode 100644 flink/v1.15/flink/src/main/java/com/netease/arctic/flink/read/source/MergeOnReadDataIterator.java create mode 100644 flink/v1.15/flink/src/test/java/com/netease/arctic/flink/read/hybrid/assigner/TestStaticSplitAssigner.java diff --git a/docs/engines/flink/flink-dml.md b/docs/engines/flink/flink-dml.md index 374bf328ff..826d2afd84 100644 --- a/docs/engines/flink/flink-dml.md +++ b/docs/engines/flink/flink-dml.md @@ -57,8 +57,7 @@ The supported parameters for bounded reads of non-primary-key tables in BaseStor ### Batch mode (primary key table) ```sql --- Read the current full amount and possibly unmerged ChangeStore data --- TODO: In the future, bounded full amount data will be read using the MOR method. +-- Merge on Read the current mixed-format table and return append-only data. SELECT * FROM keyed /*+ OPTIONS('streaming'='false', 'scan.startup.mode'='earliest')*/; ``` diff --git a/flink/v1.12/flink/src/main/java/com/netease/arctic/flink/read/ArcticSource.java b/flink/v1.12/flink/src/main/java/com/netease/arctic/flink/read/ArcticSource.java index afd65ea745..589cd63ed3 100644 --- a/flink/v1.12/flink/src/main/java/com/netease/arctic/flink/read/ArcticSource.java +++ b/flink/v1.12/flink/src/main/java/com/netease/arctic/flink/read/ArcticSource.java @@ -80,13 +80,13 @@ public Boundedness getBoundedness() { } @Override - public SourceReader createReader(SourceReaderContext readerContext) throws Exception { + public SourceReader createReader(SourceReaderContext readerContext) { return new ArcticSourceReader<>(readerFunction, readerContext.getConfiguration(), readerContext, dimTable); } @Override public SplitEnumerator createEnumerator( - SplitEnumeratorContext enumContext) throws Exception { + SplitEnumeratorContext enumContext) { return createEnumerator(enumContext, null); } @@ -112,7 +112,7 @@ private SplitEnumerator createEnumerator( @Override public SplitEnumerator restoreEnumerator( SplitEnumeratorContext enumContext, - ArcticSourceEnumState checkpoint) throws Exception { + ArcticSourceEnumState checkpoint) { return createEnumerator(enumContext, checkpoint); } diff --git a/flink/v1.12/flink/src/main/java/com/netease/arctic/flink/read/hybrid/assigner/ShuffleSplitAssigner.java b/flink/v1.12/flink/src/main/java/com/netease/arctic/flink/read/hybrid/assigner/ShuffleSplitAssigner.java index ed1a5c08be..d40e7fd3af 100644 --- a/flink/v1.12/flink/src/main/java/com/netease/arctic/flink/read/hybrid/assigner/ShuffleSplitAssigner.java +++ b/flink/v1.12/flink/src/main/java/com/netease/arctic/flink/read/hybrid/assigner/ShuffleSplitAssigner.java @@ -43,7 +43,7 @@ import java.util.stream.Collectors; /** - * According to Mark,Index TreeNodes and subtaskId assigning a split to special subtask to read. + * According to Mark, Index TreeNodes and subtaskId assigning a split to special subtask to read. */ public class ShuffleSplitAssigner implements SplitAssigner { private static final Logger LOG = LoggerFactory.getLogger(ShuffleSplitAssigner.class); diff --git a/flink/v1.12/flink/src/main/java/com/netease/arctic/flink/table/FlinkSource.java b/flink/v1.12/flink/src/main/java/com/netease/arctic/flink/table/FlinkSource.java index f96962aabe..410480b01e 100644 --- a/flink/v1.12/flink/src/main/java/com/netease/arctic/flink/table/FlinkSource.java +++ b/flink/v1.12/flink/src/main/java/com/netease/arctic/flink/table/FlinkSource.java @@ -74,7 +74,7 @@ public static final class Builder { private TableSchema projectedSchema; private List filters; private ReadableConfig flinkConf = new Configuration(); - private Map properties = new HashMap<>(); + private final Map properties = new HashMap<>(); private long limit = -1L; private WatermarkStrategy watermarkStrategy = WatermarkStrategy.noWatermarks(); private final ArcticScanContext.Builder contextBuilder = ArcticScanContext.arcticBuilder(); diff --git a/flink/v1.14/flink/src/main/java/com/netease/arctic/flink/read/ArcticSource.java b/flink/v1.14/flink/src/main/java/com/netease/arctic/flink/read/ArcticSource.java index afd65ea745..7db69a074e 100644 --- a/flink/v1.14/flink/src/main/java/com/netease/arctic/flink/read/ArcticSource.java +++ b/flink/v1.14/flink/src/main/java/com/netease/arctic/flink/read/ArcticSource.java @@ -20,6 +20,7 @@ import com.netease.arctic.flink.read.hybrid.assigner.ShuffleSplitAssigner; import com.netease.arctic.flink.read.hybrid.assigner.SplitAssigner; +import com.netease.arctic.flink.read.hybrid.assigner.StaticSplitAssigner; import com.netease.arctic.flink.read.hybrid.enumerator.ArcticSourceEnumState; import com.netease.arctic.flink.read.hybrid.enumerator.ArcticSourceEnumStateSerializer; import com.netease.arctic.flink.read.hybrid.enumerator.ArcticSourceEnumerator; @@ -39,8 +40,6 @@ import org.apache.flink.api.connector.source.SplitEnumeratorContext; import org.apache.flink.api.java.typeutils.ResultTypeQueryable; import org.apache.flink.core.io.SimpleVersionedSerializer; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; /** * Arctic Source based of Flip27. @@ -51,8 +50,6 @@ */ public class ArcticSource implements Source, ResultTypeQueryable { private static final long serialVersionUID = 1L; - - private static final Logger LOG = LoggerFactory.getLogger(ArcticSource.class); private final ArcticScanContext scanContext; private final ReaderFunction readerFunction; private final TypeInformation typeInformation; @@ -80,31 +77,24 @@ public Boundedness getBoundedness() { } @Override - public SourceReader createReader(SourceReaderContext readerContext) throws Exception { + public SourceReader createReader(SourceReaderContext readerContext) { return new ArcticSourceReader<>(readerFunction, readerContext.getConfiguration(), readerContext, dimTable); } @Override public SplitEnumerator createEnumerator( - SplitEnumeratorContext enumContext) throws Exception { + SplitEnumeratorContext enumContext) { return createEnumerator(enumContext, null); } private SplitEnumerator createEnumerator( SplitEnumeratorContext enumContext, ArcticSourceEnumState enumState) { SplitAssigner splitAssigner; - if (enumState == null) { - splitAssigner = new ShuffleSplitAssigner(enumContext); - } else { - LOG.info("Arctic source restored {} splits from state for table {}", - enumState.pendingSplits().size(), tableName); - splitAssigner = new ShuffleSplitAssigner(enumContext, enumState.pendingSplits(), - enumState.shuffleSplitRelation()); - } - if (scanContext.isStreaming()) { + splitAssigner = new ShuffleSplitAssigner(enumContext, tableName, enumState); return new ArcticSourceEnumerator(enumContext, splitAssigner, loader, scanContext, enumState, dimTable); } else { + splitAssigner = new StaticSplitAssigner(enumState); return new StaticArcticSourceEnumerator(enumContext, splitAssigner, loader, scanContext, null); } } @@ -112,7 +102,7 @@ private SplitEnumerator createEnumerator( @Override public SplitEnumerator restoreEnumerator( SplitEnumeratorContext enumContext, - ArcticSourceEnumState checkpoint) throws Exception { + ArcticSourceEnumState checkpoint) { return createEnumerator(enumContext, checkpoint); } diff --git a/flink/v1.14/flink/src/main/java/com/netease/arctic/flink/read/hybrid/assigner/ShuffleSplitAssigner.java b/flink/v1.14/flink/src/main/java/com/netease/arctic/flink/read/hybrid/assigner/ShuffleSplitAssigner.java index ed1a5c08be..93b4f1ca21 100644 --- a/flink/v1.14/flink/src/main/java/com/netease/arctic/flink/read/hybrid/assigner/ShuffleSplitAssigner.java +++ b/flink/v1.14/flink/src/main/java/com/netease/arctic/flink/read/hybrid/assigner/ShuffleSplitAssigner.java @@ -20,20 +20,24 @@ import com.netease.arctic.data.DataTreeNode; import com.netease.arctic.data.PrimaryKeyedFile; +import com.netease.arctic.flink.read.hybrid.enumerator.ArcticSourceEnumState; import com.netease.arctic.flink.read.hybrid.split.ArcticSplit; import com.netease.arctic.flink.read.hybrid.split.ArcticSplitState; import com.netease.arctic.scan.ArcticFileScanTask; +import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.api.connector.source.SplitEnumeratorContext; import org.apache.flink.util.FlinkRuntimeException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import javax.annotation.Nullable; import java.io.IOException; import java.util.ArrayList; import java.util.Collection; import java.util.Collections; import java.util.List; import java.util.Map; +import java.util.Objects; import java.util.Optional; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ConcurrentHashMap; @@ -43,7 +47,7 @@ import java.util.stream.Collectors; /** - * According to Mark,Index TreeNodes and subtaskId assigning a split to special subtask to read. + * According to Mark, Index TreeNodes and subtaskId assigning a split to special subtask to read. */ public class ShuffleSplitAssigner implements SplitAssigner { private static final Logger LOG = LoggerFactory.getLogger(ShuffleSplitAssigner.class); @@ -68,6 +72,7 @@ public class ShuffleSplitAssigner implements SplitAssigner { private CompletableFuture availableFuture; + @VisibleForTesting public ShuffleSplitAssigner( SplitEnumeratorContext enumeratorContext) { this.enumeratorContext = enumeratorContext; @@ -77,15 +82,29 @@ public ShuffleSplitAssigner( } public ShuffleSplitAssigner( - SplitEnumeratorContext enumeratorContext, Collection splitStates, - long[] shuffleSplitRelation) { + SplitEnumeratorContext enumeratorContext, + String tableName, + @Nullable ArcticSourceEnumState enumState) { this.enumeratorContext = enumeratorContext; this.partitionIndexSubtaskMap = new ConcurrentHashMap<>(); this.subtaskSplitMap = new ConcurrentHashMap<>(); - deserializePartitionIndex(shuffleSplitRelation); - splitStates.forEach(state -> onDiscoveredSplits(Collections.singleton(state.toSourceSplit()))); + if (enumState == null) { + this.totalParallelism = enumeratorContext.currentParallelism(); + LOG.info( + "Arctic source enumerator current parallelism is {} for table {}", + totalParallelism, tableName); + } else { + LOG.info("Arctic source restored {} splits from state for table {}", + enumState.pendingSplits().size(), tableName); + deserializePartitionIndex( + Objects.requireNonNull( + enumState.shuffleSplitRelation(), + "The partition index and subtask state couldn't be null.")); + enumState.pendingSplits().forEach(state -> onDiscoveredSplits(Collections.singleton(state.toSourceSplit()))); + } } + @Override public Split getNext() { throw new UnsupportedOperationException("ShuffleSplitAssigner couldn't support this operation."); diff --git a/flink/v1.14/flink/src/main/java/com/netease/arctic/flink/read/hybrid/assigner/StaticSplitAssigner.java b/flink/v1.14/flink/src/main/java/com/netease/arctic/flink/read/hybrid/assigner/StaticSplitAssigner.java new file mode 100644 index 0000000000..4a0f8bc1bb --- /dev/null +++ b/flink/v1.14/flink/src/main/java/com/netease/arctic/flink/read/hybrid/assigner/StaticSplitAssigner.java @@ -0,0 +1,130 @@ +/* + * 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 com.netease.arctic.flink.read.hybrid.assigner; + +import com.netease.arctic.flink.read.hybrid.enumerator.ArcticSourceEnumState; +import com.netease.arctic.flink.read.hybrid.split.ArcticSplit; +import com.netease.arctic.flink.read.hybrid.split.ArcticSplitState; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.Nullable; +import java.io.IOException; +import java.util.Collection; +import java.util.Collections; +import java.util.Optional; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.PriorityBlockingQueue; +import java.util.concurrent.TimeUnit; +import java.util.stream.Collectors; + +/** + * This is a static split assigner which is used for batch mode. + */ +public class StaticSplitAssigner implements SplitAssigner { + private static final Logger LOG = LoggerFactory.getLogger(StaticSplitAssigner.class); + + private static final long POLL_TIMEOUT = 200; + private int totalSplitNum; + + private final PriorityBlockingQueue splitQueue; + + private CompletableFuture availableFuture; + + public StaticSplitAssigner(@Nullable ArcticSourceEnumState enumState) { + this.splitQueue = new PriorityBlockingQueue<>(); + if (enumState != null) { + Collection splitStates = enumState.pendingSplits(); + splitStates.forEach(state -> onDiscoveredSplits(Collections.singleton(state.toSourceSplit()))); + } + } + + @Override + public Split getNext() { + return getNextSplit().map(Split::of).orElseGet(Split::unavailable); + } + + @Override + public Split getNext(int subtaskId) { + return getNext(); + } + + private Optional getNextSplit() { + ArcticSplit arcticSplit = null; + try { + arcticSplit = splitQueue.poll(POLL_TIMEOUT, TimeUnit.MILLISECONDS); + } catch (InterruptedException e) { + LOG.warn("Interrupted when polling splits from the split queue", e); + } + if (arcticSplit == null) { + LOG.debug("Couldn't retrieve arctic source split from the queue, as the queue is empty."); + return Optional.empty(); + } else { + LOG.info("Assigning the arctic split, task index is {}, total number of splits is {}, arctic split is {}.", + arcticSplit.taskIndex(), totalSplitNum, arcticSplit); + return Optional.of(arcticSplit); + } + } + + @Override + public void onDiscoveredSplits(Collection splits) { + splits.forEach(this::putArcticIntoQueue); + totalSplitNum += splits.size(); + // only complete pending future if new splits are discovered + completeAvailableFuturesIfNeeded(); + } + + @Override + public void onUnassignedSplits(Collection splits) { + onDiscoveredSplits(splits); + } + + void putArcticIntoQueue(final ArcticSplit split) { + splitQueue.put(split); + } + + @Override + public Collection state() { + return splitQueue.stream().map(ArcticSplitState::new).collect(Collectors.toList()); + } + + @Override + public synchronized CompletableFuture isAvailable() { + if (availableFuture == null) { + availableFuture = new CompletableFuture<>(); + } + return availableFuture; + } + + public boolean isEmpty() { + return splitQueue.isEmpty(); + } + + @Override + public void close() throws IOException { + splitQueue.clear(); + } + + private synchronized void completeAvailableFuturesIfNeeded() { + if (availableFuture != null && !isEmpty()) { + availableFuture.complete(null); + } + availableFuture = null; + } +} diff --git a/flink/v1.14/flink/src/main/java/com/netease/arctic/flink/read/hybrid/enumerator/ContinuousSplitPlannerImpl.java b/flink/v1.14/flink/src/main/java/com/netease/arctic/flink/read/hybrid/enumerator/ContinuousSplitPlannerImpl.java index a911de23d0..297c2d5680 100644 --- a/flink/v1.14/flink/src/main/java/com/netease/arctic/flink/read/hybrid/enumerator/ContinuousSplitPlannerImpl.java +++ b/flink/v1.14/flink/src/main/java/com/netease/arctic/flink/read/hybrid/enumerator/ContinuousSplitPlannerImpl.java @@ -61,6 +61,9 @@ public ContinuousSplitPlannerImpl(ArcticTableLoader loader) { @Override public void close() throws IOException { + if (loader != null) { + loader.close(); + } } @Override @@ -109,6 +112,9 @@ protected ContinuousEnumerationResult discoverIncrementalSplits( protected ContinuousEnumerationResult discoverInitialSplits(List filters) { Snapshot changeSnapshot = table.changeTable().currentSnapshot(); + // todo ShuffleSplitAssigner doesn't support MergeOnReadSplit right now, + // because it doesn't implement the dataTreeNode() method + // fix AMORO-1950 in the future. List arcticSplits = FlinkSplitPlanner.planFullTable(table, filters, splitCount); long changeStartSnapshotId = changeSnapshot != null ? changeSnapshot.snapshotId() : EARLIEST_SNAPSHOT_ID; diff --git a/flink/v1.14/flink/src/main/java/com/netease/arctic/flink/read/hybrid/enumerator/MergeOnReadPlannerImpl.java b/flink/v1.14/flink/src/main/java/com/netease/arctic/flink/read/hybrid/enumerator/MergeOnReadPlannerImpl.java new file mode 100644 index 0000000000..6d95b335e9 --- /dev/null +++ b/flink/v1.14/flink/src/main/java/com/netease/arctic/flink/read/hybrid/enumerator/MergeOnReadPlannerImpl.java @@ -0,0 +1,85 @@ +/* + * 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 com.netease.arctic.flink.read.hybrid.enumerator; + +import com.netease.arctic.flink.read.FlinkSplitPlanner; +import com.netease.arctic.flink.read.hybrid.split.ArcticSplit; +import com.netease.arctic.flink.table.ArcticTableLoader; +import com.netease.arctic.table.KeyedTable; +import org.apache.commons.collections.CollectionUtils; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.expressions.Expression; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.List; +import java.util.concurrent.atomic.AtomicInteger; + +import static com.netease.arctic.flink.read.hybrid.enumerator.ArcticEnumeratorOffset.EARLIEST_SNAPSHOT_ID; +import static com.netease.arctic.flink.util.ArcticUtils.loadArcticTable; + +/** + * Used for MergeOnRead, only for the bounded reading and return append stream. + */ +public class MergeOnReadPlannerImpl implements ContinuousSplitPlanner { + private static final Logger LOG = LoggerFactory.getLogger(MergeOnReadPlannerImpl.class); + + protected transient KeyedTable table; + protected final ArcticTableLoader loader; + protected static final AtomicInteger splitCount = new AtomicInteger(); + + public MergeOnReadPlannerImpl(ArcticTableLoader loader) { + this.loader = loader; + } + + @Override + public ContinuousEnumerationResult planSplits( + ArcticEnumeratorOffset ignored, List filters) { + // todo support mor the table from the specific offset in the future + if (table == null) { + table = loadArcticTable(loader).asKeyedTable(); + } + table.refresh(); + return discoverInitialSplits(filters); + } + + protected ContinuousEnumerationResult discoverInitialSplits(List filters) { + Snapshot changeSnapshot = table.changeTable().currentSnapshot(); + List arcticSplits = FlinkSplitPlanner.mergeOnReadPlan(table, filters, splitCount); + + long changeStartSnapshotId = changeSnapshot != null ? changeSnapshot.snapshotId() : EARLIEST_SNAPSHOT_ID; + if (changeSnapshot == null && CollectionUtils.isEmpty(arcticSplits)) { + LOG.info("There have no change snapshot, and no base splits in table: {}.", table); + return ContinuousEnumerationResult.EMPTY; + } + + return new ContinuousEnumerationResult( + arcticSplits, + null, + ArcticEnumeratorOffset.of(changeStartSnapshotId, null)); + } + + @Override + public void close() throws IOException { + if (loader != null) { + loader.close(); + } + } +} diff --git a/flink/v1.14/flink/src/main/java/com/netease/arctic/flink/read/hybrid/enumerator/StaticArcticSourceEnumerator.java b/flink/v1.14/flink/src/main/java/com/netease/arctic/flink/read/hybrid/enumerator/StaticArcticSourceEnumerator.java index a4db0edb83..7bec0a8597 100644 --- a/flink/v1.14/flink/src/main/java/com/netease/arctic/flink/read/hybrid/enumerator/StaticArcticSourceEnumerator.java +++ b/flink/v1.14/flink/src/main/java/com/netease/arctic/flink/read/hybrid/enumerator/StaticArcticSourceEnumerator.java @@ -18,19 +18,18 @@ package com.netease.arctic.flink.read.hybrid.enumerator; -import com.netease.arctic.flink.read.FlinkSplitPlanner; import com.netease.arctic.flink.read.hybrid.assigner.SplitAssigner; import com.netease.arctic.flink.read.hybrid.split.ArcticSplit; import com.netease.arctic.flink.read.source.ArcticScanContext; import com.netease.arctic.flink.table.ArcticTableLoader; import com.netease.arctic.table.KeyedTable; import org.apache.flink.api.connector.source.SplitEnumeratorContext; +import org.apache.iceberg.flink.source.ScanContext; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import javax.annotation.Nullable; -import java.util.List; -import java.util.concurrent.atomic.AtomicInteger; +import java.util.Collection; import static com.netease.arctic.flink.util.ArcticUtils.loadArcticTable; @@ -45,6 +44,7 @@ public class StaticArcticSourceEnumerator extends AbstractArcticEnumerator { private transient KeyedTable keyedTable; private final ArcticScanContext scanContext; private final boolean shouldEnumerate; + private final ContinuousSplitPlanner splitPlanner; public StaticArcticSourceEnumerator( SplitEnumeratorContext enumeratorContext, @@ -56,8 +56,9 @@ public StaticArcticSourceEnumerator( this.loader = loader; this.assigner = assigner; this.scanContext = scanContext; - // split enumeration is not needed during restore scenario + // split enumeration is not needed during a restore scenario this.shouldEnumerate = enumState == null; + this.splitPlanner = new MergeOnReadPlannerImpl(loader); } @Override @@ -69,7 +70,7 @@ public void start() { if (shouldEnumerate) { keyedTable.baseTable().refresh(); keyedTable.changeTable().refresh(); - List splits = FlinkSplitPlanner.planFullTable(keyedTable, new AtomicInteger()); + Collection splits = splitPlanner.planSplits(null, scanContext.filters()).splits(); assigner.onDiscoveredSplits(splits); LOG.info("Discovered {} splits from table {} during job initialization", splits.size(), keyedTable.name()); diff --git a/flink/v1.14/flink/src/main/java/com/netease/arctic/flink/read/hybrid/reader/DataIteratorReaderFunction.java b/flink/v1.14/flink/src/main/java/com/netease/arctic/flink/read/hybrid/reader/DataIteratorReaderFunction.java index cee14a9479..8338d10c32 100644 --- a/flink/v1.14/flink/src/main/java/com/netease/arctic/flink/read/hybrid/reader/DataIteratorReaderFunction.java +++ b/flink/v1.14/flink/src/main/java/com/netease/arctic/flink/read/hybrid/reader/DataIteratorReaderFunction.java @@ -23,6 +23,7 @@ import com.netease.arctic.flink.read.hybrid.split.ChangelogSplit; import com.netease.arctic.flink.read.source.ChangeLogDataIterator; import com.netease.arctic.flink.read.source.DataIterator; +import com.netease.arctic.flink.read.source.MergeOnReadDataIterator; import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds; import org.apache.iceberg.io.CloseableIterator; @@ -41,7 +42,12 @@ public DataIteratorReaderFunction(DataIteratorBatcher batcher) { @Override public CloseableIterator>> apply(ArcticSplit split) { DataIterator inputIterator = createDataIterator(split); - if (inputIterator instanceof ChangeLogDataIterator) { + if (inputIterator instanceof MergeOnReadDataIterator) { + inputIterator.seek( + 0, + split.asMergeOnReadSplit().recordOffset() + ); + } else if (inputIterator instanceof ChangeLogDataIterator) { ChangeLogDataIterator changelogInputIterator = (ChangeLogDataIterator) inputIterator; ChangelogSplit changelogSplit = split.asChangelogSplit(); changelogInputIterator.seek( diff --git a/flink/v1.14/flink/src/main/java/com/netease/arctic/flink/read/hybrid/reader/HybridSplitReader.java b/flink/v1.14/flink/src/main/java/com/netease/arctic/flink/read/hybrid/reader/HybridSplitReader.java index 609fd8f8d7..72e239e7f5 100644 --- a/flink/v1.14/flink/src/main/java/com/netease/arctic/flink/read/hybrid/reader/HybridSplitReader.java +++ b/flink/v1.14/flink/src/main/java/com/netease/arctic/flink/read/hybrid/reader/HybridSplitReader.java @@ -20,6 +20,7 @@ import com.netease.arctic.flink.read.hybrid.split.ArcticSplit; import com.netease.arctic.flink.read.hybrid.split.ChangelogSplit; +import com.netease.arctic.flink.read.hybrid.split.MergeOnReadSplit; import com.netease.arctic.flink.read.hybrid.split.SnapshotSplit; import org.apache.flink.api.connector.source.SourceReaderContext; import org.apache.flink.connector.base.source.reader.RecordsBySplits; @@ -91,7 +92,9 @@ public void handleSplitsChanges(SplitsChange splitsChange) { LOG.info("Handling a split change {}.", splitsChange); splitsChange.splits().forEach(arcticSplit -> { - if (arcticSplit instanceof SnapshotSplit || arcticSplit instanceof ChangelogSplit) { + if (arcticSplit instanceof SnapshotSplit || + arcticSplit instanceof ChangelogSplit || + arcticSplit instanceof MergeOnReadSplit) { splits.add(arcticSplit); } else { throw new IllegalArgumentException( diff --git a/flink/v1.14/flink/src/main/java/com/netease/arctic/flink/read/hybrid/reader/RowDataReaderFunction.java b/flink/v1.14/flink/src/main/java/com/netease/arctic/flink/read/hybrid/reader/RowDataReaderFunction.java index d179a2b135..84ade6cafb 100644 --- a/flink/v1.14/flink/src/main/java/com/netease/arctic/flink/read/hybrid/reader/RowDataReaderFunction.java +++ b/flink/v1.14/flink/src/main/java/com/netease/arctic/flink/read/hybrid/reader/RowDataReaderFunction.java @@ -24,6 +24,8 @@ import com.netease.arctic.flink.read.source.DataIterator; import com.netease.arctic.flink.read.source.FileScanTaskReader; import com.netease.arctic.flink.read.source.FlinkArcticDataReader; +import com.netease.arctic.flink.read.source.FlinkArcticMORDataReader; +import com.netease.arctic.flink.read.source.MergeOnReadDataIterator; import com.netease.arctic.flink.util.ArcticUtils; import com.netease.arctic.io.ArcticFileIO; import com.netease.arctic.table.PrimaryKeySpec; @@ -89,8 +91,20 @@ public RowDataReaderFunction( @Override public DataIterator createDataIterator(ArcticSplit split) { - if (split.isSnapshotSplit()) { - + if (split.isMergeOnReadSplit()) { + FlinkArcticMORDataReader morDataReader = + new FlinkArcticMORDataReader( + io, + tableSchema, + readSchema, + primaryKeySpec, + nameMapping, + caseSensitive, + RowDataUtil::convertConstant, + reuse + ); + return new MergeOnReadDataIterator(morDataReader, split.asMergeOnReadSplit().keyedTableScanTask(), io); + } else if (split.isSnapshotSplit()) { FileScanTaskReader rowDataReader = new FlinkArcticDataReader( io, tableSchema, readSchema, primaryKeySpec, nameMapping, caseSensitive, RowDataUtil::convertConstant, @@ -150,7 +164,7 @@ RowData transformRowKind(ChangeLogDataIterator.ChangeActionTrans trans) * * @param tableSchema table schema * @param projectedSchema projected schema - * @return a new Schema on which include the identifier fields. + * @return a new Schema on which includes the identifier fields. */ private static Schema fillUpReadSchema(Schema tableSchema, Schema projectedSchema, PrimaryKeySpec primaryKeySpec) { Preconditions.checkNotNull(tableSchema, "Table schema can't be null"); diff --git a/flink/v1.14/flink/src/main/java/com/netease/arctic/flink/read/hybrid/split/ArcticSplitSerializer.java b/flink/v1.14/flink/src/main/java/com/netease/arctic/flink/read/hybrid/split/ArcticSplitSerializer.java index 4a81496b81..be3f91579d 100644 --- a/flink/v1.14/flink/src/main/java/com/netease/arctic/flink/read/hybrid/split/ArcticSplitSerializer.java +++ b/flink/v1.14/flink/src/main/java/com/netease/arctic/flink/read/hybrid/split/ArcticSplitSerializer.java @@ -34,6 +34,7 @@ public class ArcticSplitSerializer implements SimpleVersionedSerializerdeserializeObject( + content, MergeOnReadSplit.class.getClassLoader()); + } else if (flag == SNAPSHOT_SPLIT_FLAG) { return InstantiationUtil.deserializeObject(content, SnapshotSplit.class.getClassLoader()); } else if (flag == CHANGELOG_SPLIT_FLAG) { return InstantiationUtil.deserializeObject(content, ChangelogSplit.class.getClassLoader()); } else { - throw new IllegalArgumentException("this flag split is unsupported. available: 1,2."); + throw new IllegalArgumentException( + String.format( + "this flag split %s is unsupported. available: %s, %s, and %s.", + flag, + SNAPSHOT_SPLIT_FLAG, + CHANGELOG_SPLIT_FLAG, + MOR_SPLIT_FLAG)); } } } catch (ClassNotFoundException e) { diff --git a/flink/v1.14/flink/src/main/java/com/netease/arctic/flink/read/hybrid/split/ArcticSplitState.java b/flink/v1.14/flink/src/main/java/com/netease/arctic/flink/read/hybrid/split/ArcticSplitState.java index a3c73f4518..de02cc0f33 100644 --- a/flink/v1.14/flink/src/main/java/com/netease/arctic/flink/read/hybrid/split/ArcticSplitState.java +++ b/flink/v1.14/flink/src/main/java/com/netease/arctic/flink/read/hybrid/split/ArcticSplitState.java @@ -36,7 +36,11 @@ public ArcticSplitState(ArcticSplit arcticSplit) { } public ArcticSplit toSourceSplit() { - if (arcticSplit.isSnapshotSplit()) { + if (arcticSplit.isMergeOnReadSplit()) { + MergeOnReadSplit mergeOnReadSplit = (MergeOnReadSplit) arcticSplit; + mergeOnReadSplit.updateOffset(new Object[]{currentInsertFileOffset, currentInsertRecordOffset}); + return mergeOnReadSplit; + } else if (arcticSplit.isSnapshotSplit()) { SnapshotSplit snapshotSplit = (SnapshotSplit) arcticSplit; snapshotSplit.updateOffset(new Object[]{currentInsertFileOffset, currentInsertRecordOffset}); return snapshotSplit; @@ -52,10 +56,12 @@ public ArcticSplit toSourceSplit() { } throw new FlinkRuntimeException( - String.format("As of now this source split is unsupported %s, available split are %s, %s", + String.format( + "As of now this source split is unsupported %s, available split are %s, %s, and %s", arcticSplit.getClass().getSimpleName(), SnapshotSplit.class.getSimpleName(), - ChangelogSplit.class.getSimpleName())); + ChangelogSplit.class.getSimpleName(), + MergeOnReadSplit.class.getSimpleName())); } public void updateOffset(Object[] offsets) { diff --git a/flink/v1.14/flink/src/main/java/com/netease/arctic/flink/read/hybrid/split/MergeOnReadSplit.java b/flink/v1.14/flink/src/main/java/com/netease/arctic/flink/read/hybrid/split/MergeOnReadSplit.java index 7f32973c89..068a553910 100644 --- a/flink/v1.14/flink/src/main/java/com/netease/arctic/flink/read/hybrid/split/MergeOnReadSplit.java +++ b/flink/v1.14/flink/src/main/java/com/netease/arctic/flink/read/hybrid/split/MergeOnReadSplit.java @@ -20,13 +20,14 @@ import com.netease.arctic.scan.KeyedTableScanTask; import com.netease.arctic.utils.FileScanTaskUtil; -import org.apache.flink.util.FlinkRuntimeException; +import org.apache.flink.util.Preconditions; import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; public class MergeOnReadSplit extends ArcticSplit { private static final long serialVersionUID = 1L; private final int taskIndex; private final KeyedTableScanTask keyedTableScanTask; + private long recordOffset; public MergeOnReadSplit(int taskIndex, KeyedTableScanTask keyedTableScanTask) { this.taskIndex = taskIndex; @@ -43,8 +44,10 @@ public Integer taskIndex() { } @Override - public void updateOffset(Object[] recordOffsets) { - throw new FlinkRuntimeException("Merge On Read not support offset state right now."); + public void updateOffset(Object[] offsets) { + Preconditions.checkArgument(offsets.length == 2); + // offsets[0] is file offset, but we don't need it + recordOffset = (long) offsets[1]; } @Override @@ -61,6 +64,21 @@ public String splitId() { .toString(); } + public long recordOffset() { + return recordOffset; + } + + @Override + public boolean equals(Object obj) { + if (!(obj instanceof MergeOnReadSplit)) { + return false; + } + MergeOnReadSplit other = (MergeOnReadSplit) obj; + return splitId().equals(other.splitId()) && + recordOffset == other.recordOffset && + taskIndex == other.taskIndex; + } + @Override public String toString() { return MoreObjects.toStringHelper(this) diff --git a/flink/v1.14/flink/src/main/java/com/netease/arctic/flink/read/source/ArcticScanContext.java b/flink/v1.14/flink/src/main/java/com/netease/arctic/flink/read/source/ArcticScanContext.java index 1e7b81feba..1fd87e154b 100644 --- a/flink/v1.14/flink/src/main/java/com/netease/arctic/flink/read/source/ArcticScanContext.java +++ b/flink/v1.14/flink/src/main/java/com/netease/arctic/flink/read/source/ArcticScanContext.java @@ -52,60 +52,36 @@ public class ArcticScanContext extends ScanContext implements Serializable { private static final long serialVersionUID = 1L; private final String scanStartupMode; - - protected ArcticScanContext( - boolean caseSensitive, - Long snapshotId, - StreamingStartingStrategy startingStrategy, - Long startSnapshotTimestamp, - Long startSnapshotId, - Long endSnapshotId, - Long asOfTimestamp, - Long splitSize, - Integer splitLookback, - Long splitOpenFileCost, - boolean isStreaming, - Duration monitorInterval, - String nameMapping, - Schema schema, - List filters, - long limit, - boolean includeColumnStats, - boolean exposeLocality, - Integer planParallelism, - int maxPlanningSnapshotCount, - String scanStartupMode, - int maxAllowedPlanningFailures, - String branch, - String tag, - String startTag, - String endTag) { - super(caseSensitive, - snapshotId, - startingStrategy, - startSnapshotTimestamp, - startSnapshotId, - endSnapshotId, - asOfTimestamp, - splitSize, - splitLookback, - splitOpenFileCost, - isStreaming, - monitorInterval, - nameMapping, - schema, - filters, - limit, - includeColumnStats, - exposeLocality, - planParallelism, - maxPlanningSnapshotCount, - maxAllowedPlanningFailures, - branch, - tag, - startTag, - endTag); - this.scanStartupMode = scanStartupMode; + private final boolean batchMode; + + protected ArcticScanContext(Builder builder) { + super(builder.caseSensitive, + builder.snapshotId, + builder.startingStrategy, + builder.startSnapshotTimestamp, + builder.startSnapshotId, + builder.endSnapshotId, + builder.asOfTimestamp, + builder.splitSize, + builder.splitLookback, + builder.splitOpenFileCost, + builder.isStreaming, + builder.monitorInterval, + builder.nameMapping, + builder.projectedSchema, + builder.filters, + builder.limit, + builder.includeColumnStats, + builder.exposeLocality, + builder.planParallelism, + builder.maxPlanningSnapshotCount, + builder.maxAllowedPlanningFailures, + builder.branch, + builder.tag, + builder.startTag, + builder.endTag); + this.scanStartupMode = builder.scanStartupMode; + this.batchMode = builder.batchMode; } public boolean caseSensitive() { @@ -175,6 +151,10 @@ public String scanStartupMode() { return scanStartupMode; } + public boolean isBatchMode() { + return batchMode; + } + public static class Builder { private boolean caseSensitive = FlinkReadOptions.CASE_SENSITIVE_OPTION.defaultValue(); private Long snapshotId = FlinkReadOptions.SNAPSHOT_ID.defaultValue(); @@ -212,6 +192,7 @@ public static class Builder { private String endTag = FlinkReadOptions.END_TAG.defaultValue(); private String scanStartupMode; + private boolean batchMode = false; private Builder() { } @@ -346,6 +327,11 @@ public Builder includeColumnStats(boolean newIncludeColumnStats) { return this; } + public Builder batchMode(boolean batchMode) { + this.batchMode = batchMode; + return this; + } + public Builder fromProperties(Map properties) { Configuration config = new Configuration(); properties.forEach(config::setString); @@ -380,34 +366,10 @@ public ArcticScanContext build() { Objects.equals(scanStartupMode, SCAN_STARTUP_MODE_LATEST), String.format("only support %s, %s when %s is %s", SCAN_STARTUP_MODE_EARLIEST, SCAN_STARTUP_MODE_LATEST, ARCTIC_READ_MODE, ARCTIC_READ_FILE)); - return new ArcticScanContext( - caseSensitive, - snapshotId, - startingStrategy, - startSnapshotTimestamp, - startSnapshotId, - endSnapshotId, - asOfTimestamp, - splitSize, - splitLookback, - splitOpenFileCost, - isStreaming, - monitorInterval, - nameMapping, - projectedSchema, - filters, - limit, - includeColumnStats, - exposeLocality, - planParallelism, - maxPlanningSnapshotCount, - scanStartupMode, - maxAllowedPlanningFailures, - branch, - tag, - startTag, - endTag - ); + Preconditions.checkArgument( + !(isStreaming && batchMode), + String.format("only support %s = false when execution.runtime-mode is batch", STREAMING.key())); + return new ArcticScanContext(this); } } } diff --git a/flink/v1.14/flink/src/main/java/com/netease/arctic/flink/read/source/DataIterator.java b/flink/v1.14/flink/src/main/java/com/netease/arctic/flink/read/source/DataIterator.java index 4355428e52..e7ec41977a 100644 --- a/flink/v1.14/flink/src/main/java/com/netease/arctic/flink/read/source/DataIterator.java +++ b/flink/v1.14/flink/src/main/java/com/netease/arctic/flink/read/source/DataIterator.java @@ -35,7 +35,7 @@ /** * Flink data iterator that reads {@link ArcticFileScanTask} into a {@link CloseableIterator} * - * @param is the output data type returned by this iterator. + * @param T is the output data type returned by this iterator. */ @Internal public class DataIterator implements CloseableIterator { @@ -51,6 +51,10 @@ public class DataIterator implements CloseableIterator { private final Function arcticFileOffsetGetter; private final Function arcticMetaColumnRemover; + public DataIterator() { + this(null, Collections.emptyList(), t -> Long.MIN_VALUE, t -> t); + } + public DataIterator( FileScanTaskReader fileScanTaskReader, Collection tasks, @@ -69,14 +73,15 @@ public DataIterator( this.fileOffset = -1; // record offset points to the record that next() should return when called this.recordOffset = 0L; - // actual record offset in data file. it's incremental within insert and delete files in the same tree node group. + // actual record offset in data file. + // it's incremental within inserting and deleting files in the same tree node group. this.currentArcticFileOffset = 0L; } /** - * (startingFileOffset, startingRecordOffset) points to the next row that reader should resume from. - * E.g., if the seek position is (file=0, record=1), seek moves the iterator position to the 2nd row - * in file 0. When next() is called after seek, 2nd row from file 0 should be returned. + * (startingFileOffset, startingRecordOffset) points to the next row that the reader should resume from. + * E.g., if the seek position is (file=0, record=1), seek moves the iterator position to the second row + * in file 0. When next() is called after seek; the second row from file 0 should be returned. */ public void seek(int startingFileOffset, long startingRecordOffset) { // It means file is empty. diff --git a/flink/v1.14/flink/src/main/java/com/netease/arctic/flink/read/source/MergeOnReadDataIterator.java b/flink/v1.14/flink/src/main/java/com/netease/arctic/flink/read/source/MergeOnReadDataIterator.java new file mode 100644 index 0000000000..3807147eb0 --- /dev/null +++ b/flink/v1.14/flink/src/main/java/com/netease/arctic/flink/read/source/MergeOnReadDataIterator.java @@ -0,0 +1,133 @@ +/* + * 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 com.netease.arctic.flink.read.source; + +import com.netease.arctic.io.ArcticFileIO; +import com.netease.arctic.scan.KeyedTableScanTask; +import org.apache.flink.table.data.RowData; +import org.apache.flink.util.Preconditions; +import org.apache.iceberg.io.CloseableIterator; + +import java.io.IOException; + + +/** + * Iterator for reading data in a Merge on Read (MOR) way. + * This iterator handles reading data from an Amoro mix-format table + * while keeping track of file and record offsets for efficient data retrieval. + */ +public class MergeOnReadDataIterator extends DataIterator { + private int fileOffset; + private long recordOffset; + private final CloseableIterator iterator; + + public MergeOnReadDataIterator( + FlinkArcticMORDataReader flinkArcticMORDataReader, + KeyedTableScanTask keyedTableScanTask, + ArcticFileIO io) { + super(); + this.iterator = + IteratorWithIO.of( + io, + io.doAs(() -> flinkArcticMORDataReader.readData(keyedTableScanTask))); + } + + @Override + public void seek(int startingFileOffset, long startingRecordOffset) { + // startingFileOffset is not used, because we only have one file per task + Preconditions.checkNotNull(iterator, "iterator is null in the MergeOnReadDataIterator."); + // skip records within the file + for (long i = 0; i < startingRecordOffset; ++i) { + if (hasNext()) { + next(); + } else { + throw new IllegalStateException(String.format( + "Invalid starting record offset %d for file %d from KeyedTableScanTask.", + startingRecordOffset, startingFileOffset)); + } + } + this.fileOffset = startingFileOffset; + this.recordOffset = startingRecordOffset; + } + + @Override + public boolean hasNext() { + return iterator.hasNext(); + } + + @Override + public RowData next() { + return iterator.next(); + } + + public boolean currentFileHasNext() { + return iterator.hasNext(); + } + + @Override + public int fileOffset() { + return fileOffset; + } + + @Override + public long recordOffset() { + return recordOffset; + } + + @Override + public void close() throws IOException { + // close the current iterator + if (iterator != null) { + iterator.close(); + } + } + + static class IteratorWithIO implements CloseableIterator { + private final ArcticFileIO io; + private final CloseableIterator iterator; + + private IteratorWithIO(ArcticFileIO io, CloseableIterator iterator) { + this.io = io; + this.iterator = iterator; + } + + static IteratorWithIO of(ArcticFileIO io, CloseableIterator iterator) { + Preconditions.checkNotNull(io); + return new IteratorWithIO(io, iterator); + } + + @Override + public void close() throws IOException { + io.doAs(() -> { + iterator.close(); + return null; + }); + } + + @Override + public boolean hasNext() { + return io.doAs(iterator::hasNext); + } + + @Override + public RowData next() { + return io.doAs(iterator::next); + } + } +} diff --git a/flink/v1.14/flink/src/main/java/com/netease/arctic/flink/table/ArcticFileSource.java b/flink/v1.14/flink/src/main/java/com/netease/arctic/flink/table/ArcticFileSource.java index 6317d1dc93..47f0a6f082 100644 --- a/flink/v1.14/flink/src/main/java/com/netease/arctic/flink/table/ArcticFileSource.java +++ b/flink/v1.14/flink/src/main/java/com/netease/arctic/flink/table/ArcticFileSource.java @@ -51,6 +51,8 @@ import java.util.List; import static com.netease.arctic.flink.table.descriptors.ArcticValidator.DIM_TABLE_ENABLE; +import static org.apache.flink.api.common.RuntimeExecutionMode.BATCH; +import static org.apache.flink.configuration.ExecutionOptions.RUNTIME_MODE; /** * Flink table api that generates arctic base/change file source operators. @@ -63,13 +65,14 @@ public class ArcticFileSource implements ScanTableSource, SupportsFilterPushDown private int[] projectedFields; private long limit; private List filters; - private ArcticTable table; + private final ArcticTable table; @Nullable protected WatermarkStrategy watermarkStrategy; private final ArcticTableLoader loader; private final TableSchema tableSchema; private final ReadableConfig readableConfig; + private final boolean batchMode; private ArcticFileSource(ArcticFileSource toCopy) { this.loader = toCopy.loader; @@ -79,6 +82,8 @@ private ArcticFileSource(ArcticFileSource toCopy) { this.filters = toCopy.filters; this.readableConfig = toCopy.readableConfig; this.table = toCopy.table; + this.watermarkStrategy = toCopy.watermarkStrategy; + this.batchMode = toCopy.batchMode; } public ArcticFileSource(ArcticTableLoader loader, @@ -87,7 +92,8 @@ public ArcticFileSource(ArcticTableLoader loader, ArcticTable table, long limit, List filters, - ReadableConfig readableConfig) { + ReadableConfig readableConfig, + boolean batchMode) { this.loader = loader; this.tableSchema = tableSchema; this.projectedFields = projectedFields; @@ -95,11 +101,16 @@ public ArcticFileSource(ArcticTableLoader loader, this.table = table; this.filters = filters; this.readableConfig = readableConfig; + this.batchMode = batchMode; } - public ArcticFileSource(ArcticTableLoader loader, TableSchema tableSchema, ArcticTable table, - ReadableConfig readableConfig) { - this(loader, tableSchema, null, table, -1, ImmutableList.of(), readableConfig); + public ArcticFileSource( + ArcticTableLoader loader, + TableSchema tableSchema, + ArcticTable table, + ReadableConfig readableConfig, + boolean batchMode) { + this(loader, tableSchema, null, table, -1, ImmutableList.of(), readableConfig, batchMode); } @Override @@ -121,6 +132,7 @@ private DataStream createDataStream(StreamExecutionEnvironment execEnv) .limit(limit) .filters(filters) .flinkConf(readableConfig) + .batchMode(execEnv.getConfiguration().get(RUNTIME_MODE).equals(BATCH)) .watermarkStrategy(watermarkStrategy) .build(); } @@ -168,7 +180,7 @@ public boolean supportsNestedProjection() { @Override public ChangelogMode getChangelogMode() { - if (table.isUnkeyedTable()) { + if (table.isUnkeyedTable() || batchMode) { return ChangelogMode.insertOnly(); } return ChangelogMode.newBuilder() diff --git a/flink/v1.14/flink/src/main/java/com/netease/arctic/flink/table/DynamicTableFactory.java b/flink/v1.14/flink/src/main/java/com/netease/arctic/flink/table/DynamicTableFactory.java index 71d4e0ee29..e999c7918f 100644 --- a/flink/v1.14/flink/src/main/java/com/netease/arctic/flink/table/DynamicTableFactory.java +++ b/flink/v1.14/flink/src/main/java/com/netease/arctic/flink/table/DynamicTableFactory.java @@ -83,6 +83,8 @@ import static com.netease.arctic.table.TableProperties.LOG_STORE_STORAGE_TYPE_DEFAULT; import static com.netease.arctic.table.TableProperties.LOG_STORE_STORAGE_TYPE_KAFKA; import static com.netease.arctic.table.TableProperties.LOG_STORE_TYPE; +import static org.apache.flink.api.common.RuntimeExecutionMode.BATCH; +import static org.apache.flink.configuration.ExecutionOptions.RUNTIME_MODE; import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.KEY_FIELDS_PREFIX; import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.KEY_FORMAT; import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.PROPS_BOOTSTRAP_SERVERS; @@ -101,13 +103,10 @@ public class DynamicTableFactory implements DynamicTableSourceFactory, DynamicTableSinkFactory { private static final Logger LOG = LoggerFactory.getLogger(DynamicTableFactory.class); public static final String IDENTIFIER = "arctic"; - private ArcticCatalog arcticCatalog; private InternalCatalogBuilder internalCatalogBuilder; private String internalCatalogName; - public DynamicTableFactory( - ArcticCatalog arcticCatalog) { - this.arcticCatalog = arcticCatalog; + public DynamicTableFactory(ArcticCatalog arcticCatalog) { this.internalCatalogBuilder = arcticCatalog.catalogBuilder(); this.internalCatalogName = arcticCatalog.amsCatalogName(); } @@ -162,8 +161,9 @@ public DynamicTableSource createDynamicTableSource(Context context) { dimTable); switch (readMode) { case ArcticValidator.ARCTIC_READ_FILE: - LOG.info("build file reader"); - arcticDynamicSource = new ArcticFileSource(tableLoader, tableSchema, arcticTable, confWithAll); + boolean batchMode = context.getConfiguration().get(RUNTIME_MODE).equals(BATCH); + LOG.info("Building a file reader in {} runtime mode", batchMode ? "batch" : "streaming"); + arcticDynamicSource = new ArcticFileSource(tableLoader, tableSchema, arcticTable, confWithAll, batchMode); break; case ArcticValidator.ARCTIC_READ_LOG: default: @@ -225,8 +225,7 @@ public String factoryIdentifier() { @Override public Set> requiredOptions() { - final Set> options = new HashSet<>(); - return options; + return new HashSet<>(); } @Override @@ -356,7 +355,7 @@ private static Optional>> getKeyDe /** * Return true only if {@link ArcticValidator#ARCTIC_LOG_KAFKA_COMPATIBLE_ENABLE} is true and - * {@link LOG_STORE_TYPE} is kafka. + * {@link TableProperties#LOG_STORE_TYPE} is kafka. */ private static boolean adaptLegacySource(ArcticTable arcticTable) { boolean legacySourceEnabled = CompatibleFlinkPropertyUtil.propertyAsBoolean(arcticTable.properties(), diff --git a/flink/v1.14/flink/src/main/java/com/netease/arctic/flink/table/FlinkSource.java b/flink/v1.14/flink/src/main/java/com/netease/arctic/flink/table/FlinkSource.java index 13f80c6647..c8d862eeda 100644 --- a/flink/v1.14/flink/src/main/java/com/netease/arctic/flink/table/FlinkSource.java +++ b/flink/v1.14/flink/src/main/java/com/netease/arctic/flink/table/FlinkSource.java @@ -75,9 +75,10 @@ public static final class Builder { private TableSchema projectedSchema; private List filters; private ReadableConfig flinkConf = new Configuration(); - private Map properties = new HashMap<>(); + private final Map properties = new HashMap<>(); private long limit = -1L; private WatermarkStrategy watermarkStrategy = WatermarkStrategy.noWatermarks(); + private boolean batchMode = false; private final ArcticScanContext.Builder contextBuilder = ArcticScanContext.arcticBuilder(); private Builder() { @@ -133,6 +134,11 @@ public Builder watermarkStrategy(WatermarkStrategy watermarkStrategy) { return this; } + public Builder batchMode(boolean batchMode) { + this.batchMode = batchMode; + return this; + } + public DataStream build() { Preconditions.checkNotNull(env, "StreamExecutionEnvironment should not be null"); loadTableIfNeeded(); @@ -157,8 +163,11 @@ public DataStream build() { rowType = toRowType(filterWatermark(projectedSchema)); } } - contextBuilder.fromProperties(properties); - ArcticScanContext scanContext = contextBuilder.build(); + ArcticScanContext scanContext = + contextBuilder + .fromProperties(properties) + .batchMode(batchMode) + .build(); RowDataReaderFunction rowDataReaderFunction = new RowDataReaderFunction( flinkConf, diff --git a/flink/v1.14/flink/src/test/java/com/netease/arctic/flink/read/TestArcticSource.java b/flink/v1.14/flink/src/test/java/com/netease/arctic/flink/read/TestArcticSource.java index 934b6f6c72..b70d702e65 100644 --- a/flink/v1.14/flink/src/test/java/com/netease/arctic/flink/read/TestArcticSource.java +++ b/flink/v1.14/flink/src/test/java/com/netease/arctic/flink/read/TestArcticSource.java @@ -81,6 +81,7 @@ import org.junit.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; + import java.io.IOException; import java.io.Serializable; import java.time.Duration; @@ -94,6 +95,7 @@ import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; import java.util.concurrent.atomic.AtomicInteger; + import static com.netease.arctic.ams.api.MockArcticMetastoreServer.TEST_CATALOG_NAME; import static com.netease.arctic.ams.api.MockArcticMetastoreServer.TEST_DB_NAME; import static com.netease.arctic.flink.table.descriptors.ArcticValidator.SCAN_STARTUP_MODE_EARLIEST; @@ -168,8 +170,8 @@ public void testArcticSourceStatic() throws Exception { GenericRowData rowData = convert(row); actualResult.add(rowData); }); - Assert.assertEquals(8, actualResult.size()); - assertArrayEquals(excepts(), actualResult); + RowData[] expected = expectedAfterMOR(); + assertArrayEquals(expected, actualResult); } @Test @@ -183,7 +185,7 @@ public void testArcticSourceStaticTaskManagerFailover() throws Exception { } public void testArcticSource(FailoverType failoverType) throws Exception { - List expected = new ArrayList<>(exceptsCollection()); + List expected = new ArrayList<>(expectedCollection()); List updated = updateRecords(); writeUpdate(updated); List records = generateRecords(2, 1); @@ -203,8 +205,9 @@ public void testArcticSource(FailoverType failoverType) throws Exception { "ArcticParallelSource") .setParallelism(PARALLELISM); + List expectedAfterMoR = new ArrayList<>(mor(expected)); DataStream streamFailingInTheMiddleOfReading = - RecordCounterToFail.wrapWithFailureAfter(input, expected.size() / 2); + RecordCounterToFail.wrapWithFailureAfter(input, expectedAfterMoR.size() / 2); FlinkSink .forRowData(streamFailingInTheMiddleOfReading) @@ -223,7 +226,7 @@ public void testArcticSource(FailoverType failoverType) throws Exception { RecordCounterToFail::continueProcessing, miniClusterResource.getMiniCluster()); - assertRecords(testFailoverTable, expected, Duration.ofMillis(10), 12000); + assertRecords(testFailoverTable, expectedAfterMoR, Duration.ofMillis(10), 12000); } @Test(timeout = 60000) @@ -704,12 +707,12 @@ private List collectRecordsFromUnboundedStream( // TODO a more proper timeout strategy? long timeFlies = System.currentTimeMillis() - start; if (timeFlies / 1000 >= intervalOneSecond) { - LOG.info("time flies: {} ms.", timeFlies); + LOG.info("Time flies: {} ms.", timeFlies); intervalOneSecond++; } if (System.currentTimeMillis() - start > timeout) { LOG.error( - "this task [{}] try to collect records from unbounded stream but timeout {}. As of now, collect result:{}.", + "This task [{}] try to collect records from unbounded stream but timeout {}. As of now, collect result:{}.", client.client.getJobID().toString(), timeout, result.toArray()); diff --git a/flink/v1.14/flink/src/test/java/com/netease/arctic/flink/read/hybrid/assigner/TestShuffleSplitAssigner.java b/flink/v1.14/flink/src/test/java/com/netease/arctic/flink/read/hybrid/assigner/TestShuffleSplitAssigner.java index 1703b6fc90..e6afcd661b 100644 --- a/flink/v1.14/flink/src/test/java/com/netease/arctic/flink/read/hybrid/assigner/TestShuffleSplitAssigner.java +++ b/flink/v1.14/flink/src/test/java/com/netease/arctic/flink/read/hybrid/assigner/TestShuffleSplitAssigner.java @@ -83,7 +83,7 @@ public void testMultiParallelism() { if (splitOpt.isAvailable()) { actual.add(splitOpt.split()); } else { - LOG.info("subtask id {}, splits {}.\n {}", subtaskId, actual.size(), actual); + LOG.info("Subtask id {}, splits {}.\n {}", subtaskId, actual.size(), actual); --subtaskId; } } @@ -183,7 +183,7 @@ public void testNodeUpMoved() throws IOException { } while (subtaskId < totalParallelism); - List excepts = exceptsCollection(); + List excepts = expectedCollection(); excepts.addAll(generateRecords()); RowData[] array = excepts.stream().sorted(Comparator.comparing(RowData::toString)) .collect(Collectors.toList()) diff --git a/flink/v1.14/flink/src/test/java/com/netease/arctic/flink/read/hybrid/assigner/TestStaticSplitAssigner.java b/flink/v1.14/flink/src/test/java/com/netease/arctic/flink/read/hybrid/assigner/TestStaticSplitAssigner.java new file mode 100644 index 0000000000..b678032713 --- /dev/null +++ b/flink/v1.14/flink/src/test/java/com/netease/arctic/flink/read/hybrid/assigner/TestStaticSplitAssigner.java @@ -0,0 +1,85 @@ +/* + * 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 com.netease.arctic.flink.read.hybrid.assigner; + +import com.netease.arctic.flink.read.FlinkSplitPlanner; +import com.netease.arctic.flink.read.hybrid.reader.TestRowDataReaderFunction; +import com.netease.arctic.flink.read.hybrid.split.ArcticSplit; +import org.junit.Assert; +import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.concurrent.atomic.AtomicInteger; + +public class TestStaticSplitAssigner extends TestRowDataReaderFunction { + private static final Logger LOG = LoggerFactory.getLogger(TestStaticSplitAssigner.class); + + @Test + public void testSingleParallelism() throws IOException { + try (StaticSplitAssigner staticSplitAssigner = instanceStaticSplitAssigner()) { + List splitList = FlinkSplitPlanner.mergeOnReadPlan(testKeyedTable, Collections.emptyList(), + new AtomicInteger()); + staticSplitAssigner.onDiscoveredSplits(splitList); + List actual = new ArrayList<>(); + + while (true) { + Split splitOpt = staticSplitAssigner.getNext(0); + if (splitOpt.isAvailable()) { + actual.add(splitOpt.split()); + } else { + break; + } + } + + Assert.assertEquals(splitList.size(), actual.size()); + } + } + + @Test + public void testMultiParallelism() throws IOException { + try (StaticSplitAssigner staticSplitAssigner = instanceStaticSplitAssigner()) { + List splitList = FlinkSplitPlanner.mergeOnReadPlan(testKeyedTable, Collections.emptyList(), + new AtomicInteger()); + staticSplitAssigner.onDiscoveredSplits(splitList); + List actual = new ArrayList<>(); + + int subtaskId = 2; + while (subtaskId >= 0) { + Split splitOpt = staticSplitAssigner.getNext(subtaskId); + if (splitOpt.isAvailable()) { + actual.add(splitOpt.split()); + } else { + LOG.info("Subtask id {}, splits {}.\n {}", subtaskId, actual.size(), actual); + --subtaskId; + } + } + + Assert.assertEquals(splitList.size(), actual.size()); + } + } + + protected StaticSplitAssigner instanceStaticSplitAssigner() { + return new StaticSplitAssigner(null); + } +} diff --git a/flink/v1.14/flink/src/test/java/com/netease/arctic/flink/read/hybrid/enumerator/TestArcticSourceEnumStateSerializer.java b/flink/v1.14/flink/src/test/java/com/netease/arctic/flink/read/hybrid/enumerator/TestArcticSourceEnumStateSerializer.java index 366cd96bdc..81945e6942 100644 --- a/flink/v1.14/flink/src/test/java/com/netease/arctic/flink/read/hybrid/enumerator/TestArcticSourceEnumStateSerializer.java +++ b/flink/v1.14/flink/src/test/java/com/netease/arctic/flink/read/hybrid/enumerator/TestArcticSourceEnumStateSerializer.java @@ -20,8 +20,8 @@ import com.netease.arctic.flink.read.FlinkSplitPlanner; import com.netease.arctic.flink.read.hybrid.assigner.ShuffleSplitAssigner; -import com.netease.arctic.flink.read.hybrid.assigner.TestShuffleSplitAssigner; import com.netease.arctic.flink.read.hybrid.assigner.Split; +import com.netease.arctic.flink.read.hybrid.assigner.TestShuffleSplitAssigner; import com.netease.arctic.flink.read.hybrid.split.ArcticSplit; import com.netease.arctic.flink.read.hybrid.split.TemporalJoinSplits; import org.apache.flink.api.connector.source.SplitEnumeratorContext; @@ -67,25 +67,28 @@ public void testArcticEnumState() throws IOException { Objects.requireNonNull(actual.shuffleSplitRelation()).length); SplitEnumeratorContext splitEnumeratorContext = new InternalSplitEnumeratorContext(3); - ShuffleSplitAssigner actualAssigner = new ShuffleSplitAssigner(splitEnumeratorContext, - actual.pendingSplits(), actual.shuffleSplitRelation()); - - List actualSplits = new ArrayList<>(); - - int subtaskId = 2; - while (subtaskId >= 0) { - Split splitOpt = actualAssigner.getNext(subtaskId); - if (splitOpt.isAvailable()) { - actualSplits.add(splitOpt.split()); - } else { - LOG.info("subtask id {}, splits {}.\n {}", subtaskId, actualSplits.size(), actualSplits); - --subtaskId; + try (ShuffleSplitAssigner actualAssigner = + new ShuffleSplitAssigner( + splitEnumeratorContext, + getArcticTable().name(), + actual)) { + List actualSplits = new ArrayList<>(); + + int subtaskId = 2; + while (subtaskId >= 0) { + Split splitOpt = actualAssigner.getNext(subtaskId); + if (splitOpt.isAvailable()) { + actualSplits.add(splitOpt.split()); + } else { + LOG.info("subtask id {}, splits {}.\n {}", subtaskId, actualSplits.size(), actualSplits); + --subtaskId; + } } - } - Assert.assertEquals(splitList.size(), actualSplits.size()); + Assert.assertEquals(splitList.size(), actualSplits.size()); - TemporalJoinSplits temporalJoinSplits = actual.temporalJoinSplits(); - Assert.assertEquals(expect.temporalJoinSplits(), temporalJoinSplits); + TemporalJoinSplits temporalJoinSplits = actual.temporalJoinSplits(); + Assert.assertEquals(expect.temporalJoinSplits(), temporalJoinSplits); + } } } \ No newline at end of file diff --git a/flink/v1.14/flink/src/test/java/com/netease/arctic/flink/read/hybrid/reader/TestRowDataReaderFunction.java b/flink/v1.14/flink/src/test/java/com/netease/arctic/flink/read/hybrid/reader/TestRowDataReaderFunction.java index 03764fd437..b92cb68cff 100644 --- a/flink/v1.14/flink/src/test/java/com/netease/arctic/flink/read/hybrid/reader/TestRowDataReaderFunction.java +++ b/flink/v1.14/flink/src/test/java/com/netease/arctic/flink/read/hybrid/reader/TestRowDataReaderFunction.java @@ -34,6 +34,7 @@ import com.netease.arctic.scan.TableEntriesScan; import com.netease.arctic.table.KeyedTable; import org.apache.flink.configuration.Configuration; +import org.apache.flink.shaded.guava30.com.google.common.collect.Maps; import org.apache.flink.table.data.GenericRowData; import org.apache.flink.table.data.RowData; import org.apache.flink.table.data.StringData; @@ -56,6 +57,7 @@ import java.util.Comparator; import java.util.HashSet; import java.util.List; +import java.util.Map; import java.util.Set; import java.util.concurrent.atomic.AtomicInteger; import java.util.stream.Collectors; @@ -66,7 +68,7 @@ public class TestRowDataReaderFunction extends TestContinuousSplitPlannerImpl { public TestRowDataReaderFunction() { super(new BasicCatalogTestHelper(TableFormat.MIXED_ICEBERG), - new BasicTableTestHelper(true, true)); + new BasicTableTestHelper(true, true)); } @Test @@ -172,7 +174,7 @@ public void testReadNodesUpMoved() throws IOException { } }); - List excepts = exceptsCollection(); + List excepts = expectedCollection(); excepts.addAll(generateRecords()); RowData[] array = excepts.stream().sorted(Comparator.comparing(RowData::toString)) .collect(Collectors.toList()) @@ -254,14 +256,37 @@ protected RowData[] excepts2() { } protected RowData[] excepts() { - List excepts = exceptsCollection(); + List excepts = expectedCollection(); return excepts.stream().sorted(Comparator.comparing(RowData::toString)) .collect(Collectors.toList()) .toArray(new RowData[excepts.size()]); } - protected List exceptsCollection() { + protected RowData[] expectedAfterMOR() { + List expected = expectedCollection(); + return mor(expected).stream().sorted(Comparator.comparing(RowData::toString)).toArray(RowData[]::new); + } + + protected Collection mor(final Collection changelog) { + Map map = Maps.newHashMap(); + + changelog.forEach(rowData -> { + int key = rowData.getInt(0); + RowKind kind = rowData.getRowKind(); + + if ((kind == RowKind.INSERT || kind == RowKind.UPDATE_AFTER) && !map.containsKey(key)) { + rowData.setRowKind(RowKind.INSERT); + map.put(key, rowData); + } else if ((kind == RowKind.DELETE || kind == RowKind.UPDATE_BEFORE)) { + map.remove(key); + } + }); + + return map.values(); + } + + protected List expectedCollection() { List excepts = new ArrayList<>(); excepts.add(GenericRowData.ofKind(RowKind.INSERT, 1, StringData.fromString("john"), ldt.toEpochSecond(ZoneOffset.UTC), TimestampData.fromLocalDateTime(ldt))); excepts.add(GenericRowData.ofKind(RowKind.INSERT, 2, StringData.fromString("lily"), ldt.toEpochSecond(ZoneOffset.UTC), TimestampData.fromLocalDateTime(ldt))); diff --git a/flink/v1.14/flink/src/test/java/com/netease/arctic/flink/read/hybrid/split/TestArcticSplitSerializer.java b/flink/v1.14/flink/src/test/java/com/netease/arctic/flink/read/hybrid/split/TestArcticSplitSerializer.java index 3b6c625786..cf26b6628f 100644 --- a/flink/v1.14/flink/src/test/java/com/netease/arctic/flink/read/hybrid/split/TestArcticSplitSerializer.java +++ b/flink/v1.14/flink/src/test/java/com/netease/arctic/flink/read/hybrid/split/TestArcticSplitSerializer.java @@ -33,9 +33,18 @@ public class TestArcticSplitSerializer extends TestRowDataReaderFunction { @Test public void testSerAndDes() { List arcticSplits = FlinkSplitPlanner.planFullTable(testKeyedTable, new AtomicInteger(0)); + assertSerializedSplitEquals(arcticSplits); + } + + @Test + public void testSerAndDesMoRSplit() { + List arcticSplits = FlinkSplitPlanner.mergeOnReadPlan(testKeyedTable, null, new AtomicInteger(0)); + assertSerializedSplitEquals(arcticSplits); + } + private void assertSerializedSplitEquals(List expected) { ArcticSplitSerializer serializer = new ArcticSplitSerializer(); - List contents = arcticSplits.stream().map(split -> { + List contents = expected.stream().map(split -> { try { return serializer.serialize(split); } catch (IOException e) { @@ -44,7 +53,7 @@ public void testSerAndDes() { } }).collect(Collectors.toList()); - Assert.assertArrayEquals(arcticSplits.toArray(new ArcticSplit[0]), contents.stream().map(data -> { + Assert.assertArrayEquals(expected.toArray(new ArcticSplit[0]), contents.stream().map(data -> { if (data.length == 0) { throw new FlinkRuntimeException("failed cause data length is 0."); } diff --git a/flink/v1.14/flink/src/test/java/com/netease/arctic/flink/table/TestKeyed.java b/flink/v1.14/flink/src/test/java/com/netease/arctic/flink/table/TestKeyed.java index e8bb5f4d3c..a2f63d481a 100644 --- a/flink/v1.14/flink/src/test/java/com/netease/arctic/flink/table/TestKeyed.java +++ b/flink/v1.14/flink/src/test/java/com/netease/arctic/flink/table/TestKeyed.java @@ -113,30 +113,30 @@ public TestKeyed( public static Collection parameters() { return Arrays.asList( new Object[][]{ - { - new HiveCatalogTestHelper(TableFormat.MIXED_HIVE, TEST_HMS.getHiveConf()), - new HiveTableTestHelper(true, true), - true, - true - }, - { - new HiveCatalogTestHelper(TableFormat.MIXED_HIVE, TEST_HMS.getHiveConf()), - new HiveTableTestHelper(true, true), - true, - false - }, - { - new BasicCatalogTestHelper(TableFormat.MIXED_ICEBERG), - new BasicTableTestHelper(true, true), - false, - true - }, - { - new BasicCatalogTestHelper(TableFormat.MIXED_ICEBERG), - new BasicTableTestHelper(true, true), - false, - false - } + { + new HiveCatalogTestHelper(TableFormat.MIXED_HIVE, TEST_HMS.getHiveConf()), + new HiveTableTestHelper(true, true), + true, + true + }, + { + new HiveCatalogTestHelper(TableFormat.MIXED_HIVE, TEST_HMS.getHiveConf()), + new HiveTableTestHelper(true, true), + true, + false + }, + { + new BasicCatalogTestHelper(TableFormat.MIXED_ICEBERG), + new BasicTableTestHelper(true, true), + false, + true + }, + { + new BasicCatalogTestHelper(TableFormat.MIXED_ICEBERG), + new BasicTableTestHelper(true, true), + false, + false + } }); } @@ -239,9 +239,7 @@ public void testSinkSourceFile() throws IOException { List expected = new LinkedList<>(); expected.add(new Object[]{RowKind.INSERT, 1000004, LocalDateTime.parse("2022-06-17T10:10:11.0"), LocalDateTime.parse("2022-06-17T10:10:11.0").atZone(ZoneId.systemDefault()).toInstant()}); - expected.add(new Object[]{RowKind.UPDATE_BEFORE, 1000021, LocalDateTime.parse("2022-06-17T10:11:11.0"), - LocalDateTime.parse("2022-06-17T10:11:11.0").atZone(ZoneId.systemDefault()).toInstant()}); - expected.add(new Object[]{RowKind.UPDATE_AFTER, 1000021, LocalDateTime.parse("2022-06-17T10:11:11.0"), + expected.add(new Object[]{RowKind.INSERT, 1000021, LocalDateTime.parse("2022-06-17T10:11:11.0"), LocalDateTime.parse("2022-06-17T10:11:11.0").atZone(ZoneId.systemDefault()).toInstant()}); expected.add(new Object[]{RowKind.INSERT, 1000015, LocalDateTime.parse("2022-06-17T10:10:11.0"), LocalDateTime.parse("2022-06-17T10:10:11.0").atZone(ZoneId.systemDefault()).toInstant()}); @@ -457,28 +455,28 @@ public void testSinkSourceFileWithoutSelectPK() throws Exception { List rows = DataUtil.toRows(data); Table input = getTableEnv().fromValues(DataTypes.ROW( - DataTypes.FIELD("id", DataTypes.INT()), - DataTypes.FIELD("name", DataTypes.STRING()), - DataTypes.FIELD("op_time", DataTypes.TIMESTAMP()) - ), - rows + DataTypes.FIELD("id", DataTypes.INT()), + DataTypes.FIELD("name", DataTypes.STRING()), + DataTypes.FIELD("op_time", DataTypes.TIMESTAMP()) + ), + rows ); getTableEnv().createTemporaryView("input", input); sql("CREATE CATALOG arcticCatalog WITH %s", toWithClause(props)); sql("CREATE TABLE IF NOT EXISTS arcticCatalog." + db + "." + TABLE + "(" + - " id INT, name STRING, op_time TIMESTAMP, PRIMARY KEY (id) NOT ENFORCED " + - ") WITH ('connector' = 'arctic')"); + " id INT, name STRING, op_time TIMESTAMP, PRIMARY KEY (id) NOT ENFORCED " + + ") WITH ('connector' = 'arctic')"); sql("insert into arcticCatalog." + db + "." + TABLE + - "/*+ OPTIONS(" + - "'arctic.emit.mode'='file'" + - ")*/" + " select * from input"); + "/*+ OPTIONS(" + + "'arctic.emit.mode'='file'" + + ")*/" + " select * from input"); TableResult result = exec("select name, op_time from arcticCatalog." + db + "." + TABLE + " /*+ OPTIONS(" + - "'streaming'='false'" + - ") */"); + "'streaming'='false'" + + ") */"); LinkedList actual = new LinkedList<>(); try (CloseableIterator iterator = result.collect()) { while (iterator.hasNext()) { @@ -496,7 +494,7 @@ public void testSinkSourceFileWithoutSelectPK() throws Exception { expected.add(new Object[]{"e", LocalDateTime.parse("2022-06-18T10:10:11.0")}); Assert.assertEquals(DataUtil.toRowSet(expected), - new HashSet<>(actual)); + new HashSet<>(actual)); } @Test @@ -540,19 +538,8 @@ public void testFileUpsert() { List expected = new LinkedList<>(); expected.add(new Object[]{RowKind.INSERT, 1000004, "a", LocalDateTime.parse("2022-06-17T10:10:11.0")}); - expected.add(new Object[]{RowKind.DELETE, 1000004, "a", LocalDateTime.parse("2022-06-17T10:10:11.0")}); - expected.add(new Object[]{RowKind.UPDATE_BEFORE, 1000021, "d", LocalDateTime.parse("2022-06-17T10:11:11.0")}); - expected.add(new Object[]{RowKind.UPDATE_AFTER, 1000021, "e", LocalDateTime.parse("2022-06-17T10:11:11.0")}); - expected.add(new Object[]{RowKind.DELETE, 1000021, "e", LocalDateTime.parse("2022-06-17T10:11:11.0")}); + // key = 1000021 locate in two partitions. expected.add(new Object[]{RowKind.INSERT, 1000021, "e", LocalDateTime.parse("2022-06-17T10:11:11.0")}); - expected.add(new Object[]{RowKind.DELETE, 1000015, "e", LocalDateTime.parse("2022-06-17T10:10:11.0")}); - expected.add(new Object[]{RowKind.INSERT, 1000015, "e", LocalDateTime.parse("2022-06-17T10:10:11.0")}); - expected.add(new Object[]{RowKind.DELETE, 1000021, "e", LocalDateTime.parse("2022-06-17T10:10:11.0")}); - expected.add(new Object[]{RowKind.INSERT, 1000021, "e", LocalDateTime.parse("2022-06-17T10:10:11.0")}); - expected.add(new Object[]{RowKind.UPDATE_BEFORE, 1000021, "e", LocalDateTime.parse("2022-06-17T10:10:11.0")}); - expected.add(new Object[]{RowKind.UPDATE_AFTER, 1000021, "d", LocalDateTime.parse("2022-06-17T10:10:11.0")}); - expected.add(new Object[]{RowKind.DELETE, 1000015, "e", LocalDateTime.parse("2022-06-17T10:10:11.0")}); - expected.add(new Object[]{RowKind.DELETE, 1000021, "f", LocalDateTime.parse("2022-06-17T10:10:11.0")}); expected.add(new Object[]{RowKind.INSERT, 1000021, "f", LocalDateTime.parse("2022-06-17T10:10:11.0")}); Assert.assertEquals(DataUtil.toRowSet(expected), new HashSet<>(sql("select * from arcticCatalog." + db + "." + TABLE + " /*+ OPTIONS(" + @@ -605,22 +592,13 @@ public void testFileCDC() { ")*/" + " select * from input"); List expected = new LinkedList<>(); + // upsert is disEnabled, key=1000021 locate in two diff partitions. expected.add(new Object[]{RowKind.INSERT, 1000004, "a", LocalDateTime.parse("2022-06-17T10:10:11.0")}); - expected.add(new Object[]{RowKind.UPDATE_BEFORE, 1000021, "d", LocalDateTime.parse("2022-06-17T10:11:11.0")}); - expected.add(new Object[]{RowKind.UPDATE_AFTER, 1000021, "e", LocalDateTime.parse("2022-06-17T10:11:11.0")}); expected.add(new Object[]{RowKind.INSERT, 1000021, "e", LocalDateTime.parse("2022-06-17T10:11:11.0")}); - expected.add(new Object[]{RowKind.INSERT, 1000015, "e", LocalDateTime.parse("2022-06-17T10:10:11.0")}); - expected.add(new Object[]{RowKind.INSERT, 1000021, "e", LocalDateTime.parse("2022-06-17T10:10:11.0")}); - expected.add(new Object[]{RowKind.UPDATE_BEFORE, 1000021, "e", LocalDateTime.parse("2022-06-17T10:10:11.0")}); - expected.add(new Object[]{RowKind.UPDATE_AFTER, 1000021, "d", LocalDateTime.parse("2022-06-17T10:10:11.0")}); - expected.add(new Object[]{RowKind.DELETE, 1000015, "e", LocalDateTime.parse("2022-06-17T10:10:11.0")}); + expected.add(new Object[]{RowKind.INSERT, 1000021, "d", LocalDateTime.parse("2022-06-17T10:10:11.0")}); expected.add(new Object[]{RowKind.INSERT, 1000021, "f", LocalDateTime.parse("2022-06-17T10:10:11.0")}); - expected.add(new Object[]{RowKind.INSERT, 1000031, "g", LocalDateTime.parse("2022-06-17T10:10:11.0")}); - expected.add(new Object[]{RowKind.INSERT, 1000032, "h", LocalDateTime.parse("2022-06-17T10:10:11.0")}); - expected.add(new Object[]{RowKind.DELETE, 1000031, "g", LocalDateTime.parse("2022-06-17T10:10:11.0")}); - expected.add(new Object[]{RowKind.UPDATE_BEFORE, 1000032, "h", LocalDateTime.parse("2022-06-17T10:10:11.0")}); expected.add(new Object[]{RowKind.INSERT, 1000031, "f", LocalDateTime.parse("2022-06-17T10:10:11.0")}); - expected.add(new Object[]{RowKind.UPDATE_AFTER, 1000032, "e", LocalDateTime.parse("2022-06-17T10:10:11.0")}); + expected.add(new Object[]{RowKind.INSERT, 1000032, "e", LocalDateTime.parse("2022-06-17T10:10:11.0")}); Assert.assertEquals(DataUtil.toRowSet(expected), new HashSet<>(sql("select * from arcticCatalog." + db + "." + TABLE + " /*+ OPTIONS(" + "'streaming'='false'" + @@ -672,13 +650,7 @@ public void testFileUpsertWithSamePrimaryKey() throws Exception { LinkedList expected = new LinkedList<>(); - expected.add(new Object[]{RowKind.DELETE, 1000004, "a", LocalDateTime.parse("2022-06-17T10:10:11.0")}); - expected.add(new Object[]{RowKind.INSERT, 1000004, "a", LocalDateTime.parse("2022-06-17T10:10:11.0")}); - expected.add(new Object[]{RowKind.DELETE, 1000004, "b", LocalDateTime.parse("2022-06-17T10:10:11.0")}); expected.add(new Object[]{RowKind.INSERT, 1000004, "b", LocalDateTime.parse("2022-06-17T10:10:11.0")}); - expected.add(new Object[]{RowKind.DELETE, 1000011, "e", LocalDateTime.parse("2022-06-17T10:10:11.0")}); - expected.add(new Object[]{RowKind.INSERT, 1000011, "e", LocalDateTime.parse("2022-06-17T10:10:11.0")}); - expected.add(new Object[]{RowKind.DELETE, 1000011, "f", LocalDateTime.parse("2022-06-17T10:10:11.0")}); expected.add(new Object[]{RowKind.INSERT, 1000011, "f", LocalDateTime.parse("2022-06-17T10:10:11.0")}); Map> actualMap = DataUtil.groupByPrimaryKey(actual, 0); diff --git a/flink/v1.15/flink/src/main/java/com/netease/arctic/flink/read/ArcticSource.java b/flink/v1.15/flink/src/main/java/com/netease/arctic/flink/read/ArcticSource.java index cce1658d69..7db69a074e 100644 --- a/flink/v1.15/flink/src/main/java/com/netease/arctic/flink/read/ArcticSource.java +++ b/flink/v1.15/flink/src/main/java/com/netease/arctic/flink/read/ArcticSource.java @@ -20,6 +20,7 @@ import com.netease.arctic.flink.read.hybrid.assigner.ShuffleSplitAssigner; import com.netease.arctic.flink.read.hybrid.assigner.SplitAssigner; +import com.netease.arctic.flink.read.hybrid.assigner.StaticSplitAssigner; import com.netease.arctic.flink.read.hybrid.enumerator.ArcticSourceEnumState; import com.netease.arctic.flink.read.hybrid.enumerator.ArcticSourceEnumStateSerializer; import com.netease.arctic.flink.read.hybrid.enumerator.ArcticSourceEnumerator; @@ -39,8 +40,6 @@ import org.apache.flink.api.connector.source.SplitEnumeratorContext; import org.apache.flink.api.java.typeutils.ResultTypeQueryable; import org.apache.flink.core.io.SimpleVersionedSerializer; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; /** * Arctic Source based of Flip27. @@ -51,8 +50,6 @@ */ public class ArcticSource implements Source, ResultTypeQueryable { private static final long serialVersionUID = 1L; - - private static final Logger LOG = LoggerFactory.getLogger(ArcticSource.class); private final ArcticScanContext scanContext; private final ReaderFunction readerFunction; private final TypeInformation typeInformation; @@ -65,7 +62,7 @@ public class ArcticSource implements Source readerFunction, - TypeInformation typeInformation, String tableName, boolean dimTable) { + TypeInformation typeInformation, String tableName, boolean dimTable) { this.loader = loader; this.scanContext = scanContext; this.readerFunction = readerFunction; @@ -80,31 +77,24 @@ public Boundedness getBoundedness() { } @Override - public SourceReader createReader(SourceReaderContext readerContext) throws Exception { + public SourceReader createReader(SourceReaderContext readerContext) { return new ArcticSourceReader<>(readerFunction, readerContext.getConfiguration(), readerContext, dimTable); } @Override public SplitEnumerator createEnumerator( - SplitEnumeratorContext enumContext) throws Exception { + SplitEnumeratorContext enumContext) { return createEnumerator(enumContext, null); } private SplitEnumerator createEnumerator( SplitEnumeratorContext enumContext, ArcticSourceEnumState enumState) { SplitAssigner splitAssigner; - if (enumState == null) { - splitAssigner = new ShuffleSplitAssigner(enumContext); - } else { - LOG.info("Arctic source restored {} splits from state for table {}", - enumState.pendingSplits().size(), tableName); - splitAssigner = new ShuffleSplitAssigner(enumContext, enumState.pendingSplits(), - enumState.shuffleSplitRelation()); - } - if (scanContext.isStreaming()) { + splitAssigner = new ShuffleSplitAssigner(enumContext, tableName, enumState); return new ArcticSourceEnumerator(enumContext, splitAssigner, loader, scanContext, enumState, dimTable); } else { + splitAssigner = new StaticSplitAssigner(enumState); return new StaticArcticSourceEnumerator(enumContext, splitAssigner, loader, scanContext, null); } } @@ -112,7 +102,7 @@ private SplitEnumerator createEnumerator( @Override public SplitEnumerator restoreEnumerator( SplitEnumeratorContext enumContext, - ArcticSourceEnumState checkpoint) throws Exception { + ArcticSourceEnumState checkpoint) { return createEnumerator(enumContext, checkpoint); } diff --git a/flink/v1.15/flink/src/main/java/com/netease/arctic/flink/read/hybrid/assigner/ShuffleSplitAssigner.java b/flink/v1.15/flink/src/main/java/com/netease/arctic/flink/read/hybrid/assigner/ShuffleSplitAssigner.java index ed1a5c08be..93b4f1ca21 100644 --- a/flink/v1.15/flink/src/main/java/com/netease/arctic/flink/read/hybrid/assigner/ShuffleSplitAssigner.java +++ b/flink/v1.15/flink/src/main/java/com/netease/arctic/flink/read/hybrid/assigner/ShuffleSplitAssigner.java @@ -20,20 +20,24 @@ import com.netease.arctic.data.DataTreeNode; import com.netease.arctic.data.PrimaryKeyedFile; +import com.netease.arctic.flink.read.hybrid.enumerator.ArcticSourceEnumState; import com.netease.arctic.flink.read.hybrid.split.ArcticSplit; import com.netease.arctic.flink.read.hybrid.split.ArcticSplitState; import com.netease.arctic.scan.ArcticFileScanTask; +import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.api.connector.source.SplitEnumeratorContext; import org.apache.flink.util.FlinkRuntimeException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import javax.annotation.Nullable; import java.io.IOException; import java.util.ArrayList; import java.util.Collection; import java.util.Collections; import java.util.List; import java.util.Map; +import java.util.Objects; import java.util.Optional; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ConcurrentHashMap; @@ -43,7 +47,7 @@ import java.util.stream.Collectors; /** - * According to Mark,Index TreeNodes and subtaskId assigning a split to special subtask to read. + * According to Mark, Index TreeNodes and subtaskId assigning a split to special subtask to read. */ public class ShuffleSplitAssigner implements SplitAssigner { private static final Logger LOG = LoggerFactory.getLogger(ShuffleSplitAssigner.class); @@ -68,6 +72,7 @@ public class ShuffleSplitAssigner implements SplitAssigner { private CompletableFuture availableFuture; + @VisibleForTesting public ShuffleSplitAssigner( SplitEnumeratorContext enumeratorContext) { this.enumeratorContext = enumeratorContext; @@ -77,15 +82,29 @@ public ShuffleSplitAssigner( } public ShuffleSplitAssigner( - SplitEnumeratorContext enumeratorContext, Collection splitStates, - long[] shuffleSplitRelation) { + SplitEnumeratorContext enumeratorContext, + String tableName, + @Nullable ArcticSourceEnumState enumState) { this.enumeratorContext = enumeratorContext; this.partitionIndexSubtaskMap = new ConcurrentHashMap<>(); this.subtaskSplitMap = new ConcurrentHashMap<>(); - deserializePartitionIndex(shuffleSplitRelation); - splitStates.forEach(state -> onDiscoveredSplits(Collections.singleton(state.toSourceSplit()))); + if (enumState == null) { + this.totalParallelism = enumeratorContext.currentParallelism(); + LOG.info( + "Arctic source enumerator current parallelism is {} for table {}", + totalParallelism, tableName); + } else { + LOG.info("Arctic source restored {} splits from state for table {}", + enumState.pendingSplits().size(), tableName); + deserializePartitionIndex( + Objects.requireNonNull( + enumState.shuffleSplitRelation(), + "The partition index and subtask state couldn't be null.")); + enumState.pendingSplits().forEach(state -> onDiscoveredSplits(Collections.singleton(state.toSourceSplit()))); + } } + @Override public Split getNext() { throw new UnsupportedOperationException("ShuffleSplitAssigner couldn't support this operation."); diff --git a/flink/v1.15/flink/src/main/java/com/netease/arctic/flink/read/hybrid/assigner/StaticSplitAssigner.java b/flink/v1.15/flink/src/main/java/com/netease/arctic/flink/read/hybrid/assigner/StaticSplitAssigner.java new file mode 100644 index 0000000000..4a0f8bc1bb --- /dev/null +++ b/flink/v1.15/flink/src/main/java/com/netease/arctic/flink/read/hybrid/assigner/StaticSplitAssigner.java @@ -0,0 +1,130 @@ +/* + * 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 com.netease.arctic.flink.read.hybrid.assigner; + +import com.netease.arctic.flink.read.hybrid.enumerator.ArcticSourceEnumState; +import com.netease.arctic.flink.read.hybrid.split.ArcticSplit; +import com.netease.arctic.flink.read.hybrid.split.ArcticSplitState; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.Nullable; +import java.io.IOException; +import java.util.Collection; +import java.util.Collections; +import java.util.Optional; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.PriorityBlockingQueue; +import java.util.concurrent.TimeUnit; +import java.util.stream.Collectors; + +/** + * This is a static split assigner which is used for batch mode. + */ +public class StaticSplitAssigner implements SplitAssigner { + private static final Logger LOG = LoggerFactory.getLogger(StaticSplitAssigner.class); + + private static final long POLL_TIMEOUT = 200; + private int totalSplitNum; + + private final PriorityBlockingQueue splitQueue; + + private CompletableFuture availableFuture; + + public StaticSplitAssigner(@Nullable ArcticSourceEnumState enumState) { + this.splitQueue = new PriorityBlockingQueue<>(); + if (enumState != null) { + Collection splitStates = enumState.pendingSplits(); + splitStates.forEach(state -> onDiscoveredSplits(Collections.singleton(state.toSourceSplit()))); + } + } + + @Override + public Split getNext() { + return getNextSplit().map(Split::of).orElseGet(Split::unavailable); + } + + @Override + public Split getNext(int subtaskId) { + return getNext(); + } + + private Optional getNextSplit() { + ArcticSplit arcticSplit = null; + try { + arcticSplit = splitQueue.poll(POLL_TIMEOUT, TimeUnit.MILLISECONDS); + } catch (InterruptedException e) { + LOG.warn("Interrupted when polling splits from the split queue", e); + } + if (arcticSplit == null) { + LOG.debug("Couldn't retrieve arctic source split from the queue, as the queue is empty."); + return Optional.empty(); + } else { + LOG.info("Assigning the arctic split, task index is {}, total number of splits is {}, arctic split is {}.", + arcticSplit.taskIndex(), totalSplitNum, arcticSplit); + return Optional.of(arcticSplit); + } + } + + @Override + public void onDiscoveredSplits(Collection splits) { + splits.forEach(this::putArcticIntoQueue); + totalSplitNum += splits.size(); + // only complete pending future if new splits are discovered + completeAvailableFuturesIfNeeded(); + } + + @Override + public void onUnassignedSplits(Collection splits) { + onDiscoveredSplits(splits); + } + + void putArcticIntoQueue(final ArcticSplit split) { + splitQueue.put(split); + } + + @Override + public Collection state() { + return splitQueue.stream().map(ArcticSplitState::new).collect(Collectors.toList()); + } + + @Override + public synchronized CompletableFuture isAvailable() { + if (availableFuture == null) { + availableFuture = new CompletableFuture<>(); + } + return availableFuture; + } + + public boolean isEmpty() { + return splitQueue.isEmpty(); + } + + @Override + public void close() throws IOException { + splitQueue.clear(); + } + + private synchronized void completeAvailableFuturesIfNeeded() { + if (availableFuture != null && !isEmpty()) { + availableFuture.complete(null); + } + availableFuture = null; + } +} diff --git a/flink/v1.15/flink/src/main/java/com/netease/arctic/flink/read/hybrid/enumerator/ContinuousSplitPlannerImpl.java b/flink/v1.15/flink/src/main/java/com/netease/arctic/flink/read/hybrid/enumerator/ContinuousSplitPlannerImpl.java index a911de23d0..297c2d5680 100644 --- a/flink/v1.15/flink/src/main/java/com/netease/arctic/flink/read/hybrid/enumerator/ContinuousSplitPlannerImpl.java +++ b/flink/v1.15/flink/src/main/java/com/netease/arctic/flink/read/hybrid/enumerator/ContinuousSplitPlannerImpl.java @@ -61,6 +61,9 @@ public ContinuousSplitPlannerImpl(ArcticTableLoader loader) { @Override public void close() throws IOException { + if (loader != null) { + loader.close(); + } } @Override @@ -109,6 +112,9 @@ protected ContinuousEnumerationResult discoverIncrementalSplits( protected ContinuousEnumerationResult discoverInitialSplits(List filters) { Snapshot changeSnapshot = table.changeTable().currentSnapshot(); + // todo ShuffleSplitAssigner doesn't support MergeOnReadSplit right now, + // because it doesn't implement the dataTreeNode() method + // fix AMORO-1950 in the future. List arcticSplits = FlinkSplitPlanner.planFullTable(table, filters, splitCount); long changeStartSnapshotId = changeSnapshot != null ? changeSnapshot.snapshotId() : EARLIEST_SNAPSHOT_ID; diff --git a/flink/v1.15/flink/src/main/java/com/netease/arctic/flink/read/hybrid/enumerator/MergeOnReadPlannerImpl.java b/flink/v1.15/flink/src/main/java/com/netease/arctic/flink/read/hybrid/enumerator/MergeOnReadPlannerImpl.java new file mode 100644 index 0000000000..6d95b335e9 --- /dev/null +++ b/flink/v1.15/flink/src/main/java/com/netease/arctic/flink/read/hybrid/enumerator/MergeOnReadPlannerImpl.java @@ -0,0 +1,85 @@ +/* + * 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 com.netease.arctic.flink.read.hybrid.enumerator; + +import com.netease.arctic.flink.read.FlinkSplitPlanner; +import com.netease.arctic.flink.read.hybrid.split.ArcticSplit; +import com.netease.arctic.flink.table.ArcticTableLoader; +import com.netease.arctic.table.KeyedTable; +import org.apache.commons.collections.CollectionUtils; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.expressions.Expression; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.List; +import java.util.concurrent.atomic.AtomicInteger; + +import static com.netease.arctic.flink.read.hybrid.enumerator.ArcticEnumeratorOffset.EARLIEST_SNAPSHOT_ID; +import static com.netease.arctic.flink.util.ArcticUtils.loadArcticTable; + +/** + * Used for MergeOnRead, only for the bounded reading and return append stream. + */ +public class MergeOnReadPlannerImpl implements ContinuousSplitPlanner { + private static final Logger LOG = LoggerFactory.getLogger(MergeOnReadPlannerImpl.class); + + protected transient KeyedTable table; + protected final ArcticTableLoader loader; + protected static final AtomicInteger splitCount = new AtomicInteger(); + + public MergeOnReadPlannerImpl(ArcticTableLoader loader) { + this.loader = loader; + } + + @Override + public ContinuousEnumerationResult planSplits( + ArcticEnumeratorOffset ignored, List filters) { + // todo support mor the table from the specific offset in the future + if (table == null) { + table = loadArcticTable(loader).asKeyedTable(); + } + table.refresh(); + return discoverInitialSplits(filters); + } + + protected ContinuousEnumerationResult discoverInitialSplits(List filters) { + Snapshot changeSnapshot = table.changeTable().currentSnapshot(); + List arcticSplits = FlinkSplitPlanner.mergeOnReadPlan(table, filters, splitCount); + + long changeStartSnapshotId = changeSnapshot != null ? changeSnapshot.snapshotId() : EARLIEST_SNAPSHOT_ID; + if (changeSnapshot == null && CollectionUtils.isEmpty(arcticSplits)) { + LOG.info("There have no change snapshot, and no base splits in table: {}.", table); + return ContinuousEnumerationResult.EMPTY; + } + + return new ContinuousEnumerationResult( + arcticSplits, + null, + ArcticEnumeratorOffset.of(changeStartSnapshotId, null)); + } + + @Override + public void close() throws IOException { + if (loader != null) { + loader.close(); + } + } +} diff --git a/flink/v1.15/flink/src/main/java/com/netease/arctic/flink/read/hybrid/enumerator/StaticArcticSourceEnumerator.java b/flink/v1.15/flink/src/main/java/com/netease/arctic/flink/read/hybrid/enumerator/StaticArcticSourceEnumerator.java index a4db0edb83..7bec0a8597 100644 --- a/flink/v1.15/flink/src/main/java/com/netease/arctic/flink/read/hybrid/enumerator/StaticArcticSourceEnumerator.java +++ b/flink/v1.15/flink/src/main/java/com/netease/arctic/flink/read/hybrid/enumerator/StaticArcticSourceEnumerator.java @@ -18,19 +18,18 @@ package com.netease.arctic.flink.read.hybrid.enumerator; -import com.netease.arctic.flink.read.FlinkSplitPlanner; import com.netease.arctic.flink.read.hybrid.assigner.SplitAssigner; import com.netease.arctic.flink.read.hybrid.split.ArcticSplit; import com.netease.arctic.flink.read.source.ArcticScanContext; import com.netease.arctic.flink.table.ArcticTableLoader; import com.netease.arctic.table.KeyedTable; import org.apache.flink.api.connector.source.SplitEnumeratorContext; +import org.apache.iceberg.flink.source.ScanContext; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import javax.annotation.Nullable; -import java.util.List; -import java.util.concurrent.atomic.AtomicInteger; +import java.util.Collection; import static com.netease.arctic.flink.util.ArcticUtils.loadArcticTable; @@ -45,6 +44,7 @@ public class StaticArcticSourceEnumerator extends AbstractArcticEnumerator { private transient KeyedTable keyedTable; private final ArcticScanContext scanContext; private final boolean shouldEnumerate; + private final ContinuousSplitPlanner splitPlanner; public StaticArcticSourceEnumerator( SplitEnumeratorContext enumeratorContext, @@ -56,8 +56,9 @@ public StaticArcticSourceEnumerator( this.loader = loader; this.assigner = assigner; this.scanContext = scanContext; - // split enumeration is not needed during restore scenario + // split enumeration is not needed during a restore scenario this.shouldEnumerate = enumState == null; + this.splitPlanner = new MergeOnReadPlannerImpl(loader); } @Override @@ -69,7 +70,7 @@ public void start() { if (shouldEnumerate) { keyedTable.baseTable().refresh(); keyedTable.changeTable().refresh(); - List splits = FlinkSplitPlanner.planFullTable(keyedTable, new AtomicInteger()); + Collection splits = splitPlanner.planSplits(null, scanContext.filters()).splits(); assigner.onDiscoveredSplits(splits); LOG.info("Discovered {} splits from table {} during job initialization", splits.size(), keyedTable.name()); diff --git a/flink/v1.15/flink/src/main/java/com/netease/arctic/flink/read/hybrid/reader/DataIteratorReaderFunction.java b/flink/v1.15/flink/src/main/java/com/netease/arctic/flink/read/hybrid/reader/DataIteratorReaderFunction.java index cee14a9479..8338d10c32 100644 --- a/flink/v1.15/flink/src/main/java/com/netease/arctic/flink/read/hybrid/reader/DataIteratorReaderFunction.java +++ b/flink/v1.15/flink/src/main/java/com/netease/arctic/flink/read/hybrid/reader/DataIteratorReaderFunction.java @@ -23,6 +23,7 @@ import com.netease.arctic.flink.read.hybrid.split.ChangelogSplit; import com.netease.arctic.flink.read.source.ChangeLogDataIterator; import com.netease.arctic.flink.read.source.DataIterator; +import com.netease.arctic.flink.read.source.MergeOnReadDataIterator; import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds; import org.apache.iceberg.io.CloseableIterator; @@ -41,7 +42,12 @@ public DataIteratorReaderFunction(DataIteratorBatcher batcher) { @Override public CloseableIterator>> apply(ArcticSplit split) { DataIterator inputIterator = createDataIterator(split); - if (inputIterator instanceof ChangeLogDataIterator) { + if (inputIterator instanceof MergeOnReadDataIterator) { + inputIterator.seek( + 0, + split.asMergeOnReadSplit().recordOffset() + ); + } else if (inputIterator instanceof ChangeLogDataIterator) { ChangeLogDataIterator changelogInputIterator = (ChangeLogDataIterator) inputIterator; ChangelogSplit changelogSplit = split.asChangelogSplit(); changelogInputIterator.seek( diff --git a/flink/v1.15/flink/src/main/java/com/netease/arctic/flink/read/hybrid/reader/HybridSplitReader.java b/flink/v1.15/flink/src/main/java/com/netease/arctic/flink/read/hybrid/reader/HybridSplitReader.java index 609fd8f8d7..72e239e7f5 100644 --- a/flink/v1.15/flink/src/main/java/com/netease/arctic/flink/read/hybrid/reader/HybridSplitReader.java +++ b/flink/v1.15/flink/src/main/java/com/netease/arctic/flink/read/hybrid/reader/HybridSplitReader.java @@ -20,6 +20,7 @@ import com.netease.arctic.flink.read.hybrid.split.ArcticSplit; import com.netease.arctic.flink.read.hybrid.split.ChangelogSplit; +import com.netease.arctic.flink.read.hybrid.split.MergeOnReadSplit; import com.netease.arctic.flink.read.hybrid.split.SnapshotSplit; import org.apache.flink.api.connector.source.SourceReaderContext; import org.apache.flink.connector.base.source.reader.RecordsBySplits; @@ -91,7 +92,9 @@ public void handleSplitsChanges(SplitsChange splitsChange) { LOG.info("Handling a split change {}.", splitsChange); splitsChange.splits().forEach(arcticSplit -> { - if (arcticSplit instanceof SnapshotSplit || arcticSplit instanceof ChangelogSplit) { + if (arcticSplit instanceof SnapshotSplit || + arcticSplit instanceof ChangelogSplit || + arcticSplit instanceof MergeOnReadSplit) { splits.add(arcticSplit); } else { throw new IllegalArgumentException( diff --git a/flink/v1.15/flink/src/main/java/com/netease/arctic/flink/read/hybrid/reader/RowDataReaderFunction.java b/flink/v1.15/flink/src/main/java/com/netease/arctic/flink/read/hybrid/reader/RowDataReaderFunction.java index d179a2b135..84ade6cafb 100644 --- a/flink/v1.15/flink/src/main/java/com/netease/arctic/flink/read/hybrid/reader/RowDataReaderFunction.java +++ b/flink/v1.15/flink/src/main/java/com/netease/arctic/flink/read/hybrid/reader/RowDataReaderFunction.java @@ -24,6 +24,8 @@ import com.netease.arctic.flink.read.source.DataIterator; import com.netease.arctic.flink.read.source.FileScanTaskReader; import com.netease.arctic.flink.read.source.FlinkArcticDataReader; +import com.netease.arctic.flink.read.source.FlinkArcticMORDataReader; +import com.netease.arctic.flink.read.source.MergeOnReadDataIterator; import com.netease.arctic.flink.util.ArcticUtils; import com.netease.arctic.io.ArcticFileIO; import com.netease.arctic.table.PrimaryKeySpec; @@ -89,8 +91,20 @@ public RowDataReaderFunction( @Override public DataIterator createDataIterator(ArcticSplit split) { - if (split.isSnapshotSplit()) { - + if (split.isMergeOnReadSplit()) { + FlinkArcticMORDataReader morDataReader = + new FlinkArcticMORDataReader( + io, + tableSchema, + readSchema, + primaryKeySpec, + nameMapping, + caseSensitive, + RowDataUtil::convertConstant, + reuse + ); + return new MergeOnReadDataIterator(morDataReader, split.asMergeOnReadSplit().keyedTableScanTask(), io); + } else if (split.isSnapshotSplit()) { FileScanTaskReader rowDataReader = new FlinkArcticDataReader( io, tableSchema, readSchema, primaryKeySpec, nameMapping, caseSensitive, RowDataUtil::convertConstant, @@ -150,7 +164,7 @@ RowData transformRowKind(ChangeLogDataIterator.ChangeActionTrans trans) * * @param tableSchema table schema * @param projectedSchema projected schema - * @return a new Schema on which include the identifier fields. + * @return a new Schema on which includes the identifier fields. */ private static Schema fillUpReadSchema(Schema tableSchema, Schema projectedSchema, PrimaryKeySpec primaryKeySpec) { Preconditions.checkNotNull(tableSchema, "Table schema can't be null"); diff --git a/flink/v1.15/flink/src/main/java/com/netease/arctic/flink/read/hybrid/split/ArcticSplitSerializer.java b/flink/v1.15/flink/src/main/java/com/netease/arctic/flink/read/hybrid/split/ArcticSplitSerializer.java index 4a81496b81..be3f91579d 100644 --- a/flink/v1.15/flink/src/main/java/com/netease/arctic/flink/read/hybrid/split/ArcticSplitSerializer.java +++ b/flink/v1.15/flink/src/main/java/com/netease/arctic/flink/read/hybrid/split/ArcticSplitSerializer.java @@ -34,6 +34,7 @@ public class ArcticSplitSerializer implements SimpleVersionedSerializerdeserializeObject( + content, MergeOnReadSplit.class.getClassLoader()); + } else if (flag == SNAPSHOT_SPLIT_FLAG) { return InstantiationUtil.deserializeObject(content, SnapshotSplit.class.getClassLoader()); } else if (flag == CHANGELOG_SPLIT_FLAG) { return InstantiationUtil.deserializeObject(content, ChangelogSplit.class.getClassLoader()); } else { - throw new IllegalArgumentException("this flag split is unsupported. available: 1,2."); + throw new IllegalArgumentException( + String.format( + "this flag split %s is unsupported. available: %s, %s, and %s.", + flag, + SNAPSHOT_SPLIT_FLAG, + CHANGELOG_SPLIT_FLAG, + MOR_SPLIT_FLAG)); } } } catch (ClassNotFoundException e) { diff --git a/flink/v1.15/flink/src/main/java/com/netease/arctic/flink/read/hybrid/split/ArcticSplitState.java b/flink/v1.15/flink/src/main/java/com/netease/arctic/flink/read/hybrid/split/ArcticSplitState.java index a3c73f4518..de02cc0f33 100644 --- a/flink/v1.15/flink/src/main/java/com/netease/arctic/flink/read/hybrid/split/ArcticSplitState.java +++ b/flink/v1.15/flink/src/main/java/com/netease/arctic/flink/read/hybrid/split/ArcticSplitState.java @@ -36,7 +36,11 @@ public ArcticSplitState(ArcticSplit arcticSplit) { } public ArcticSplit toSourceSplit() { - if (arcticSplit.isSnapshotSplit()) { + if (arcticSplit.isMergeOnReadSplit()) { + MergeOnReadSplit mergeOnReadSplit = (MergeOnReadSplit) arcticSplit; + mergeOnReadSplit.updateOffset(new Object[]{currentInsertFileOffset, currentInsertRecordOffset}); + return mergeOnReadSplit; + } else if (arcticSplit.isSnapshotSplit()) { SnapshotSplit snapshotSplit = (SnapshotSplit) arcticSplit; snapshotSplit.updateOffset(new Object[]{currentInsertFileOffset, currentInsertRecordOffset}); return snapshotSplit; @@ -52,10 +56,12 @@ public ArcticSplit toSourceSplit() { } throw new FlinkRuntimeException( - String.format("As of now this source split is unsupported %s, available split are %s, %s", + String.format( + "As of now this source split is unsupported %s, available split are %s, %s, and %s", arcticSplit.getClass().getSimpleName(), SnapshotSplit.class.getSimpleName(), - ChangelogSplit.class.getSimpleName())); + ChangelogSplit.class.getSimpleName(), + MergeOnReadSplit.class.getSimpleName())); } public void updateOffset(Object[] offsets) { diff --git a/flink/v1.15/flink/src/main/java/com/netease/arctic/flink/read/hybrid/split/MergeOnReadSplit.java b/flink/v1.15/flink/src/main/java/com/netease/arctic/flink/read/hybrid/split/MergeOnReadSplit.java index 7f32973c89..068a553910 100644 --- a/flink/v1.15/flink/src/main/java/com/netease/arctic/flink/read/hybrid/split/MergeOnReadSplit.java +++ b/flink/v1.15/flink/src/main/java/com/netease/arctic/flink/read/hybrid/split/MergeOnReadSplit.java @@ -20,13 +20,14 @@ import com.netease.arctic.scan.KeyedTableScanTask; import com.netease.arctic.utils.FileScanTaskUtil; -import org.apache.flink.util.FlinkRuntimeException; +import org.apache.flink.util.Preconditions; import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; public class MergeOnReadSplit extends ArcticSplit { private static final long serialVersionUID = 1L; private final int taskIndex; private final KeyedTableScanTask keyedTableScanTask; + private long recordOffset; public MergeOnReadSplit(int taskIndex, KeyedTableScanTask keyedTableScanTask) { this.taskIndex = taskIndex; @@ -43,8 +44,10 @@ public Integer taskIndex() { } @Override - public void updateOffset(Object[] recordOffsets) { - throw new FlinkRuntimeException("Merge On Read not support offset state right now."); + public void updateOffset(Object[] offsets) { + Preconditions.checkArgument(offsets.length == 2); + // offsets[0] is file offset, but we don't need it + recordOffset = (long) offsets[1]; } @Override @@ -61,6 +64,21 @@ public String splitId() { .toString(); } + public long recordOffset() { + return recordOffset; + } + + @Override + public boolean equals(Object obj) { + if (!(obj instanceof MergeOnReadSplit)) { + return false; + } + MergeOnReadSplit other = (MergeOnReadSplit) obj; + return splitId().equals(other.splitId()) && + recordOffset == other.recordOffset && + taskIndex == other.taskIndex; + } + @Override public String toString() { return MoreObjects.toStringHelper(this) diff --git a/flink/v1.15/flink/src/main/java/com/netease/arctic/flink/read/source/ArcticScanContext.java b/flink/v1.15/flink/src/main/java/com/netease/arctic/flink/read/source/ArcticScanContext.java index 56b4dfb702..1fd87e154b 100644 --- a/flink/v1.15/flink/src/main/java/com/netease/arctic/flink/read/source/ArcticScanContext.java +++ b/flink/v1.15/flink/src/main/java/com/netease/arctic/flink/read/source/ArcticScanContext.java @@ -52,60 +52,36 @@ public class ArcticScanContext extends ScanContext implements Serializable { private static final long serialVersionUID = 1L; private final String scanStartupMode; - - protected ArcticScanContext( - boolean caseSensitive, - Long snapshotId, - StreamingStartingStrategy startingStrategy, - Long startSnapshotTimestamp, - Long startSnapshotId, - Long endSnapshotId, - Long asOfTimestamp, - Long splitSize, - Integer splitLookback, - Long splitOpenFileCost, - boolean isStreaming, - Duration monitorInterval, - String nameMapping, - Schema schema, - List filters, - long limit, - boolean includeColumnStats, - boolean exposeLocality, - Integer planParallelism, - int maxPlanningSnapshotCount, - String scanStartupMode, - int maxAllowedPlanningFailures, - String branch, - String tag, - String startTag, - String endTag) { - super(caseSensitive, - snapshotId, - startingStrategy, - startSnapshotTimestamp, - startSnapshotId, - endSnapshotId, - asOfTimestamp, - splitSize, - splitLookback, - splitOpenFileCost, - isStreaming, - monitorInterval, - nameMapping, - schema, - filters, - limit, - includeColumnStats, - exposeLocality, - planParallelism, - maxPlanningSnapshotCount, - maxAllowedPlanningFailures, - branch, - tag, - startTag, - endTag); - this.scanStartupMode = scanStartupMode; + private final boolean batchMode; + + protected ArcticScanContext(Builder builder) { + super(builder.caseSensitive, + builder.snapshotId, + builder.startingStrategy, + builder.startSnapshotTimestamp, + builder.startSnapshotId, + builder.endSnapshotId, + builder.asOfTimestamp, + builder.splitSize, + builder.splitLookback, + builder.splitOpenFileCost, + builder.isStreaming, + builder.monitorInterval, + builder.nameMapping, + builder.projectedSchema, + builder.filters, + builder.limit, + builder.includeColumnStats, + builder.exposeLocality, + builder.planParallelism, + builder.maxPlanningSnapshotCount, + builder.maxAllowedPlanningFailures, + builder.branch, + builder.tag, + builder.startTag, + builder.endTag); + this.scanStartupMode = builder.scanStartupMode; + this.batchMode = builder.batchMode; } public boolean caseSensitive() { @@ -175,6 +151,10 @@ public String scanStartupMode() { return scanStartupMode; } + public boolean isBatchMode() { + return batchMode; + } + public static class Builder { private boolean caseSensitive = FlinkReadOptions.CASE_SENSITIVE_OPTION.defaultValue(); private Long snapshotId = FlinkReadOptions.SNAPSHOT_ID.defaultValue(); @@ -199,8 +179,7 @@ public static class Builder { private boolean exposeLocality; private Integer planParallelism = FlinkConfigOptions.TABLE_EXEC_ICEBERG_WORKER_POOL_SIZE.defaultValue(); - private int maxPlanningSnapshotCount = - FlinkReadOptions.MAX_PLANNING_SNAPSHOT_COUNT_OPTION.defaultValue(); + private int maxPlanningSnapshotCount = MAX_PLANNING_SNAPSHOT_COUNT.defaultValue(); private int maxAllowedPlanningFailures = FlinkReadOptions.MAX_ALLOWED_PLANNING_FAILURES_OPTION.defaultValue(); @@ -213,6 +192,7 @@ public static class Builder { private String endTag = FlinkReadOptions.END_TAG.defaultValue(); private String scanStartupMode; + private boolean batchMode = false; private Builder() { } @@ -347,6 +327,11 @@ public Builder includeColumnStats(boolean newIncludeColumnStats) { return this; } + public Builder batchMode(boolean batchMode) { + this.batchMode = batchMode; + return this; + } + public Builder fromProperties(Map properties) { Configuration config = new Configuration(); properties.forEach(config::setString); @@ -381,33 +366,10 @@ public ArcticScanContext build() { Objects.equals(scanStartupMode, SCAN_STARTUP_MODE_LATEST), String.format("only support %s, %s when %s is %s", SCAN_STARTUP_MODE_EARLIEST, SCAN_STARTUP_MODE_LATEST, ARCTIC_READ_MODE, ARCTIC_READ_FILE)); - return new ArcticScanContext( - caseSensitive, - snapshotId, - startingStrategy, - startSnapshotTimestamp, - startSnapshotId, - endSnapshotId, - asOfTimestamp, - splitSize, - splitLookback, - splitOpenFileCost, - isStreaming, - monitorInterval, - nameMapping, - projectedSchema, - filters, - limit, - includeColumnStats, - exposeLocality, - planParallelism, - maxPlanningSnapshotCount, - scanStartupMode, - maxAllowedPlanningFailures, - branch, - tag, - startTag, - endTag); + Preconditions.checkArgument( + !(isStreaming && batchMode), + String.format("only support %s = false when execution.runtime-mode is batch", STREAMING.key())); + return new ArcticScanContext(this); } } } diff --git a/flink/v1.15/flink/src/main/java/com/netease/arctic/flink/read/source/DataIterator.java b/flink/v1.15/flink/src/main/java/com/netease/arctic/flink/read/source/DataIterator.java index 4355428e52..e7ec41977a 100644 --- a/flink/v1.15/flink/src/main/java/com/netease/arctic/flink/read/source/DataIterator.java +++ b/flink/v1.15/flink/src/main/java/com/netease/arctic/flink/read/source/DataIterator.java @@ -35,7 +35,7 @@ /** * Flink data iterator that reads {@link ArcticFileScanTask} into a {@link CloseableIterator} * - * @param is the output data type returned by this iterator. + * @param T is the output data type returned by this iterator. */ @Internal public class DataIterator implements CloseableIterator { @@ -51,6 +51,10 @@ public class DataIterator implements CloseableIterator { private final Function arcticFileOffsetGetter; private final Function arcticMetaColumnRemover; + public DataIterator() { + this(null, Collections.emptyList(), t -> Long.MIN_VALUE, t -> t); + } + public DataIterator( FileScanTaskReader fileScanTaskReader, Collection tasks, @@ -69,14 +73,15 @@ public DataIterator( this.fileOffset = -1; // record offset points to the record that next() should return when called this.recordOffset = 0L; - // actual record offset in data file. it's incremental within insert and delete files in the same tree node group. + // actual record offset in data file. + // it's incremental within inserting and deleting files in the same tree node group. this.currentArcticFileOffset = 0L; } /** - * (startingFileOffset, startingRecordOffset) points to the next row that reader should resume from. - * E.g., if the seek position is (file=0, record=1), seek moves the iterator position to the 2nd row - * in file 0. When next() is called after seek, 2nd row from file 0 should be returned. + * (startingFileOffset, startingRecordOffset) points to the next row that the reader should resume from. + * E.g., if the seek position is (file=0, record=1), seek moves the iterator position to the second row + * in file 0. When next() is called after seek; the second row from file 0 should be returned. */ public void seek(int startingFileOffset, long startingRecordOffset) { // It means file is empty. diff --git a/flink/v1.15/flink/src/main/java/com/netease/arctic/flink/read/source/MergeOnReadDataIterator.java b/flink/v1.15/flink/src/main/java/com/netease/arctic/flink/read/source/MergeOnReadDataIterator.java new file mode 100644 index 0000000000..a4e7104f8f --- /dev/null +++ b/flink/v1.15/flink/src/main/java/com/netease/arctic/flink/read/source/MergeOnReadDataIterator.java @@ -0,0 +1,132 @@ +/* + * 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 com.netease.arctic.flink.read.source; + +import com.netease.arctic.io.ArcticFileIO; +import com.netease.arctic.scan.KeyedTableScanTask; +import org.apache.flink.table.data.RowData; +import org.apache.flink.util.Preconditions; +import org.apache.iceberg.io.CloseableIterator; + +import java.io.IOException; + +/** + * Iterator for reading data in a Merge on Read (MOR) way. + * This iterator handles reading data from an Amoro mix-format table + * while keeping track of file and record offsets for efficient data retrieval. + */ +public class MergeOnReadDataIterator extends DataIterator { + private int fileOffset; + private long recordOffset; + private final CloseableIterator iterator; + + public MergeOnReadDataIterator( + FlinkArcticMORDataReader flinkArcticMORDataReader, + KeyedTableScanTask keyedTableScanTask, + ArcticFileIO io) { + super(); + this.iterator = + IteratorWithIO.of( + io, + io.doAs(() -> flinkArcticMORDataReader.readData(keyedTableScanTask))); + } + + @Override + public void seek(int startingFileOffset, long startingRecordOffset) { + // startingFileOffset is not used, because we only have one file per task + Preconditions.checkNotNull(iterator, "iterator is null in the MergeOnReadDataIterator."); + // skip records within the file + for (long i = 0; i < startingRecordOffset; ++i) { + if (hasNext()) { + next(); + } else { + throw new IllegalStateException(String.format( + "Invalid starting record offset %d for file %d from KeyedTableScanTask.", + startingRecordOffset, startingFileOffset)); + } + } + this.fileOffset = startingFileOffset; + this.recordOffset = startingRecordOffset; + } + + @Override + public boolean hasNext() { + return iterator.hasNext(); + } + + @Override + public RowData next() { + return iterator.next(); + } + + public boolean currentFileHasNext() { + return iterator.hasNext(); + } + + @Override + public int fileOffset() { + return fileOffset; + } + + @Override + public long recordOffset() { + return recordOffset; + } + + @Override + public void close() throws IOException { + // close the current iterator + if (iterator != null) { + iterator.close(); + } + } + + static class IteratorWithIO implements CloseableIterator { + private final ArcticFileIO io; + private final CloseableIterator iterator; + + private IteratorWithIO(ArcticFileIO io, CloseableIterator iterator) { + this.io = io; + this.iterator = iterator; + } + + static IteratorWithIO of(ArcticFileIO io, CloseableIterator iterator) { + Preconditions.checkNotNull(io); + return new IteratorWithIO(io, iterator); + } + + @Override + public void close() throws IOException { + io.doAs(() -> { + iterator.close(); + return null; + }); + } + + @Override + public boolean hasNext() { + return io.doAs(iterator::hasNext); + } + + @Override + public RowData next() { + return io.doAs(iterator::next); + } + } +} diff --git a/flink/v1.15/flink/src/main/java/com/netease/arctic/flink/table/ArcticFileSource.java b/flink/v1.15/flink/src/main/java/com/netease/arctic/flink/table/ArcticFileSource.java index 4b94c46b3e..d86d3da910 100644 --- a/flink/v1.15/flink/src/main/java/com/netease/arctic/flink/table/ArcticFileSource.java +++ b/flink/v1.15/flink/src/main/java/com/netease/arctic/flink/table/ArcticFileSource.java @@ -52,6 +52,8 @@ import java.util.List; import static com.netease.arctic.flink.table.descriptors.ArcticValidator.DIM_TABLE_ENABLE; +import static org.apache.flink.api.common.RuntimeExecutionMode.BATCH; +import static org.apache.flink.configuration.ExecutionOptions.RUNTIME_MODE; /** * Flink table api that generates arctic base/change file source operators. @@ -64,13 +66,14 @@ public class ArcticFileSource implements ScanTableSource, SupportsFilterPushDown private int[] projectedFields; private long limit; private List filters; - private ArcticTable table; + private final ArcticTable table; @Nullable protected WatermarkStrategy watermarkStrategy; private final ArcticTableLoader loader; private final TableSchema tableSchema; private final ReadableConfig readableConfig; + private final boolean batchMode; private ArcticFileSource(ArcticFileSource toCopy) { this.loader = toCopy.loader; @@ -80,6 +83,8 @@ private ArcticFileSource(ArcticFileSource toCopy) { this.filters = toCopy.filters; this.readableConfig = toCopy.readableConfig; this.table = toCopy.table; + this.watermarkStrategy = toCopy.watermarkStrategy; + this.batchMode = toCopy.batchMode; } public ArcticFileSource(ArcticTableLoader loader, @@ -88,7 +93,8 @@ public ArcticFileSource(ArcticTableLoader loader, ArcticTable table, long limit, List filters, - ReadableConfig readableConfig) { + ReadableConfig readableConfig, + boolean batchMode) { this.loader = loader; this.tableSchema = tableSchema; this.projectedFields = projectedFields; @@ -96,11 +102,16 @@ public ArcticFileSource(ArcticTableLoader loader, this.table = table; this.filters = filters; this.readableConfig = readableConfig; + this.batchMode = batchMode; } - public ArcticFileSource(ArcticTableLoader loader, TableSchema tableSchema, ArcticTable table, - ReadableConfig readableConfig) { - this(loader, tableSchema, null, table, -1, ImmutableList.of(), readableConfig); + public ArcticFileSource( + ArcticTableLoader loader, + TableSchema tableSchema, + ArcticTable table, + ReadableConfig readableConfig, + boolean batchMode) { + this(loader, tableSchema, null, table, -1, ImmutableList.of(), readableConfig, batchMode); } @Override @@ -123,6 +134,7 @@ private DataStream createDataStream(ProviderContext providerContext, St .limit(limit) .filters(filters) .flinkConf(readableConfig) + .batchMode(execEnv.getConfiguration().get(RUNTIME_MODE).equals(BATCH)) .watermarkStrategy(watermarkStrategy) .build(); } @@ -170,7 +182,7 @@ public boolean supportsNestedProjection() { @Override public ChangelogMode getChangelogMode() { - if (table.isUnkeyedTable()) { + if (table.isUnkeyedTable() || batchMode) { return ChangelogMode.insertOnly(); } return ChangelogMode.newBuilder() diff --git a/flink/v1.15/flink/src/main/java/com/netease/arctic/flink/table/DynamicTableFactory.java b/flink/v1.15/flink/src/main/java/com/netease/arctic/flink/table/DynamicTableFactory.java index 2cf5c25230..e999c7918f 100644 --- a/flink/v1.15/flink/src/main/java/com/netease/arctic/flink/table/DynamicTableFactory.java +++ b/flink/v1.15/flink/src/main/java/com/netease/arctic/flink/table/DynamicTableFactory.java @@ -83,6 +83,8 @@ import static com.netease.arctic.table.TableProperties.LOG_STORE_STORAGE_TYPE_DEFAULT; import static com.netease.arctic.table.TableProperties.LOG_STORE_STORAGE_TYPE_KAFKA; import static com.netease.arctic.table.TableProperties.LOG_STORE_TYPE; +import static org.apache.flink.api.common.RuntimeExecutionMode.BATCH; +import static org.apache.flink.configuration.ExecutionOptions.RUNTIME_MODE; import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.KEY_FIELDS_PREFIX; import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.KEY_FORMAT; import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.PROPS_BOOTSTRAP_SERVERS; @@ -101,13 +103,10 @@ public class DynamicTableFactory implements DynamicTableSourceFactory, DynamicTableSinkFactory { private static final Logger LOG = LoggerFactory.getLogger(DynamicTableFactory.class); public static final String IDENTIFIER = "arctic"; - private ArcticCatalog arcticCatalog; private InternalCatalogBuilder internalCatalogBuilder; private String internalCatalogName; - public DynamicTableFactory( - ArcticCatalog arcticCatalog) { - this.arcticCatalog = arcticCatalog; + public DynamicTableFactory(ArcticCatalog arcticCatalog) { this.internalCatalogBuilder = arcticCatalog.catalogBuilder(); this.internalCatalogName = arcticCatalog.amsCatalogName(); } @@ -162,8 +161,9 @@ public DynamicTableSource createDynamicTableSource(Context context) { dimTable); switch (readMode) { case ArcticValidator.ARCTIC_READ_FILE: - LOG.info("build file reader"); - arcticDynamicSource = new ArcticFileSource(tableLoader, tableSchema, arcticTable, confWithAll); + boolean batchMode = context.getConfiguration().get(RUNTIME_MODE).equals(BATCH); + LOG.info("Building a file reader in {} runtime mode", batchMode ? "batch" : "streaming"); + arcticDynamicSource = new ArcticFileSource(tableLoader, tableSchema, arcticTable, confWithAll, batchMode); break; case ArcticValidator.ARCTIC_READ_LOG: default: @@ -225,8 +225,7 @@ public String factoryIdentifier() { @Override public Set> requiredOptions() { - final Set> options = new HashSet<>(); - return options; + return new HashSet<>(); } @Override @@ -356,7 +355,7 @@ private static Optional>> getKeyDe /** * Return true only if {@link ArcticValidator#ARCTIC_LOG_KAFKA_COMPATIBLE_ENABLE} is true and - * {@link LOG_STORE_TYPE} is kafka. + * {@link TableProperties#LOG_STORE_TYPE} is kafka. */ private static boolean adaptLegacySource(ArcticTable arcticTable) { boolean legacySourceEnabled = CompatibleFlinkPropertyUtil.propertyAsBoolean(arcticTable.properties(), @@ -385,4 +384,4 @@ private static DecodingFormat> getValueDecodingFo DeserializationFormatFactory.class, VALUE_FORMAT)); } -} \ No newline at end of file +} diff --git a/flink/v1.15/flink/src/main/java/com/netease/arctic/flink/table/FlinkSource.java b/flink/v1.15/flink/src/main/java/com/netease/arctic/flink/table/FlinkSource.java index 8a211f6eb6..ce5647452a 100644 --- a/flink/v1.15/flink/src/main/java/com/netease/arctic/flink/table/FlinkSource.java +++ b/flink/v1.15/flink/src/main/java/com/netease/arctic/flink/table/FlinkSource.java @@ -80,10 +80,11 @@ public static final class Builder { private TableSchema projectedSchema; private List filters; private ReadableConfig flinkConf = new Configuration(); - private Map properties = new HashMap<>(); + private final Map properties = new HashMap<>(); private long limit = -1L; private WatermarkStrategy watermarkStrategy = WatermarkStrategy.noWatermarks(); private final ArcticScanContext.Builder contextBuilder = ArcticScanContext.arcticBuilder(); + private boolean batchMode = false; private Builder() { } @@ -143,6 +144,11 @@ public Builder watermarkStrategy(WatermarkStrategy watermarkStrategy) { return this; } + public Builder batchMode(boolean batchMode) { + this.batchMode = batchMode; + return this; + } + public DataStream build() { Preconditions.checkNotNull(env, "StreamExecutionEnvironment should not be null"); loadTableIfNeeded(); @@ -167,8 +173,11 @@ public DataStream build() { rowType = toRowType(filterWatermark(projectedSchema)); } } - contextBuilder.fromProperties(properties); - ArcticScanContext scanContext = contextBuilder.build(); + ArcticScanContext scanContext = + contextBuilder + .fromProperties(properties) + .batchMode(batchMode) + .build(); RowDataReaderFunction rowDataReaderFunction = new RowDataReaderFunction( flinkConf, diff --git a/flink/v1.15/flink/src/test/java/com/netease/arctic/flink/read/TestArcticSource.java b/flink/v1.15/flink/src/test/java/com/netease/arctic/flink/read/TestArcticSource.java index ba8da7f9b9..c7fdfcfe68 100644 --- a/flink/v1.15/flink/src/test/java/com/netease/arctic/flink/read/TestArcticSource.java +++ b/flink/v1.15/flink/src/test/java/com/netease/arctic/flink/read/TestArcticSource.java @@ -122,7 +122,8 @@ public class TestArcticSource extends TestRowDataReaderFunction implements Seria protected KeyedTable testFailoverTable; protected static final String sinkTableName = "test_sink_exactly_once"; protected static final TableIdentifier FAIL_TABLE_ID = - TableIdentifier.of(TEST_CATALOG_NAME, TEST_DB_NAME, sinkTableName); + TableIdentifier.of(TableTestHelper.TEST_CATALOG_NAME, TableTestHelper.TEST_DB_NAME, sinkTableName); + ; @Before public void testSetup() throws IOException { @@ -170,8 +171,8 @@ public void testArcticSourceStatic() throws Exception { GenericRowData rowData = convert(row); actualResult.add(rowData); }); - Assert.assertEquals(8, actualResult.size()); - assertArrayEquals(excepts(), actualResult); + RowData[] expected = expectedAfterMOR(); + assertArrayEquals(expected, actualResult); } @Test @@ -185,7 +186,7 @@ public void testArcticSourceStaticTaskManagerFailover() throws Exception { } public void testArcticSource(FailoverType failoverType) throws Exception { - List expected = new ArrayList<>(exceptsCollection()); + List expected = new ArrayList<>(expectedCollection()); List updated = updateRecords(); writeUpdate(updated); List records = generateRecords(2, 1); @@ -205,8 +206,9 @@ public void testArcticSource(FailoverType failoverType) throws Exception { "ArcticParallelSource") .setParallelism(PARALLELISM); + List expectedAfterMoR = new ArrayList<>(mor(expected)); DataStream streamFailingInTheMiddleOfReading = - RecordCounterToFail.wrapWithFailureAfter(input, expected.size() / 2); + RecordCounterToFail.wrapWithFailureAfter(input, expectedAfterMoR.size() / 2); FlinkSink .forRowData(streamFailingInTheMiddleOfReading) @@ -226,7 +228,7 @@ public void testArcticSource(FailoverType failoverType) throws Exception { RecordCounterToFail::continueProcessing, miniClusterResource.getMiniCluster()); - assertRecords(testFailoverTable, expected, Duration.ofMillis(10), 12000); + assertRecords(testFailoverTable, expectedAfterMoR, Duration.ofMillis(10), 12000); } @Test(timeout = 60000) @@ -690,7 +692,7 @@ private static void restartTaskManager(Runnable afterFailAction, MiniCluster min } private List collectRecordsFromUnboundedStream( - final ClientAndIterator client, final int numElements) throws InterruptedException { + final ClientAndIterator client, final int numElements) { checkNotNull(client, "client"); checkArgument(numElements > 0, "numElement must be > 0"); @@ -698,8 +700,6 @@ private List collectRecordsFromUnboundedStream( final ArrayList result = new ArrayList<>(numElements); final Iterator iterator = client.iterator; - LOG.info("begin collect records by the UT {}, ", name.getMethodName()); - CollectTask collectTask = new CollectTask(result, iterator, numElements); new Thread(collectTask).start(); @@ -710,13 +710,12 @@ private List collectRecordsFromUnboundedStream( // TODO a more proper timeout strategy? long timeFlies = System.currentTimeMillis() - start; if (timeFlies / 1000 >= intervalOneSecond) { - LOG.info("time flies: {} ms.", timeFlies); + LOG.info("Time flies: {} ms.", timeFlies); intervalOneSecond++; } - Thread.sleep(10); - if (timeFlies > timeout) { + if (System.currentTimeMillis() - start > timeout) { LOG.error( - "this task [{}] try to collect records from unbounded stream but timeout {}. As of now, collect result:{}.", + "This task [{}] try to collect records from unbounded stream but timeout {}. As of now, collect result:{}.", client.client.getJobID().toString(), timeout, result.toArray()); @@ -751,7 +750,6 @@ public CollectTask(ArrayList result, Iterator iterator, int li public void run() { while (iterator.hasNext()) { result.add(convert(iterator.next())); - LOG.info("collected records size:{}.", result.size()); if (result.size() == limit) { running = false; return; diff --git a/flink/v1.15/flink/src/test/java/com/netease/arctic/flink/read/hybrid/assigner/TestShuffleSplitAssigner.java b/flink/v1.15/flink/src/test/java/com/netease/arctic/flink/read/hybrid/assigner/TestShuffleSplitAssigner.java index 1703b6fc90..e6afcd661b 100644 --- a/flink/v1.15/flink/src/test/java/com/netease/arctic/flink/read/hybrid/assigner/TestShuffleSplitAssigner.java +++ b/flink/v1.15/flink/src/test/java/com/netease/arctic/flink/read/hybrid/assigner/TestShuffleSplitAssigner.java @@ -83,7 +83,7 @@ public void testMultiParallelism() { if (splitOpt.isAvailable()) { actual.add(splitOpt.split()); } else { - LOG.info("subtask id {}, splits {}.\n {}", subtaskId, actual.size(), actual); + LOG.info("Subtask id {}, splits {}.\n {}", subtaskId, actual.size(), actual); --subtaskId; } } @@ -183,7 +183,7 @@ public void testNodeUpMoved() throws IOException { } while (subtaskId < totalParallelism); - List excepts = exceptsCollection(); + List excepts = expectedCollection(); excepts.addAll(generateRecords()); RowData[] array = excepts.stream().sorted(Comparator.comparing(RowData::toString)) .collect(Collectors.toList()) diff --git a/flink/v1.15/flink/src/test/java/com/netease/arctic/flink/read/hybrid/assigner/TestStaticSplitAssigner.java b/flink/v1.15/flink/src/test/java/com/netease/arctic/flink/read/hybrid/assigner/TestStaticSplitAssigner.java new file mode 100644 index 0000000000..b678032713 --- /dev/null +++ b/flink/v1.15/flink/src/test/java/com/netease/arctic/flink/read/hybrid/assigner/TestStaticSplitAssigner.java @@ -0,0 +1,85 @@ +/* + * 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 com.netease.arctic.flink.read.hybrid.assigner; + +import com.netease.arctic.flink.read.FlinkSplitPlanner; +import com.netease.arctic.flink.read.hybrid.reader.TestRowDataReaderFunction; +import com.netease.arctic.flink.read.hybrid.split.ArcticSplit; +import org.junit.Assert; +import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.concurrent.atomic.AtomicInteger; + +public class TestStaticSplitAssigner extends TestRowDataReaderFunction { + private static final Logger LOG = LoggerFactory.getLogger(TestStaticSplitAssigner.class); + + @Test + public void testSingleParallelism() throws IOException { + try (StaticSplitAssigner staticSplitAssigner = instanceStaticSplitAssigner()) { + List splitList = FlinkSplitPlanner.mergeOnReadPlan(testKeyedTable, Collections.emptyList(), + new AtomicInteger()); + staticSplitAssigner.onDiscoveredSplits(splitList); + List actual = new ArrayList<>(); + + while (true) { + Split splitOpt = staticSplitAssigner.getNext(0); + if (splitOpt.isAvailable()) { + actual.add(splitOpt.split()); + } else { + break; + } + } + + Assert.assertEquals(splitList.size(), actual.size()); + } + } + + @Test + public void testMultiParallelism() throws IOException { + try (StaticSplitAssigner staticSplitAssigner = instanceStaticSplitAssigner()) { + List splitList = FlinkSplitPlanner.mergeOnReadPlan(testKeyedTable, Collections.emptyList(), + new AtomicInteger()); + staticSplitAssigner.onDiscoveredSplits(splitList); + List actual = new ArrayList<>(); + + int subtaskId = 2; + while (subtaskId >= 0) { + Split splitOpt = staticSplitAssigner.getNext(subtaskId); + if (splitOpt.isAvailable()) { + actual.add(splitOpt.split()); + } else { + LOG.info("Subtask id {}, splits {}.\n {}", subtaskId, actual.size(), actual); + --subtaskId; + } + } + + Assert.assertEquals(splitList.size(), actual.size()); + } + } + + protected StaticSplitAssigner instanceStaticSplitAssigner() { + return new StaticSplitAssigner(null); + } +} diff --git a/flink/v1.15/flink/src/test/java/com/netease/arctic/flink/read/hybrid/enumerator/TestArcticSourceEnumStateSerializer.java b/flink/v1.15/flink/src/test/java/com/netease/arctic/flink/read/hybrid/enumerator/TestArcticSourceEnumStateSerializer.java index 366cd96bdc..81945e6942 100644 --- a/flink/v1.15/flink/src/test/java/com/netease/arctic/flink/read/hybrid/enumerator/TestArcticSourceEnumStateSerializer.java +++ b/flink/v1.15/flink/src/test/java/com/netease/arctic/flink/read/hybrid/enumerator/TestArcticSourceEnumStateSerializer.java @@ -20,8 +20,8 @@ import com.netease.arctic.flink.read.FlinkSplitPlanner; import com.netease.arctic.flink.read.hybrid.assigner.ShuffleSplitAssigner; -import com.netease.arctic.flink.read.hybrid.assigner.TestShuffleSplitAssigner; import com.netease.arctic.flink.read.hybrid.assigner.Split; +import com.netease.arctic.flink.read.hybrid.assigner.TestShuffleSplitAssigner; import com.netease.arctic.flink.read.hybrid.split.ArcticSplit; import com.netease.arctic.flink.read.hybrid.split.TemporalJoinSplits; import org.apache.flink.api.connector.source.SplitEnumeratorContext; @@ -67,25 +67,28 @@ public void testArcticEnumState() throws IOException { Objects.requireNonNull(actual.shuffleSplitRelation()).length); SplitEnumeratorContext splitEnumeratorContext = new InternalSplitEnumeratorContext(3); - ShuffleSplitAssigner actualAssigner = new ShuffleSplitAssigner(splitEnumeratorContext, - actual.pendingSplits(), actual.shuffleSplitRelation()); - - List actualSplits = new ArrayList<>(); - - int subtaskId = 2; - while (subtaskId >= 0) { - Split splitOpt = actualAssigner.getNext(subtaskId); - if (splitOpt.isAvailable()) { - actualSplits.add(splitOpt.split()); - } else { - LOG.info("subtask id {}, splits {}.\n {}", subtaskId, actualSplits.size(), actualSplits); - --subtaskId; + try (ShuffleSplitAssigner actualAssigner = + new ShuffleSplitAssigner( + splitEnumeratorContext, + getArcticTable().name(), + actual)) { + List actualSplits = new ArrayList<>(); + + int subtaskId = 2; + while (subtaskId >= 0) { + Split splitOpt = actualAssigner.getNext(subtaskId); + if (splitOpt.isAvailable()) { + actualSplits.add(splitOpt.split()); + } else { + LOG.info("subtask id {}, splits {}.\n {}", subtaskId, actualSplits.size(), actualSplits); + --subtaskId; + } } - } - Assert.assertEquals(splitList.size(), actualSplits.size()); + Assert.assertEquals(splitList.size(), actualSplits.size()); - TemporalJoinSplits temporalJoinSplits = actual.temporalJoinSplits(); - Assert.assertEquals(expect.temporalJoinSplits(), temporalJoinSplits); + TemporalJoinSplits temporalJoinSplits = actual.temporalJoinSplits(); + Assert.assertEquals(expect.temporalJoinSplits(), temporalJoinSplits); + } } } \ No newline at end of file diff --git a/flink/v1.15/flink/src/test/java/com/netease/arctic/flink/read/hybrid/reader/TestRowDataReaderFunction.java b/flink/v1.15/flink/src/test/java/com/netease/arctic/flink/read/hybrid/reader/TestRowDataReaderFunction.java index 03764fd437..b92cb68cff 100644 --- a/flink/v1.15/flink/src/test/java/com/netease/arctic/flink/read/hybrid/reader/TestRowDataReaderFunction.java +++ b/flink/v1.15/flink/src/test/java/com/netease/arctic/flink/read/hybrid/reader/TestRowDataReaderFunction.java @@ -34,6 +34,7 @@ import com.netease.arctic.scan.TableEntriesScan; import com.netease.arctic.table.KeyedTable; import org.apache.flink.configuration.Configuration; +import org.apache.flink.shaded.guava30.com.google.common.collect.Maps; import org.apache.flink.table.data.GenericRowData; import org.apache.flink.table.data.RowData; import org.apache.flink.table.data.StringData; @@ -56,6 +57,7 @@ import java.util.Comparator; import java.util.HashSet; import java.util.List; +import java.util.Map; import java.util.Set; import java.util.concurrent.atomic.AtomicInteger; import java.util.stream.Collectors; @@ -66,7 +68,7 @@ public class TestRowDataReaderFunction extends TestContinuousSplitPlannerImpl { public TestRowDataReaderFunction() { super(new BasicCatalogTestHelper(TableFormat.MIXED_ICEBERG), - new BasicTableTestHelper(true, true)); + new BasicTableTestHelper(true, true)); } @Test @@ -172,7 +174,7 @@ public void testReadNodesUpMoved() throws IOException { } }); - List excepts = exceptsCollection(); + List excepts = expectedCollection(); excepts.addAll(generateRecords()); RowData[] array = excepts.stream().sorted(Comparator.comparing(RowData::toString)) .collect(Collectors.toList()) @@ -254,14 +256,37 @@ protected RowData[] excepts2() { } protected RowData[] excepts() { - List excepts = exceptsCollection(); + List excepts = expectedCollection(); return excepts.stream().sorted(Comparator.comparing(RowData::toString)) .collect(Collectors.toList()) .toArray(new RowData[excepts.size()]); } - protected List exceptsCollection() { + protected RowData[] expectedAfterMOR() { + List expected = expectedCollection(); + return mor(expected).stream().sorted(Comparator.comparing(RowData::toString)).toArray(RowData[]::new); + } + + protected Collection mor(final Collection changelog) { + Map map = Maps.newHashMap(); + + changelog.forEach(rowData -> { + int key = rowData.getInt(0); + RowKind kind = rowData.getRowKind(); + + if ((kind == RowKind.INSERT || kind == RowKind.UPDATE_AFTER) && !map.containsKey(key)) { + rowData.setRowKind(RowKind.INSERT); + map.put(key, rowData); + } else if ((kind == RowKind.DELETE || kind == RowKind.UPDATE_BEFORE)) { + map.remove(key); + } + }); + + return map.values(); + } + + protected List expectedCollection() { List excepts = new ArrayList<>(); excepts.add(GenericRowData.ofKind(RowKind.INSERT, 1, StringData.fromString("john"), ldt.toEpochSecond(ZoneOffset.UTC), TimestampData.fromLocalDateTime(ldt))); excepts.add(GenericRowData.ofKind(RowKind.INSERT, 2, StringData.fromString("lily"), ldt.toEpochSecond(ZoneOffset.UTC), TimestampData.fromLocalDateTime(ldt))); diff --git a/flink/v1.15/flink/src/test/java/com/netease/arctic/flink/read/hybrid/split/TestArcticSplitSerializer.java b/flink/v1.15/flink/src/test/java/com/netease/arctic/flink/read/hybrid/split/TestArcticSplitSerializer.java index 3b6c625786..cf26b6628f 100644 --- a/flink/v1.15/flink/src/test/java/com/netease/arctic/flink/read/hybrid/split/TestArcticSplitSerializer.java +++ b/flink/v1.15/flink/src/test/java/com/netease/arctic/flink/read/hybrid/split/TestArcticSplitSerializer.java @@ -33,9 +33,18 @@ public class TestArcticSplitSerializer extends TestRowDataReaderFunction { @Test public void testSerAndDes() { List arcticSplits = FlinkSplitPlanner.planFullTable(testKeyedTable, new AtomicInteger(0)); + assertSerializedSplitEquals(arcticSplits); + } + + @Test + public void testSerAndDesMoRSplit() { + List arcticSplits = FlinkSplitPlanner.mergeOnReadPlan(testKeyedTable, null, new AtomicInteger(0)); + assertSerializedSplitEquals(arcticSplits); + } + private void assertSerializedSplitEquals(List expected) { ArcticSplitSerializer serializer = new ArcticSplitSerializer(); - List contents = arcticSplits.stream().map(split -> { + List contents = expected.stream().map(split -> { try { return serializer.serialize(split); } catch (IOException e) { @@ -44,7 +53,7 @@ public void testSerAndDes() { } }).collect(Collectors.toList()); - Assert.assertArrayEquals(arcticSplits.toArray(new ArcticSplit[0]), contents.stream().map(data -> { + Assert.assertArrayEquals(expected.toArray(new ArcticSplit[0]), contents.stream().map(data -> { if (data.length == 0) { throw new FlinkRuntimeException("failed cause data length is 0."); } diff --git a/flink/v1.15/flink/src/test/java/com/netease/arctic/flink/table/TestKeyed.java b/flink/v1.15/flink/src/test/java/com/netease/arctic/flink/table/TestKeyed.java index 3dfd53c6be..a2f63d481a 100644 --- a/flink/v1.15/flink/src/test/java/com/netease/arctic/flink/table/TestKeyed.java +++ b/flink/v1.15/flink/src/test/java/com/netease/arctic/flink/table/TestKeyed.java @@ -45,6 +45,7 @@ import org.junit.AfterClass; import org.junit.Assert; import org.junit.Assume; +import org.junit.Before; import org.junit.BeforeClass; import org.junit.ClassRule; import org.junit.Rule; @@ -90,6 +91,8 @@ public class TestKeyed extends FlinkTestBase { private static final String DB = TableTestHelper.TEST_TABLE_ID.getDatabase(); private static final String TABLE = "test_keyed"; + + private String catalog; private String db; private String topic; private Map tableProperties = new HashMap<>(); @@ -97,10 +100,10 @@ public class TestKeyed extends FlinkTestBase { public boolean kafkaLegacyEnable; public TestKeyed( - CatalogTestHelper catalogTestHelper, - TableTestHelper tableTestHelper, - boolean isHive, - boolean kafkaLegacyEnable) { + CatalogTestHelper catalogTestHelper, + TableTestHelper tableTestHelper, + boolean isHive, + boolean kafkaLegacyEnable) { super(catalogTestHelper, tableTestHelper); this.isHive = isHive; this.kafkaLegacyEnable = kafkaLegacyEnable; @@ -109,32 +112,32 @@ public TestKeyed( @Parameterized.Parameters(name = "{0}, {1}, {2}, {3}") public static Collection parameters() { return Arrays.asList( - new Object[][]{ - { - new HiveCatalogTestHelper(TableFormat.MIXED_HIVE, TEST_HMS.getHiveConf()), - new HiveTableTestHelper(true, true), - true, - true - }, - { - new HiveCatalogTestHelper(TableFormat.MIXED_HIVE, TEST_HMS.getHiveConf()), - new HiveTableTestHelper(true, true), - true, - false - }, - { - new BasicCatalogTestHelper(TableFormat.MIXED_ICEBERG), - new BasicTableTestHelper(true, true), - false, - true - }, - { - new BasicCatalogTestHelper(TableFormat.MIXED_ICEBERG), - new BasicTableTestHelper(true, true), - false, - false - } - }); + new Object[][]{ + { + new HiveCatalogTestHelper(TableFormat.MIXED_HIVE, TEST_HMS.getHiveConf()), + new HiveTableTestHelper(true, true), + true, + true + }, + { + new HiveCatalogTestHelper(TableFormat.MIXED_HIVE, TEST_HMS.getHiveConf()), + new HiveTableTestHelper(true, true), + true, + false + }, + { + new BasicCatalogTestHelper(TableFormat.MIXED_ICEBERG), + new BasicTableTestHelper(true, true), + false, + true + }, + { + new BasicCatalogTestHelper(TableFormat.MIXED_ICEBERG), + new BasicTableTestHelper(true, true), + false, + false + } + }); } @BeforeClass @@ -147,6 +150,7 @@ public static void afterClass() throws Exception { FlinkTestBase.shutdown(); } + @Before public void before() throws Exception { if (isHive) { db = HiveTableTestHelper.TEST_DB_NAME; @@ -158,6 +162,11 @@ public void before() throws Exception { super.config(); } + @After + public void after() { + sql("DROP TABLE IF EXISTS arcticCatalog." + db + "." + TABLE); + } + private void prepareLog() { topic = TestUtil.getUtMethodName(testName) + isHive + kafkaLegacyEnable; tableProperties.clear(); @@ -173,11 +182,6 @@ private void prepareLog() { } } - @After - public void after() { - sql("DROP TABLE IF EXISTS arcticCatalog." + db + "." + TABLE); - } - @Test public void testSinkSourceFile() throws IOException { Assume.assumeFalse(kafkaLegacyEnable); @@ -235,9 +239,7 @@ public void testSinkSourceFile() throws IOException { List expected = new LinkedList<>(); expected.add(new Object[]{RowKind.INSERT, 1000004, LocalDateTime.parse("2022-06-17T10:10:11.0"), LocalDateTime.parse("2022-06-17T10:10:11.0").atZone(ZoneId.systemDefault()).toInstant()}); - expected.add(new Object[]{RowKind.UPDATE_BEFORE, 1000021, LocalDateTime.parse("2022-06-17T10:11:11.0"), - LocalDateTime.parse("2022-06-17T10:11:11.0").atZone(ZoneId.systemDefault()).toInstant()}); - expected.add(new Object[]{RowKind.UPDATE_AFTER, 1000021, LocalDateTime.parse("2022-06-17T10:11:11.0"), + expected.add(new Object[]{RowKind.INSERT, 1000021, LocalDateTime.parse("2022-06-17T10:11:11.0"), LocalDateTime.parse("2022-06-17T10:11:11.0").atZone(ZoneId.systemDefault()).toInstant()}); expected.add(new Object[]{RowKind.INSERT, 1000015, LocalDateTime.parse("2022-06-17T10:10:11.0"), LocalDateTime.parse("2022-06-17T10:10:11.0").atZone(ZoneId.systemDefault()).toInstant()}); @@ -453,28 +455,28 @@ public void testSinkSourceFileWithoutSelectPK() throws Exception { List rows = DataUtil.toRows(data); Table input = getTableEnv().fromValues(DataTypes.ROW( - DataTypes.FIELD("id", DataTypes.INT()), - DataTypes.FIELD("name", DataTypes.STRING()), - DataTypes.FIELD("op_time", DataTypes.TIMESTAMP()) - ), - rows + DataTypes.FIELD("id", DataTypes.INT()), + DataTypes.FIELD("name", DataTypes.STRING()), + DataTypes.FIELD("op_time", DataTypes.TIMESTAMP()) + ), + rows ); getTableEnv().createTemporaryView("input", input); sql("CREATE CATALOG arcticCatalog WITH %s", toWithClause(props)); sql("CREATE TABLE IF NOT EXISTS arcticCatalog." + db + "." + TABLE + "(" + - " id INT, name STRING, op_time TIMESTAMP, PRIMARY KEY (id) NOT ENFORCED " + - ") WITH ('connector' = 'arctic')"); + " id INT, name STRING, op_time TIMESTAMP, PRIMARY KEY (id) NOT ENFORCED " + + ") WITH ('connector' = 'arctic')"); sql("insert into arcticCatalog." + db + "." + TABLE + - "/*+ OPTIONS(" + - "'arctic.emit.mode'='file'" + - ")*/" + " select * from input"); + "/*+ OPTIONS(" + + "'arctic.emit.mode'='file'" + + ")*/" + " select * from input"); TableResult result = exec("select name, op_time from arcticCatalog." + db + "." + TABLE + " /*+ OPTIONS(" + - "'streaming'='false'" + - ") */"); + "'streaming'='false'" + + ") */"); LinkedList actual = new LinkedList<>(); try (CloseableIterator iterator = result.collect()) { while (iterator.hasNext()) { @@ -492,7 +494,7 @@ public void testSinkSourceFileWithoutSelectPK() throws Exception { expected.add(new Object[]{"e", LocalDateTime.parse("2022-06-18T10:10:11.0")}); Assert.assertEquals(DataUtil.toRowSet(expected), - new HashSet<>(actual)); + new HashSet<>(actual)); } @Test @@ -536,19 +538,8 @@ public void testFileUpsert() { List expected = new LinkedList<>(); expected.add(new Object[]{RowKind.INSERT, 1000004, "a", LocalDateTime.parse("2022-06-17T10:10:11.0")}); - expected.add(new Object[]{RowKind.DELETE, 1000004, "a", LocalDateTime.parse("2022-06-17T10:10:11.0")}); - expected.add(new Object[]{RowKind.UPDATE_BEFORE, 1000021, "d", LocalDateTime.parse("2022-06-17T10:11:11.0")}); - expected.add(new Object[]{RowKind.UPDATE_AFTER, 1000021, "e", LocalDateTime.parse("2022-06-17T10:11:11.0")}); - expected.add(new Object[]{RowKind.DELETE, 1000021, "e", LocalDateTime.parse("2022-06-17T10:11:11.0")}); + // key = 1000021 locate in two partitions. expected.add(new Object[]{RowKind.INSERT, 1000021, "e", LocalDateTime.parse("2022-06-17T10:11:11.0")}); - expected.add(new Object[]{RowKind.DELETE, 1000015, "e", LocalDateTime.parse("2022-06-17T10:10:11.0")}); - expected.add(new Object[]{RowKind.INSERT, 1000015, "e", LocalDateTime.parse("2022-06-17T10:10:11.0")}); - expected.add(new Object[]{RowKind.DELETE, 1000021, "e", LocalDateTime.parse("2022-06-17T10:10:11.0")}); - expected.add(new Object[]{RowKind.INSERT, 1000021, "e", LocalDateTime.parse("2022-06-17T10:10:11.0")}); - expected.add(new Object[]{RowKind.UPDATE_BEFORE, 1000021, "e", LocalDateTime.parse("2022-06-17T10:10:11.0")}); - expected.add(new Object[]{RowKind.UPDATE_AFTER, 1000021, "d", LocalDateTime.parse("2022-06-17T10:10:11.0")}); - expected.add(new Object[]{RowKind.DELETE, 1000015, "e", LocalDateTime.parse("2022-06-17T10:10:11.0")}); - expected.add(new Object[]{RowKind.DELETE, 1000021, "f", LocalDateTime.parse("2022-06-17T10:10:11.0")}); expected.add(new Object[]{RowKind.INSERT, 1000021, "f", LocalDateTime.parse("2022-06-17T10:10:11.0")}); Assert.assertEquals(DataUtil.toRowSet(expected), new HashSet<>(sql("select * from arcticCatalog." + db + "." + TABLE + " /*+ OPTIONS(" + @@ -601,22 +592,13 @@ public void testFileCDC() { ")*/" + " select * from input"); List expected = new LinkedList<>(); + // upsert is disEnabled, key=1000021 locate in two diff partitions. expected.add(new Object[]{RowKind.INSERT, 1000004, "a", LocalDateTime.parse("2022-06-17T10:10:11.0")}); - expected.add(new Object[]{RowKind.UPDATE_BEFORE, 1000021, "d", LocalDateTime.parse("2022-06-17T10:11:11.0")}); - expected.add(new Object[]{RowKind.UPDATE_AFTER, 1000021, "e", LocalDateTime.parse("2022-06-17T10:11:11.0")}); expected.add(new Object[]{RowKind.INSERT, 1000021, "e", LocalDateTime.parse("2022-06-17T10:11:11.0")}); - expected.add(new Object[]{RowKind.INSERT, 1000015, "e", LocalDateTime.parse("2022-06-17T10:10:11.0")}); - expected.add(new Object[]{RowKind.INSERT, 1000021, "e", LocalDateTime.parse("2022-06-17T10:10:11.0")}); - expected.add(new Object[]{RowKind.UPDATE_BEFORE, 1000021, "e", LocalDateTime.parse("2022-06-17T10:10:11.0")}); - expected.add(new Object[]{RowKind.UPDATE_AFTER, 1000021, "d", LocalDateTime.parse("2022-06-17T10:10:11.0")}); - expected.add(new Object[]{RowKind.DELETE, 1000015, "e", LocalDateTime.parse("2022-06-17T10:10:11.0")}); + expected.add(new Object[]{RowKind.INSERT, 1000021, "d", LocalDateTime.parse("2022-06-17T10:10:11.0")}); expected.add(new Object[]{RowKind.INSERT, 1000021, "f", LocalDateTime.parse("2022-06-17T10:10:11.0")}); - expected.add(new Object[]{RowKind.INSERT, 1000031, "g", LocalDateTime.parse("2022-06-17T10:10:11.0")}); - expected.add(new Object[]{RowKind.INSERT, 1000032, "h", LocalDateTime.parse("2022-06-17T10:10:11.0")}); - expected.add(new Object[]{RowKind.DELETE, 1000031, "g", LocalDateTime.parse("2022-06-17T10:10:11.0")}); - expected.add(new Object[]{RowKind.UPDATE_BEFORE, 1000032, "h", LocalDateTime.parse("2022-06-17T10:10:11.0")}); expected.add(new Object[]{RowKind.INSERT, 1000031, "f", LocalDateTime.parse("2022-06-17T10:10:11.0")}); - expected.add(new Object[]{RowKind.UPDATE_AFTER, 1000032, "e", LocalDateTime.parse("2022-06-17T10:10:11.0")}); + expected.add(new Object[]{RowKind.INSERT, 1000032, "e", LocalDateTime.parse("2022-06-17T10:10:11.0")}); Assert.assertEquals(DataUtil.toRowSet(expected), new HashSet<>(sql("select * from arcticCatalog." + db + "." + TABLE + " /*+ OPTIONS(" + "'streaming'='false'" + @@ -668,13 +650,7 @@ public void testFileUpsertWithSamePrimaryKey() throws Exception { LinkedList expected = new LinkedList<>(); - expected.add(new Object[]{RowKind.DELETE, 1000004, "a", LocalDateTime.parse("2022-06-17T10:10:11.0")}); - expected.add(new Object[]{RowKind.INSERT, 1000004, "a", LocalDateTime.parse("2022-06-17T10:10:11.0")}); - expected.add(new Object[]{RowKind.DELETE, 1000004, "b", LocalDateTime.parse("2022-06-17T10:10:11.0")}); expected.add(new Object[]{RowKind.INSERT, 1000004, "b", LocalDateTime.parse("2022-06-17T10:10:11.0")}); - expected.add(new Object[]{RowKind.DELETE, 1000011, "e", LocalDateTime.parse("2022-06-17T10:10:11.0")}); - expected.add(new Object[]{RowKind.INSERT, 1000011, "e", LocalDateTime.parse("2022-06-17T10:10:11.0")}); - expected.add(new Object[]{RowKind.DELETE, 1000011, "f", LocalDateTime.parse("2022-06-17T10:10:11.0")}); expected.add(new Object[]{RowKind.INSERT, 1000011, "f", LocalDateTime.parse("2022-06-17T10:10:11.0")}); Map> actualMap = DataUtil.groupByPrimaryKey(actual, 0);