pvary commented on code in PR #11144: URL: https://github.com/apache/iceberg/pull/11144#discussion_r1765720264
########## flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/stream/TestTableMaintenance.java: ########## @@ -0,0 +1,490 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.maintenance.stream; + +import static org.apache.iceberg.flink.SimpleDataUtil.createRowData; +import static org.apache.iceberg.flink.maintenance.operator.TableMaintenanceMetrics.CONCURRENT_RUN_THROTTLED; +import static org.apache.iceberg.flink.maintenance.operator.TableMaintenanceMetrics.FAILED_TASK_COUNTER; +import static org.apache.iceberg.flink.maintenance.operator.TableMaintenanceMetrics.GROUP_VALUE_DEFAULT; +import static org.apache.iceberg.flink.maintenance.operator.TableMaintenanceMetrics.NOTHING_TO_TRIGGER; +import static org.apache.iceberg.flink.maintenance.operator.TableMaintenanceMetrics.RATE_LIMITER_TRIGGERED; +import static org.apache.iceberg.flink.maintenance.operator.TableMaintenanceMetrics.SUCCEEDED_TASK_COUNTER; +import static org.apache.iceberg.flink.maintenance.operator.TableMaintenanceMetrics.TRIGGERED; +import static org.apache.iceberg.flink.maintenance.stream.TableMaintenance.LOCK_REMOVER_TASK_NAME; +import static org.apache.iceberg.flink.maintenance.stream.TableMaintenance.SOURCE_NAME; +import static org.apache.iceberg.flink.maintenance.stream.TableMaintenance.TRIGGER_MANAGER_TASK_NAME; +import static org.assertj.core.api.Assertions.assertThat; + +import java.io.File; +import java.io.Serializable; +import java.time.Duration; +import java.util.Collections; +import java.util.List; +import org.apache.flink.api.common.functions.MapFunction; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.dag.Transformation; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.api.java.typeutils.ResultTypeQueryable; +import org.apache.flink.configuration.CheckpointingOptions; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.core.execution.JobClient; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.transformations.SourceTransformation; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.runtime.typeutils.InternalTypeInfo; +import org.apache.iceberg.Table; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.TableLoader; +import org.apache.iceberg.flink.maintenance.operator.ManualSource; +import org.apache.iceberg.flink.maintenance.operator.MetricsReporterFactoryForTests; +import org.apache.iceberg.flink.maintenance.operator.OperatorTestBase; +import org.apache.iceberg.flink.maintenance.operator.TableChange; +import org.apache.iceberg.flink.maintenance.operator.TaskResult; +import org.apache.iceberg.flink.maintenance.operator.Trigger; +import org.apache.iceberg.flink.maintenance.operator.TriggerLockFactory; +import org.apache.iceberg.flink.sink.FlinkSink; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.awaitility.Awaitility; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; + +class TestTableMaintenance extends OperatorTestBase { + private static final String[] TASKS = + new String[] { + MaintenanceTaskBuilderForTest.class.getSimpleName() + " [0]", + MaintenanceTaskBuilderForTest.class.getSimpleName() + " [1]" + }; + private static final TableChange DUMMY_CHANGE = TableChange.builder().commitCount(1).build(); + private static final List<Trigger> PROCESSED = + Collections.synchronizedList(Lists.newArrayListWithCapacity(1)); + + private StreamExecutionEnvironment env; + + @TempDir private File checkpointDir; + + @BeforeEach + public void beforeEach() { + Configuration config = new Configuration(); + config.set(CheckpointingOptions.CHECKPOINT_STORAGE, "filesystem"); + config.set(CheckpointingOptions.CHECKPOINTS_DIRECTORY, "file://" + checkpointDir.getPath()); + env = StreamExecutionEnvironment.getExecutionEnvironment(config); + PROCESSED.clear(); + MaintenanceTaskBuilderForTest.counter = 0; + } + + @Test + void testFromStream() throws Exception { + sql.exec("CREATE TABLE %s (id int, data varchar)", TABLE_NAME); + sql.exec("INSERT INTO %s VALUES (1, 'a')", TABLE_NAME); + + TableLoader tableLoader = sql.tableLoader(TABLE_NAME); + tableLoader.open(); + + ManualSource<TableChange> schedulerSource = + new ManualSource<>(env, TypeInformation.of(TableChange.class)); + + TableMaintenance.Builder streamBuilder = + TableMaintenance.forChangeStream(schedulerSource.dataStream(), tableLoader, LOCK_FACTORY) + .rateLimit(Duration.ofMillis(2)) + .lockCheckDelay(Duration.ofSeconds(3)) + .add( + new MaintenanceTaskBuilderForTest(true) + .scheduleOnCommitCount(1) + .scheduleOnDataFileCount(2) + .scheduleOnDataFileSize(3L) + .scheduleOnEqDeleteFileCount(4) + .scheduleOnEqDeleteRecordCount(5L) + .scheduleOnPosDeleteFileCount(6) + .scheduleOnPosDeleteRecordCount(7L) + .scheduleOnInterval(Duration.ofHours(1))); + + sendEvents(schedulerSource, streamBuilder, ImmutableList.of(Tuple2.of(DUMMY_CHANGE, 1))); + } + + @Test + void testFromEnv() throws Exception { + sql.exec( + "CREATE TABLE %s (id int, data varchar)" + + "WITH ('flink.max-continuous-empty-commits'='100000')", + TABLE_NAME); + sql.exec("INSERT INTO %s VALUES (1, 'a')", TABLE_NAME); + + TableLoader tableLoader = sql.tableLoader(TABLE_NAME); + tableLoader.open(); + Table table = tableLoader.loadTable(); + + env.enableCheckpointing(10); + + TableMaintenance.forTable(env, tableLoader, LOCK_FACTORY) + .rateLimit(Duration.ofMillis(2)) + .maxReadBack(2) + .add(new MaintenanceTaskBuilderForTest(true).scheduleOnCommitCount(2)) + .append(); + + // Creating a stream for inserting data into the table concurrently + ManualSource<RowData> insertSource = + new ManualSource<>(env, InternalTypeInfo.of(FlinkSchemaUtil.convert(table.schema()))); + FlinkSink.forRowData(insertSource.dataStream()) + .tableLoader(tableLoader) + .uidPrefix(UID_SUFFIX + "-iceberg-sink") + .append(); + + JobClient jobClient = null; + try { + jobClient = env.executeAsync(); + + insertSource.sendRecord(createRowData(2, "b")); + + Awaitility.await().until(() -> PROCESSED.size() == 1); + } finally { + closeJobClient(jobClient); + } + } + + @Test + void testLocking() throws Exception { Review Comment: I prefer to separate out testing the different features -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org --------------------------------------------------------------------- To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org For additional commands, e-mail: issues-h...@iceberg.apache.org