luoyuxia commented on code in PR #1984: URL: https://github.com/apache/fluss/pull/1984#discussion_r2618358550
########## fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/tiering/TieringFailoverITCase.java: ########## @@ -0,0 +1,346 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.fluss.flink.tiering; + +import org.apache.fluss.config.Configuration; +import org.apache.fluss.flink.tiering.committer.CommittableMessageTypeInfo; +import org.apache.fluss.flink.tiering.committer.TieringCommitOperatorFactory; +import org.apache.fluss.flink.tiering.source.TableBucketWriteResultTypeInfo; +import org.apache.fluss.flink.tiering.source.TieringSource; +import org.apache.fluss.lake.values.ValuesLake; +import org.apache.fluss.lake.values.tiering.ValuesLakeTieringFactory; +import org.apache.fluss.lake.writer.LakeTieringFactory; +import org.apache.fluss.metadata.Schema; +import org.apache.fluss.metadata.TableBucket; +import org.apache.fluss.metadata.TablePath; +import org.apache.fluss.row.BinaryString; +import org.apache.fluss.row.Decimal; +import org.apache.fluss.row.InternalRow; +import org.apache.fluss.row.TimestampLtz; +import org.apache.fluss.row.TimestampNtz; +import org.apache.fluss.types.DataTypes; +import org.apache.fluss.utils.TypeUtils; + +import org.apache.flink.api.common.eventtime.WatermarkStrategy; +import org.apache.flink.configuration.PipelineOptions; +import org.apache.flink.core.execution.JobClient; +import org.apache.flink.streaming.api.datastream.DataStreamSource; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.functions.sink.v2.DiscardingSink; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; + +import java.time.Duration; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; + +import static org.apache.fluss.flink.tiering.source.TieringSource.TIERING_SOURCE_TRANSFORMATION_UID; +import static org.apache.fluss.flink.tiering.source.TieringSourceOptions.POLL_TIERING_TABLE_INTERVAL; +import static org.apache.fluss.lake.committer.BucketOffset.FLUSS_LAKE_SNAP_BUCKET_OFFSET_PROPERTY; +import static org.apache.fluss.testutils.DataTestUtils.row; +import static org.assertj.core.api.Assertions.assertThat; + +/** Test tiering failover. */ +class TieringFailoverITCase extends FlinkValuesTieringTestBase { + protected static final String DEFAULT_DB = "fluss"; + + private static StreamExecutionEnvironment execEnv; + + private static final Schema pkSchema = Review Comment: we don't need so complex schema, two columns, one int, one string should be enough to verify failover ########## fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/source/enumerator/TieringSourceEnumerator.java: ########## @@ -248,6 +297,9 @@ private void generateAndAssignSplits( } private void assignSplits() { + if (isFailOvering) { Review Comment: add comment like: ``` we don't assign splits during failovering ``` ########## fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/tiering/source/enumerator/TieringSourceEnumeratorTest.java: ########## @@ -654,37 +636,41 @@ void testHandleFailOverEvent() throws Throwable { createTable(tablePath2, DEFAULT_LOG_TABLE_DESCRIPTOR); appendRow(tablePath2, DEFAULT_LOG_TABLE_DESCRIPTOR, 0, 10); - int numSubtasks = 1; - try (MockSplitEnumeratorContext<TieringSplit> context = - new MockSplitEnumeratorContext<>(numSubtasks)) { + try (FlussMockSplitEnumeratorContext<TieringSplit> context = + new FlussMockSplitEnumeratorContext<>(numSubtasks)) { TieringSourceEnumerator enumerator = new TieringSourceEnumerator(flussConf, context, 500); enumerator.start(); assertThat(context.getSplitsAssignmentSequence()).isEmpty(); - // register one reader - int subtaskId = 0; - registerReader(context, enumerator, subtaskId, "localhost-" + subtaskId); - - // handle split request - enumerator.handleSplitRequest(subtaskId, "localhost-" + subtaskId); + // register readers and handle split requests for attempt 0 + registerReaderAndHandleSplitRequests(context, enumerator, numSubtasks, 0); // should get one tiering split, and the split is for tablePath1 - verifyTieringSplitAssignment(context, 1, tablePath1); + verifyTieringSplitAssignment(context, numSubtasks, tablePath1); // clean assignment context.getSplitsAssignmentSequence().clear(); - // enumerator handle TieringFailOverEvent, which will mark current tiering tablePath1 as - // fail, and all pending splits should be clear - enumerator.handleSourceEvent(subtaskId, new TieringFailOverEvent()); + // first reader failover: Enumerator marks tablePath1 as failed and clears its splits Review Comment: ```suggestion // readers failover: Enumerator marks tablePath1 as failed and clears its splits ``` `first reader failover` make me feel it like the first reader fail. ########## fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/source/enumerator/TieringSourceEnumerator.java: ########## @@ -179,11 +181,53 @@ public void addSplitsBack(List<TieringSplit> splits, int subtaskId) { @Override public void addReader(int subtaskId) { LOG.info("Adding reader: {} to Tiering Source enumerator.", subtaskId); - if (context.registeredReaders().containsKey(subtaskId)) { + Map<Integer, ReaderInfo> readerByAttempt = + context.registeredReadersOfAttempts().get(subtaskId); + if (readerByAttempt != null && !readerByAttempt.isEmpty()) { readersAwaitingSplit.add(subtaskId); + int maxAttempt = max(readerByAttempt.keySet()); + if (maxAttempt >= 1) { + if (isFailOvering) { + LOG.warn( + "Subtask {} (max attempt {}) registered during ongoing failover.", + subtaskId, + maxAttempt); + } else { + LOG.warn( + "Detected failover: subtask {} has max attempt {} > 0. Triggering global failover handling.", + subtaskId, + maxAttempt); + // should be failover + isFailOvering = true; + handleSourceReaderFailOver(); + } + + // if registered readers equal to current parallelism, check whether all registered + // readers have same max attempt + if (context.registeredReadersOfAttempts().size() == context.currentParallelism()) { + // Check if all readers have the same max attempt number + Set<Integer> maxAttempts = + context.registeredReadersOfAttempts().values().stream() + .map(_readerByAttempt -> max(_readerByAttempt.keySet())) + .collect(Collectors.toSet()); + int globalMaxAttempt = max(maxAttempts); + if (maxAttempts.size() == 1 && globalMaxAttempt >= 1) { + LOG.warn( Review Comment: ```suggestion LOG.info( ``` ########## fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/lake/values/ValuesLake.java: ########## @@ -0,0 +1,312 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.fluss.lake.values; + +import org.apache.fluss.metadata.Schema; +import org.apache.fluss.record.LogRecord; +import org.apache.fluss.row.InternalRow; +import org.apache.fluss.utils.MapUtils; +import org.apache.fluss.utils.types.Tuple2; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Comparator; +import java.util.HashMap; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.atomic.AtomicInteger; + +import static org.apache.fluss.utils.Preconditions.checkArgument; +import static org.apache.fluss.utils.Preconditions.checkNotNull; + +/** + * In-memory implementation of a lake storage for testing purposes. + * + * <p>Provides utilities for managing tables, writing records, committing stages, and retrieving + * results in a test environment. + */ +public class ValuesLake { Review Comment: I found some methods are not used. Could you please remove them? ########## fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/tiering/TieringFailoverITCase.java: ########## @@ -0,0 +1,346 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.fluss.flink.tiering; + +import org.apache.fluss.config.Configuration; +import org.apache.fluss.flink.tiering.committer.CommittableMessageTypeInfo; +import org.apache.fluss.flink.tiering.committer.TieringCommitOperatorFactory; +import org.apache.fluss.flink.tiering.source.TableBucketWriteResultTypeInfo; +import org.apache.fluss.flink.tiering.source.TieringSource; +import org.apache.fluss.lake.values.ValuesLake; +import org.apache.fluss.lake.values.tiering.ValuesLakeTieringFactory; +import org.apache.fluss.lake.writer.LakeTieringFactory; +import org.apache.fluss.metadata.Schema; +import org.apache.fluss.metadata.TableBucket; +import org.apache.fluss.metadata.TablePath; +import org.apache.fluss.row.BinaryString; +import org.apache.fluss.row.Decimal; +import org.apache.fluss.row.InternalRow; +import org.apache.fluss.row.TimestampLtz; +import org.apache.fluss.row.TimestampNtz; +import org.apache.fluss.types.DataTypes; +import org.apache.fluss.utils.TypeUtils; + +import org.apache.flink.api.common.eventtime.WatermarkStrategy; +import org.apache.flink.configuration.PipelineOptions; +import org.apache.flink.core.execution.JobClient; +import org.apache.flink.streaming.api.datastream.DataStreamSource; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.functions.sink.v2.DiscardingSink; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; + +import java.time.Duration; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; + +import static org.apache.fluss.flink.tiering.source.TieringSource.TIERING_SOURCE_TRANSFORMATION_UID; +import static org.apache.fluss.flink.tiering.source.TieringSourceOptions.POLL_TIERING_TABLE_INTERVAL; +import static org.apache.fluss.lake.committer.BucketOffset.FLUSS_LAKE_SNAP_BUCKET_OFFSET_PROPERTY; +import static org.apache.fluss.testutils.DataTestUtils.row; +import static org.assertj.core.api.Assertions.assertThat; + +/** Test tiering failover. */ +class TieringFailoverITCase extends FlinkValuesTieringTestBase { + protected static final String DEFAULT_DB = "fluss"; + + private static StreamExecutionEnvironment execEnv; + + private static final Schema pkSchema = + Schema.newBuilder() + .column("f_boolean", DataTypes.BOOLEAN()) + .column("f_byte", DataTypes.TINYINT()) + .column("f_short", DataTypes.SMALLINT()) + .column("f_int", DataTypes.INT()) + .column("f_long", DataTypes.BIGINT()) + .column("f_float", DataTypes.FLOAT()) + .column("f_double", DataTypes.DOUBLE()) + .column("f_string", DataTypes.STRING()) + .column("f_decimal1", DataTypes.DECIMAL(5, 2)) + .column("f_decimal2", DataTypes.DECIMAL(20, 0)) + .column("f_timestamp_ltz1", DataTypes.TIMESTAMP_LTZ(3)) + .column("f_timestamp_ltz2", DataTypes.TIMESTAMP_LTZ(6)) + .column("f_timestamp_ntz1", DataTypes.TIMESTAMP(3)) + .column("f_timestamp_ntz2", DataTypes.TIMESTAMP(6)) + .column("f_binary", DataTypes.BINARY(4)) + .column("f_date", DataTypes.DATE()) + .column("f_time", DataTypes.TIME()) + .column("f_char", DataTypes.CHAR(3)) + .column("f_bytes", DataTypes.BYTES()) + .primaryKey("f_string") + .build(); + + @BeforeAll + protected static void beforeAll() { + FlinkValuesTieringTestBase.beforeAll(); + execEnv = StreamExecutionEnvironment.getExecutionEnvironment(); + execEnv.setParallelism(2); + execEnv.enableCheckpointing(1000); + } + + @Test + void testTiering() throws Exception { + // create a pk table, write some records and wait until snapshot finished + TablePath t1 = TablePath.of(DEFAULT_DB, "pkTable"); + long t1Id = createPkTable(t1, 1, false, pkSchema); + TableBucket t1Bucket = new TableBucket(t1Id, 0); + // write records + List<InternalRow> rows = + Arrays.asList( + row( + true, + (byte) 100, + (short) 200, + 1, + 1 + 400L, + 500.1f, + 600.0d, + "v1", + Decimal.fromUnscaledLong(900, 5, 2), + Decimal.fromBigDecimal(new java.math.BigDecimal(1000), 20, 0), + TimestampLtz.fromEpochMillis(1698235273400L), + TimestampLtz.fromEpochMillis(1698235273400L, 7000), + TimestampNtz.fromMillis(1698235273501L), + TimestampNtz.fromMillis(1698235273501L, 8000), + new byte[] {5, 6, 7, 8}, + TypeUtils.castFromString("2023-10-25", DataTypes.DATE()), + TypeUtils.castFromString("09:30:00.0", DataTypes.TIME()), + BinaryString.fromString("abc"), + new byte[] {1, 2, 3, 4, 5, 6, 7, 8, 9, 10}), + row( + true, + (byte) 100, + (short) 200, + 2, + 2 + 400L, + 500.1f, + 600.0d, + "v2", + Decimal.fromUnscaledLong(900, 5, 2), + Decimal.fromBigDecimal(new java.math.BigDecimal(1000), 20, 0), + TimestampLtz.fromEpochMillis(1698235273400L), + TimestampLtz.fromEpochMillis(1698235273400L, 7000), + TimestampNtz.fromMillis(1698235273501L), + TimestampNtz.fromMillis(1698235273501L, 8000), + new byte[] {5, 6, 7, 8}, + TypeUtils.castFromString("2023-10-25", DataTypes.DATE()), + TypeUtils.castFromString("09:30:00.0", DataTypes.TIME()), + BinaryString.fromString("abc"), + new byte[] {1, 2, 3, 4, 5, 6, 7, 8, 9, 10}), + row( + true, + (byte) 100, + (short) 200, + 3, + 3 + 400L, + 500.1f, + 600.0d, + "v3", + Decimal.fromUnscaledLong(900, 5, 2), + Decimal.fromBigDecimal(new java.math.BigDecimal(1000), 20, 0), + TimestampLtz.fromEpochMillis(1698235273400L), + TimestampLtz.fromEpochMillis(1698235273400L, 7000), + TimestampNtz.fromMillis(1698235273501L), + TimestampNtz.fromMillis(1698235273501L, 8000), + new byte[] {5, 6, 7, 8}, + TypeUtils.castFromString("2023-10-25", DataTypes.DATE()), + TypeUtils.castFromString("09:30:00.0", DataTypes.TIME()), + BinaryString.fromString("abc"), + new byte[] {1, 2, 3, 4, 5, 6, 7, 8, 9, 10})); + List<InternalRow> expectedRows = new ArrayList<>(rows); + writeRows(t1, rows); + waitUntilSnapshot(t1Id, 1, 0); + + // fail the first write to the pk table + ValuesLake.failWhen(t1.toString()).failWriteOnce(); + + // then start tiering job + JobClient jobClient = buildTieringJob(execEnv); + try { + // check the status of replica after synced + assertReplicaStatus(t1Bucket, 3); + + checkDataInValuesPrimaryKeyTable(t1, rows); + // check snapshot property in values lake Review Comment: don't need to check this since we don't care ########## fluss-common/src/main/java/org/apache/fluss/metadata/DataLakeFormat.java: ########## @@ -21,7 +21,8 @@ public enum DataLakeFormat { PAIMON("paimon"), LANCE("lance"), - ICEBERG("iceberg"); + ICEBERG("iceberg"), + VALUES("values"); Review Comment: I'm still concerned on introduing a format just for testing. Actaully, we don't need to introduce a new one format. We already have a `org.apache.fluss.server.lakehouse.TestingPaimonStoragePlugin` for test purpose. I think we can resue `TestingPaimonStoragePlugin`? fluss-flink module introduce fluss-server test module which contains `TestingPaimonStoragePlugin` ########## fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/lake/values/ValuesLake.java: ########## @@ -0,0 +1,312 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.fluss.lake.values; + +import org.apache.fluss.metadata.Schema; +import org.apache.fluss.record.LogRecord; +import org.apache.fluss.row.InternalRow; +import org.apache.fluss.utils.MapUtils; +import org.apache.fluss.utils.types.Tuple2; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Comparator; +import java.util.HashMap; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.atomic.AtomicInteger; + +import static org.apache.fluss.utils.Preconditions.checkArgument; +import static org.apache.fluss.utils.Preconditions.checkNotNull; + +/** + * In-memory implementation of a lake storage for testing purposes. + * + * <p>Provides utilities for managing tables, writing records, committing stages, and retrieving + * results in a test environment. + */ +public class ValuesLake { + private static final Logger LOG = LoggerFactory.getLogger(ValuesLake.class); + + private static final Map<String, ValuesTable> globalTables = MapUtils.newConcurrentHashMap(); + private static final Map<String, TableFailureController> FAILURE_CONTROLLERS = + MapUtils.newConcurrentHashMap(); + + public static TableFailureController failWhen(String tableId) { + return FAILURE_CONTROLLERS.computeIfAbsent(tableId, k -> new TableFailureController()); + } + + public static void clearAllFailureControls() { + FAILURE_CONTROLLERS.clear(); + } + + public static Schema getTableSchema(String tableId) { + return globalTables.get(tableId).schema; + } + + public static List<InternalRow> getResults(String tableId) { + ValuesTable table = globalTables.get(tableId); + checkNotNull(table, tableId + " does not exist"); + return table.getResult(); + } + + public static void writeRecord(String tableId, String stageId, LogRecord record) + throws IOException { + ValuesTable table = globalTables.get(tableId); + checkNotNull(table, tableId + " does not exist"); + TableFailureController controller = FAILURE_CONTROLLERS.get(tableId); + if (controller != null) { + controller.checkWriteShouldFail(tableId); + } + table.writeRecord(stageId, record); + LOG.info("Write record to stage {}: {}", stageId, record); + } + + public static long commit( + String tableId, List<String> stageIds, Map<String, String> snapshotProperties) + throws IOException { + ValuesTable table = globalTables.get(tableId); + checkNotNull(table, "commit stage %s failed, table %s does not exist", stageIds, tableId); + table.commit(stageIds, snapshotProperties); + LOG.info("Commit table {} stage {}", tableId, stageIds); + return table.getSnapshotId(); + } + + public static void abort(String tableId, List<String> stageIds) { + ValuesTable table = globalTables.get(tableId); + checkNotNull( + table, "abort stage record %s failed, table %s does not exist", stageIds, tableId); + table.abort(stageIds); + LOG.info("Abort table {} stage {}", tableId, stageIds); + } + + public static ValuesTable getTable(String tableId) { + return globalTables.get(tableId); + } + + public static void createTable(String tableId, Schema schema) { + if (!globalTables.containsKey(tableId)) { + globalTables.put(tableId, new ValuesTable(schema)); + ValuesTable table = globalTables.get(tableId); + checkNotNull(table, "create table %s failed", tableId); + } + } + + public static void dropTable(String tableId) { + globalTables.remove(tableId); + } + + public static void truncateTable(String tableId) { + ValuesTable table = globalTables.get(tableId); + checkNotNull(table, "truncate table %s failed", tableId); + table.truncateTable(); + } + + public static void clear() { + globalTables.clear(); + } + + /** maintain the columns, primaryKeys and records of a specific table in memory. */ + public static class ValuesTable { + + private final Object lock; + + // [primaryKeys, rowValue] + private final Map<String, InternalRow> records; + private final List<InternalRow> logRecords; + private final Map<String, List<Tuple2<String, LogRecord>>> stageRecords; + private final Map<String, List<LogRecord>> stageLogRecords; + private final Schema schema; + + private final List<Schema.Column> columns; + + private final List<String> primaryKeys; + + // indexes of primaryKeys in columns + private final List<Integer> primaryKeyIndexes; + + private long snapshotId = 0L; + + private final Map<Long, Map<String, String>> snapshotProperties = new HashMap<>(); Review Comment: we can remove `snapshotProperties` ########## fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/lake/values/ValuesLake.java: ########## @@ -0,0 +1,312 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.fluss.lake.values; + +import org.apache.fluss.metadata.Schema; +import org.apache.fluss.record.LogRecord; +import org.apache.fluss.row.InternalRow; +import org.apache.fluss.utils.MapUtils; +import org.apache.fluss.utils.types.Tuple2; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Comparator; +import java.util.HashMap; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.atomic.AtomicInteger; + +import static org.apache.fluss.utils.Preconditions.checkArgument; +import static org.apache.fluss.utils.Preconditions.checkNotNull; + +/** + * In-memory implementation of a lake storage for testing purposes. + * + * <p>Provides utilities for managing tables, writing records, committing stages, and retrieving + * results in a test environment. + */ +public class ValuesLake { + private static final Logger LOG = LoggerFactory.getLogger(ValuesLake.class); + + private static final Map<String, ValuesTable> globalTables = MapUtils.newConcurrentHashMap(); + private static final Map<String, TableFailureController> FAILURE_CONTROLLERS = + MapUtils.newConcurrentHashMap(); + + public static TableFailureController failWhen(String tableId) { + return FAILURE_CONTROLLERS.computeIfAbsent(tableId, k -> new TableFailureController()); + } + + public static void clearAllFailureControls() { + FAILURE_CONTROLLERS.clear(); + } + + public static Schema getTableSchema(String tableId) { + return globalTables.get(tableId).schema; + } + + public static List<InternalRow> getResults(String tableId) { + ValuesTable table = globalTables.get(tableId); + checkNotNull(table, tableId + " does not exist"); + return table.getResult(); + } + + public static void writeRecord(String tableId, String stageId, LogRecord record) + throws IOException { + ValuesTable table = globalTables.get(tableId); + checkNotNull(table, tableId + " does not exist"); + TableFailureController controller = FAILURE_CONTROLLERS.get(tableId); + if (controller != null) { + controller.checkWriteShouldFail(tableId); + } + table.writeRecord(stageId, record); + LOG.info("Write record to stage {}: {}", stageId, record); + } + + public static long commit( + String tableId, List<String> stageIds, Map<String, String> snapshotProperties) + throws IOException { + ValuesTable table = globalTables.get(tableId); + checkNotNull(table, "commit stage %s failed, table %s does not exist", stageIds, tableId); + table.commit(stageIds, snapshotProperties); + LOG.info("Commit table {} stage {}", tableId, stageIds); + return table.getSnapshotId(); + } + + public static void abort(String tableId, List<String> stageIds) { + ValuesTable table = globalTables.get(tableId); + checkNotNull( + table, "abort stage record %s failed, table %s does not exist", stageIds, tableId); + table.abort(stageIds); + LOG.info("Abort table {} stage {}", tableId, stageIds); + } + + public static ValuesTable getTable(String tableId) { + return globalTables.get(tableId); + } + + public static void createTable(String tableId, Schema schema) { + if (!globalTables.containsKey(tableId)) { + globalTables.put(tableId, new ValuesTable(schema)); + ValuesTable table = globalTables.get(tableId); + checkNotNull(table, "create table %s failed", tableId); + } + } + + public static void dropTable(String tableId) { + globalTables.remove(tableId); + } + + public static void truncateTable(String tableId) { + ValuesTable table = globalTables.get(tableId); + checkNotNull(table, "truncate table %s failed", tableId); + table.truncateTable(); + } + + public static void clear() { + globalTables.clear(); + } + + /** maintain the columns, primaryKeys and records of a specific table in memory. */ + public static class ValuesTable { Review Comment: we can minmum the code of `ValuesTable`. Only support append-only table, it's enough to verify the failover case. ########## fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/tiering/TieringFailoverITCase.java: ########## @@ -0,0 +1,346 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.fluss.flink.tiering; + +import org.apache.fluss.config.Configuration; +import org.apache.fluss.flink.tiering.committer.CommittableMessageTypeInfo; +import org.apache.fluss.flink.tiering.committer.TieringCommitOperatorFactory; +import org.apache.fluss.flink.tiering.source.TableBucketWriteResultTypeInfo; +import org.apache.fluss.flink.tiering.source.TieringSource; +import org.apache.fluss.lake.values.ValuesLake; +import org.apache.fluss.lake.values.tiering.ValuesLakeTieringFactory; +import org.apache.fluss.lake.writer.LakeTieringFactory; +import org.apache.fluss.metadata.Schema; +import org.apache.fluss.metadata.TableBucket; +import org.apache.fluss.metadata.TablePath; +import org.apache.fluss.row.BinaryString; +import org.apache.fluss.row.Decimal; +import org.apache.fluss.row.InternalRow; +import org.apache.fluss.row.TimestampLtz; +import org.apache.fluss.row.TimestampNtz; +import org.apache.fluss.types.DataTypes; +import org.apache.fluss.utils.TypeUtils; + +import org.apache.flink.api.common.eventtime.WatermarkStrategy; +import org.apache.flink.configuration.PipelineOptions; +import org.apache.flink.core.execution.JobClient; +import org.apache.flink.streaming.api.datastream.DataStreamSource; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.functions.sink.v2.DiscardingSink; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; + +import java.time.Duration; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; + +import static org.apache.fluss.flink.tiering.source.TieringSource.TIERING_SOURCE_TRANSFORMATION_UID; +import static org.apache.fluss.flink.tiering.source.TieringSourceOptions.POLL_TIERING_TABLE_INTERVAL; +import static org.apache.fluss.lake.committer.BucketOffset.FLUSS_LAKE_SNAP_BUCKET_OFFSET_PROPERTY; +import static org.apache.fluss.testutils.DataTestUtils.row; +import static org.assertj.core.api.Assertions.assertThat; + +/** Test tiering failover. */ +class TieringFailoverITCase extends FlinkValuesTieringTestBase { + protected static final String DEFAULT_DB = "fluss"; + + private static StreamExecutionEnvironment execEnv; + + private static final Schema pkSchema = + Schema.newBuilder() + .column("f_boolean", DataTypes.BOOLEAN()) + .column("f_byte", DataTypes.TINYINT()) + .column("f_short", DataTypes.SMALLINT()) + .column("f_int", DataTypes.INT()) + .column("f_long", DataTypes.BIGINT()) + .column("f_float", DataTypes.FLOAT()) + .column("f_double", DataTypes.DOUBLE()) + .column("f_string", DataTypes.STRING()) + .column("f_decimal1", DataTypes.DECIMAL(5, 2)) + .column("f_decimal2", DataTypes.DECIMAL(20, 0)) + .column("f_timestamp_ltz1", DataTypes.TIMESTAMP_LTZ(3)) + .column("f_timestamp_ltz2", DataTypes.TIMESTAMP_LTZ(6)) + .column("f_timestamp_ntz1", DataTypes.TIMESTAMP(3)) + .column("f_timestamp_ntz2", DataTypes.TIMESTAMP(6)) + .column("f_binary", DataTypes.BINARY(4)) + .column("f_date", DataTypes.DATE()) + .column("f_time", DataTypes.TIME()) + .column("f_char", DataTypes.CHAR(3)) + .column("f_bytes", DataTypes.BYTES()) + .primaryKey("f_string") + .build(); + + @BeforeAll + protected static void beforeAll() { + FlinkValuesTieringTestBase.beforeAll(); + execEnv = StreamExecutionEnvironment.getExecutionEnvironment(); + execEnv.setParallelism(2); + execEnv.enableCheckpointing(1000); + } + + @Test + void testTiering() throws Exception { + // create a pk table, write some records and wait until snapshot finished + TablePath t1 = TablePath.of(DEFAULT_DB, "pkTable"); + long t1Id = createPkTable(t1, 1, false, pkSchema); + TableBucket t1Bucket = new TableBucket(t1Id, 0); + // write records + List<InternalRow> rows = + Arrays.asList( + row( + true, + (byte) 100, + (short) 200, + 1, + 1 + 400L, + 500.1f, + 600.0d, + "v1", + Decimal.fromUnscaledLong(900, 5, 2), + Decimal.fromBigDecimal(new java.math.BigDecimal(1000), 20, 0), + TimestampLtz.fromEpochMillis(1698235273400L), + TimestampLtz.fromEpochMillis(1698235273400L, 7000), + TimestampNtz.fromMillis(1698235273501L), + TimestampNtz.fromMillis(1698235273501L, 8000), + new byte[] {5, 6, 7, 8}, + TypeUtils.castFromString("2023-10-25", DataTypes.DATE()), + TypeUtils.castFromString("09:30:00.0", DataTypes.TIME()), + BinaryString.fromString("abc"), + new byte[] {1, 2, 3, 4, 5, 6, 7, 8, 9, 10}), + row( + true, + (byte) 100, + (short) 200, + 2, + 2 + 400L, + 500.1f, + 600.0d, + "v2", + Decimal.fromUnscaledLong(900, 5, 2), + Decimal.fromBigDecimal(new java.math.BigDecimal(1000), 20, 0), + TimestampLtz.fromEpochMillis(1698235273400L), + TimestampLtz.fromEpochMillis(1698235273400L, 7000), + TimestampNtz.fromMillis(1698235273501L), + TimestampNtz.fromMillis(1698235273501L, 8000), + new byte[] {5, 6, 7, 8}, + TypeUtils.castFromString("2023-10-25", DataTypes.DATE()), + TypeUtils.castFromString("09:30:00.0", DataTypes.TIME()), + BinaryString.fromString("abc"), + new byte[] {1, 2, 3, 4, 5, 6, 7, 8, 9, 10}), + row( + true, + (byte) 100, + (short) 200, + 3, + 3 + 400L, + 500.1f, + 600.0d, + "v3", + Decimal.fromUnscaledLong(900, 5, 2), + Decimal.fromBigDecimal(new java.math.BigDecimal(1000), 20, 0), + TimestampLtz.fromEpochMillis(1698235273400L), + TimestampLtz.fromEpochMillis(1698235273400L, 7000), + TimestampNtz.fromMillis(1698235273501L), + TimestampNtz.fromMillis(1698235273501L, 8000), + new byte[] {5, 6, 7, 8}, + TypeUtils.castFromString("2023-10-25", DataTypes.DATE()), + TypeUtils.castFromString("09:30:00.0", DataTypes.TIME()), + BinaryString.fromString("abc"), + new byte[] {1, 2, 3, 4, 5, 6, 7, 8, 9, 10})); + List<InternalRow> expectedRows = new ArrayList<>(rows); + writeRows(t1, rows); + waitUntilSnapshot(t1Id, 1, 0); + + // fail the first write to the pk table + ValuesLake.failWhen(t1.toString()).failWriteOnce(); + + // then start tiering job + JobClient jobClient = buildTieringJob(execEnv); + try { + // check the status of replica after synced + assertReplicaStatus(t1Bucket, 3); + + checkDataInValuesPrimaryKeyTable(t1, rows); + // check snapshot property in values lake + Map<String, String> properties = + new HashMap<String, String>() { + { + put( + FLUSS_LAKE_SNAP_BUCKET_OFFSET_PROPERTY, + "[{\"bucket\":0,\"offset\":3}]"); + } + }; + checkSnapshotPropertyInValues(t1, properties); + + // then write data to the pk tables + // write records + rows = + Arrays.asList( + row( + true, + (byte) 100, + (short) 200, + 1, + 1 + 400L, + 500.1f, + 600.0d, + "v111", + Decimal.fromUnscaledLong(900, 5, 2), + Decimal.fromBigDecimal(new java.math.BigDecimal(1000), 20, 0), + TimestampLtz.fromEpochMillis(1698235273400L), + TimestampLtz.fromEpochMillis(1698235273400L, 7000), + TimestampNtz.fromMillis(1698235273501L), + TimestampNtz.fromMillis(1698235273501L, 8000), + new byte[] {5, 6, 7, 8}, + TypeUtils.castFromString("2023-10-25", DataTypes.DATE()), + TypeUtils.castFromString("09:30:00.0", DataTypes.TIME()), + BinaryString.fromString("abc"), + new byte[] {1, 2, 3, 4, 5, 6, 7, 8, 9, 10}), + row( + true, + (byte) 100, + (short) 200, + 2, + 2 + 400L, + 500.1f, + 600.0d, + "v222", + Decimal.fromUnscaledLong(900, 5, 2), + Decimal.fromBigDecimal(new java.math.BigDecimal(1000), 20, 0), + TimestampLtz.fromEpochMillis(1698235273400L), + TimestampLtz.fromEpochMillis(1698235273400L, 7000), + TimestampNtz.fromMillis(1698235273501L), + TimestampNtz.fromMillis(1698235273501L, 8000), + new byte[] {5, 6, 7, 8}, + TypeUtils.castFromString("2023-10-25", DataTypes.DATE()), + TypeUtils.castFromString("09:30:00.0", DataTypes.TIME()), + BinaryString.fromString("abc"), + new byte[] {1, 2, 3, 4, 5, 6, 7, 8, 9, 10}), + row( + true, + (byte) 100, + (short) 200, + 3, + 3 + 400L, + 500.1f, + 600.0d, + "v333", + Decimal.fromUnscaledLong(900, 5, 2), + Decimal.fromBigDecimal(new java.math.BigDecimal(1000), 20, 0), + TimestampLtz.fromEpochMillis(1698235273400L), + TimestampLtz.fromEpochMillis(1698235273400L, 7000), + TimestampNtz.fromMillis(1698235273501L), + TimestampNtz.fromMillis(1698235273501L, 8000), + new byte[] {5, 6, 7, 8}, + TypeUtils.castFromString("2023-10-25", DataTypes.DATE()), + TypeUtils.castFromString("09:30:00.0", DataTypes.TIME()), + BinaryString.fromString("abc"), + new byte[] {1, 2, 3, 4, 5, 6, 7, 8, 9, 10})); + expectedRows.addAll(rows); + // write records + writeRows(t1, rows); + + // check the status of replica of t1 after synced + // not check start offset since we won't + // update start log offset for primary key table + assertReplicaStatus(t1Bucket, expectedRows.size()); + + checkDataInValuesPrimaryKeyTable(t1, expectedRows); + } finally { + jobClient.cancel().get(); + } + } + + protected JobClient buildTieringJob(StreamExecutionEnvironment execEnv) throws Exception { Review Comment: add `@SuppressWarnings({"rawtypes", "unchecked"})` to suppress warning. ########## fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/tiering/source/enumerator/TieringSourceEnumeratorTest.java: ########## @@ -654,37 +636,41 @@ void testHandleFailOverEvent() throws Throwable { createTable(tablePath2, DEFAULT_LOG_TABLE_DESCRIPTOR); appendRow(tablePath2, DEFAULT_LOG_TABLE_DESCRIPTOR, 0, 10); - int numSubtasks = 1; - try (MockSplitEnumeratorContext<TieringSplit> context = - new MockSplitEnumeratorContext<>(numSubtasks)) { + try (FlussMockSplitEnumeratorContext<TieringSplit> context = + new FlussMockSplitEnumeratorContext<>(numSubtasks)) { TieringSourceEnumerator enumerator = new TieringSourceEnumerator(flussConf, context, 500); enumerator.start(); assertThat(context.getSplitsAssignmentSequence()).isEmpty(); - // register one reader - int subtaskId = 0; - registerReader(context, enumerator, subtaskId, "localhost-" + subtaskId); - - // handle split request - enumerator.handleSplitRequest(subtaskId, "localhost-" + subtaskId); + // register readers and handle split requests for attempt 0 + registerReaderAndHandleSplitRequests(context, enumerator, numSubtasks, 0); // should get one tiering split, and the split is for tablePath1 - verifyTieringSplitAssignment(context, 1, tablePath1); + verifyTieringSplitAssignment(context, numSubtasks, tablePath1); // clean assignment context.getSplitsAssignmentSequence().clear(); - // enumerator handle TieringFailOverEvent, which will mark current tiering tablePath1 as - // fail, and all pending splits should be clear - enumerator.handleSourceEvent(subtaskId, new TieringFailOverEvent()); + // first reader failover: Enumerator marks tablePath1 as failed and clears its splits + // register readers and handle split requests (attempt 1) + registerReaderAndHandleSplitRequests(context, enumerator, numSubtasks, 1); - // handle split request - enumerator.handleSplitRequest(subtaskId, "localhost-" + subtaskId); // now, should get another one tiering split, the split is for tablePath2 since all // pending split for tablePath1 is clear - verifyTieringSplitAssignment(context, 1, tablePath2); + verifyTieringSplitAssignment(context, numSubtasks, tablePath2); + + // clean assignment + context.getSplitsAssignmentSequence().clear(); + + // second reader failover: Enumerator marks tablePath2 as failed and clears its splits Review Comment: ```suggestion // readers failover again: Enumerator marks tablePath2 as failed and clears its splits ``` ########## fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/tiering/TieringFailoverITCase.java: ########## @@ -0,0 +1,346 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.fluss.flink.tiering; + +import org.apache.fluss.config.Configuration; +import org.apache.fluss.flink.tiering.committer.CommittableMessageTypeInfo; +import org.apache.fluss.flink.tiering.committer.TieringCommitOperatorFactory; +import org.apache.fluss.flink.tiering.source.TableBucketWriteResultTypeInfo; +import org.apache.fluss.flink.tiering.source.TieringSource; +import org.apache.fluss.lake.values.ValuesLake; +import org.apache.fluss.lake.values.tiering.ValuesLakeTieringFactory; +import org.apache.fluss.lake.writer.LakeTieringFactory; +import org.apache.fluss.metadata.Schema; +import org.apache.fluss.metadata.TableBucket; +import org.apache.fluss.metadata.TablePath; +import org.apache.fluss.row.BinaryString; +import org.apache.fluss.row.Decimal; +import org.apache.fluss.row.InternalRow; +import org.apache.fluss.row.TimestampLtz; +import org.apache.fluss.row.TimestampNtz; +import org.apache.fluss.types.DataTypes; +import org.apache.fluss.utils.TypeUtils; + +import org.apache.flink.api.common.eventtime.WatermarkStrategy; +import org.apache.flink.configuration.PipelineOptions; +import org.apache.flink.core.execution.JobClient; +import org.apache.flink.streaming.api.datastream.DataStreamSource; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.functions.sink.v2.DiscardingSink; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; + +import java.time.Duration; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; + +import static org.apache.fluss.flink.tiering.source.TieringSource.TIERING_SOURCE_TRANSFORMATION_UID; +import static org.apache.fluss.flink.tiering.source.TieringSourceOptions.POLL_TIERING_TABLE_INTERVAL; +import static org.apache.fluss.lake.committer.BucketOffset.FLUSS_LAKE_SNAP_BUCKET_OFFSET_PROPERTY; +import static org.apache.fluss.testutils.DataTestUtils.row; +import static org.assertj.core.api.Assertions.assertThat; + +/** Test tiering failover. */ +class TieringFailoverITCase extends FlinkValuesTieringTestBase { + protected static final String DEFAULT_DB = "fluss"; + + private static StreamExecutionEnvironment execEnv; + + private static final Schema pkSchema = + Schema.newBuilder() + .column("f_boolean", DataTypes.BOOLEAN()) + .column("f_byte", DataTypes.TINYINT()) + .column("f_short", DataTypes.SMALLINT()) + .column("f_int", DataTypes.INT()) + .column("f_long", DataTypes.BIGINT()) + .column("f_float", DataTypes.FLOAT()) + .column("f_double", DataTypes.DOUBLE()) + .column("f_string", DataTypes.STRING()) + .column("f_decimal1", DataTypes.DECIMAL(5, 2)) + .column("f_decimal2", DataTypes.DECIMAL(20, 0)) + .column("f_timestamp_ltz1", DataTypes.TIMESTAMP_LTZ(3)) + .column("f_timestamp_ltz2", DataTypes.TIMESTAMP_LTZ(6)) + .column("f_timestamp_ntz1", DataTypes.TIMESTAMP(3)) + .column("f_timestamp_ntz2", DataTypes.TIMESTAMP(6)) + .column("f_binary", DataTypes.BINARY(4)) + .column("f_date", DataTypes.DATE()) + .column("f_time", DataTypes.TIME()) + .column("f_char", DataTypes.CHAR(3)) + .column("f_bytes", DataTypes.BYTES()) + .primaryKey("f_string") + .build(); + + @BeforeAll + protected static void beforeAll() { + FlinkValuesTieringTestBase.beforeAll(); + execEnv = StreamExecutionEnvironment.getExecutionEnvironment(); + execEnv.setParallelism(2); + execEnv.enableCheckpointing(1000); + } + + @Test + void testTiering() throws Exception { + // create a pk table, write some records and wait until snapshot finished + TablePath t1 = TablePath.of(DEFAULT_DB, "pkTable"); + long t1Id = createPkTable(t1, 1, false, pkSchema); + TableBucket t1Bucket = new TableBucket(t1Id, 0); + // write records + List<InternalRow> rows = + Arrays.asList( + row( + true, + (byte) 100, + (short) 200, + 1, + 1 + 400L, + 500.1f, + 600.0d, + "v1", + Decimal.fromUnscaledLong(900, 5, 2), + Decimal.fromBigDecimal(new java.math.BigDecimal(1000), 20, 0), + TimestampLtz.fromEpochMillis(1698235273400L), + TimestampLtz.fromEpochMillis(1698235273400L, 7000), + TimestampNtz.fromMillis(1698235273501L), + TimestampNtz.fromMillis(1698235273501L, 8000), + new byte[] {5, 6, 7, 8}, + TypeUtils.castFromString("2023-10-25", DataTypes.DATE()), + TypeUtils.castFromString("09:30:00.0", DataTypes.TIME()), + BinaryString.fromString("abc"), + new byte[] {1, 2, 3, 4, 5, 6, 7, 8, 9, 10}), + row( + true, + (byte) 100, + (short) 200, + 2, + 2 + 400L, + 500.1f, + 600.0d, + "v2", + Decimal.fromUnscaledLong(900, 5, 2), + Decimal.fromBigDecimal(new java.math.BigDecimal(1000), 20, 0), + TimestampLtz.fromEpochMillis(1698235273400L), + TimestampLtz.fromEpochMillis(1698235273400L, 7000), + TimestampNtz.fromMillis(1698235273501L), + TimestampNtz.fromMillis(1698235273501L, 8000), + new byte[] {5, 6, 7, 8}, + TypeUtils.castFromString("2023-10-25", DataTypes.DATE()), + TypeUtils.castFromString("09:30:00.0", DataTypes.TIME()), + BinaryString.fromString("abc"), + new byte[] {1, 2, 3, 4, 5, 6, 7, 8, 9, 10}), + row( + true, + (byte) 100, + (short) 200, + 3, + 3 + 400L, + 500.1f, + 600.0d, + "v3", + Decimal.fromUnscaledLong(900, 5, 2), + Decimal.fromBigDecimal(new java.math.BigDecimal(1000), 20, 0), + TimestampLtz.fromEpochMillis(1698235273400L), + TimestampLtz.fromEpochMillis(1698235273400L, 7000), + TimestampNtz.fromMillis(1698235273501L), + TimestampNtz.fromMillis(1698235273501L, 8000), + new byte[] {5, 6, 7, 8}, + TypeUtils.castFromString("2023-10-25", DataTypes.DATE()), + TypeUtils.castFromString("09:30:00.0", DataTypes.TIME()), + BinaryString.fromString("abc"), + new byte[] {1, 2, 3, 4, 5, 6, 7, 8, 9, 10})); + List<InternalRow> expectedRows = new ArrayList<>(rows); + writeRows(t1, rows); + waitUntilSnapshot(t1Id, 1, 0); + + // fail the first write to the pk table + ValuesLake.failWhen(t1.toString()).failWriteOnce(); + + // then start tiering job + JobClient jobClient = buildTieringJob(execEnv); + try { + // check the status of replica after synced + assertReplicaStatus(t1Bucket, 3); + + checkDataInValuesPrimaryKeyTable(t1, rows); + // check snapshot property in values lake + Map<String, String> properties = + new HashMap<String, String>() { + { + put( + FLUSS_LAKE_SNAP_BUCKET_OFFSET_PROPERTY, + "[{\"bucket\":0,\"offset\":3}]"); + } + }; + checkSnapshotPropertyInValues(t1, properties); + + // then write data to the pk tables + // write records + rows = + Arrays.asList( + row( + true, + (byte) 100, + (short) 200, + 1, + 1 + 400L, + 500.1f, + 600.0d, + "v111", + Decimal.fromUnscaledLong(900, 5, 2), + Decimal.fromBigDecimal(new java.math.BigDecimal(1000), 20, 0), + TimestampLtz.fromEpochMillis(1698235273400L), + TimestampLtz.fromEpochMillis(1698235273400L, 7000), + TimestampNtz.fromMillis(1698235273501L), + TimestampNtz.fromMillis(1698235273501L, 8000), + new byte[] {5, 6, 7, 8}, + TypeUtils.castFromString("2023-10-25", DataTypes.DATE()), + TypeUtils.castFromString("09:30:00.0", DataTypes.TIME()), + BinaryString.fromString("abc"), + new byte[] {1, 2, 3, 4, 5, 6, 7, 8, 9, 10}), + row( + true, + (byte) 100, + (short) 200, + 2, + 2 + 400L, + 500.1f, + 600.0d, + "v222", + Decimal.fromUnscaledLong(900, 5, 2), + Decimal.fromBigDecimal(new java.math.BigDecimal(1000), 20, 0), + TimestampLtz.fromEpochMillis(1698235273400L), + TimestampLtz.fromEpochMillis(1698235273400L, 7000), + TimestampNtz.fromMillis(1698235273501L), + TimestampNtz.fromMillis(1698235273501L, 8000), + new byte[] {5, 6, 7, 8}, + TypeUtils.castFromString("2023-10-25", DataTypes.DATE()), + TypeUtils.castFromString("09:30:00.0", DataTypes.TIME()), + BinaryString.fromString("abc"), + new byte[] {1, 2, 3, 4, 5, 6, 7, 8, 9, 10}), + row( + true, + (byte) 100, + (short) 200, + 3, + 3 + 400L, + 500.1f, + 600.0d, + "v333", + Decimal.fromUnscaledLong(900, 5, 2), + Decimal.fromBigDecimal(new java.math.BigDecimal(1000), 20, 0), + TimestampLtz.fromEpochMillis(1698235273400L), + TimestampLtz.fromEpochMillis(1698235273400L, 7000), + TimestampNtz.fromMillis(1698235273501L), + TimestampNtz.fromMillis(1698235273501L, 8000), + new byte[] {5, 6, 7, 8}, + TypeUtils.castFromString("2023-10-25", DataTypes.DATE()), + TypeUtils.castFromString("09:30:00.0", DataTypes.TIME()), + BinaryString.fromString("abc"), + new byte[] {1, 2, 3, 4, 5, 6, 7, 8, 9, 10})); + expectedRows.addAll(rows); + // write records + writeRows(t1, rows); + + // check the status of replica of t1 after synced + // not check start offset since we won't + // update start log offset for primary key table + assertReplicaStatus(t1Bucket, expectedRows.size()); + + checkDataInValuesPrimaryKeyTable(t1, expectedRows); + } finally { + jobClient.cancel().get(); + } + } + + protected JobClient buildTieringJob(StreamExecutionEnvironment execEnv) throws Exception { Review Comment: ``` @SuppressWarnings({"rawtypes", "unchecked"}) protected JobClient buildTieringJob(StreamExecutionEnvironment execEnv) throws Exception { Configuration flinkConfig = new Configuration(); LakeTieringFactory lakeTieringFactory = new ValuesLakeTieringFactory(); // build tiering source TieringSource<?> tieringSource = new TieringSource.Builder<>(flinkConfig, lakeTieringFactory) .withPollTieringTableIntervalMs(500L) .build(); DataStreamSource<?> source = execEnv.fromSource( tieringSource, WatermarkStrategy.noWatermarks(), "TieringSource", TableBucketWriteResultTypeInfo.of( () -> lakeTieringFactory.getWriteResultSerializer())); source.getTransformation().setUid(TIERING_SOURCE_TRANSFORMATION_UID); source.transform( "TieringCommitter", CommittableMessageTypeInfo.of( () -> lakeTieringFactory.getCommittableSerializer()), new TieringCommitOperatorFactory(flinkConfig, lakeTieringFactory)) .setParallelism(1) .setMaxParallelism(1) .sinkTo(new DiscardingSink()) .name("end") .setParallelism(1); return execEnv.executeAsync("Fluss Lake Tiering FailOver IT Test."); } ``` -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected]
