From 70e078328d00dbb427259300877961dde563fc01 Mon Sep 17 00:00:00 2001 From: Peter Vary Date: Mon, 26 Aug 2024 08:29:52 +0200 Subject: [PATCH 01/12] Flink: Maintenance - TableManager + ExpireSnapshots --- .../operator/AsyncDeleteFiles.java | 106 ++++ .../operator/ExpireSnapshotsProcessor.java | 99 ++++ .../maintenance/operator/MonitorSource.java | 4 +- .../maintenance/operator/TableChange.java | 8 +- .../operator/TableMaintenanceMetrics.java | 4 + .../flink/maintenance/operator/Trigger.java | 8 +- .../operator/TriggerEvaluator.java | 8 +- .../operator/TriggerLockFactory.java | 5 +- .../maintenance/operator/TriggerManager.java | 4 +- .../maintenance/stream/ExpireSnapshots.java | 161 ++++++ .../stream/MaintenanceTaskBuilder.java | 238 +++++++++ .../maintenance/stream/TableMaintenance.java | 356 +++++++++++++ .../maintenance/operator/CollectingSink.java | 6 +- .../operator/ConstantsForTests.java | 29 -- .../operator/FlinkSqlExtension.java | 4 + .../operator/FlinkStreamingTestUtils.java | 73 --- .../maintenance/operator/ManualSource.java | 12 +- .../operator/OperatorTestBase.java | 124 +++-- .../operator/TestAsyncDeleteFiles.java | 154 ++++++ .../TestExpireSnapshotsProcessor.java | 86 +++ .../maintenance/operator/TestLockRemover.java | 32 +- .../operator/TestTriggerManager.java | 50 +- .../stream/ScheduledBuilderTestBase.java | 85 +++ .../stream/ScheduledInfraExtension.java | 80 +++ .../stream/TestExpireSnapshots.java | 248 +++++++++ .../stream/TestMaintenanceE2E.java | 69 +++ .../stream/TestTableMaintenance.java | 490 ++++++++++++++++++ 27 files changed, 2338 insertions(+), 205 deletions(-) create mode 100644 flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/AsyncDeleteFiles.java create mode 100644 flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/ExpireSnapshotsProcessor.java create mode 100644 flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/stream/ExpireSnapshots.java create mode 100644 flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/stream/MaintenanceTaskBuilder.java create mode 100644 flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/stream/TableMaintenance.java delete mode 100644 flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/ConstantsForTests.java delete mode 100644 flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/FlinkStreamingTestUtils.java create mode 100644 flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestAsyncDeleteFiles.java create mode 100644 flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestExpireSnapshotsProcessor.java create mode 100644 flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/stream/ScheduledBuilderTestBase.java create mode 100644 flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/stream/ScheduledInfraExtension.java create mode 100644 flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/stream/TestExpireSnapshots.java create mode 100644 flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/stream/TestMaintenanceE2E.java create mode 100644 flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/stream/TestTableMaintenance.java diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/AsyncDeleteFiles.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/AsyncDeleteFiles.java new file mode 100644 index 000000000000..af40919a8f96 --- /dev/null +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/AsyncDeleteFiles.java @@ -0,0 +1,106 @@ +/* + * 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.operator; + +import java.io.Serializable; +import java.util.Collection; +import java.util.Collections; +import java.util.concurrent.ExecutorService; +import java.util.function.Predicate; +import org.apache.flink.annotation.Internal; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.metrics.Counter; +import org.apache.flink.streaming.api.functions.async.ResultFuture; +import org.apache.flink.streaming.api.functions.async.RichAsyncFunction; +import org.apache.iceberg.Table; +import org.apache.iceberg.flink.TableLoader; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.util.ThreadPools; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** Delete the files using the {@link FileIO}. */ +@Internal +public class AsyncDeleteFiles extends RichAsyncFunction { + private static final Logger LOG = LoggerFactory.getLogger(AsyncDeleteFiles.class); + public static final Predicate> FAILED_PREDICATE = new FailedPredicate(); + + private final String name; + private final FileIO io; + private final int workerPoolSize; + private final String tableName; + + private transient ExecutorService workerPool; + private transient Counter failedCounter; + private transient Counter succeededCounter; + + public AsyncDeleteFiles(String name, TableLoader tableLoader, int workerPoolSize) { + Preconditions.checkNotNull(name, "Name should no be null"); + Preconditions.checkNotNull(tableLoader, "Table loader should no be null"); + + this.name = name; + tableLoader.open(); + Table table = tableLoader.loadTable(); + this.io = table.io(); + this.workerPoolSize = workerPoolSize; + this.tableName = table.name(); + } + + @Override + public void open(Configuration parameters) throws Exception { + this.failedCounter = + getRuntimeContext() + .getMetricGroup() + .addGroup(TableMaintenanceMetrics.GROUP_KEY, name) + .counter(TableMaintenanceMetrics.DELETE_FILE_FAILED_COUNTER); + this.succeededCounter = + getRuntimeContext() + .getMetricGroup() + .addGroup(TableMaintenanceMetrics.GROUP_KEY, name) + .counter(TableMaintenanceMetrics.DELETE_FILE_SUCCEEDED_COUNTER); + + this.workerPool = + ThreadPools.newWorkerPool(tableName + "-" + name + "-async-delete-files", workerPoolSize); + } + + @Override + public void asyncInvoke(String fileName, ResultFuture resultFuture) { + workerPool.execute( + () -> { + try { + LOG.info("Deleting file: {} with {}", fileName, name); + io.deleteFile(fileName); + resultFuture.complete(Collections.singletonList(true)); + succeededCounter.inc(); + } catch (Throwable e) { + LOG.info("Failed to delete file {} with {}", fileName, name, e); + resultFuture.complete(Collections.singletonList(false)); + failedCounter.inc(); + } + }); + } + + private static class FailedPredicate implements Predicate>, Serializable { + @Override + public boolean test(Collection collection) { + return collection.size() != 1 || !collection.iterator().next(); + } + } +} diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/ExpireSnapshotsProcessor.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/ExpireSnapshotsProcessor.java new file mode 100644 index 000000000000..bbd6e9775557 --- /dev/null +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/ExpireSnapshotsProcessor.java @@ -0,0 +1,99 @@ +/* + * 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.operator; + +import java.util.Collections; +import java.util.concurrent.ExecutorService; +import org.apache.flink.api.common.typeinfo.Types; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.streaming.api.functions.ProcessFunction; +import org.apache.flink.util.Collector; +import org.apache.flink.util.OutputTag; +import org.apache.iceberg.ExpireSnapshots; +import org.apache.iceberg.Table; +import org.apache.iceberg.flink.TableLoader; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.util.ThreadPools; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Calls the {@link ExpireSnapshots} to remove the old snapshots and emits the filenames which could + * be removed in the {@link #DELETE_STREAM} side output. + */ +public class ExpireSnapshotsProcessor extends ProcessFunction { + private static final Logger LOG = LoggerFactory.getLogger(ExpireSnapshotsProcessor.class); + public static final OutputTag DELETE_STREAM = + new OutputTag<>("delete-stream", Types.STRING); + + private final TableLoader tableLoader; + private final Long minAgeMs; + private final Integer retainLast; + private final int plannerPoolSize; + private transient ExecutorService plannerPool; + private transient Table table; + + public ExpireSnapshotsProcessor( + TableLoader tableLoader, Long minAgeMs, Integer retainLast, int plannerPoolSize) { + Preconditions.checkNotNull(tableLoader, "Table loader should no be null"); + + this.tableLoader = tableLoader; + this.minAgeMs = minAgeMs; + this.retainLast = retainLast; + this.plannerPoolSize = plannerPoolSize; + } + + @Override + public void open(Configuration parameters) throws Exception { + tableLoader.open(); + this.table = tableLoader.loadTable(); + this.plannerPool = ThreadPools.newWorkerPool(table.name() + "-table--planner", plannerPoolSize); + } + + @Override + public void processElement(Trigger trigger, Context ctx, Collector out) + throws Exception { + try { + table.refresh(); + ExpireSnapshots expireSnapshots = table.expireSnapshots(); + if (minAgeMs != null) { + expireSnapshots = expireSnapshots.expireOlderThan(ctx.timestamp() - minAgeMs); + } + + if (retainLast != null) { + expireSnapshots = expireSnapshots.retainLast(retainLast); + } + + expireSnapshots + .planWith(plannerPool) + .deleteWith(file -> ctx.output(DELETE_STREAM, file)) + .cleanExpiredFiles(true) + .commit(); + + LOG.info("Successfully finished expiring snapshots for {} at {}", table, ctx.timestamp()); + out.collect( + new TaskResult(trigger.taskId(), trigger.timestamp(), true, Collections.emptyList())); + } catch (Exception e) { + LOG.info("Exception expiring snapshots for {} at {}", table, ctx.timestamp(), e); + out.collect( + new TaskResult(trigger.taskId(), trigger.timestamp(), false, Lists.newArrayList(e))); + } + } +} diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/MonitorSource.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/MonitorSource.java index 89efffa15f16..d74b2349b1de 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/MonitorSource.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/MonitorSource.java @@ -43,7 +43,7 @@ /** Monitors an Iceberg table for changes */ @Internal -class MonitorSource extends SingleThreadedIteratorSource { +public class MonitorSource extends SingleThreadedIteratorSource { private static final Logger LOG = LoggerFactory.getLogger(MonitorSource.class); private final TableLoader tableLoader; @@ -58,7 +58,7 @@ class MonitorSource extends SingleThreadedIteratorSource { * @param rateLimiterStrategy limits the frequency the table is checked * @param maxReadBack sets the number of snapshots read before stopping change collection */ - MonitorSource( + public MonitorSource( TableLoader tableLoader, RateLimiterStrategy rateLimiterStrategy, long maxReadBack) { Preconditions.checkNotNull(tableLoader, "Table loader should no be null"); Preconditions.checkNotNull(rateLimiterStrategy, "Rate limiter strategy should no be null"); diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TableChange.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TableChange.java index 773b34b6c495..25c3c1028113 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TableChange.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TableChange.java @@ -28,7 +28,7 @@ /** Event describing changes in an Iceberg table */ @Internal -class TableChange { +public class TableChange { private int dataFileCount; private long dataFileSizeInBytes; private int posDeleteFileCount; @@ -87,7 +87,7 @@ static TableChange empty() { return new TableChange(0, 0L, 0, 0L, 0, 0L, 0); } - static Builder builder() { + public static Builder builder() { return new Builder(); } @@ -115,7 +115,7 @@ long eqDeleteRecordCount() { return eqDeleteRecordCount; } - public int commitCount() { + int commitCount() { return commitCount; } @@ -183,7 +183,7 @@ public int hashCode() { commitCount); } - static class Builder { + public static class Builder { private int dataFileCount = 0; private long dataFileSizeInBytes = 0L; private int posDeleteFileCount = 0; diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TableMaintenanceMetrics.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TableMaintenanceMetrics.java index 1a04461aed43..c57ed5092504 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TableMaintenanceMetrics.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TableMaintenanceMetrics.java @@ -33,6 +33,10 @@ public class TableMaintenanceMetrics { public static final String FAILED_TASK_COUNTER = "failedTasks"; public static final String LAST_RUN_DURATION_MS = "lastRunDurationMs"; + // DeleteFiles metrics + public static final String DELETE_FILE_FAILED_COUNTER = "deleteFailed"; + public static final String DELETE_FILE_SUCCEEDED_COUNTER = "deleteSucceeded"; + private TableMaintenanceMetrics() { // do not instantiate } diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/Trigger.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/Trigger.java index 85c6c8dbdd55..b4556dc1beab 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/Trigger.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/Trigger.java @@ -23,7 +23,7 @@ import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; @Internal -class Trigger { +public class Trigger { private final long timestamp; private final SerializableTable table; private final Integer taskId; @@ -36,7 +36,7 @@ private Trigger(long timestamp, SerializableTable table, Integer taskId, boolean this.isRecovery = isRecovery; } - static Trigger create(long timestamp, SerializableTable table, int taskId) { + public static Trigger create(long timestamp, SerializableTable table, int taskId) { return new Trigger(timestamp, table, taskId, false); } @@ -44,7 +44,7 @@ static Trigger recovery(long timestamp) { return new Trigger(timestamp, null, null, true); } - long timestamp() { + public long timestamp() { return timestamp; } @@ -52,7 +52,7 @@ SerializableTable table() { return table; } - Integer taskId() { + public Integer taskId() { return taskId; } diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TriggerEvaluator.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TriggerEvaluator.java index dba33b22a42a..d448898bdfe6 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TriggerEvaluator.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TriggerEvaluator.java @@ -28,7 +28,7 @@ import org.slf4j.LoggerFactory; @Internal -class TriggerEvaluator implements Serializable { +public class TriggerEvaluator implements Serializable { private static final Logger LOG = LoggerFactory.getLogger(TriggerEvaluator.class); private final List predicates; @@ -50,7 +50,7 @@ boolean check(TableChange event, long lastTimeMs, long currentTimeMs) { return result; } - static class Builder implements Serializable { + public static class Builder implements Serializable { private Integer dataFileCount; private Long dataFileSizeInBytes; private Integer posDeleteFileCount; @@ -95,12 +95,12 @@ public Builder commitCount(int newCommitCount) { return this; } - Builder timeout(Duration newTimeout) { + public Builder timeout(Duration newTimeout) { this.timeout = newTimeout; return this; } - TriggerEvaluator build() { + public TriggerEvaluator build() { List predicates = Lists.newArrayList(); if (dataFileCount != null) { predicates.add((change, unused, unused2) -> change.dataFileCount() >= dataFileCount); diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TriggerLockFactory.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TriggerLockFactory.java index 329223d27ccf..446e8ce2f2a8 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TriggerLockFactory.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TriggerLockFactory.java @@ -51,12 +51,11 @@ interface Lock { */ boolean isHeld(); - // TODO: Fix the link to the LockRemover when we have a final name and implementation /** * Releases the lock. Should not fail if the lock is not held by anyone. * - *

Called by LockRemover. Implementations could assume that are no concurrent calls for this - * method. + *

Called by {@link LockRemover}. Implementations could assume that are no concurrent calls + * for this method. */ void unlock(); } diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TriggerManager.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TriggerManager.java index dc95b27af0a6..6b658d122734 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TriggerManager.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TriggerManager.java @@ -57,7 +57,7 @@ * the timer functions are available, but the key is not used. */ @Internal -class TriggerManager extends KeyedProcessFunction +public class TriggerManager extends KeyedProcessFunction implements CheckpointedFunction { private static final Logger LOG = LoggerFactory.getLogger(TriggerManager.class); @@ -89,7 +89,7 @@ class TriggerManager extends KeyedProcessFunction private transient int startsFrom = 0; private transient boolean triggered = false; - TriggerManager( + public TriggerManager( TableLoader tableLoader, TriggerLockFactory lockFactory, List maintenanceTaskNames, diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/stream/ExpireSnapshots.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/stream/ExpireSnapshots.java new file mode 100644 index 000000000000..056d8e7d0b7f --- /dev/null +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/stream/ExpireSnapshots.java @@ -0,0 +1,161 @@ +/* + * 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 java.time.Duration; +import java.util.concurrent.TimeUnit; +import org.apache.flink.streaming.api.datastream.AsyncDataStream; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator; +import org.apache.flink.streaming.api.functions.async.AsyncRetryStrategy; +import org.apache.flink.streaming.util.retryable.AsyncRetryStrategies; +import org.apache.iceberg.flink.maintenance.operator.AsyncDeleteFiles; +import org.apache.iceberg.flink.maintenance.operator.ExpireSnapshotsProcessor; +import org.apache.iceberg.flink.maintenance.operator.TaskResult; +import org.apache.iceberg.flink.maintenance.operator.Trigger; +import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; + +public class ExpireSnapshots { + private static final long DELETE_INITIAL_DELAY_MS = 10L; + private static final long DELETE_MAX_RETRY_DELAY_MS = 1000L; + private static final double DELETE_BACKOFF_MULTIPLIER = 1.5; + private static final long DELETE_TIMEOUT_MS = 10000L; + private static final int DELETE_PLANNING_WORKER_POOL_SIZE_DEFAULT = 10; + private static final int DELETE_ATTEMPT_NUM = 10; + private static final int DELETE_WORKER_POOL_SIZE_DEFAULT = 10; + private static final String EXECUTOR_TASK_NAME = "ES Executor"; + @VisibleForTesting static final String DELETE_FILES_TASK_NAME = "Delete file"; + + private ExpireSnapshots() { + // Do not instantiate directly + } + + /** Creates the builder for creating a stream which expires snapshots for the table. */ + public static Builder builder() { + return new Builder(); + } + + public static class Builder extends MaintenanceTaskBuilder { + private Duration minAge = null; + private Integer retainLast = null; + private int planningWorkerPoolSize = DELETE_PLANNING_WORKER_POOL_SIZE_DEFAULT; + private int deleteAttemptNum = DELETE_ATTEMPT_NUM; + private int deleteWorkerPoolSize = DELETE_WORKER_POOL_SIZE_DEFAULT; + + /** + * The snapshots newer than this age will not be removed. + * + * @param newMinAge of the files to be removed + * @return for chained calls + */ + public Builder minAge(Duration newMinAge) { + this.minAge = newMinAge; + return this; + } + + /** + * The minimum {@link org.apache.iceberg.Snapshot}s to retain. For more details description see + * {@link org.apache.iceberg.ExpireSnapshots#retainLast(int)}. + * + * @param newRetainLast number of snapshots to retain + * @return for chained calls + */ + public Builder retainLast(int newRetainLast) { + this.retainLast = newRetainLast; + return this; + } + + /** + * The worker pool size used to calculate the files to delete. + * + * @param newPlanningWorkerPoolSize for planning files to delete + * @return for chained calls + */ + public Builder planningWorkerPoolSize(int newPlanningWorkerPoolSize) { + this.planningWorkerPoolSize = newPlanningWorkerPoolSize; + return this; + } + + /** + * The number of retries on the failed delete attempts. + * + * @param newDeleteAttemptNum number of retries + * @return for chained calls + */ + public Builder deleteAttemptNum(int newDeleteAttemptNum) { + this.deleteAttemptNum = newDeleteAttemptNum; + return this; + } + + /** + * The worker pool size used for deleting files. + * + * @param newDeleteWorkerPoolSize for scanning + * @return for chained calls + */ + public Builder deleteWorkerPoolSize(int newDeleteWorkerPoolSize) { + this.deleteWorkerPoolSize = newDeleteWorkerPoolSize; + return this; + } + + @Override + DataStream buildInternal(DataStream trigger) { + Preconditions.checkNotNull(tableLoader(), "TableLoader should not be null"); + + SingleOutputStreamOperator result = + trigger + .process( + new ExpireSnapshotsProcessor( + tableLoader(), + minAge == null ? null : minAge.toMillis(), + retainLast, + planningWorkerPoolSize)) + .name(EXECUTOR_TASK_NAME) + .uid(uidPrefix() + "-expire-snapshots") + .slotSharingGroup(slotSharingGroup()) + .forceNonParallel(); + + AsyncRetryStrategy retryStrategy = + new AsyncRetryStrategies.ExponentialBackoffDelayRetryStrategyBuilder( + deleteAttemptNum, + DELETE_INITIAL_DELAY_MS, + DELETE_MAX_RETRY_DELAY_MS, + DELETE_BACKOFF_MULTIPLIER) + .ifResult(AsyncDeleteFiles.FAILED_PREDICATE) + .build(); + + AsyncDataStream.unorderedWaitWithRetry( + result.getSideOutput(ExpireSnapshotsProcessor.DELETE_STREAM).rebalance(), + new AsyncDeleteFiles(name(), tableLoader(), deleteWorkerPoolSize), + DELETE_TIMEOUT_MS, + TimeUnit.MILLISECONDS, + deleteWorkerPoolSize, + retryStrategy) + .name(DELETE_FILES_TASK_NAME) + .uid(uidPrefix() + "-delete-expired-files") + .slotSharingGroup(slotSharingGroup()) + .setParallelism(parallelism()); + + // Deleting the files is asynchronous, so we ignore the results when calculating the return + // value + return result; + } + } +} diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/stream/MaintenanceTaskBuilder.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/stream/MaintenanceTaskBuilder.java new file mode 100644 index 000000000000..d02ba7662b38 --- /dev/null +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/stream/MaintenanceTaskBuilder.java @@ -0,0 +1,238 @@ +/* + * 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 java.time.Duration; +import org.apache.flink.annotation.Internal; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator; +import org.apache.iceberg.flink.TableLoader; +import org.apache.iceberg.flink.maintenance.operator.TaskResult; +import org.apache.iceberg.flink.maintenance.operator.Trigger; +import org.apache.iceberg.flink.maintenance.operator.TriggerEvaluator; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; + +public abstract class MaintenanceTaskBuilder { + private int id; + private String name; + private TableLoader tableLoader; + private String uidPrefix = null; + private String slotSharingGroup = null; + private Integer parallelism = null; + private TriggerEvaluator.Builder triggerEvaluator = new TriggerEvaluator.Builder(); + + abstract DataStream buildInternal(DataStream sourceStream); + + /** + * After a given number of Iceberg table commits since the last run, starts the downstream job. + * + * @param commitCount after the downstream job should be started + * @return for chained calls + */ + public T scheduleOnCommitCount(int commitCount) { + triggerEvaluator.commitCount(commitCount); + return (T) this; + } + + /** + * After a given number of new data files since the last run, starts the downstream job. + * + * @param dataFileCount after the downstream job should be started + * @return for chained calls + */ + public T scheduleOnDataFileCount(int dataFileCount) { + triggerEvaluator.dataFileCount(dataFileCount); + return (T) this; + } + + /** + * After a given aggregated data file size since the last run, starts the downstream job. + * + * @param dataFileSizeInBytes after the downstream job should be started + * @return for chained calls + */ + public T scheduleOnDataFileSize(long dataFileSizeInBytes) { + triggerEvaluator.dataFileSizeInBytes(dataFileSizeInBytes); + return (T) this; + } + + /** + * After a given number of new positional delete files since the last run, starts the downstream + * job. + * + * @param posDeleteFileCount after the downstream job should be started + * @return for chained calls + */ + public T schedulerOnPosDeleteFileCount(int posDeleteFileCount) { + triggerEvaluator.posDeleteFileCount(posDeleteFileCount); + return (T) this; + } + + /** + * After a given number of new positional delete records since the last run, starts the downstream + * job. + * + * @param posDeleteRecordCount after the downstream job should be started + * @return for chained calls + */ + public T schedulerOnPosDeleteRecordCount(long posDeleteRecordCount) { + triggerEvaluator.posDeleteRecordCount(posDeleteRecordCount); + return (T) this; + } + + /** + * After a given number of new equality delete files since the last run, starts the downstream + * job. + * + * @param eqDeleteFileCount after the downstream job should be started + * @return for chained calls + */ + public T scheduleOnEqDeleteFileCount(int eqDeleteFileCount) { + triggerEvaluator.eqDeleteFileCount(eqDeleteFileCount); + return (T) this; + } + + /** + * After a given number of new equality delete records since the last run, starts the downstream + * job. + * + * @param eqDeleteRecordCount after the downstream job should be started + * @return for chained calls + */ + public T scheduleOnEqDeleteRecordCount(long eqDeleteRecordCount) { + triggerEvaluator.eqDeleteRecordCount(eqDeleteRecordCount); + return (T) this; + } + + /** + * After a given time since the last run, starts the downstream job. + * + * @param time after the downstream job should be started + * @return for chained calls + */ + public T scheduleOnTime(Duration time) { + triggerEvaluator.timeout(time); + return (T) this; + } + + /** + * The prefix used for the generated {@link org.apache.flink.api.dag.Transformation}'s uid. + * + * @param newUidPrefix for the transformations + * @return for chained calls + */ + public T uidPrefix(String newUidPrefix) { + this.uidPrefix = newUidPrefix; + return (T) this; + } + + /** + * The {@link SingleOutputStreamOperator#slotSharingGroup(String)} for all the operators of the + * generated stream. Could be used to separate the resources used by this task. + * + * @param newSlotSharingGroup to be used for the operators + * @return for chained calls + */ + public T slotSharingGroup(String newSlotSharingGroup) { + this.slotSharingGroup = newSlotSharingGroup; + return (T) this; + } + + /** + * Sets the parallelism for the stream. + * + * @param newParallelism the required parallelism + * @return for chained calls + */ + public T parallelism(int newParallelism) { + this.parallelism = newParallelism; + return (T) this; + } + + @Internal + int id() { + return id; + } + + @Internal + String name() { + return name; + } + + @Internal + TableLoader tableLoader() { + return tableLoader; + } + + @Internal + String uidPrefix() { + return uidPrefix; + } + + @Internal + String slotSharingGroup() { + return slotSharingGroup; + } + + @Internal + Integer parallelism() { + return parallelism; + } + + @Internal + TriggerEvaluator evaluator() { + return triggerEvaluator.build(); + } + + @Internal + DataStream build( + DataStream sourceStream, + int newId, + String newName, + TableLoader newTableLoader, + String mainUidPrefix, + String mainSlotSharingGroup, + int mainParallelism) { + Preconditions.checkArgument( + parallelism == null || parallelism == -1 || parallelism > 0, + "Parallelism should be left to default (-1/null) or greater than 0"); + Preconditions.checkNotNull(newName, "Name should not be null"); + Preconditions.checkNotNull(newTableLoader, "TableLoader should not be null"); + + this.id = newId; + this.name = newName; + this.tableLoader = newTableLoader; + + if (uidPrefix == null) { + uidPrefix = mainUidPrefix + "_" + name + "_" + id; + } + + if (parallelism == null) { + parallelism = mainParallelism; + } + + if (slotSharingGroup == null) { + slotSharingGroup = mainSlotSharingGroup; + } + + tableLoader.open(); + + return buildInternal(sourceStream); + } +} diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/stream/TableMaintenance.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/stream/TableMaintenance.java new file mode 100644 index 000000000000..9ba6a9f67eff --- /dev/null +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/stream/TableMaintenance.java @@ -0,0 +1,356 @@ +/* + * 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 java.time.Duration; +import java.util.List; +import java.util.UUID; +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.eventtime.TimestampAssigner; +import org.apache.flink.api.common.eventtime.TimestampAssignerSupplier; +import org.apache.flink.api.common.eventtime.Watermark; +import org.apache.flink.api.common.eventtime.WatermarkGenerator; +import org.apache.flink.api.common.eventtime.WatermarkGeneratorSupplier; +import org.apache.flink.api.common.eventtime.WatermarkOutput; +import org.apache.flink.api.common.eventtime.WatermarkStrategy; +import org.apache.flink.api.common.functions.FlatMapFunction; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.connector.source.util.ratelimit.RateLimiterStrategy; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.datastream.DataStreamUtils; +import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.graph.StreamGraphGenerator; +import org.apache.flink.util.Collector; +import org.apache.iceberg.flink.TableLoader; +import org.apache.iceberg.flink.maintenance.operator.LockRemover; +import org.apache.iceberg.flink.maintenance.operator.MonitorSource; +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.TriggerEvaluator; +import org.apache.iceberg.flink.maintenance.operator.TriggerLockFactory; +import org.apache.iceberg.flink.maintenance.operator.TriggerManager; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; + +/** Creates the table maintenance graph. */ +public class TableMaintenance { + private static final String TASK_NAME_FORMAT = "%s [%d]"; + static final String SOURCE_NAME = "Monitor source"; + static final String TRIGGER_MANAGER_TASK_NAME = "Trigger manager"; + static final String LOCK_REMOVER_TASK_NAME = "Lock remover"; + + private TableMaintenance() { + // Do not instantiate directly + } + + /** + * Use when the change stream is already provided, like in the {@link + * org.apache.iceberg.flink.sink.IcebergSink#addPostCommitTopology(DataStream)}. + * + * @param changeStream the table changes + * @param tableLoader used for accessing the table + * @param lockFactory used for preventing concurrent task runs + * @return builder for the maintenance stream + */ + public static Builder builder( + DataStream changeStream, + TableLoader tableLoader, + TriggerLockFactory lockFactory) { + Preconditions.checkNotNull(changeStream, "The change stream should not be null"); + Preconditions.checkNotNull(tableLoader, "TableLoader should not be null"); + Preconditions.checkNotNull(lockFactory, "LockFactory should not be null"); + + return new Builder(changeStream, tableLoader, lockFactory); + } + + /** + * Creates the default monitor source for collecting the table changes and returns a builder for + * the maintenance stream. + * + * @param env used to register the monitor source + * @param tableLoader used for accessing the table + * @param lockFactory used for preventing concurrent task runs + * @return builder for the maintenance stream + */ + public static Builder builder( + StreamExecutionEnvironment env, TableLoader tableLoader, TriggerLockFactory lockFactory) { + Preconditions.checkNotNull(env, "StreamExecutionEnvironment should not be null"); + Preconditions.checkNotNull(tableLoader, "TableLoader should not be null"); + Preconditions.checkNotNull(lockFactory, "LockFactory should not be null"); + + return new Builder(env, tableLoader, lockFactory); + } + + public static class Builder { + private final StreamExecutionEnvironment env; + private final DataStream changeStream; + private final TableLoader tableLoader; + private final List> taskBuilders; + private final TriggerLockFactory lockFactory; + + private String uidPrefix = "TableMaintenance-" + UUID.randomUUID(); + private String slotSharingGroup = StreamGraphGenerator.DEFAULT_SLOT_SHARING_GROUP; + private Duration rateLimit = Duration.ofMillis(1); + private Duration concurrentCheckDelay = Duration.ofSeconds(30); + private Integer parallelism = ExecutionConfig.PARALLELISM_DEFAULT; + private int maxReadBack = 100; + + private Builder( + StreamExecutionEnvironment env, TableLoader tableLoader, TriggerLockFactory lockFactory) { + this.env = env; + this.changeStream = null; + this.tableLoader = tableLoader; + this.lockFactory = lockFactory; + this.taskBuilders = Lists.newArrayListWithCapacity(4); + } + + private Builder( + DataStream changeStream, + TableLoader tableLoader, + TriggerLockFactory lockFactory) { + this.env = null; + this.changeStream = changeStream; + this.tableLoader = tableLoader; + this.lockFactory = lockFactory; + this.taskBuilders = Lists.newArrayListWithCapacity(4); + } + + /** + * The prefix used for the generated {@link org.apache.flink.api.dag.Transformation}'s uid. + * + * @param newUidPrefix for the transformations + * @return for chained calls + */ + public Builder uidPrefix(String newUidPrefix) { + this.uidPrefix = newUidPrefix; + return this; + } + + /** + * The {@link SingleOutputStreamOperator#slotSharingGroup(String)} for all the operators of the + * generated stream. Could be used to separate the resources used by this task. + * + * @param newSlotSharingGroup to be used for the operators + * @return for chained calls + */ + public Builder slotSharingGroup(String newSlotSharingGroup) { + this.slotSharingGroup = newSlotSharingGroup; + return this; + } + + /** + * Limits the firing frequency for the task triggers. + * + * @param newRateLimit firing frequency + * @return for chained calls + */ + public Builder rateLimit(Duration newRateLimit) { + Preconditions.checkNotNull(rateLimit.toMillis() > 0, "Rate limit should be greater than 0"); + this.rateLimit = newRateLimit; + return this; + } + + /** + * Sets the delay for checking lock availability when a concurrent run is detected. + * + * @param newConcurrentCheckDelay firing frequency + * @return for chained calls + */ + public Builder concurrentCheckDelay(Duration newConcurrentCheckDelay) { + this.concurrentCheckDelay = newConcurrentCheckDelay; + return this; + } + + /** + * Sets the global parallelism of maintenance tasks. Could be overwritten by the {@link + * MaintenanceTaskBuilder#parallelism(int)}. + * + * @param newParallelism task parallelism + * @return for chained calls + */ + public Builder parallelism(int newParallelism) { + this.parallelism = newParallelism; + return this; + } + + /** + * Maximum number of snapshots checked when started with an embedded {@link MonitorSource} at + * the first time. Only available when the {@link MonitorSource} is generated by the builder. + * + * @param newMaxReadBack snapshots to consider when initializing + * @return for chained calls + */ + public Builder maxReadBack(int newMaxReadBack) { + Preconditions.checkArgument( + changeStream == null, "Can't set maxReadBack when change stream is provided"); + this.maxReadBack = newMaxReadBack; + return this; + } + + /** + * Adds a specific task with the given schedule. + * + * @param task to add + * @return for chained calls + */ + public Builder add(MaintenanceTaskBuilder task) { + taskBuilders.add(task); + return this; + } + + /** Builds the task graph for the maintenance tasks. */ + public void append() { + Preconditions.checkArgument(!taskBuilders.isEmpty(), "Provide at least one task"); + Preconditions.checkNotNull(uidPrefix, "Uid prefix should no be null"); + + DataStream sourceStream; + if (changeStream == null) { + // Create a monitor source to provide the TableChange stream + MonitorSource source = + new MonitorSource( + tableLoader, + RateLimiterStrategy.perSecond(1.0 / rateLimit.getSeconds()), + maxReadBack); + sourceStream = + env.fromSource(source, WatermarkStrategy.noWatermarks(), SOURCE_NAME) + .uid(uidPrefix + "-monitor-source") + .slotSharingGroup(slotSharingGroup) + .forceNonParallel(); + } else { + sourceStream = changeStream.global(); + } + + // Chain the TriggerManager + List taskNames = Lists.newArrayListWithCapacity(taskBuilders.size()); + List evaluators = Lists.newArrayListWithCapacity(taskBuilders.size()); + for (int i = 0; i < taskBuilders.size(); ++i) { + taskNames.add(nameFor(taskBuilders.get(i), i)); + evaluators.add(taskBuilders.get(i).evaluator()); + } + + DataStream triggers = + // Add TriggerManager to schedule the tasks + DataStreamUtils.reinterpretAsKeyedStream(sourceStream, unused -> true) + .process( + new TriggerManager( + tableLoader, + lockFactory, + taskNames, + evaluators, + rateLimit.toMillis(), + concurrentCheckDelay.toMillis())) + .name(TRIGGER_MANAGER_TASK_NAME) + .uid(uidPrefix + "-trigger-manager") + .slotSharingGroup(slotSharingGroup) + .forceNonParallel() + // Add a watermark after every trigger + .assignTimestampsAndWatermarks(new WindowClosingWatermarkStrategy()) + .name("Watermark Assigner") + .uid(uidPrefix + "-watermark-assigner") + .slotSharingGroup(slotSharingGroup) + .forceNonParallel(); + + // Add the specific tasks + DataStream unioned = null; + for (int i = 0; i < taskBuilders.size(); ++i) { + DataStream filtered = + triggers + .flatMap(new TaskFilter(i)) + .name("Filter " + i) + .forceNonParallel() + .uid(uidPrefix + "-filter-" + i) + .slotSharingGroup(slotSharingGroup); + MaintenanceTaskBuilder builder = taskBuilders.get(i); + DataStream result = + builder.build( + filtered, + i, + taskNames.get(i), + tableLoader, + uidPrefix, + slotSharingGroup, + parallelism); + if (unioned == null) { + unioned = result; + } else { + unioned = unioned.union(result); + } + } + + // Add the LockRemover to the end + unioned + .global() + .transform( + LOCK_REMOVER_TASK_NAME, + TypeInformation.of(Void.class), + new LockRemover(lockFactory, taskNames)) + .forceNonParallel() + .uid(uidPrefix + "-lock-remover") + .slotSharingGroup(slotSharingGroup); + } + } + + private static String nameFor(MaintenanceTaskBuilder streamBuilder, int taskId) { + return String.format(TASK_NAME_FORMAT, streamBuilder.getClass().getSimpleName(), taskId); + } + + @Internal + public static class WindowClosingWatermarkStrategy implements WatermarkStrategy { + @Override + public WatermarkGenerator createWatermarkGenerator( + WatermarkGeneratorSupplier.Context context) { + return new WatermarkGenerator<>() { + @Override + public void onEvent(Trigger event, long eventTimestamp, WatermarkOutput output) { + output.emitWatermark(new Watermark(event.timestamp())); + } + + @Override + public void onPeriodicEmit(WatermarkOutput output) { + // No periodic watermarks + } + }; + } + + @Override + public TimestampAssigner createTimestampAssigner( + TimestampAssignerSupplier.Context context) { + return (element, unused) -> element.timestamp(); + } + } + + @Internal + private static class TaskFilter implements FlatMapFunction { + private final int taskId; + + private TaskFilter(int taskId) { + this.taskId = taskId; + } + + @Override + public void flatMap(Trigger trigger, Collector out) { + if (trigger.taskId() != null && trigger.taskId() == taskId) { + out.collect(trigger); + } + } + } +} diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/CollectingSink.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/CollectingSink.java index 9b6580fad0bf..e7e818ba6887 100644 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/CollectingSink.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/CollectingSink.java @@ -31,7 +31,7 @@ import org.apache.iceberg.relocated.com.google.common.collect.Lists; /** Sink for collecting output during testing. */ -class CollectingSink implements Sink { +public class CollectingSink implements Sink { private static final long serialVersionUID = 1L; private static final List> QUEUES = Collections.synchronizedList(Lists.newArrayListWithExpectedSize(1)); @@ -39,7 +39,7 @@ class CollectingSink implements Sink { private final int index; /** Creates a new sink which collects the elements received. */ - CollectingSink() { + public CollectingSink() { this.index = NUM_SINKS.incrementAndGet(); QUEUES.add(new LinkedBlockingQueue<>()); } @@ -69,7 +69,7 @@ boolean isEmpty() { * @return The first element received by this {@link Sink} * @throws TimeoutException if no element received until the timeout */ - T poll(Duration timeout) throws TimeoutException { + public T poll(Duration timeout) throws TimeoutException { Object element; try { diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/ConstantsForTests.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/ConstantsForTests.java deleted file mode 100644 index 36e162d4f068..000000000000 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/ConstantsForTests.java +++ /dev/null @@ -1,29 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.iceberg.flink.maintenance.operator; - -class ConstantsForTests { - public static final long EVENT_TIME = 10L; - static final long EVENT_TIME_2 = 11L; - static final String DUMMY_NAME = "dummy"; - - private ConstantsForTests() { - // Do not instantiate - } -} diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/FlinkSqlExtension.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/FlinkSqlExtension.java index 91d36aa3e85d..1e8ebcdeb256 100644 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/FlinkSqlExtension.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/FlinkSqlExtension.java @@ -125,6 +125,10 @@ public TableLoader tableLoader(String tableName) { return tableLoader; } + public CatalogLoader catalogLoader() { + return catalogLoader; + } + private static String toWithClause(Map props) { return String.format( "(%s)", diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/FlinkStreamingTestUtils.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/FlinkStreamingTestUtils.java deleted file mode 100644 index 9cdc55cb0cce..000000000000 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/FlinkStreamingTestUtils.java +++ /dev/null @@ -1,73 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.iceberg.flink.maintenance.operator; - -import java.io.File; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.core.execution.JobClient; -import org.apache.flink.core.execution.SavepointFormatType; -import org.apache.flink.runtime.jobgraph.SavepointConfigOptions; -import org.awaitility.Awaitility; - -class FlinkStreamingTestUtils { - private FlinkStreamingTestUtils() { - // Do not instantiate - } - - /** - * Close the {@link JobClient} and wait for the job closure. If the savepointDir is specified, it - * stops the job with a savepoint. - * - * @param jobClient the job to close - * @param savepointDir the savepointDir to store the last savepoint. If null then - * stop without a savepoint. - * @return configuration for restarting the job from the savepoint - */ - static Configuration closeJobClient(JobClient jobClient, File savepointDir) { - Configuration conf = new Configuration(); - if (jobClient != null) { - if (savepointDir != null) { - // Stop with savepoint - jobClient.stopWithSavepoint(false, savepointDir.getPath(), SavepointFormatType.CANONICAL); - // Wait until the savepoint is created and the job has been stopped - Awaitility.await().until(() -> savepointDir.listFiles(File::isDirectory).length == 1); - conf.set( - SavepointConfigOptions.SAVEPOINT_PATH, - savepointDir.listFiles(File::isDirectory)[0].getAbsolutePath()); - } else { - jobClient.cancel(); - } - - // Wait until the job has been stopped - Awaitility.await().until(() -> jobClient.getJobStatus().get().isTerminalState()); - return conf; - } - - return null; - } - - /** - * Close the {@link JobClient} and wait for the job closure. - * - * @param jobClient the job to close - */ - static void closeJobClient(JobClient jobClient) { - closeJobClient(jobClient, null); - } -} diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/ManualSource.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/ManualSource.java index 679b3ec508a2..eff32fcfa118 100644 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/ManualSource.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/ManualSource.java @@ -44,7 +44,7 @@ import org.jetbrains.annotations.Nullable; /** Testing source implementation for Flink sources which can be triggered manually. */ -class ManualSource +public class ManualSource implements Source, ResultTypeQueryable { @@ -65,7 +65,7 @@ class ManualSource * @param env to register the source * @param type of the events returned by the source */ - ManualSource(StreamExecutionEnvironment env, TypeInformation type) { + public ManualSource(StreamExecutionEnvironment env, TypeInformation type) { this.type = type; this.env = env; this.index = numSources++; @@ -78,7 +78,7 @@ class ManualSource * * @param event to emit */ - void sendRecord(T event) { + public void sendRecord(T event) { this.sendInternal(Tuple2.of(event, null)); } @@ -88,7 +88,7 @@ void sendRecord(T event) { * @param event to emit * @param eventTime of the event */ - void sendRecord(T event, long eventTime) { + public void sendRecord(T event, long eventTime) { this.sendInternal(Tuple2.of(event, eventTime)); } @@ -97,7 +97,7 @@ void sendRecord(T event, long eventTime) { * * @param timeStamp of the watermark */ - void sendWatermark(long timeStamp) { + public void sendWatermark(long timeStamp) { this.sendInternal(Tuple2.of(null, timeStamp)); } @@ -112,7 +112,7 @@ void markFinished() { * * @return the stream emitted by this source */ - DataStream dataStream() { + public DataStream dataStream() { if (this.stream == null) { this.stream = this.env diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/OperatorTestBase.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/OperatorTestBase.java index 225853086545..63ba6f587462 100644 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/OperatorTestBase.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/OperatorTestBase.java @@ -19,27 +19,41 @@ package org.apache.iceberg.flink.maintenance.operator; import static org.apache.iceberg.flink.MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG; +import static org.assertj.core.api.Assertions.assertThat; import java.io.File; +import java.io.IOException; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.MetricOptions; import org.apache.flink.core.execution.JobClient; import org.apache.flink.core.execution.SavepointFormatType; import org.apache.flink.runtime.jobgraph.SavepointConfigOptions; import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.graph.StreamGraphGenerator; +import org.apache.flink.streaming.api.transformations.SinkTransformation; import org.apache.flink.test.junit5.MiniClusterExtension; import org.apache.iceberg.flink.FlinkCatalogFactory; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.awaitility.Awaitility; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.extension.RegisterExtension; -class OperatorTestBase { +public class OperatorTestBase { private static final int NUMBER_TASK_MANAGERS = 1; private static final int SLOTS_PER_TASK_MANAGER = 8; - private static final TriggerLockFactory.Lock MAINTENANCE_LOCK = new MemoryLock(); - private static final TriggerLockFactory.Lock RECOVERY_LOCK = new MemoryLock(); - static final String TABLE_NAME = "test_table"; + protected static final String UID_PREFIX = "UID-Dummy"; + protected static final String SLOT_SHARING_GROUP = "SlotSharingGroup"; + protected static final String TABLE_NAME = "test_table"; + protected static final TriggerLockFactory LOCK_FACTORY = new MemoryLockFactory(); + + public static final String IGNORED_OPERATOR_NAME = "Ignore"; + + static final long EVENT_TIME = 10L; + static final long EVENT_TIME_2 = 11L; + protected static final String DUMMY_NAME = "dummy"; @RegisterExtension protected static final MiniClusterExtension MINI_CLUSTER_EXTENSION = @@ -51,42 +65,21 @@ class OperatorTestBase { .build()); @RegisterExtension - final FlinkSqlExtension sql = + public final FlinkSqlExtension sql = new FlinkSqlExtension( "catalog", ImmutableMap.of("type", "iceberg", FlinkCatalogFactory.ICEBERG_CATALOG_TYPE, "hadoop"), "db"); - private static Configuration config() { - Configuration config = new Configuration(DISABLE_CLASSLOADER_CHECK_CONFIG); - MetricOptions.forReporter(config, "test_reporter") - .set(MetricOptions.REPORTER_FACTORY_CLASS, MetricsReporterFactoryForTests.class.getName()); - return config; + @BeforeEach + void before() { + LOCK_FACTORY.open(); + MetricsReporterFactoryForTests.reset(); } - protected static TriggerLockFactory lockFactory() { - return new TriggerLockFactory() { - @Override - public void open() { - MAINTENANCE_LOCK.unlock(); - RECOVERY_LOCK.unlock(); - } - - @Override - public Lock createLock() { - return MAINTENANCE_LOCK; - } - - @Override - public Lock createRecoveryLock() { - return RECOVERY_LOCK; - } - - @Override - public void close() { - // do nothing - } - }; + @AfterEach + void after() throws IOException { + LOCK_FACTORY.close(); } /** @@ -98,7 +91,7 @@ public void close() { * stop without a savepoint. * @return configuration for restarting the job from the savepoint */ - public static Configuration closeJobClient(JobClient jobClient, File savepointDir) { + protected static Configuration closeJobClient(JobClient jobClient, File savepointDir) { Configuration conf = new Configuration(); if (jobClient != null) { if (savepointDir != null) { @@ -126,12 +119,45 @@ public static Configuration closeJobClient(JobClient jobClient, File savepointDi * * @param jobClient the job to close */ - public static void closeJobClient(JobClient jobClient) { + protected static void closeJobClient(JobClient jobClient) { closeJobClient(jobClient, null); } + protected static void checkUidsAreSet(StreamExecutionEnvironment env, String uidPrefix) { + env.getTransformations().stream() + .filter( + t -> !(t instanceof SinkTransformation) && !(t.getName().equals(IGNORED_OPERATOR_NAME))) + .forEach( + transformation -> { + assertThat(transformation.getUid()).isNotNull(); + if (uidPrefix != null) { + assertThat(transformation.getUid()).contains(UID_PREFIX); + } + }); + } + + protected static void checkSlotSharingGroupsAreSet(StreamExecutionEnvironment env, String name) { + String nameToCheck = name != null ? name : StreamGraphGenerator.DEFAULT_SLOT_SHARING_GROUP; + + env.getTransformations().stream() + .filter( + t -> !(t instanceof SinkTransformation) && !(t.getName().equals(IGNORED_OPERATOR_NAME))) + .forEach( + t -> { + assertThat(t.getSlotSharingGroup()).isPresent(); + assertThat(t.getSlotSharingGroup().get().getName()).isEqualTo(nameToCheck); + }); + } + + private static Configuration config() { + Configuration config = new Configuration(DISABLE_CLASSLOADER_CHECK_CONFIG); + MetricOptions.forReporter(config, "test_reporter") + .set(MetricOptions.REPORTER_FACTORY_CLASS, MetricsReporterFactoryForTests.class.getName()); + return config; + } + private static class MemoryLock implements TriggerLockFactory.Lock { - boolean locked = false; + volatile boolean locked = false; @Override public boolean tryLock() { @@ -153,4 +179,30 @@ public void unlock() { locked = false; } } + + private static class MemoryLockFactory implements TriggerLockFactory { + private static final TriggerLockFactory.Lock MAINTENANCE_LOCK = new MemoryLock(); + private static final TriggerLockFactory.Lock RECOVERY_LOCK = new MemoryLock(); + + @Override + public void open() { + MAINTENANCE_LOCK.unlock(); + RECOVERY_LOCK.unlock(); + } + + @Override + public Lock createLock() { + return MAINTENANCE_LOCK; + } + + @Override + public Lock createRecoveryLock() { + return RECOVERY_LOCK; + } + + @Override + public void close() { + // do nothing + } + } } diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestAsyncDeleteFiles.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestAsyncDeleteFiles.java new file mode 100644 index 000000000000..3e443fdbbc31 --- /dev/null +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestAsyncDeleteFiles.java @@ -0,0 +1,154 @@ +/* + * 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.operator; + +import static org.assertj.core.api.Assertions.assertThat; + +import java.io.Serializable; +import java.nio.charset.StandardCharsets; +import java.nio.file.FileSystems; +import java.nio.file.Files; +import java.nio.file.Path; +import java.util.Collection; +import java.util.List; +import java.util.function.Predicate; +import java.util.stream.Collectors; +import org.apache.flink.api.common.typeinfo.BasicTypeInfo; +import org.apache.flink.api.common.typeutils.base.StringSerializer; +import org.apache.flink.streaming.api.datastream.AsyncDataStream; +import org.apache.flink.streaming.api.operators.async.AsyncWaitOperatorFactory; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.apache.flink.streaming.runtime.tasks.OneInputStreamTask; +import org.apache.flink.streaming.runtime.tasks.StreamTaskMailboxTestHarness; +import org.apache.flink.streaming.runtime.tasks.StreamTaskMailboxTestHarnessBuilder; +import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness; +import org.apache.flink.streaming.util.retryable.AsyncRetryStrategies; +import org.apache.iceberg.Table; +import org.apache.iceberg.flink.TableLoader; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.awaitility.Awaitility; +import org.junit.jupiter.api.Test; + +class TestAsyncDeleteFiles extends OperatorTestBase { + private static final String DUMMY_FILE_NAME = "dummy"; + + @Test + void testDelete() throws Exception { + sql.exec("CREATE TABLE %s (id int, data varchar)", TABLE_NAME); + + TableLoader tableLoader = sql.tableLoader(TABLE_NAME); + Table table = tableLoader.loadTable(); + + // Write an extra files + Path dummyFile = + FileSystems.getDefault().getPath(table.location().substring(5), DUMMY_FILE_NAME); + Files.write(dummyFile, "DUMMY".getBytes(StandardCharsets.UTF_8)); + + List actual = deleteDummyFileAndWait(tableLoader); + + assertThat(actual).isEqualTo(ImmutableList.of(Boolean.TRUE)); + assertThat(Files.exists(dummyFile)).isFalse(); + } + + @Test + void testDeleteMissingFile() throws Exception { + sql.exec("CREATE TABLE %s (id int, data varchar)", TABLE_NAME); + + List actual = deleteDummyFileAndWait(sql.tableLoader(TABLE_NAME)); + + assertThat(actual).isEqualTo(ImmutableList.of(Boolean.TRUE)); + } + + @Test + void testWrongFile() throws Exception { + sql.exec("CREATE TABLE %s (id int, data varchar)", TABLE_NAME); + + StreamTaskMailboxTestHarnessBuilder builder = + new StreamTaskMailboxTestHarnessBuilder<>( + OneInputStreamTask::new, BasicTypeInfo.BOOLEAN_TYPE_INFO) + .addInput(BasicTypeInfo.STRING_TYPE_INFO); + CountingPredicateChecker predicate = new CountingPredicateChecker(); + + try (StreamTaskMailboxTestHarness testHarness = + builder + .setupOutputForSingletonOperatorChain( + asyncWaitOperatorFactory(sql.tableLoader(TABLE_NAME), predicate)) + .build()) { + testHarness.processElement(new StreamRecord<>("wrong://", System.currentTimeMillis())); + + while (testHarness.getOutput().isEmpty()) { + Awaitility.await().until(() -> testHarness.getOutput().isEmpty()); + testHarness.processAll(); + } + + // Make sure that we do not complete immediately + assertThat(CountingPredicateChecker.calls).isEqualTo(3); + + // The result still should be fail + assertThat( + testHarness.getOutput().stream() + .map(r -> ((StreamRecord) r).getValue()) + .collect(Collectors.toList())) + .isEqualTo(ImmutableList.of(Boolean.FALSE)); + } + } + + private List deleteDummyFileAndWait(TableLoader tableLoader) throws Exception { + Table table = tableLoader.loadTable(); + + try (OneInputStreamOperatorTestHarness testHarness = + new OneInputStreamOperatorTestHarness<>( + asyncWaitOperatorFactory(tableLoader, new CountingPredicateChecker()), + StringSerializer.INSTANCE)) { + testHarness.open(); + testHarness.processElement( + table.location() + "/" + DUMMY_FILE_NAME, System.currentTimeMillis()); + + // wait until all async collectors in the buffer have been emitted out. + testHarness.endInput(); + testHarness.close(); + + return testHarness.extractOutputValues(); + } + } + + private AsyncWaitOperatorFactory asyncWaitOperatorFactory( + TableLoader tableLoader, Predicate> predicate) { + return new AsyncWaitOperatorFactory<>( + new AsyncDeleteFiles(DUMMY_NAME, tableLoader, 10), + 1000000, + 10, + AsyncDataStream.OutputMode.ORDERED, + new AsyncRetryStrategies.ExponentialBackoffDelayRetryStrategyBuilder( + 2, 10, 1000, 1.5) + .ifResult(predicate) + .build()); + } + + private static class CountingPredicateChecker + implements Predicate>, Serializable { + private static int calls = 0; + + @Override + public boolean test(Collection param) { + ++calls; + return AsyncDeleteFiles.FAILED_PREDICATE.test(param); + } + } +} diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestExpireSnapshotsProcessor.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestExpireSnapshotsProcessor.java new file mode 100644 index 000000000000..79af082a0d44 --- /dev/null +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestExpireSnapshotsProcessor.java @@ -0,0 +1,86 @@ +/* + * 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.operator; + +import static org.assertj.core.api.Assertions.assertThat; + +import java.util.List; +import java.util.Queue; +import java.util.Set; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness; +import org.apache.flink.streaming.util.ProcessFunctionTestHarnesses; +import org.apache.iceberg.SerializableTable; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.Table; +import org.apache.iceberg.flink.TableLoader; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.ValueSource; + +class TestExpireSnapshotsProcessor extends OperatorTestBase { + @ParameterizedTest + @ValueSource(booleans = {true, false}) + void testExpire(boolean success) throws Exception { + sql.exec("CREATE TABLE %s (id int, data varchar, spec varchar)", TABLE_NAME); + sql.exec("INSERT INTO %s VALUES (1, 'a', 'p1')", TABLE_NAME); + sql.exec("INSERT INTO %s VALUES (2, 'b', 'p2')", TABLE_NAME); + + TableLoader tableLoader = sql.tableLoader(TABLE_NAME); + Table table = tableLoader.loadTable(); + SerializableTable serializableTable = (SerializableTable) SerializableTable.copyOf(table); + + List actual; + Queue> deletes; + try (OneInputStreamOperatorTestHarness testHarness = + ProcessFunctionTestHarnesses.forProcessFunction( + new ExpireSnapshotsProcessor(tableLoader, 0L, 1, 10))) { + testHarness.open(); + + if (!success) { + // Cause an exception + sql.exec("DROP TABLE IF EXISTS %s", TABLE_NAME); + } + + testHarness.processElement( + Trigger.create(10, serializableTable, 11), System.currentTimeMillis()); + deletes = testHarness.getSideOutput(ExpireSnapshotsProcessor.DELETE_STREAM); + actual = testHarness.extractOutputValues(); + } + + assertThat(actual).hasSize(1); + TaskResult result = actual.get(0); + assertThat(result.startEpoch()).isEqualTo(10); + assertThat(result.taskIndex()).isEqualTo(11); + assertThat(result.success()).isEqualTo(success); + + if (success) { + assertThat(result.exceptions()).isNotNull().isEmpty(); + + table.refresh(); + Set snapshots = Sets.newHashSet(table.snapshots()); + assertThat(snapshots).hasSize(1); + assertThat(deletes).hasSize(1); + } else { + assertThat(result.exceptions()).isNotNull().hasSize(1); + + assertThat(deletes).isNull(); + } + } +} diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestLockRemover.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestLockRemover.java index ccb90ec33d9c..f6f6f932e9d0 100644 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestLockRemover.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestLockRemover.java @@ -18,8 +18,8 @@ */ package org.apache.iceberg.flink.maintenance.operator; -import static org.apache.iceberg.flink.maintenance.operator.ConstantsForTests.DUMMY_NAME; import static org.apache.iceberg.flink.maintenance.operator.TableMaintenanceMetrics.FAILED_TASK_COUNTER; +import static org.apache.iceberg.flink.maintenance.operator.TableMaintenanceMetrics.LAST_RUN_DURATION_MS; import static org.apache.iceberg.flink.maintenance.operator.TableMaintenanceMetrics.SUCCEEDED_TASK_COUNTER; import static org.assertj.core.api.Assertions.assertThat; @@ -54,7 +54,7 @@ @Timeout(value = 10) class TestLockRemover extends OperatorTestBase { - private static final String[] TASKS = new String[] {"task0", "task1"}; + private static final String[] TASKS = new String[] {"task0", "task1", "task2"}; private static final TriggerLockFactory.Lock LOCK = new TestingLock(); private static final TriggerLockFactory.Lock RECOVERY_LOCK = new TestingLock(); @@ -135,15 +135,16 @@ void testMetrics() throws Exception { .setParallelism(1); JobClient jobClient = null; + long time = System.currentTimeMillis(); try { jobClient = env.executeAsync(); // Start the 2 successful and one failed result trigger for task1, and 3 successful for task2 - processAndCheck(source, new TaskResult(0, 0L, true, Lists.newArrayList())); - processAndCheck(source, new TaskResult(1, 1L, true, Lists.newArrayList())); - processAndCheck(source, new TaskResult(1, 2L, true, Lists.newArrayList())); - processAndCheck(source, new TaskResult(0, 3L, false, Lists.newArrayList())); - processAndCheck(source, new TaskResult(0, 4L, true, Lists.newArrayList())); - processAndCheck(source, new TaskResult(1, 5L, true, Lists.newArrayList())); + processAndCheck(source, new TaskResult(0, time, true, Lists.newArrayList())); + processAndCheck(source, new TaskResult(1, 0L, true, Lists.newArrayList())); + processAndCheck(source, new TaskResult(1, 0L, true, Lists.newArrayList())); + processAndCheck(source, new TaskResult(0, time, false, Lists.newArrayList())); + processAndCheck(source, new TaskResult(0, time, true, Lists.newArrayList())); + processAndCheck(source, new TaskResult(1, 0L, true, Lists.newArrayList())); Awaitility.await() .until( @@ -159,7 +160,22 @@ void testMetrics() throws Exception { .put(DUMMY_NAME + "." + TASKS[0] + "." + FAILED_TASK_COUNTER, 1L) .put(DUMMY_NAME + "." + TASKS[1] + "." + SUCCEEDED_TASK_COUNTER, 3L) .put(DUMMY_NAME + "." + TASKS[1] + "." + FAILED_TASK_COUNTER, 0L) + .put(DUMMY_NAME + "." + TASKS[2] + "." + SUCCEEDED_TASK_COUNTER, 0L) + .put(DUMMY_NAME + "." + TASKS[2] + "." + FAILED_TASK_COUNTER, 0L) .build()); + + assertThat( + MetricsReporterFactoryForTests.gauge( + DUMMY_NAME + "." + TASKS[0] + "." + LAST_RUN_DURATION_MS)) + .isPositive(); + assertThat( + MetricsReporterFactoryForTests.gauge( + DUMMY_NAME + "." + TASKS[1] + "." + LAST_RUN_DURATION_MS)) + .isGreaterThan(time); + assertThat( + MetricsReporterFactoryForTests.gauge( + DUMMY_NAME + "." + TASKS[2] + "." + LAST_RUN_DURATION_MS)) + .isZero(); } finally { closeJobClient(jobClient); } diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestTriggerManager.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestTriggerManager.java index fba4a12d9c6b..c761c2904186 100644 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestTriggerManager.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestTriggerManager.java @@ -18,9 +18,6 @@ */ package org.apache.iceberg.flink.maintenance.operator; -import static org.apache.iceberg.flink.maintenance.operator.ConstantsForTests.DUMMY_NAME; -import static org.apache.iceberg.flink.maintenance.operator.ConstantsForTests.EVENT_TIME; -import static org.apache.iceberg.flink.maintenance.operator.ConstantsForTests.EVENT_TIME_2; import static org.apache.iceberg.flink.maintenance.operator.TableMaintenanceMetrics.CONCURRENT_RUN_THROTTLED; import static org.apache.iceberg.flink.maintenance.operator.TableMaintenanceMetrics.GROUP_VALUE_DEFAULT; import static org.apache.iceberg.flink.maintenance.operator.TableMaintenanceMetrics.NOTHING_TO_TRIGGER; @@ -28,7 +25,6 @@ import static org.apache.iceberg.flink.maintenance.operator.TableMaintenanceMetrics.TRIGGERED; import static org.assertj.core.api.Assertions.assertThat; -import java.io.IOException; import java.time.Duration; import java.util.Iterator; import java.util.List; @@ -47,7 +43,6 @@ 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.AfterEach; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import org.junit.jupiter.params.ParameterizedTest; @@ -56,28 +51,16 @@ class TestTriggerManager extends OperatorTestBase { private static final long DELAY = 10L; - private static final String NAME_1 = "name1"; - private static final String NAME_2 = "name2"; + private static final String[] TASKS = new String[] {"task0", "task1"}; private long processingTime = 0L; - private TriggerLockFactory lockFactory; private TriggerLockFactory.Lock lock; private TriggerLockFactory.Lock recoveringLock; @BeforeEach void before() { sql.exec("CREATE TABLE %s (id int, data varchar)", TABLE_NAME); - this.lockFactory = lockFactory(); - lockFactory.open(); - this.lock = lockFactory.createLock(); - this.recoveringLock = lockFactory.createRecoveryLock(); - lock.unlock(); - recoveringLock.unlock(); - MetricsReporterFactoryForTests.reset(); - } - - @AfterEach - void after() throws IOException { - lockFactory.close(); + this.lock = LOCK_FACTORY.createLock(); + this.recoveringLock = LOCK_FACTORY.createRecoveryLock(); } @Test @@ -444,8 +427,8 @@ void testTriggerMetrics() throws Exception { TriggerManager manager = new TriggerManager( tableLoader, - lockFactory, - Lists.newArrayList(NAME_1, NAME_2), + LOCK_FACTORY, + Lists.newArrayList(TASKS), Lists.newArrayList( new TriggerEvaluator.Builder().commitCount(2).build(), new TriggerEvaluator.Builder().commitCount(4).build()), @@ -480,7 +463,7 @@ void testTriggerMetrics() throws Exception { // Wait until we receive the trigger assertThat(sink.poll(Duration.ofSeconds(5))).isNotNull(); assertThat( - MetricsReporterFactoryForTests.counter(DUMMY_NAME + "." + NAME_1 + "." + TRIGGERED)) + MetricsReporterFactoryForTests.counter(DUMMY_NAME + "." + TASKS[0] + "." + TRIGGERED)) .isEqualTo(1L); lock.unlock(); @@ -492,10 +475,10 @@ void testTriggerMetrics() throws Exception { assertThat(sink.poll(Duration.ofSeconds(5))).isNotNull(); lock.unlock(); assertThat( - MetricsReporterFactoryForTests.counter(DUMMY_NAME + "." + NAME_1 + "." + TRIGGERED)) + MetricsReporterFactoryForTests.counter(DUMMY_NAME + "." + TASKS[0] + "." + TRIGGERED)) .isEqualTo(2L); assertThat( - MetricsReporterFactoryForTests.counter(DUMMY_NAME + "." + NAME_2 + "." + TRIGGERED)) + MetricsReporterFactoryForTests.counter(DUMMY_NAME + "." + TASKS[1] + "." + TRIGGERED)) .isEqualTo(1L); // Final check all the counters @@ -503,8 +486,8 @@ void testTriggerMetrics() throws Exception { new ImmutableMap.Builder() .put(DUMMY_NAME + "." + GROUP_VALUE_DEFAULT + "." + RATE_LIMITER_TRIGGERED, -1L) .put(DUMMY_NAME + "." + GROUP_VALUE_DEFAULT + "." + CONCURRENT_RUN_THROTTLED, -1L) - .put(DUMMY_NAME + "." + NAME_1 + "." + TRIGGERED, 2L) - .put(DUMMY_NAME + "." + NAME_2 + "." + TRIGGERED, 1L) + .put(DUMMY_NAME + "." + TASKS[0] + "." + TRIGGERED, 2L) + .put(DUMMY_NAME + "." + TASKS[1] + "." + TRIGGERED, 1L) .put(DUMMY_NAME + "." + GROUP_VALUE_DEFAULT + "." + NOTHING_TO_TRIGGER, 1L) .build()); } finally { @@ -618,7 +601,7 @@ private void assertCounters(long rateLimiterTrigger, long concurrentRunTrigger) .put( DUMMY_NAME + "." + GROUP_VALUE_DEFAULT + "." + CONCURRENT_RUN_THROTTLED, concurrentRunTrigger) - .put(DUMMY_NAME + "." + NAME_1 + "." + TRIGGERED, 1L) + .put(DUMMY_NAME + "." + TASKS[0] + "." + TRIGGERED, 1L) .put(DUMMY_NAME + "." + GROUP_VALUE_DEFAULT + "." + NOTHING_TO_TRIGGER, 0L) .build()); } @@ -644,15 +627,20 @@ private void addEventAndCheckResult( private TriggerManager manager(TableLoader tableLoader, TriggerEvaluator evaluator) { return new TriggerManager( - tableLoader, lockFactory, Lists.newArrayList(NAME_1), Lists.newArrayList(evaluator), 1, 1); + tableLoader, + LOCK_FACTORY, + Lists.newArrayList(TASKS[0]), + Lists.newArrayList(evaluator), + 1, + 1); } private TriggerManager manager( TableLoader tableLoader, long minFireDelayMs, long lockCheckDelayMs) { return new TriggerManager( tableLoader, - lockFactory, - Lists.newArrayList(NAME_1), + LOCK_FACTORY, + Lists.newArrayList(TASKS[0]), Lists.newArrayList(new TriggerEvaluator.Builder().commitCount(2).build()), minFireDelayMs, lockCheckDelayMs); diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/stream/ScheduledBuilderTestBase.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/stream/ScheduledBuilderTestBase.java new file mode 100644 index 000000000000..a2474c0f0809 --- /dev/null +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/stream/ScheduledBuilderTestBase.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 org.apache.iceberg.flink.maintenance.stream; + +import static org.assertj.core.api.Assertions.assertThat; + +import java.time.Duration; +import java.util.function.Supplier; +import org.apache.flink.core.execution.JobClient; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.iceberg.SerializableTable; +import org.apache.iceberg.Table; +import org.apache.iceberg.flink.maintenance.operator.CollectingSink; +import org.apache.iceberg.flink.maintenance.operator.ManualSource; +import org.apache.iceberg.flink.maintenance.operator.OperatorTestBase; +import org.apache.iceberg.flink.maintenance.operator.TaskResult; +import org.apache.iceberg.flink.maintenance.operator.Trigger; +import org.awaitility.Awaitility; +import org.junit.jupiter.api.extension.RegisterExtension; + +class ScheduledBuilderTestBase extends OperatorTestBase { + private static final int TESTING_TASK_ID = 0; + private static final Duration POLL_DURATION = Duration.ofSeconds(5); + static final String DB_NAME = "db"; + + @RegisterExtension ScheduledInfraExtension infra = new ScheduledInfraExtension(); + + /** + * Triggers a maintenance tasks and waits for the successful result. The {@link Table} is + * refreshed for convenience reasons. + * + * @param env used for testing + * @param triggerSource used for manually emitting the trigger + * @param collectingSink used for collecting the result + * @param table used for generating the payload + * @throws Exception if any + */ + void runAndWaitForSuccess( + StreamExecutionEnvironment env, + ManualSource triggerSource, + CollectingSink collectingSink, + Supplier checkSideEffect, + Table table) + throws Exception { + table.refresh(); + SerializableTable payload = (SerializableTable) SerializableTable.copyOf(table); + + JobClient jobClient = null; + try { + jobClient = env.executeAsync(); + + // Do a single task run + long time = System.currentTimeMillis(); + triggerSource.sendRecord(Trigger.create(time, payload, TESTING_TASK_ID), time); + + TaskResult result = collectingSink.poll(POLL_DURATION); + + assertThat(result.startEpoch()).isEqualTo(time); + assertThat(result.success()).isTrue(); + assertThat(result.taskIndex()).isEqualTo(TESTING_TASK_ID); + + Awaitility.await().until(() -> checkSideEffect.get()); + } finally { + closeJobClient(jobClient); + } + + table.refresh(); + } +} diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/stream/ScheduledInfraExtension.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/stream/ScheduledInfraExtension.java new file mode 100644 index 000000000000..210c003f183a --- /dev/null +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/stream/ScheduledInfraExtension.java @@ -0,0 +1,80 @@ +/* + * 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.maintenance.operator.OperatorTestBase.IGNORED_OPERATOR_NAME; + +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.iceberg.flink.maintenance.operator.CollectingSink; +import org.apache.iceberg.flink.maintenance.operator.ManualSource; +import org.apache.iceberg.flink.maintenance.operator.TaskResult; +import org.apache.iceberg.flink.maintenance.operator.Trigger; +import org.junit.jupiter.api.extension.BeforeEachCallback; +import org.junit.jupiter.api.extension.ExtensionContext; + +/** + * {@link org.junit.jupiter.api.extension.Extension} used to generate the common elements for the + * {@link MaintenanceTaskBuilder} implementations. These are the following: + * + *

    + *
  • {@link StreamExecutionEnvironment} - environment for testing + *
  • {@link ManualSource} - source for manually emitting {@link Trigger}s + *
  • {@link DataStream} - which generated from the {@link ManualSource} + *
  • {@link CollectingSink} - which could be used poll for the records emitted by the + * maintenance tasks + *
+ */ +class ScheduledInfraExtension implements BeforeEachCallback { + private StreamExecutionEnvironment env; + private ManualSource source; + private DataStream triggerStream; + private CollectingSink sink; + + @Override + public void beforeEach(ExtensionContext context) { + env = StreamExecutionEnvironment.getExecutionEnvironment(); + source = new ManualSource<>(env, TypeInformation.of(Trigger.class)); + // Adds the watermark to mimic the behaviour expected for the input of the maintenance tasks + triggerStream = + source + .dataStream() + .assignTimestampsAndWatermarks(new TableMaintenance.WindowClosingWatermarkStrategy()) + .name(IGNORED_OPERATOR_NAME) + .forceNonParallel(); + sink = new CollectingSink<>(); + } + + StreamExecutionEnvironment env() { + return env; + } + + ManualSource source() { + return source; + } + + DataStream triggerStream() { + return triggerStream; + } + + CollectingSink sink() { + return sink; + } +} diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/stream/TestExpireSnapshots.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/stream/TestExpireSnapshots.java new file mode 100644 index 000000000000..8d3c8864ebe3 --- /dev/null +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/stream/TestExpireSnapshots.java @@ -0,0 +1,248 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.maintenance.stream; + +import static org.apache.iceberg.flink.SimpleDataUtil.createRecord; +import static org.apache.iceberg.flink.maintenance.operator.TableMaintenanceMetrics.DELETE_FILE_FAILED_COUNTER; +import static org.apache.iceberg.flink.maintenance.operator.TableMaintenanceMetrics.DELETE_FILE_SUCCEEDED_COUNTER; +import static org.apache.iceberg.flink.maintenance.stream.ExpireSnapshots.DELETE_FILES_TASK_NAME; +import static org.assertj.core.api.Assertions.assertThat; + +import java.time.Duration; +import java.util.Set; +import org.apache.flink.core.execution.JobClient; +import org.apache.flink.streaming.api.graph.StreamGraphGenerator; +import org.apache.iceberg.SerializableTable; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.Table; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.flink.SimpleDataUtil; +import org.apache.iceberg.flink.TableLoader; +import org.apache.iceberg.flink.maintenance.operator.MetricsReporterFactoryForTests; +import org.apache.iceberg.flink.maintenance.operator.Trigger; +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.Sets; +import org.awaitility.Awaitility; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +class TestExpireSnapshots extends ScheduledBuilderTestBase { + @BeforeEach + void before() { + MetricsReporterFactoryForTests.reset(); + } + + @Test + void testExpireSnapshots() throws Exception { + sql.exec("CREATE TABLE %s (id int, data varchar)", TABLE_NAME); + sql.exec("INSERT INTO %s VALUES (1, 'a')", TABLE_NAME); + sql.exec("INSERT INTO %s VALUES (2, 'b')", TABLE_NAME); + sql.exec("INSERT INTO %s VALUES (3, 'c')", TABLE_NAME); + sql.exec("INSERT INTO %s VALUES (4, 'd')", TABLE_NAME); + + TableLoader tableLoader = sql.tableLoader(TABLE_NAME); + Table table = tableLoader.loadTable(); + Set snapshots = Sets.newHashSet(table.snapshots()); + assertThat(snapshots).hasSize(4); + + ExpireSnapshots.builder() + .parallelism(1) + .planningWorkerPoolSize(2) + .deleteAttemptNum(2) + .deleteWorkerPoolSize(5) + .minAge(Duration.ZERO) + .retainLast(1) + .uidPrefix(UID_PREFIX) + .build( + infra.triggerStream(), + 0, + DUMMY_NAME, + tableLoader, + "OTHER", + StreamGraphGenerator.DEFAULT_SLOT_SHARING_GROUP, + 1) + .sinkTo(infra.sink()); + + runAndWaitForSuccess( + infra.env(), infra.source(), infra.sink(), () -> checkDeleteFinished(3L), table); + + // Check that the table data not changed + table.refresh(); + assertThat(Sets.newHashSet(table.snapshots())).hasSize(1); + SimpleDataUtil.assertTableRecords( + table, + ImmutableList.of( + createRecord(1, "a"), + createRecord(2, "b"), + createRecord(3, "c"), + createRecord(4, "d"))); + } + + @Test + void testFailure() throws Exception { + sql.exec("CREATE TABLE %s (id int, data varchar)", TABLE_NAME); + sql.exec("INSERT INTO %s VALUES (1, 'a')", TABLE_NAME); + sql.exec("INSERT INTO %s VALUES (2, 'b')", TABLE_NAME); + + TableLoader tableLoader = sql.tableLoader(TABLE_NAME); + Table table = tableLoader.loadTable(); + SerializableTable serializableTable = (SerializableTable) SerializableTable.copyOf(table); + + ExpireSnapshots.builder() + .build( + infra.triggerStream(), + 0, + DUMMY_NAME, + tableLoader, + UID_PREFIX, + StreamGraphGenerator.DEFAULT_SLOT_SHARING_GROUP, + 1) + .sinkTo(infra.sink()); + + JobClient jobClient = null; + try { + jobClient = infra.env().executeAsync(); + + // Do a single task run + long time = System.currentTimeMillis(); + infra + .source() + .sendRecord(Trigger.create(time, serializableTable, 1), System.currentTimeMillis()); + + // First successful run (ensure that the operators are loaded/opened etc.) + assertThat(infra.sink().poll(Duration.ofSeconds(5)).success()).isTrue(); + + // Drop the table, so it will cause an exception + sql.catalogLoader().loadCatalog().dropTable(TableIdentifier.of(DB_NAME, TABLE_NAME)); + + // Failed run + infra.source().sendRecord(Trigger.create(time + 1, serializableTable, 1), time + 1); + + assertThat(infra.sink().poll(Duration.ofSeconds(5)).success()).isFalse(); + } finally { + closeJobClient(jobClient); + } + + // Check the metrics + MetricsReporterFactoryForTests.assertCounters( + new ImmutableMap.Builder() + .put(DELETE_FILES_TASK_NAME + "." + DUMMY_NAME + "." + DELETE_FILE_FAILED_COUNTER, 0L) + .put( + DELETE_FILES_TASK_NAME + "." + DUMMY_NAME + "." + DELETE_FILE_SUCCEEDED_COUNTER, 0L) + .build()); + } + + @Test + void testUidAndSlotSharingGroup() { + sql.exec("CREATE TABLE %s (id int, data varchar)", TABLE_NAME); + TableLoader tableLoader = sql.tableLoader(TABLE_NAME); + + ExpireSnapshots.builder() + .slotSharingGroup(SLOT_SHARING_GROUP) + .uidPrefix(UID_PREFIX) + .build( + infra.triggerStream(), + 0, + DUMMY_NAME, + tableLoader, + UID_PREFIX, + StreamGraphGenerator.DEFAULT_SLOT_SHARING_GROUP, + 1) + .sinkTo(infra.sink()); + + checkUidsAreSet(infra.env(), UID_PREFIX); + checkSlotSharingGroupsAreSet(infra.env(), SLOT_SHARING_GROUP); + } + + @Test + void testUidAndSlotSharingGroupUnset() { + sql.exec("CREATE TABLE %s (id int, data varchar)", TABLE_NAME); + TableLoader tableLoader = sql.tableLoader(TABLE_NAME); + + ExpireSnapshots.builder() + .build( + infra.triggerStream(), + 0, + DUMMY_NAME, + tableLoader, + UID_PREFIX, + StreamGraphGenerator.DEFAULT_SLOT_SHARING_GROUP, + 1) + .sinkTo(infra.sink()); + + checkUidsAreSet(infra.env(), null); + checkSlotSharingGroupsAreSet(infra.env(), null); + } + + @Test + void testMetrics() throws Exception { + sql.exec("CREATE TABLE %s (id int, data varchar)", TABLE_NAME); + sql.exec("INSERT INTO %s VALUES (1, 'a')", TABLE_NAME); + sql.exec("INSERT INTO %s VALUES (2, 'b')", TABLE_NAME); + + TableLoader tableLoader = sql.tableLoader(TABLE_NAME); + Table table = tableLoader.loadTable(); + + ExpireSnapshots.builder() + .minAge(Duration.ZERO) + .retainLast(1) + .parallelism(1) + .build( + infra.triggerStream(), + 0, + DUMMY_NAME, + tableLoader, + UID_PREFIX, + StreamGraphGenerator.DEFAULT_SLOT_SHARING_GROUP, + 1) + .sinkTo(infra.sink()); + + runAndWaitForSuccess( + infra.env(), infra.source(), infra.sink(), () -> checkDeleteFinished(1L), table); + + // Check the metrics + Awaitility.await() + .untilAsserted( + () -> + MetricsReporterFactoryForTests.assertCounters( + new ImmutableMap.Builder() + .put( + DELETE_FILES_TASK_NAME + + "." + + DUMMY_NAME + + "." + + DELETE_FILE_FAILED_COUNTER, + 0L) + .put( + DELETE_FILES_TASK_NAME + + "." + + DUMMY_NAME + + "." + + DELETE_FILE_SUCCEEDED_COUNTER, + 1L) + .build())); + } + + private static boolean checkDeleteFinished(Long expectedDeleteNum) { + return expectedDeleteNum.equals( + MetricsReporterFactoryForTests.counter( + DELETE_FILES_TASK_NAME + "." + DUMMY_NAME + "." + DELETE_FILE_SUCCEEDED_COUNTER)); + } +} diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/stream/TestMaintenanceE2E.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/stream/TestMaintenanceE2E.java new file mode 100644 index 000000000000..fd9b136af21d --- /dev/null +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/stream/TestMaintenanceE2E.java @@ -0,0 +1,69 @@ +/* + * 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.assertj.core.api.Assertions.assertThat; + +import java.time.Duration; +import org.apache.flink.core.execution.JobClient; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.iceberg.flink.TableLoader; +import org.apache.iceberg.flink.maintenance.operator.OperatorTestBase; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +class TestMaintenanceE2E extends OperatorTestBase { + private StreamExecutionEnvironment env; + + @BeforeEach + public void beforeEach() { + env = StreamExecutionEnvironment.getExecutionEnvironment(); + } + + @Test + void testE2e() 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); + + TableMaintenance.builder(env, tableLoader, LOCK_FACTORY) + .uidPrefix("E2eTestUID") + .rateLimit(Duration.ofMinutes(10)) + .concurrentCheckDelay(Duration.ofSeconds(10)) + .add( + ExpireSnapshots.builder() + .scheduleOnCommitCount(10) + .minAge(Duration.ofMinutes(10)) + .retainLast(5) + .deleteWorkerPoolSize(5) + .parallelism(8)) + .append(); + + JobClient jobClient = null; + try { + jobClient = env.executeAsync(); + + // Just make sure that we are able to instantiate the flow + assertThat(jobClient).isNotNull(); + } finally { + closeJobClient(jobClient); + } + } +} diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/stream/TestTableMaintenance.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/stream/TestTableMaintenance.java new file mode 100644 index 000000000000..c7fa51675886 --- /dev/null +++ b/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 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 schedulerSource = + new ManualSource<>(env, TypeInformation.of(TableChange.class)); + + TableMaintenance.Builder streamBuilder = + TableMaintenance.builder(schedulerSource.dataStream(), tableLoader, LOCK_FACTORY) + .rateLimit(Duration.ofMillis(2)) + .concurrentCheckDelay(Duration.ofSeconds(3)) + .add( + new MaintenanceTaskBuilderForTest(true) + .scheduleOnCommitCount(1) + .scheduleOnDataFileCount(2) + .scheduleOnDataFileSize(3L) + .scheduleOnEqDeleteFileCount(4) + .scheduleOnEqDeleteRecordCount(5L) + .schedulerOnPosDeleteFileCount(6) + .schedulerOnPosDeleteRecordCount(7L) + .scheduleOnTime(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.builder(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 insertSource = + new ManualSource<>(env, InternalTypeInfo.of(FlinkSchemaUtil.convert(table.schema()))); + FlinkSink.forRowData(insertSource.dataStream()) + .tableLoader(tableLoader) + .uidPrefix(UID_PREFIX + "-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 { + 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(); + + TriggerLockFactory.Lock lock = LOCK_FACTORY.createLock(); + + ManualSource schedulerSource = + new ManualSource<>(env, TypeInformation.of(TableChange.class)); + + TableMaintenance.Builder streamBuilder = + TableMaintenance.builder(schedulerSource.dataStream(), tableLoader, LOCK_FACTORY) + .rateLimit(Duration.ofMillis(2)) + .add(new MaintenanceTaskBuilderForTest(true).scheduleOnCommitCount(1)); + + assertThat(lock.isHeld()).isFalse(); + sendEvents(schedulerSource, streamBuilder, ImmutableList.of(Tuple2.of(DUMMY_CHANGE, 1))); + + assertThat(lock.isHeld()).isFalse(); + } + + @Test + void testMetrics() 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 schedulerSource = + new ManualSource<>(env, TypeInformation.of(TableChange.class)); + + TableMaintenance.Builder streamBuilder = + TableMaintenance.builder(schedulerSource.dataStream(), tableLoader, LOCK_FACTORY) + .rateLimit(Duration.ofMillis(2)) + .concurrentCheckDelay(Duration.ofMillis(2)) + .add(new MaintenanceTaskBuilderForTest(true).scheduleOnCommitCount(1)) + .add(new MaintenanceTaskBuilderForTest(false).scheduleOnCommitCount(2)); + + sendEvents( + schedulerSource, + streamBuilder, + ImmutableList.of(Tuple2.of(DUMMY_CHANGE, 1), Tuple2.of(DUMMY_CHANGE, 2))); + + Awaitility.await() + .until( + () -> + MetricsReporterFactoryForTests.counter( + LOCK_REMOVER_TASK_NAME + "." + TASKS[0] + "." + SUCCEEDED_TASK_COUNTER) + .equals(2L)); + + MetricsReporterFactoryForTests.assertCounters( + new ImmutableMap.Builder() + .put(LOCK_REMOVER_TASK_NAME + "." + TASKS[0] + "." + SUCCEEDED_TASK_COUNTER, 2L) + .put(LOCK_REMOVER_TASK_NAME + "." + TASKS[0] + "." + FAILED_TASK_COUNTER, 0L) + .put(TRIGGER_MANAGER_TASK_NAME + "." + TASKS[0] + "." + TRIGGERED, 2L) + .put(LOCK_REMOVER_TASK_NAME + "." + TASKS[1] + "." + SUCCEEDED_TASK_COUNTER, 0L) + .put(LOCK_REMOVER_TASK_NAME + "." + TASKS[1] + "." + FAILED_TASK_COUNTER, 1L) + .put(TRIGGER_MANAGER_TASK_NAME + "." + TASKS[1] + "." + TRIGGERED, 1L) + .put( + TRIGGER_MANAGER_TASK_NAME + "." + GROUP_VALUE_DEFAULT + "." + NOTHING_TO_TRIGGER, + -1L) + .put( + TRIGGER_MANAGER_TASK_NAME + + "." + + GROUP_VALUE_DEFAULT + + "." + + CONCURRENT_RUN_THROTTLED, + -1L) + .put( + TRIGGER_MANAGER_TASK_NAME + + "." + + GROUP_VALUE_DEFAULT + + "." + + RATE_LIMITER_TRIGGERED, + -1L) + .build()); + } + + @Test + void testUidAndSlotSharingGroup() { + 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(); + + TableMaintenance.builder( + new ManualSource<>(env, TypeInformation.of(TableChange.class)).dataStream(), + tableLoader, + LOCK_FACTORY) + .uidPrefix(UID_PREFIX) + .slotSharingGroup(SLOT_SHARING_GROUP) + .add( + new MaintenanceTaskBuilderForTest(true) + .scheduleOnCommitCount(1) + .uidPrefix(UID_PREFIX) + .slotSharingGroup(SLOT_SHARING_GROUP)) + .append(); + + checkUidsAreSet(env, UID_PREFIX); + checkSlotSharingGroupsAreSet(env, SLOT_SHARING_GROUP); + } + + @Test + void testUidAndSlotSharingGroupUnset() { + 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(); + + TableMaintenance.builder( + new ManualSource<>(env, TypeInformation.of(TableChange.class)).dataStream(), + tableLoader, + LOCK_FACTORY) + .add(new MaintenanceTaskBuilderForTest(true).scheduleOnCommitCount(1)) + .append(); + + checkUidsAreSet(env, null); + checkSlotSharingGroupsAreSet(env, null); + } + + @Test + void testUidAndSlotSharingGroupInherit() { + 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(); + + TableMaintenance.builder( + new ManualSource<>(env, TypeInformation.of(TableChange.class)).dataStream(), + tableLoader, + LOCK_FACTORY) + .uidPrefix(UID_PREFIX) + .slotSharingGroup(SLOT_SHARING_GROUP) + .add(new MaintenanceTaskBuilderForTest(true).scheduleOnCommitCount(1)) + .append(); + + checkUidsAreSet(env, UID_PREFIX); + checkSlotSharingGroupsAreSet(env, SLOT_SHARING_GROUP); + } + + @Test + void testUidAndSlotSharingGroupOverWrite() { + String anotherUid = "Another-UID"; + String anotherSlotSharingGroup = "Another-SlotSharingGroup"; + 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(); + + TableMaintenance.builder( + new ManualSource<>(env, TypeInformation.of(TableChange.class)).dataStream(), + tableLoader, + LOCK_FACTORY) + .uidPrefix(UID_PREFIX) + .slotSharingGroup(SLOT_SHARING_GROUP) + .add( + new MaintenanceTaskBuilderForTest(true) + .scheduleOnCommitCount(1) + .uidPrefix(anotherUid) + .slotSharingGroup(anotherSlotSharingGroup)) + .append(); + + // Something from the scheduler + Transformation schedulerTransformation = + env.getTransformations().stream() + .filter(t -> t.getName().equals("Trigger manager")) + .findFirst() + .orElseThrow(); + assertThat(schedulerTransformation.getUid()).contains(UID_PREFIX); + assertThat(schedulerTransformation.getSlotSharingGroup()).isPresent(); + assertThat(schedulerTransformation.getSlotSharingGroup().get().getName()) + .isEqualTo(SLOT_SHARING_GROUP); + + // Something from the scheduled stream + Transformation scheduledTransformation = + env.getTransformations().stream() + .filter( + t -> t.getName().startsWith(MaintenanceTaskBuilderForTest.class.getSimpleName())) + .findFirst() + .orElseThrow(); + assertThat(scheduledTransformation.getUid()).contains(anotherUid); + assertThat(scheduledTransformation.getSlotSharingGroup()).isPresent(); + assertThat(scheduledTransformation.getSlotSharingGroup().get().getName()) + .isEqualTo(anotherSlotSharingGroup); + } + + @Test + void testUidAndSlotSharingGroupForMonitor() { + 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(); + + TableMaintenance.builder(env, tableLoader, LOCK_FACTORY) + .uidPrefix(UID_PREFIX) + .slotSharingGroup(SLOT_SHARING_GROUP) + .add( + new MaintenanceTaskBuilderForTest(true) + .scheduleOnCommitCount(1) + .uidPrefix(UID_PREFIX) + .slotSharingGroup(SLOT_SHARING_GROUP)) + .append(); + + Transformation source = monitorSource(); + assertThat(source).isNotNull(); + assertThat(source.getUid()).contains(UID_PREFIX); + assertThat(source.getSlotSharingGroup()).isPresent(); + assertThat(source.getSlotSharingGroup().get().getName()).isEqualTo(SLOT_SHARING_GROUP); + + checkUidsAreSet(env, UID_PREFIX); + checkSlotSharingGroupsAreSet(env, SLOT_SHARING_GROUP); + } + + /** + * Sends the events though the {@link ManualSource} provided, and waits until the given number of + * records are processed. + * + * @param schedulerSource used for sending the events + * @param streamBuilder used for generating the job + * @param eventsAndResultNumbers the pair of the event and the expected processed records + * @throws Exception if any + */ + private void sendEvents( + ManualSource schedulerSource, + TableMaintenance.Builder streamBuilder, + List> eventsAndResultNumbers) + throws Exception { + streamBuilder.append(); + + JobClient jobClient = null; + try { + jobClient = env.executeAsync(); + + eventsAndResultNumbers.forEach( + eventsAndResultNumber -> { + int expectedSize = PROCESSED.size() + eventsAndResultNumber.f1; + schedulerSource.sendRecord(eventsAndResultNumber.f0); + Awaitility.await() + .until( + () -> PROCESSED.size() == expectedSize && !LOCK_FACTORY.createLock().isHeld()); + }); + } finally { + closeJobClient(jobClient); + } + } + + private static class MaintenanceTaskBuilderForTest + extends MaintenanceTaskBuilder { + private final boolean success; + private final int id; + private static int counter = 0; + + MaintenanceTaskBuilderForTest(boolean success) { + this.success = success; + this.id = counter; + ++counter; + } + + @Override + DataStream buildInternal(DataStream trigger) { + String name = TASKS[id]; + return trigger + .map(new DummyMaintenanceTask(success)) + .name(name) + .uid(uidPrefix() + "-test-mapper-" + name) + .slotSharingGroup(slotSharingGroup()) + .forceNonParallel(); + } + } + + /** + * Finds the {@link org.apache.iceberg.flink.maintenance.operator.MonitorSource} for testing + * purposes by parsing the transformation tree. + * + * @return The monitor source if we found it + */ + private Transformation monitorSource() { + assertThat(env.getTransformations()).isNotEmpty(); + assertThat(env.getTransformations().get(0).getInputs()).isNotEmpty(); + assertThat(env.getTransformations().get(0).getInputs().get(0).getInputs()).isNotEmpty(); + + Transformation result = + env.getTransformations().get(0).getInputs().get(0).getInputs().get(0); + + // Some checks to make sure this is the transformation we are looking for + assertThat(result).isInstanceOf(SourceTransformation.class); + assertThat(result.getName()).isEqualTo(SOURCE_NAME); + + return result; + } + + private static class DummyMaintenanceTask + implements MapFunction, ResultTypeQueryable, Serializable { + private final boolean success; + + private DummyMaintenanceTask(boolean success) { + this.success = success; + } + + @Override + public TaskResult map(Trigger trigger) { + // Ensure that the lock is held when processing + assertThat(LOCK_FACTORY.createLock().isHeld()).isTrue(); + PROCESSED.add(trigger); + + return new TaskResult( + trigger.taskId(), + trigger.timestamp(), + success, + success ? Collections.emptyList() : Lists.newArrayList(new Exception("Testing error"))); + } + + @Override + public TypeInformation getProducedType() { + return TypeInformation.of(TaskResult.class); + } + } +} From 5a1516c09f0ced07177419a16ad29d0fa71c4aff Mon Sep 17 00:00:00 2001 From: Peter Vary Date: Tue, 17 Sep 2024 10:56:03 +0200 Subject: [PATCH 02/12] Steven's comments --- .../operator/ExpireSnapshotsProcessor.java | 26 ++-- .../maintenance/stream/ExpireSnapshots.java | 23 ++-- .../stream/MaintenanceTaskBuilder.java | 50 ++++---- .../maintenance/stream/TableMaintenance.java | 114 ++++++++---------- .../operator/OperatorTestBase.java | 4 +- .../stream/ScheduledInfraExtension.java | 2 +- .../stream/TestExpireSnapshots.java | 28 ++--- .../stream/TestMaintenanceE2E.java | 8 +- .../stream/TestTableMaintenance.java | 58 ++++----- 9 files changed, 152 insertions(+), 161 deletions(-) diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/ExpireSnapshotsProcessor.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/ExpireSnapshotsProcessor.java index bbd6e9775557..54ffa0009f46 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/ExpireSnapshotsProcessor.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/ExpireSnapshotsProcessor.java @@ -20,6 +20,7 @@ import java.util.Collections; import java.util.concurrent.ExecutorService; +import java.util.concurrent.atomic.AtomicLong; import org.apache.flink.api.common.typeinfo.Types; import org.apache.flink.configuration.Configuration; import org.apache.flink.streaming.api.functions.ProcessFunction; @@ -41,21 +42,21 @@ public class ExpireSnapshotsProcessor extends ProcessFunction { private static final Logger LOG = LoggerFactory.getLogger(ExpireSnapshotsProcessor.class); public static final OutputTag DELETE_STREAM = - new OutputTag<>("delete-stream", Types.STRING); + new OutputTag<>("expire-snapshots-file-deletes-stream", Types.STRING); private final TableLoader tableLoader; - private final Long minAgeMs; + private final Long maxSnapshotAgeMs; private final Integer retainLast; private final int plannerPoolSize; private transient ExecutorService plannerPool; private transient Table table; public ExpireSnapshotsProcessor( - TableLoader tableLoader, Long minAgeMs, Integer retainLast, int plannerPoolSize) { + TableLoader tableLoader, Long maxSnapshotAgeMs, Integer retainLast, int plannerPoolSize) { Preconditions.checkNotNull(tableLoader, "Table loader should no be null"); this.tableLoader = tableLoader; - this.minAgeMs = minAgeMs; + this.maxSnapshotAgeMs = maxSnapshotAgeMs; this.retainLast = retainLast; this.plannerPoolSize = plannerPoolSize; } @@ -73,21 +74,30 @@ public void processElement(Trigger trigger, Context ctx, Collector o try { table.refresh(); ExpireSnapshots expireSnapshots = table.expireSnapshots(); - if (minAgeMs != null) { - expireSnapshots = expireSnapshots.expireOlderThan(ctx.timestamp() - minAgeMs); + if (maxSnapshotAgeMs != null) { + expireSnapshots = expireSnapshots.expireOlderThan(ctx.timestamp() - maxSnapshotAgeMs); } if (retainLast != null) { expireSnapshots = expireSnapshots.retainLast(retainLast); } + AtomicLong deleteFileCounter = new AtomicLong(0L); expireSnapshots .planWith(plannerPool) - .deleteWith(file -> ctx.output(DELETE_STREAM, file)) + .deleteWith( + file -> { + ctx.output(DELETE_STREAM, file); + deleteFileCounter.incrementAndGet(); + }) .cleanExpiredFiles(true) .commit(); - LOG.info("Successfully finished expiring snapshots for {} at {}", table, ctx.timestamp()); + LOG.info( + "Successfully finished expiring snapshots for {} at {}. Scheduled {} files for delete.", + table, + ctx.timestamp(), + deleteFileCounter.get()); out.collect( new TaskResult(trigger.taskId(), trigger.timestamp(), true, Collections.emptyList())); } catch (Exception e) { diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/stream/ExpireSnapshots.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/stream/ExpireSnapshots.java index 056d8e7d0b7f..f3765c76d4c8 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/stream/ExpireSnapshots.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/stream/ExpireSnapshots.java @@ -25,6 +25,7 @@ import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator; import org.apache.flink.streaming.api.functions.async.AsyncRetryStrategy; import org.apache.flink.streaming.util.retryable.AsyncRetryStrategies; +import org.apache.iceberg.SystemConfigs; import org.apache.iceberg.flink.maintenance.operator.AsyncDeleteFiles; import org.apache.iceberg.flink.maintenance.operator.ExpireSnapshotsProcessor; import org.apache.iceberg.flink.maintenance.operator.TaskResult; @@ -37,9 +38,7 @@ public class ExpireSnapshots { private static final long DELETE_MAX_RETRY_DELAY_MS = 1000L; private static final double DELETE_BACKOFF_MULTIPLIER = 1.5; private static final long DELETE_TIMEOUT_MS = 10000L; - private static final int DELETE_PLANNING_WORKER_POOL_SIZE_DEFAULT = 10; private static final int DELETE_ATTEMPT_NUM = 10; - private static final int DELETE_WORKER_POOL_SIZE_DEFAULT = 10; private static final String EXECUTOR_TASK_NAME = "ES Executor"; @VisibleForTesting static final String DELETE_FILES_TASK_NAME = "Delete file"; @@ -53,20 +52,20 @@ public static Builder builder() { } public static class Builder extends MaintenanceTaskBuilder { - private Duration minAge = null; + private Duration maxSnapshotAge = null; private Integer retainLast = null; - private int planningWorkerPoolSize = DELETE_PLANNING_WORKER_POOL_SIZE_DEFAULT; + private int planningWorkerPoolSize = SystemConfigs.WORKER_THREAD_POOL_SIZE.value(); private int deleteAttemptNum = DELETE_ATTEMPT_NUM; - private int deleteWorkerPoolSize = DELETE_WORKER_POOL_SIZE_DEFAULT; + private int deleteWorkerPoolSize = SystemConfigs.DELETE_WORKER_THREAD_POOL_SIZE.value(); /** * The snapshots newer than this age will not be removed. * - * @param newMinAge of the files to be removed + * @param newMaxSnapshotAge of the snapshots to be removed * @return for chained calls */ - public Builder minAge(Duration newMinAge) { - this.minAge = newMinAge; + public Builder maxSnapshotAge(Duration newMaxSnapshotAge) { + this.maxSnapshotAge = newMaxSnapshotAge; return this; } @@ -116,7 +115,7 @@ public Builder deleteWorkerPoolSize(int newDeleteWorkerPoolSize) { } @Override - DataStream buildInternal(DataStream trigger) { + DataStream append(DataStream trigger) { Preconditions.checkNotNull(tableLoader(), "TableLoader should not be null"); SingleOutputStreamOperator result = @@ -124,11 +123,11 @@ DataStream buildInternal(DataStream trigger) { .process( new ExpireSnapshotsProcessor( tableLoader(), - minAge == null ? null : minAge.toMillis(), + maxSnapshotAge == null ? null : maxSnapshotAge.toMillis(), retainLast, planningWorkerPoolSize)) .name(EXECUTOR_TASK_NAME) - .uid(uidPrefix() + "-expire-snapshots") + .uid("expire-snapshots-" + uidSuffix()) .slotSharingGroup(slotSharingGroup()) .forceNonParallel(); @@ -149,7 +148,7 @@ DataStream buildInternal(DataStream trigger) { deleteWorkerPoolSize, retryStrategy) .name(DELETE_FILES_TASK_NAME) - .uid(uidPrefix() + "-delete-expired-files") + .uid("delete-expired-files-" + uidSuffix()) .slotSharingGroup(slotSharingGroup()) .setParallelism(parallelism()); diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/stream/MaintenanceTaskBuilder.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/stream/MaintenanceTaskBuilder.java index d02ba7662b38..5ca23f8cea74 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/stream/MaintenanceTaskBuilder.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/stream/MaintenanceTaskBuilder.java @@ -29,15 +29,15 @@ import org.apache.iceberg.relocated.com.google.common.base.Preconditions; public abstract class MaintenanceTaskBuilder { - private int id; + private int index; private String name; private TableLoader tableLoader; - private String uidPrefix = null; + private String uidSuffix = null; private String slotSharingGroup = null; private Integer parallelism = null; private TriggerEvaluator.Builder triggerEvaluator = new TriggerEvaluator.Builder(); - abstract DataStream buildInternal(DataStream sourceStream); + abstract DataStream append(DataStream sourceStream); /** * After a given number of Iceberg table commits since the last run, starts the downstream job. @@ -79,7 +79,7 @@ public T scheduleOnDataFileSize(long dataFileSizeInBytes) { * @param posDeleteFileCount after the downstream job should be started * @return for chained calls */ - public T schedulerOnPosDeleteFileCount(int posDeleteFileCount) { + public T scheduleOnPosDeleteFileCount(int posDeleteFileCount) { triggerEvaluator.posDeleteFileCount(posDeleteFileCount); return (T) this; } @@ -91,7 +91,7 @@ public T schedulerOnPosDeleteFileCount(int posDeleteFileCount) { * @param posDeleteRecordCount after the downstream job should be started * @return for chained calls */ - public T schedulerOnPosDeleteRecordCount(long posDeleteRecordCount) { + public T scheduleOnPosDeleteRecordCount(long posDeleteRecordCount) { triggerEvaluator.posDeleteRecordCount(posDeleteRecordCount); return (T) this; } @@ -123,22 +123,22 @@ public T scheduleOnEqDeleteRecordCount(long eqDeleteRecordCount) { /** * After a given time since the last run, starts the downstream job. * - * @param time after the downstream job should be started + * @param interval after the downstream job should be started * @return for chained calls */ - public T scheduleOnTime(Duration time) { - triggerEvaluator.timeout(time); + public T scheduleOnInterval(Duration interval) { + triggerEvaluator.timeout(interval); return (T) this; } /** - * The prefix used for the generated {@link org.apache.flink.api.dag.Transformation}'s uid. + * The suffix used for the generated {@link org.apache.flink.api.dag.Transformation}'s uid. * - * @param newUidPrefix for the transformations + * @param newUidSuffix for the transformations * @return for chained calls */ - public T uidPrefix(String newUidPrefix) { - this.uidPrefix = newUidPrefix; + public T uidSuffix(String newUidSuffix) { + this.uidSuffix = newUidSuffix; return (T) this; } @@ -167,7 +167,7 @@ public T parallelism(int newParallelism) { @Internal int id() { - return id; + return index; } @Internal @@ -181,8 +181,8 @@ TableLoader tableLoader() { } @Internal - String uidPrefix() { - return uidPrefix; + String uidSuffix() { + return uidSuffix; } @Internal @@ -201,26 +201,26 @@ TriggerEvaluator evaluator() { } @Internal - DataStream build( + DataStream append( DataStream sourceStream, - int newId, - String newName, + int maintenanceTaskIndex, + String maintainanceTaskName, TableLoader newTableLoader, - String mainUidPrefix, + String mainUidSuffix, String mainSlotSharingGroup, int mainParallelism) { Preconditions.checkArgument( parallelism == null || parallelism == -1 || parallelism > 0, "Parallelism should be left to default (-1/null) or greater than 0"); - Preconditions.checkNotNull(newName, "Name should not be null"); + Preconditions.checkNotNull(maintainanceTaskName, "Name should not be null"); Preconditions.checkNotNull(newTableLoader, "TableLoader should not be null"); - this.id = newId; - this.name = newName; + this.index = maintenanceTaskIndex; + this.name = maintainanceTaskName; this.tableLoader = newTableLoader; - if (uidPrefix == null) { - uidPrefix = mainUidPrefix + "_" + name + "_" + id; + if (uidSuffix == null) { + uidSuffix = name + "_" + index + "_" + mainUidSuffix; } if (parallelism == null) { @@ -233,6 +233,6 @@ DataStream build( tableLoader.open(); - return buildInternal(sourceStream); + return append(sourceStream); } } diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/stream/TableMaintenance.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/stream/TableMaintenance.java index 9ba6a9f67eff..3add7adbabb1 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/stream/TableMaintenance.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/stream/TableMaintenance.java @@ -30,7 +30,6 @@ import org.apache.flink.api.common.eventtime.WatermarkGeneratorSupplier; import org.apache.flink.api.common.eventtime.WatermarkOutput; import org.apache.flink.api.common.eventtime.WatermarkStrategy; -import org.apache.flink.api.common.functions.FlatMapFunction; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.connector.source.util.ratelimit.RateLimiterStrategy; import org.apache.flink.streaming.api.datastream.DataStream; @@ -38,7 +37,6 @@ import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.streaming.api.graph.StreamGraphGenerator; -import org.apache.flink.util.Collector; import org.apache.iceberg.flink.TableLoader; import org.apache.iceberg.flink.maintenance.operator.LockRemover; import org.apache.iceberg.flink.maintenance.operator.MonitorSource; @@ -71,7 +69,7 @@ private TableMaintenance() { * @param lockFactory used for preventing concurrent task runs * @return builder for the maintenance stream */ - public static Builder builder( + public static Builder forChangeStream( DataStream changeStream, TableLoader tableLoader, TriggerLockFactory lockFactory) { @@ -91,7 +89,7 @@ public static Builder builder( * @param lockFactory used for preventing concurrent task runs * @return builder for the maintenance stream */ - public static Builder builder( + public static Builder forTable( StreamExecutionEnvironment env, TableLoader tableLoader, TriggerLockFactory lockFactory) { Preconditions.checkNotNull(env, "StreamExecutionEnvironment should not be null"); Preconditions.checkNotNull(tableLoader, "TableLoader should not be null"); @@ -102,33 +100,33 @@ public static Builder builder( public static class Builder { private final StreamExecutionEnvironment env; - private final DataStream changeStream; + private final DataStream inputStream; private final TableLoader tableLoader; private final List> taskBuilders; private final TriggerLockFactory lockFactory; - private String uidPrefix = "TableMaintenance-" + UUID.randomUUID(); + private String uidSuffix = "TableMaintenance-" + UUID.randomUUID(); private String slotSharingGroup = StreamGraphGenerator.DEFAULT_SLOT_SHARING_GROUP; private Duration rateLimit = Duration.ofMillis(1); - private Duration concurrentCheckDelay = Duration.ofSeconds(30); + private Duration lockCheckDelay = Duration.ofSeconds(30); private Integer parallelism = ExecutionConfig.PARALLELISM_DEFAULT; private int maxReadBack = 100; private Builder( StreamExecutionEnvironment env, TableLoader tableLoader, TriggerLockFactory lockFactory) { this.env = env; - this.changeStream = null; + this.inputStream = null; this.tableLoader = tableLoader; this.lockFactory = lockFactory; this.taskBuilders = Lists.newArrayListWithCapacity(4); } private Builder( - DataStream changeStream, + DataStream inputStream, TableLoader tableLoader, TriggerLockFactory lockFactory) { this.env = null; - this.changeStream = changeStream; + this.inputStream = inputStream; this.tableLoader = tableLoader; this.lockFactory = lockFactory; this.taskBuilders = Lists.newArrayListWithCapacity(4); @@ -137,11 +135,11 @@ private Builder( /** * The prefix used for the generated {@link org.apache.flink.api.dag.Transformation}'s uid. * - * @param newUidPrefix for the transformations + * @param newUidSuffix for the transformations * @return for chained calls */ - public Builder uidPrefix(String newUidPrefix) { - this.uidPrefix = newUidPrefix; + public Builder uidSuffix(String newUidSuffix) { + this.uidSuffix = newUidSuffix; return this; } @@ -172,11 +170,11 @@ public Builder rateLimit(Duration newRateLimit) { /** * Sets the delay for checking lock availability when a concurrent run is detected. * - * @param newConcurrentCheckDelay firing frequency + * @param newLockCheckDelay lock checking frequency * @return for chained calls */ - public Builder concurrentCheckDelay(Duration newConcurrentCheckDelay) { - this.concurrentCheckDelay = newConcurrentCheckDelay; + public Builder lockCheckDelay(Duration newLockCheckDelay) { + this.lockCheckDelay = newLockCheckDelay; return this; } @@ -194,14 +192,16 @@ public Builder parallelism(int newParallelism) { /** * Maximum number of snapshots checked when started with an embedded {@link MonitorSource} at - * the first time. Only available when the {@link MonitorSource} is generated by the builder. + * the first time. Only available when the {@link + * TableMaintenance#forTable(StreamExecutionEnvironment, TableLoader, TriggerLockFactory)} is + * used. * * @param newMaxReadBack snapshots to consider when initializing * @return for chained calls */ public Builder maxReadBack(int newMaxReadBack) { Preconditions.checkArgument( - changeStream == null, "Can't set maxReadBack when change stream is provided"); + inputStream == null, "Can't set maxReadBack when change stream is provided"); this.maxReadBack = newMaxReadBack; return this; } @@ -220,26 +220,8 @@ public Builder add(MaintenanceTaskBuilder task) { /** Builds the task graph for the maintenance tasks. */ public void append() { Preconditions.checkArgument(!taskBuilders.isEmpty(), "Provide at least one task"); - Preconditions.checkNotNull(uidPrefix, "Uid prefix should no be null"); + Preconditions.checkNotNull(uidSuffix, "Uid suffix should no be null"); - DataStream sourceStream; - if (changeStream == null) { - // Create a monitor source to provide the TableChange stream - MonitorSource source = - new MonitorSource( - tableLoader, - RateLimiterStrategy.perSecond(1.0 / rateLimit.getSeconds()), - maxReadBack); - sourceStream = - env.fromSource(source, WatermarkStrategy.noWatermarks(), SOURCE_NAME) - .uid(uidPrefix + "-monitor-source") - .slotSharingGroup(slotSharingGroup) - .forceNonParallel(); - } else { - sourceStream = changeStream.global(); - } - - // Chain the TriggerManager List taskNames = Lists.newArrayListWithCapacity(taskBuilders.size()); List evaluators = Lists.newArrayListWithCapacity(taskBuilders.size()); for (int i = 0; i < taskBuilders.size(); ++i) { @@ -248,8 +230,7 @@ public void append() { } DataStream triggers = - // Add TriggerManager to schedule the tasks - DataStreamUtils.reinterpretAsKeyedStream(sourceStream, unused -> true) + DataStreamUtils.reinterpretAsKeyedStream(changeStream(), unused -> true) .process( new TriggerManager( tableLoader, @@ -257,36 +238,36 @@ public void append() { taskNames, evaluators, rateLimit.toMillis(), - concurrentCheckDelay.toMillis())) + lockCheckDelay.toMillis())) .name(TRIGGER_MANAGER_TASK_NAME) - .uid(uidPrefix + "-trigger-manager") + .uid("trigger-manager-" + uidSuffix) .slotSharingGroup(slotSharingGroup) .forceNonParallel() - // Add a watermark after every trigger - .assignTimestampsAndWatermarks(new WindowClosingWatermarkStrategy()) + .assignTimestampsAndWatermarks(new PunctuatedWatermarkStrategy()) .name("Watermark Assigner") - .uid(uidPrefix + "-watermark-assigner") + .uid("watermark-assigner-" + uidSuffix) .slotSharingGroup(slotSharingGroup) .forceNonParallel(); // Add the specific tasks DataStream unioned = null; for (int i = 0; i < taskBuilders.size(); ++i) { + int finalIndex = i; DataStream filtered = triggers - .flatMap(new TaskFilter(i)) + .filter(t -> t.taskId() != null && t.taskId() == finalIndex) .name("Filter " + i) .forceNonParallel() - .uid(uidPrefix + "-filter-" + i) + .uid("filter-" + i + "-" + uidSuffix) .slotSharingGroup(slotSharingGroup); MaintenanceTaskBuilder builder = taskBuilders.get(i); DataStream result = - builder.build( + builder.append( filtered, i, taskNames.get(i), tableLoader, - uidPrefix, + uidSuffix, slotSharingGroup, parallelism); if (unioned == null) { @@ -304,9 +285,26 @@ public void append() { TypeInformation.of(Void.class), new LockRemover(lockFactory, taskNames)) .forceNonParallel() - .uid(uidPrefix + "-lock-remover") + .uid("lock-remover-" + uidSuffix) .slotSharingGroup(slotSharingGroup); } + + private DataStream changeStream() { + if (inputStream == null) { + // Create a monitor source to provide the TableChange stream + MonitorSource source = + new MonitorSource( + tableLoader, + RateLimiterStrategy.perSecond(1.0 / rateLimit.getSeconds()), + maxReadBack); + return env.fromSource(source, WatermarkStrategy.noWatermarks(), SOURCE_NAME) + .uid("monitor-source-" + uidSuffix) + .slotSharingGroup(slotSharingGroup) + .forceNonParallel(); + } else { + return inputStream.global(); + } + } } private static String nameFor(MaintenanceTaskBuilder streamBuilder, int taskId) { @@ -314,7 +312,7 @@ private static String nameFor(MaintenanceTaskBuilder streamBuilder, int taskI } @Internal - public static class WindowClosingWatermarkStrategy implements WatermarkStrategy { + public static class PunctuatedWatermarkStrategy implements WatermarkStrategy { @Override public WatermarkGenerator createWatermarkGenerator( WatermarkGeneratorSupplier.Context context) { @@ -337,20 +335,4 @@ public TimestampAssigner createTimestampAssigner( return (element, unused) -> element.timestamp(); } } - - @Internal - private static class TaskFilter implements FlatMapFunction { - private final int taskId; - - private TaskFilter(int taskId) { - this.taskId = taskId; - } - - @Override - public void flatMap(Trigger trigger, Collector out) { - if (trigger.taskId() != null && trigger.taskId() == taskId) { - out.collect(trigger); - } - } - } } diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/OperatorTestBase.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/OperatorTestBase.java index 63ba6f587462..1cae679b2587 100644 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/OperatorTestBase.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/OperatorTestBase.java @@ -44,7 +44,7 @@ public class OperatorTestBase { private static final int NUMBER_TASK_MANAGERS = 1; private static final int SLOTS_PER_TASK_MANAGER = 8; - protected static final String UID_PREFIX = "UID-Dummy"; + protected static final String UID_SUFFIX = "UID-Dummy"; protected static final String SLOT_SHARING_GROUP = "SlotSharingGroup"; protected static final String TABLE_NAME = "test_table"; protected static final TriggerLockFactory LOCK_FACTORY = new MemoryLockFactory(); @@ -131,7 +131,7 @@ protected static void checkUidsAreSet(StreamExecutionEnvironment env, String uid transformation -> { assertThat(transformation.getUid()).isNotNull(); if (uidPrefix != null) { - assertThat(transformation.getUid()).contains(UID_PREFIX); + assertThat(transformation.getUid()).contains(UID_SUFFIX); } }); } diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/stream/ScheduledInfraExtension.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/stream/ScheduledInfraExtension.java index 210c003f183a..55ea6161517c 100644 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/stream/ScheduledInfraExtension.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/stream/ScheduledInfraExtension.java @@ -56,7 +56,7 @@ public void beforeEach(ExtensionContext context) { triggerStream = source .dataStream() - .assignTimestampsAndWatermarks(new TableMaintenance.WindowClosingWatermarkStrategy()) + .assignTimestampsAndWatermarks(new TableMaintenance.PunctuatedWatermarkStrategy()) .name(IGNORED_OPERATOR_NAME) .forceNonParallel(); sink = new CollectingSink<>(); diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/stream/TestExpireSnapshots.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/stream/TestExpireSnapshots.java index 8d3c8864ebe3..dff176d6c7fa 100644 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/stream/TestExpireSnapshots.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/stream/TestExpireSnapshots.java @@ -67,10 +67,10 @@ void testExpireSnapshots() throws Exception { .planningWorkerPoolSize(2) .deleteAttemptNum(2) .deleteWorkerPoolSize(5) - .minAge(Duration.ZERO) + .maxSnapshotAge(Duration.ZERO) .retainLast(1) - .uidPrefix(UID_PREFIX) - .build( + .uidSuffix(UID_SUFFIX) + .append( infra.triggerStream(), 0, DUMMY_NAME, @@ -106,12 +106,12 @@ void testFailure() throws Exception { SerializableTable serializableTable = (SerializableTable) SerializableTable.copyOf(table); ExpireSnapshots.builder() - .build( + .append( infra.triggerStream(), 0, DUMMY_NAME, tableLoader, - UID_PREFIX, + UID_SUFFIX, StreamGraphGenerator.DEFAULT_SLOT_SHARING_GROUP, 1) .sinkTo(infra.sink()); @@ -156,18 +156,18 @@ void testUidAndSlotSharingGroup() { ExpireSnapshots.builder() .slotSharingGroup(SLOT_SHARING_GROUP) - .uidPrefix(UID_PREFIX) - .build( + .uidSuffix(UID_SUFFIX) + .append( infra.triggerStream(), 0, DUMMY_NAME, tableLoader, - UID_PREFIX, + UID_SUFFIX, StreamGraphGenerator.DEFAULT_SLOT_SHARING_GROUP, 1) .sinkTo(infra.sink()); - checkUidsAreSet(infra.env(), UID_PREFIX); + checkUidsAreSet(infra.env(), UID_SUFFIX); checkSlotSharingGroupsAreSet(infra.env(), SLOT_SHARING_GROUP); } @@ -177,12 +177,12 @@ void testUidAndSlotSharingGroupUnset() { TableLoader tableLoader = sql.tableLoader(TABLE_NAME); ExpireSnapshots.builder() - .build( + .append( infra.triggerStream(), 0, DUMMY_NAME, tableLoader, - UID_PREFIX, + UID_SUFFIX, StreamGraphGenerator.DEFAULT_SLOT_SHARING_GROUP, 1) .sinkTo(infra.sink()); @@ -201,15 +201,15 @@ void testMetrics() throws Exception { Table table = tableLoader.loadTable(); ExpireSnapshots.builder() - .minAge(Duration.ZERO) + .maxSnapshotAge(Duration.ZERO) .retainLast(1) .parallelism(1) - .build( + .append( infra.triggerStream(), 0, DUMMY_NAME, tableLoader, - UID_PREFIX, + UID_SUFFIX, StreamGraphGenerator.DEFAULT_SLOT_SHARING_GROUP, 1) .sinkTo(infra.sink()); diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/stream/TestMaintenanceE2E.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/stream/TestMaintenanceE2E.java index fd9b136af21d..3a6a7909eca2 100644 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/stream/TestMaintenanceE2E.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/stream/TestMaintenanceE2E.java @@ -43,14 +43,14 @@ void testE2e() throws Exception { TableLoader tableLoader = sql.tableLoader(TABLE_NAME); - TableMaintenance.builder(env, tableLoader, LOCK_FACTORY) - .uidPrefix("E2eTestUID") + TableMaintenance.forTable(env, tableLoader, LOCK_FACTORY) + .uidSuffix("E2eTestUID") .rateLimit(Duration.ofMinutes(10)) - .concurrentCheckDelay(Duration.ofSeconds(10)) + .lockCheckDelay(Duration.ofSeconds(10)) .add( ExpireSnapshots.builder() .scheduleOnCommitCount(10) - .minAge(Duration.ofMinutes(10)) + .maxSnapshotAge(Duration.ofMinutes(10)) .retainLast(5) .deleteWorkerPoolSize(5) .parallelism(8)) diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/stream/TestTableMaintenance.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/stream/TestTableMaintenance.java index c7fa51675886..181fa1e6dc87 100644 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/stream/TestTableMaintenance.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/stream/TestTableMaintenance.java @@ -104,9 +104,9 @@ void testFromStream() throws Exception { new ManualSource<>(env, TypeInformation.of(TableChange.class)); TableMaintenance.Builder streamBuilder = - TableMaintenance.builder(schedulerSource.dataStream(), tableLoader, LOCK_FACTORY) + TableMaintenance.forChangeStream(schedulerSource.dataStream(), tableLoader, LOCK_FACTORY) .rateLimit(Duration.ofMillis(2)) - .concurrentCheckDelay(Duration.ofSeconds(3)) + .lockCheckDelay(Duration.ofSeconds(3)) .add( new MaintenanceTaskBuilderForTest(true) .scheduleOnCommitCount(1) @@ -114,9 +114,9 @@ void testFromStream() throws Exception { .scheduleOnDataFileSize(3L) .scheduleOnEqDeleteFileCount(4) .scheduleOnEqDeleteRecordCount(5L) - .schedulerOnPosDeleteFileCount(6) - .schedulerOnPosDeleteRecordCount(7L) - .scheduleOnTime(Duration.ofHours(1))); + .scheduleOnPosDeleteFileCount(6) + .scheduleOnPosDeleteRecordCount(7L) + .scheduleOnInterval(Duration.ofHours(1))); sendEvents(schedulerSource, streamBuilder, ImmutableList.of(Tuple2.of(DUMMY_CHANGE, 1))); } @@ -135,7 +135,7 @@ void testFromEnv() throws Exception { env.enableCheckpointing(10); - TableMaintenance.builder(env, tableLoader, LOCK_FACTORY) + TableMaintenance.forTable(env, tableLoader, LOCK_FACTORY) .rateLimit(Duration.ofMillis(2)) .maxReadBack(2) .add(new MaintenanceTaskBuilderForTest(true).scheduleOnCommitCount(2)) @@ -146,7 +146,7 @@ void testFromEnv() throws Exception { new ManualSource<>(env, InternalTypeInfo.of(FlinkSchemaUtil.convert(table.schema()))); FlinkSink.forRowData(insertSource.dataStream()) .tableLoader(tableLoader) - .uidPrefix(UID_PREFIX + "-iceberg-sink") + .uidPrefix(UID_SUFFIX + "-iceberg-sink") .append(); JobClient jobClient = null; @@ -175,7 +175,7 @@ void testLocking() throws Exception { new ManualSource<>(env, TypeInformation.of(TableChange.class)); TableMaintenance.Builder streamBuilder = - TableMaintenance.builder(schedulerSource.dataStream(), tableLoader, LOCK_FACTORY) + TableMaintenance.forChangeStream(schedulerSource.dataStream(), tableLoader, LOCK_FACTORY) .rateLimit(Duration.ofMillis(2)) .add(new MaintenanceTaskBuilderForTest(true).scheduleOnCommitCount(1)); @@ -197,9 +197,9 @@ void testMetrics() throws Exception { new ManualSource<>(env, TypeInformation.of(TableChange.class)); TableMaintenance.Builder streamBuilder = - TableMaintenance.builder(schedulerSource.dataStream(), tableLoader, LOCK_FACTORY) + TableMaintenance.forChangeStream(schedulerSource.dataStream(), tableLoader, LOCK_FACTORY) .rateLimit(Duration.ofMillis(2)) - .concurrentCheckDelay(Duration.ofMillis(2)) + .lockCheckDelay(Duration.ofMillis(2)) .add(new MaintenanceTaskBuilderForTest(true).scheduleOnCommitCount(1)) .add(new MaintenanceTaskBuilderForTest(false).scheduleOnCommitCount(2)); @@ -251,20 +251,20 @@ void testUidAndSlotSharingGroup() { TableLoader tableLoader = sql.tableLoader(TABLE_NAME); tableLoader.open(); - TableMaintenance.builder( + TableMaintenance.forChangeStream( new ManualSource<>(env, TypeInformation.of(TableChange.class)).dataStream(), tableLoader, LOCK_FACTORY) - .uidPrefix(UID_PREFIX) + .uidSuffix(UID_SUFFIX) .slotSharingGroup(SLOT_SHARING_GROUP) .add( new MaintenanceTaskBuilderForTest(true) .scheduleOnCommitCount(1) - .uidPrefix(UID_PREFIX) + .uidSuffix(UID_SUFFIX) .slotSharingGroup(SLOT_SHARING_GROUP)) .append(); - checkUidsAreSet(env, UID_PREFIX); + checkUidsAreSet(env, UID_SUFFIX); checkSlotSharingGroupsAreSet(env, SLOT_SHARING_GROUP); } @@ -276,7 +276,7 @@ void testUidAndSlotSharingGroupUnset() { TableLoader tableLoader = sql.tableLoader(TABLE_NAME); tableLoader.open(); - TableMaintenance.builder( + TableMaintenance.forChangeStream( new ManualSource<>(env, TypeInformation.of(TableChange.class)).dataStream(), tableLoader, LOCK_FACTORY) @@ -295,16 +295,16 @@ void testUidAndSlotSharingGroupInherit() { TableLoader tableLoader = sql.tableLoader(TABLE_NAME); tableLoader.open(); - TableMaintenance.builder( + TableMaintenance.forChangeStream( new ManualSource<>(env, TypeInformation.of(TableChange.class)).dataStream(), tableLoader, LOCK_FACTORY) - .uidPrefix(UID_PREFIX) + .uidSuffix(UID_SUFFIX) .slotSharingGroup(SLOT_SHARING_GROUP) .add(new MaintenanceTaskBuilderForTest(true).scheduleOnCommitCount(1)) .append(); - checkUidsAreSet(env, UID_PREFIX); + checkUidsAreSet(env, UID_SUFFIX); checkSlotSharingGroupsAreSet(env, SLOT_SHARING_GROUP); } @@ -318,16 +318,16 @@ void testUidAndSlotSharingGroupOverWrite() { TableLoader tableLoader = sql.tableLoader(TABLE_NAME); tableLoader.open(); - TableMaintenance.builder( + TableMaintenance.forChangeStream( new ManualSource<>(env, TypeInformation.of(TableChange.class)).dataStream(), tableLoader, LOCK_FACTORY) - .uidPrefix(UID_PREFIX) + .uidSuffix(UID_SUFFIX) .slotSharingGroup(SLOT_SHARING_GROUP) .add( new MaintenanceTaskBuilderForTest(true) .scheduleOnCommitCount(1) - .uidPrefix(anotherUid) + .uidSuffix(anotherUid) .slotSharingGroup(anotherSlotSharingGroup)) .append(); @@ -337,7 +337,7 @@ void testUidAndSlotSharingGroupOverWrite() { .filter(t -> t.getName().equals("Trigger manager")) .findFirst() .orElseThrow(); - assertThat(schedulerTransformation.getUid()).contains(UID_PREFIX); + assertThat(schedulerTransformation.getUid()).contains(UID_SUFFIX); assertThat(schedulerTransformation.getSlotSharingGroup()).isPresent(); assertThat(schedulerTransformation.getSlotSharingGroup().get().getName()) .isEqualTo(SLOT_SHARING_GROUP); @@ -363,23 +363,23 @@ void testUidAndSlotSharingGroupForMonitor() { TableLoader tableLoader = sql.tableLoader(TABLE_NAME); tableLoader.open(); - TableMaintenance.builder(env, tableLoader, LOCK_FACTORY) - .uidPrefix(UID_PREFIX) + TableMaintenance.forTable(env, tableLoader, LOCK_FACTORY) + .uidSuffix(UID_SUFFIX) .slotSharingGroup(SLOT_SHARING_GROUP) .add( new MaintenanceTaskBuilderForTest(true) .scheduleOnCommitCount(1) - .uidPrefix(UID_PREFIX) + .uidSuffix(UID_SUFFIX) .slotSharingGroup(SLOT_SHARING_GROUP)) .append(); Transformation source = monitorSource(); assertThat(source).isNotNull(); - assertThat(source.getUid()).contains(UID_PREFIX); + assertThat(source.getUid()).contains(UID_SUFFIX); assertThat(source.getSlotSharingGroup()).isPresent(); assertThat(source.getSlotSharingGroup().get().getName()).isEqualTo(SLOT_SHARING_GROUP); - checkUidsAreSet(env, UID_PREFIX); + checkUidsAreSet(env, UID_SUFFIX); checkSlotSharingGroupsAreSet(env, SLOT_SHARING_GROUP); } @@ -429,12 +429,12 @@ private static class MaintenanceTaskBuilderForTest } @Override - DataStream buildInternal(DataStream trigger) { + DataStream append(DataStream trigger) { String name = TASKS[id]; return trigger .map(new DummyMaintenanceTask(success)) .name(name) - .uid(uidPrefix() + "-test-mapper-" + name) + .uid(uidSuffix() + "-test-mapper-" + name) .slotSharingGroup(slotSharingGroup()) .forceNonParallel(); } From ea3366e00acb7c5031303c3a6929e3a2afbd3f04 Mon Sep 17 00:00:00 2001 From: Peter Vary Date: Wed, 18 Sep 2024 20:31:05 +0200 Subject: [PATCH 03/12] Steven's second round of comments --- .../{stream => api}/ExpireSnapshots.java | 98 ++++------ .../MaintenanceTaskBuilder.java | 32 ++-- .../{stream => api}/TableMaintenance.java | 51 +++--- .../operator/AsyncDeleteFiles.java | 106 ----------- .../operator/DeleteFilesProcessor.java | 124 +++++++++++++ .../operator/ExpireSnapshotsProcessor.java | 14 +- .../MaintenanceTaskInfraExtension.java} | 4 +- .../MaintenanceTaskTestBase.java} | 12 +- .../{stream => api}/TestExpireSnapshots.java | 77 ++++---- .../{stream => api}/TestMaintenanceE2E.java | 20 +-- .../{stream => api}/TestTableMaintenance.java | 167 +++++++----------- .../operator/FlinkSqlExtension.java | 139 --------------- .../operator/OperatorTestBase.java | 70 ++++++-- .../operator/TestAsyncDeleteFiles.java | 154 ---------------- .../operator/TestDeleteFilesProcessor.java | 116 ++++++++++++ .../TestExpireSnapshotsProcessor.java | 16 +- .../operator/TestMonitorSource.java | 72 +++----- .../operator/TestTriggerManager.java | 44 ++--- 18 files changed, 537 insertions(+), 779 deletions(-) rename flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/{stream => api}/ExpireSnapshots.java (50%) rename flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/{stream => api}/MaintenanceTaskBuilder.java (88%) rename flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/{stream => api}/TableMaintenance.java (88%) delete mode 100644 flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/AsyncDeleteFiles.java create mode 100644 flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/DeleteFilesProcessor.java rename flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/{stream/ScheduledInfraExtension.java => api/MaintenanceTaskInfraExtension.java} (96%) rename flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/{stream/ScheduledBuilderTestBase.java => api/MaintenanceTaskTestBase.java} (87%) rename flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/{stream => api}/TestExpireSnapshots.java (75%) rename flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/{stream => api}/TestMaintenanceE2E.java (80%) rename flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/{stream => api}/TestTableMaintenance.java (80%) delete mode 100644 flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/FlinkSqlExtension.java delete mode 100644 flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestAsyncDeleteFiles.java create mode 100644 flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestDeleteFilesProcessor.java diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/stream/ExpireSnapshots.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/ExpireSnapshots.java similarity index 50% rename from flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/stream/ExpireSnapshots.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/ExpireSnapshots.java index f3765c76d4c8..9109a9f1a97f 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/stream/ExpireSnapshots.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/ExpireSnapshots.java @@ -16,31 +16,26 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.iceberg.flink.maintenance.stream; +package org.apache.iceberg.flink.maintenance.api; import java.time.Duration; -import java.util.concurrent.TimeUnit; -import org.apache.flink.streaming.api.datastream.AsyncDataStream; +import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator; -import org.apache.flink.streaming.api.functions.async.AsyncRetryStrategy; -import org.apache.flink.streaming.util.retryable.AsyncRetryStrategies; +import org.apache.iceberg.Snapshot; import org.apache.iceberg.SystemConfigs; -import org.apache.iceberg.flink.maintenance.operator.AsyncDeleteFiles; +import org.apache.iceberg.flink.maintenance.operator.DeleteFilesProcessor; import org.apache.iceberg.flink.maintenance.operator.ExpireSnapshotsProcessor; import org.apache.iceberg.flink.maintenance.operator.TaskResult; import org.apache.iceberg.flink.maintenance.operator.Trigger; import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +/** Deletes expired snapshots and the corresponding files. */ public class ExpireSnapshots { - private static final long DELETE_INITIAL_DELAY_MS = 10L; - private static final long DELETE_MAX_RETRY_DELAY_MS = 1000L; - private static final double DELETE_BACKOFF_MULTIPLIER = 1.5; - private static final long DELETE_TIMEOUT_MS = 10000L; - private static final int DELETE_ATTEMPT_NUM = 10; - private static final String EXECUTOR_TASK_NAME = "ES Executor"; - @VisibleForTesting static final String DELETE_FILES_TASK_NAME = "Delete file"; + private static final int DELETE_BATCH_SIZE_DEFAULT = 10; + private static final String EXECUTOR_OPERATOR_NAME = "Expire Snapshot"; + @VisibleForTesting static final String DELETE_FILES_OPERATOR_NAME = "Delete file"; private ExpireSnapshots() { // Do not instantiate directly @@ -53,16 +48,15 @@ public static Builder builder() { public static class Builder extends MaintenanceTaskBuilder { private Duration maxSnapshotAge = null; - private Integer retainLast = null; + private Integer numSnapshots = null; private int planningWorkerPoolSize = SystemConfigs.WORKER_THREAD_POOL_SIZE.value(); - private int deleteAttemptNum = DELETE_ATTEMPT_NUM; - private int deleteWorkerPoolSize = SystemConfigs.DELETE_WORKER_THREAD_POOL_SIZE.value(); + private int deleteBatchSize = DELETE_BATCH_SIZE_DEFAULT; + private int deleteParallelism = 1; /** - * The snapshots newer than this age will not be removed. + * The snapshots older than this age will be removed. * * @param newMaxSnapshotAge of the snapshots to be removed - * @return for chained calls */ public Builder maxSnapshotAge(Duration newMaxSnapshotAge) { this.maxSnapshotAge = newMaxSnapshotAge; @@ -70,14 +64,13 @@ public Builder maxSnapshotAge(Duration newMaxSnapshotAge) { } /** - * The minimum {@link org.apache.iceberg.Snapshot}s to retain. For more details description see - * {@link org.apache.iceberg.ExpireSnapshots#retainLast(int)}. + * The minimum number of {@link Snapshot}s to retain. For more details description see {@link + * org.apache.iceberg.ExpireSnapshots#retainLast(int)}. * - * @param newRetainLast number of snapshots to retain - * @return for chained calls + * @param newNumSnapshots number of snapshots to retain */ - public Builder retainLast(int newRetainLast) { - this.retainLast = newRetainLast; + public Builder retainLast(int newNumSnapshots) { + this.numSnapshots = newNumSnapshots; return this; } @@ -85,7 +78,6 @@ public Builder retainLast(int newRetainLast) { * The worker pool size used to calculate the files to delete. * * @param newPlanningWorkerPoolSize for planning files to delete - * @return for chained calls */ public Builder planningWorkerPoolSize(int newPlanningWorkerPoolSize) { this.planningWorkerPoolSize = newPlanningWorkerPoolSize; @@ -93,24 +85,22 @@ public Builder planningWorkerPoolSize(int newPlanningWorkerPoolSize) { } /** - * The number of retries on the failed delete attempts. + * Size of the batch used to deleting the files. * - * @param newDeleteAttemptNum number of retries - * @return for chained calls + * @param newDeleteBatchSize used for deleting */ - public Builder deleteAttemptNum(int newDeleteAttemptNum) { - this.deleteAttemptNum = newDeleteAttemptNum; + public Builder deleteBatchSize(int newDeleteBatchSize) { + this.deleteBatchSize = newDeleteBatchSize; return this; } /** - * The worker pool size used for deleting files. + * The number of subtasks which are doing the deletes. * - * @param newDeleteWorkerPoolSize for scanning - * @return for chained calls + * @param newDeleteParallelism used for deleting */ - public Builder deleteWorkerPoolSize(int newDeleteWorkerPoolSize) { - this.deleteWorkerPoolSize = newDeleteWorkerPoolSize; + public Builder deleteParallelism(int newDeleteParallelism) { + this.deleteParallelism = newDeleteParallelism; return this; } @@ -124,36 +114,26 @@ DataStream append(DataStream trigger) { new ExpireSnapshotsProcessor( tableLoader(), maxSnapshotAge == null ? null : maxSnapshotAge.toMillis(), - retainLast, + numSnapshots, planningWorkerPoolSize)) - .name(EXECUTOR_TASK_NAME) - .uid("expire-snapshots-" + uidSuffix()) + .name(EXECUTOR_OPERATOR_NAME) + .uid(EXECUTOR_OPERATOR_NAME + uidSuffix()) .slotSharingGroup(slotSharingGroup()) .forceNonParallel(); - AsyncRetryStrategy retryStrategy = - new AsyncRetryStrategies.ExponentialBackoffDelayRetryStrategyBuilder( - deleteAttemptNum, - DELETE_INITIAL_DELAY_MS, - DELETE_MAX_RETRY_DELAY_MS, - DELETE_BACKOFF_MULTIPLIER) - .ifResult(AsyncDeleteFiles.FAILED_PREDICATE) - .build(); - - AsyncDataStream.unorderedWaitWithRetry( - result.getSideOutput(ExpireSnapshotsProcessor.DELETE_STREAM).rebalance(), - new AsyncDeleteFiles(name(), tableLoader(), deleteWorkerPoolSize), - DELETE_TIMEOUT_MS, - TimeUnit.MILLISECONDS, - deleteWorkerPoolSize, - retryStrategy) - .name(DELETE_FILES_TASK_NAME) - .uid("delete-expired-files-" + uidSuffix()) + result + .getSideOutput(ExpireSnapshotsProcessor.DELETE_STREAM) + .rebalance() + .transform( + DELETE_FILES_OPERATOR_NAME, + TypeInformation.of(Void.class), + new DeleteFilesProcessor(name(), tableLoader(), deleteBatchSize)) + .name(DELETE_FILES_OPERATOR_NAME) + .uid(DELETE_FILES_OPERATOR_NAME + uidSuffix()) .slotSharingGroup(slotSharingGroup()) - .setParallelism(parallelism()); + .setParallelism(deleteParallelism); - // Deleting the files is asynchronous, so we ignore the results when calculating the return - // value + // Ignore the file deletion result and return the DataStream directly return result; } } diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/stream/MaintenanceTaskBuilder.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/MaintenanceTaskBuilder.java similarity index 88% rename from flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/stream/MaintenanceTaskBuilder.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/MaintenanceTaskBuilder.java index 5ca23f8cea74..f286ef4bd898 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/stream/MaintenanceTaskBuilder.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/MaintenanceTaskBuilder.java @@ -16,10 +16,11 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.iceberg.flink.maintenance.stream; +package org.apache.iceberg.flink.maintenance.api; import java.time.Duration; import org.apache.flink.annotation.Internal; +import org.apache.flink.annotation.PublicEvolving; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator; import org.apache.iceberg.flink.TableLoader; @@ -28,7 +29,8 @@ import org.apache.iceberg.flink.maintenance.operator.TriggerEvaluator; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; -public abstract class MaintenanceTaskBuilder { +@PublicEvolving +abstract class MaintenanceTaskBuilder { private int index; private String name; private TableLoader tableLoader; @@ -43,7 +45,6 @@ public abstract class MaintenanceTaskBuilder { * After a given number of Iceberg table commits since the last run, starts the downstream job. * * @param commitCount after the downstream job should be started - * @return for chained calls */ public T scheduleOnCommitCount(int commitCount) { triggerEvaluator.commitCount(commitCount); @@ -54,7 +55,6 @@ public T scheduleOnCommitCount(int commitCount) { * After a given number of new data files since the last run, starts the downstream job. * * @param dataFileCount after the downstream job should be started - * @return for chained calls */ public T scheduleOnDataFileCount(int dataFileCount) { triggerEvaluator.dataFileCount(dataFileCount); @@ -65,7 +65,6 @@ public T scheduleOnDataFileCount(int dataFileCount) { * After a given aggregated data file size since the last run, starts the downstream job. * * @param dataFileSizeInBytes after the downstream job should be started - * @return for chained calls */ public T scheduleOnDataFileSize(long dataFileSizeInBytes) { triggerEvaluator.dataFileSizeInBytes(dataFileSizeInBytes); @@ -77,7 +76,6 @@ public T scheduleOnDataFileSize(long dataFileSizeInBytes) { * job. * * @param posDeleteFileCount after the downstream job should be started - * @return for chained calls */ public T scheduleOnPosDeleteFileCount(int posDeleteFileCount) { triggerEvaluator.posDeleteFileCount(posDeleteFileCount); @@ -89,7 +87,6 @@ public T scheduleOnPosDeleteFileCount(int posDeleteFileCount) { * job. * * @param posDeleteRecordCount after the downstream job should be started - * @return for chained calls */ public T scheduleOnPosDeleteRecordCount(long posDeleteRecordCount) { triggerEvaluator.posDeleteRecordCount(posDeleteRecordCount); @@ -101,7 +98,6 @@ public T scheduleOnPosDeleteRecordCount(long posDeleteRecordCount) { * job. * * @param eqDeleteFileCount after the downstream job should be started - * @return for chained calls */ public T scheduleOnEqDeleteFileCount(int eqDeleteFileCount) { triggerEvaluator.eqDeleteFileCount(eqDeleteFileCount); @@ -113,7 +109,6 @@ public T scheduleOnEqDeleteFileCount(int eqDeleteFileCount) { * job. * * @param eqDeleteRecordCount after the downstream job should be started - * @return for chained calls */ public T scheduleOnEqDeleteRecordCount(long eqDeleteRecordCount) { triggerEvaluator.eqDeleteRecordCount(eqDeleteRecordCount); @@ -124,7 +119,6 @@ public T scheduleOnEqDeleteRecordCount(long eqDeleteRecordCount) { * After a given time since the last run, starts the downstream job. * * @param interval after the downstream job should be started - * @return for chained calls */ public T scheduleOnInterval(Duration interval) { triggerEvaluator.timeout(interval); @@ -135,7 +129,6 @@ public T scheduleOnInterval(Duration interval) { * The suffix used for the generated {@link org.apache.flink.api.dag.Transformation}'s uid. * * @param newUidSuffix for the transformations - * @return for chained calls */ public T uidSuffix(String newUidSuffix) { this.uidSuffix = newUidSuffix; @@ -147,7 +140,6 @@ public T uidSuffix(String newUidSuffix) { * generated stream. Could be used to separate the resources used by this task. * * @param newSlotSharingGroup to be used for the operators - * @return for chained calls */ public T slotSharingGroup(String newSlotSharingGroup) { this.slotSharingGroup = newSlotSharingGroup; @@ -158,15 +150,15 @@ public T slotSharingGroup(String newSlotSharingGroup) { * Sets the parallelism for the stream. * * @param newParallelism the required parallelism - * @return for chained calls */ public T parallelism(int newParallelism) { + Preconditions.checkArgument(newParallelism > 0, "Parallelism should be greater than 0"); this.parallelism = newParallelism; return (T) this; } @Internal - int id() { + int index() { return index; } @@ -190,8 +182,7 @@ String slotSharingGroup() { return slotSharingGroup; } - @Internal - Integer parallelism() { + protected Integer parallelism() { return parallelism; } @@ -204,19 +195,16 @@ TriggerEvaluator evaluator() { DataStream append( DataStream sourceStream, int maintenanceTaskIndex, - String maintainanceTaskName, + String maintenanceTaskName, TableLoader newTableLoader, String mainUidSuffix, String mainSlotSharingGroup, int mainParallelism) { - Preconditions.checkArgument( - parallelism == null || parallelism == -1 || parallelism > 0, - "Parallelism should be left to default (-1/null) or greater than 0"); - Preconditions.checkNotNull(maintainanceTaskName, "Name should not be null"); + Preconditions.checkNotNull(maintenanceTaskName, "Name should not be null"); Preconditions.checkNotNull(newTableLoader, "TableLoader should not be null"); this.index = maintenanceTaskIndex; - this.name = maintainanceTaskName; + this.name = maintenanceTaskName; this.tableLoader = newTableLoader; if (uidSuffix == null) { diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/stream/TableMaintenance.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/TableMaintenance.java similarity index 88% rename from flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/stream/TableMaintenance.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/TableMaintenance.java index 3add7adbabb1..6a0909c85040 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/stream/TableMaintenance.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/TableMaintenance.java @@ -16,7 +16,7 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.iceberg.flink.maintenance.stream; +package org.apache.iceberg.flink.maintenance.api; import java.time.Duration; import java.util.List; @@ -32,6 +32,7 @@ import org.apache.flink.api.common.eventtime.WatermarkStrategy; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.connector.source.util.ratelimit.RateLimiterStrategy; +import org.apache.flink.api.dag.Transformation; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.datastream.DataStreamUtils; import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator; @@ -46,15 +47,18 @@ import org.apache.iceberg.flink.maintenance.operator.TriggerEvaluator; import org.apache.iceberg.flink.maintenance.operator.TriggerLockFactory; import org.apache.iceberg.flink.maintenance.operator.TriggerManager; +import org.apache.iceberg.flink.sink.IcebergSink; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.collect.Lists; /** Creates the table maintenance graph. */ public class TableMaintenance { private static final String TASK_NAME_FORMAT = "%s [%d]"; - static final String SOURCE_NAME = "Monitor source"; - static final String TRIGGER_MANAGER_TASK_NAME = "Trigger manager"; - static final String LOCK_REMOVER_TASK_NAME = "Lock remover"; + static final String SOURCE_OPERATOR_NAME = "Monitor source"; + static final String TRIGGER_MANAGER_OPERATOR_NAME = "Trigger manager"; + static final String WATERMARK_ASSIGNER_OPERATOR_NAME = "Watermark Assigner"; + static final String FILTER_OPERATOR_NAME_PREFIX = "Filter "; + static final String LOCK_REMOVER_OPERATOR_NAME = "Lock remover"; private TableMaintenance() { // Do not instantiate directly @@ -62,7 +66,7 @@ private TableMaintenance() { /** * Use when the change stream is already provided, like in the {@link - * org.apache.iceberg.flink.sink.IcebergSink#addPostCommitTopology(DataStream)}. + * IcebergSink#addPostCommitTopology(DataStream)}. * * @param changeStream the table changes * @param tableLoader used for accessing the table @@ -81,8 +85,8 @@ public static Builder forChangeStream( } /** - * Creates the default monitor source for collecting the table changes and returns a builder for - * the maintenance stream. + * Use this for standalone maintenance job. It creates a monitor source that detect table changes + * and build the maintenance pipelines afterwards. * * @param env used to register the monitor source * @param tableLoader used for accessing the table @@ -107,7 +111,7 @@ public static class Builder { private String uidSuffix = "TableMaintenance-" + UUID.randomUUID(); private String slotSharingGroup = StreamGraphGenerator.DEFAULT_SLOT_SHARING_GROUP; - private Duration rateLimit = Duration.ofMillis(1); + private Duration rateLimit = Duration.ofMinutes(1); private Duration lockCheckDelay = Duration.ofSeconds(30); private Integer parallelism = ExecutionConfig.PARALLELISM_DEFAULT; private int maxReadBack = 100; @@ -133,10 +137,9 @@ private Builder( } /** - * The prefix used for the generated {@link org.apache.flink.api.dag.Transformation}'s uid. + * The prefix used for the generated {@link Transformation}'s uid. * * @param newUidSuffix for the transformations - * @return for chained calls */ public Builder uidSuffix(String newUidSuffix) { this.uidSuffix = newUidSuffix; @@ -148,7 +151,6 @@ public Builder uidSuffix(String newUidSuffix) { * generated stream. Could be used to separate the resources used by this task. * * @param newSlotSharingGroup to be used for the operators - * @return for chained calls */ public Builder slotSharingGroup(String newSlotSharingGroup) { this.slotSharingGroup = newSlotSharingGroup; @@ -159,7 +161,6 @@ public Builder slotSharingGroup(String newSlotSharingGroup) { * Limits the firing frequency for the task triggers. * * @param newRateLimit firing frequency - * @return for chained calls */ public Builder rateLimit(Duration newRateLimit) { Preconditions.checkNotNull(rateLimit.toMillis() > 0, "Rate limit should be greater than 0"); @@ -171,7 +172,6 @@ public Builder rateLimit(Duration newRateLimit) { * Sets the delay for checking lock availability when a concurrent run is detected. * * @param newLockCheckDelay lock checking frequency - * @return for chained calls */ public Builder lockCheckDelay(Duration newLockCheckDelay) { this.lockCheckDelay = newLockCheckDelay; @@ -179,13 +179,13 @@ public Builder lockCheckDelay(Duration newLockCheckDelay) { } /** - * Sets the global parallelism of maintenance tasks. Could be overwritten by the {@link + * Sets the default parallelism of maintenance tasks. Could be overwritten by the {@link * MaintenanceTaskBuilder#parallelism(int)}. * * @param newParallelism task parallelism - * @return for chained calls */ public Builder parallelism(int newParallelism) { + Preconditions.checkArgument(newParallelism > 0, "Parallelism should be greater than 0"); this.parallelism = newParallelism; return this; } @@ -197,7 +197,6 @@ public Builder parallelism(int newParallelism) { * used. * * @param newMaxReadBack snapshots to consider when initializing - * @return for chained calls */ public Builder maxReadBack(int newMaxReadBack) { Preconditions.checkArgument( @@ -210,7 +209,6 @@ public Builder maxReadBack(int newMaxReadBack) { * Adds a specific task with the given schedule. * * @param task to add - * @return for chained calls */ public Builder add(MaintenanceTaskBuilder task) { taskBuilders.add(task); @@ -239,13 +237,13 @@ public void append() { evaluators, rateLimit.toMillis(), lockCheckDelay.toMillis())) - .name(TRIGGER_MANAGER_TASK_NAME) - .uid("trigger-manager-" + uidSuffix) + .name(TRIGGER_MANAGER_OPERATOR_NAME) + .uid(TRIGGER_MANAGER_OPERATOR_NAME + uidSuffix) .slotSharingGroup(slotSharingGroup) .forceNonParallel() .assignTimestampsAndWatermarks(new PunctuatedWatermarkStrategy()) - .name("Watermark Assigner") - .uid("watermark-assigner-" + uidSuffix) + .name(WATERMARK_ASSIGNER_OPERATOR_NAME) + .uid(WATERMARK_ASSIGNER_OPERATOR_NAME + uidSuffix) .slotSharingGroup(slotSharingGroup) .forceNonParallel(); @@ -256,9 +254,9 @@ public void append() { DataStream filtered = triggers .filter(t -> t.taskId() != null && t.taskId() == finalIndex) - .name("Filter " + i) + .name(FILTER_OPERATOR_NAME_PREFIX + i) .forceNonParallel() - .uid("filter-" + i + "-" + uidSuffix) + .uid(FILTER_OPERATOR_NAME_PREFIX + i + "-" + uidSuffix) .slotSharingGroup(slotSharingGroup); MaintenanceTaskBuilder builder = taskBuilders.get(i); DataStream result = @@ -279,9 +277,8 @@ public void append() { // Add the LockRemover to the end unioned - .global() .transform( - LOCK_REMOVER_TASK_NAME, + LOCK_REMOVER_OPERATOR_NAME, TypeInformation.of(Void.class), new LockRemover(lockFactory, taskNames)) .forceNonParallel() @@ -297,8 +294,8 @@ private DataStream changeStream() { tableLoader, RateLimiterStrategy.perSecond(1.0 / rateLimit.getSeconds()), maxReadBack); - return env.fromSource(source, WatermarkStrategy.noWatermarks(), SOURCE_NAME) - .uid("monitor-source-" + uidSuffix) + return env.fromSource(source, WatermarkStrategy.noWatermarks(), SOURCE_OPERATOR_NAME) + .uid(SOURCE_OPERATOR_NAME + uidSuffix) .slotSharingGroup(slotSharingGroup) .forceNonParallel(); } else { diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/AsyncDeleteFiles.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/AsyncDeleteFiles.java deleted file mode 100644 index af40919a8f96..000000000000 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/AsyncDeleteFiles.java +++ /dev/null @@ -1,106 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.iceberg.flink.maintenance.operator; - -import java.io.Serializable; -import java.util.Collection; -import java.util.Collections; -import java.util.concurrent.ExecutorService; -import java.util.function.Predicate; -import org.apache.flink.annotation.Internal; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.metrics.Counter; -import org.apache.flink.streaming.api.functions.async.ResultFuture; -import org.apache.flink.streaming.api.functions.async.RichAsyncFunction; -import org.apache.iceberg.Table; -import org.apache.iceberg.flink.TableLoader; -import org.apache.iceberg.io.FileIO; -import org.apache.iceberg.relocated.com.google.common.base.Preconditions; -import org.apache.iceberg.util.ThreadPools; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** Delete the files using the {@link FileIO}. */ -@Internal -public class AsyncDeleteFiles extends RichAsyncFunction { - private static final Logger LOG = LoggerFactory.getLogger(AsyncDeleteFiles.class); - public static final Predicate> FAILED_PREDICATE = new FailedPredicate(); - - private final String name; - private final FileIO io; - private final int workerPoolSize; - private final String tableName; - - private transient ExecutorService workerPool; - private transient Counter failedCounter; - private transient Counter succeededCounter; - - public AsyncDeleteFiles(String name, TableLoader tableLoader, int workerPoolSize) { - Preconditions.checkNotNull(name, "Name should no be null"); - Preconditions.checkNotNull(tableLoader, "Table loader should no be null"); - - this.name = name; - tableLoader.open(); - Table table = tableLoader.loadTable(); - this.io = table.io(); - this.workerPoolSize = workerPoolSize; - this.tableName = table.name(); - } - - @Override - public void open(Configuration parameters) throws Exception { - this.failedCounter = - getRuntimeContext() - .getMetricGroup() - .addGroup(TableMaintenanceMetrics.GROUP_KEY, name) - .counter(TableMaintenanceMetrics.DELETE_FILE_FAILED_COUNTER); - this.succeededCounter = - getRuntimeContext() - .getMetricGroup() - .addGroup(TableMaintenanceMetrics.GROUP_KEY, name) - .counter(TableMaintenanceMetrics.DELETE_FILE_SUCCEEDED_COUNTER); - - this.workerPool = - ThreadPools.newWorkerPool(tableName + "-" + name + "-async-delete-files", workerPoolSize); - } - - @Override - public void asyncInvoke(String fileName, ResultFuture resultFuture) { - workerPool.execute( - () -> { - try { - LOG.info("Deleting file: {} with {}", fileName, name); - io.deleteFile(fileName); - resultFuture.complete(Collections.singletonList(true)); - succeededCounter.inc(); - } catch (Throwable e) { - LOG.info("Failed to delete file {} with {}", fileName, name, e); - resultFuture.complete(Collections.singletonList(false)); - failedCounter.inc(); - } - }); - } - - private static class FailedPredicate implements Predicate>, Serializable { - @Override - public boolean test(Collection collection) { - return collection.size() != 1 || !collection.iterator().next(); - } - } -} diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/DeleteFilesProcessor.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/DeleteFilesProcessor.java new file mode 100644 index 000000000000..6f01f8ac46be --- /dev/null +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/DeleteFilesProcessor.java @@ -0,0 +1,124 @@ +/* + * 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.operator; + +import java.util.Set; +import org.apache.flink.annotation.Internal; +import org.apache.flink.metrics.Counter; +import org.apache.flink.shaded.guava31.com.google.common.collect.Sets; +import org.apache.flink.streaming.api.operators.AbstractStreamOperator; +import org.apache.flink.streaming.api.operators.OneInputStreamOperator; +import org.apache.flink.streaming.api.watermark.Watermark; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.apache.iceberg.Table; +import org.apache.iceberg.flink.TableLoader; +import org.apache.iceberg.io.BulkDeletionFailureException; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.io.SupportsBulkOperations; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** Delete the files using the {@link FileIO}. */ +@Internal +public class DeleteFilesProcessor extends AbstractStreamOperator + implements OneInputStreamOperator { + private static final Logger LOG = LoggerFactory.getLogger(DeleteFilesProcessor.class); + + private final String name; + private final SupportsBulkOperations io; + private final String tableName; + private final Set filesToDelete = Sets.newHashSet(); + private final int batchSize; + + private transient Counter failedCounter; + private transient Counter succeededCounter; + + public DeleteFilesProcessor(String name, TableLoader tableLoader, int batchSize) { + Preconditions.checkNotNull(name, "Name should no be null"); + Preconditions.checkNotNull(tableLoader, "Table loader should no be null"); + + tableLoader.open(); + Table table = tableLoader.loadTable(); + FileIO fileIO = table.io(); + Preconditions.checkArgument( + fileIO instanceof SupportsBulkOperations, + "Unsupported FileIO. %s should support bulk delete", + fileIO); + + this.name = name; + this.io = (SupportsBulkOperations) fileIO; + this.tableName = table.name(); + this.batchSize = batchSize; + } + + @Override + public void open() throws Exception { + this.failedCounter = + getRuntimeContext() + .getMetricGroup() + .addGroup(TableMaintenanceMetrics.GROUP_KEY, name) + .counter(TableMaintenanceMetrics.DELETE_FILE_FAILED_COUNTER); + this.succeededCounter = + getRuntimeContext() + .getMetricGroup() + .addGroup(TableMaintenanceMetrics.GROUP_KEY, name) + .counter(TableMaintenanceMetrics.DELETE_FILE_SUCCEEDED_COUNTER); + } + + @Override + public void processElement(StreamRecord element) throws Exception { + if (element.isRecord()) { + filesToDelete.add(element.getValue()); + } + + if (filesToDelete.size() >= batchSize) { + deleteFiles(); + } + } + + @Override + public void processWatermark(Watermark mark) { + deleteFiles(); + } + + @Override + public void prepareSnapshotPreBarrier(long checkpointId) { + deleteFiles(); + } + + private void deleteFiles() { + try { + io.deleteFiles(filesToDelete); + LOG.info( + "Deleted {} files from table {} using bulk deletes", filesToDelete.size(), tableName); + succeededCounter.inc(filesToDelete.size()); + filesToDelete.clear(); + } catch (BulkDeletionFailureException e) { + int deletedFilesCount = filesToDelete.size() - e.numberFailedObjects(); + LOG.warn( + "Deleted only {} of {} files from table {} using bulk deletes", + deletedFilesCount, + filesToDelete.size(), + tableName); + succeededCounter.inc(deletedFilesCount); + failedCounter.inc(e.numberFailedObjects()); + } + } +} diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/ExpireSnapshotsProcessor.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/ExpireSnapshotsProcessor.java index 54ffa0009f46..52499572a522 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/ExpireSnapshotsProcessor.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/ExpireSnapshotsProcessor.java @@ -21,6 +21,7 @@ import java.util.Collections; import java.util.concurrent.ExecutorService; import java.util.concurrent.atomic.AtomicLong; +import org.apache.flink.annotation.Internal; import org.apache.flink.api.common.typeinfo.Types; import org.apache.flink.configuration.Configuration; import org.apache.flink.streaming.api.functions.ProcessFunction; @@ -39,6 +40,7 @@ * Calls the {@link ExpireSnapshots} to remove the old snapshots and emits the filenames which could * be removed in the {@link #DELETE_STREAM} side output. */ +@Internal public class ExpireSnapshotsProcessor extends ProcessFunction { private static final Logger LOG = LoggerFactory.getLogger(ExpireSnapshotsProcessor.class); public static final OutputTag DELETE_STREAM = @@ -46,18 +48,18 @@ public class ExpireSnapshotsProcessor extends ProcessFunction o expireSnapshots = expireSnapshots.expireOlderThan(ctx.timestamp() - maxSnapshotAgeMs); } - if (retainLast != null) { - expireSnapshots = expireSnapshots.retainLast(retainLast); + if (numSnapshots != null) { + expireSnapshots = expireSnapshots.retainLast(numSnapshots); } AtomicLong deleteFileCounter = new AtomicLong(0L); @@ -101,7 +103,7 @@ public void processElement(Trigger trigger, Context ctx, Collector o out.collect( new TaskResult(trigger.taskId(), trigger.timestamp(), true, Collections.emptyList())); } catch (Exception e) { - LOG.info("Exception expiring snapshots for {} at {}", table, ctx.timestamp(), e); + LOG.error("Failed to expiring snapshots for {} at {}", table, ctx.timestamp(), e); out.collect( new TaskResult(trigger.taskId(), trigger.timestamp(), false, Lists.newArrayList(e))); } diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/stream/ScheduledInfraExtension.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/api/MaintenanceTaskInfraExtension.java similarity index 96% rename from flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/stream/ScheduledInfraExtension.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/api/MaintenanceTaskInfraExtension.java index 55ea6161517c..392407c8a047 100644 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/stream/ScheduledInfraExtension.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/api/MaintenanceTaskInfraExtension.java @@ -16,7 +16,7 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.iceberg.flink.maintenance.stream; +package org.apache.iceberg.flink.maintenance.api; import static org.apache.iceberg.flink.maintenance.operator.OperatorTestBase.IGNORED_OPERATOR_NAME; @@ -42,7 +42,7 @@ * maintenance tasks * */ -class ScheduledInfraExtension implements BeforeEachCallback { +class MaintenanceTaskInfraExtension implements BeforeEachCallback { private StreamExecutionEnvironment env; private ManualSource source; private DataStream triggerStream; diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/stream/ScheduledBuilderTestBase.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/api/MaintenanceTaskTestBase.java similarity index 87% rename from flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/stream/ScheduledBuilderTestBase.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/api/MaintenanceTaskTestBase.java index a2474c0f0809..e42f77a7ddb7 100644 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/stream/ScheduledBuilderTestBase.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/api/MaintenanceTaskTestBase.java @@ -16,7 +16,7 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.iceberg.flink.maintenance.stream; +package org.apache.iceberg.flink.maintenance.api; import static org.assertj.core.api.Assertions.assertThat; @@ -34,12 +34,11 @@ import org.awaitility.Awaitility; import org.junit.jupiter.api.extension.RegisterExtension; -class ScheduledBuilderTestBase extends OperatorTestBase { +class MaintenanceTaskTestBase extends OperatorTestBase { private static final int TESTING_TASK_ID = 0; private static final Duration POLL_DURATION = Duration.ofSeconds(5); - static final String DB_NAME = "db"; - @RegisterExtension ScheduledInfraExtension infra = new ScheduledInfraExtension(); + @RegisterExtension MaintenanceTaskInfraExtension infra = new MaintenanceTaskInfraExtension(); /** * Triggers a maintenance tasks and waits for the successful result. The {@link Table} is @@ -48,6 +47,7 @@ class ScheduledBuilderTestBase extends OperatorTestBase { * @param env used for testing * @param triggerSource used for manually emitting the trigger * @param collectingSink used for collecting the result + * @param waitForCondition used to wait until target condition is reached before stopping the job * @param table used for generating the payload * @throws Exception if any */ @@ -55,7 +55,7 @@ void runAndWaitForSuccess( StreamExecutionEnvironment env, ManualSource triggerSource, CollectingSink collectingSink, - Supplier checkSideEffect, + Supplier waitForCondition, Table table) throws Exception { table.refresh(); @@ -75,7 +75,7 @@ void runAndWaitForSuccess( assertThat(result.success()).isTrue(); assertThat(result.taskIndex()).isEqualTo(TESTING_TASK_ID); - Awaitility.await().until(() -> checkSideEffect.get()); + Awaitility.await().until(() -> waitForCondition.get()); } finally { closeJobClient(jobClient); } diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/stream/TestExpireSnapshots.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/api/TestExpireSnapshots.java similarity index 75% rename from flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/stream/TestExpireSnapshots.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/api/TestExpireSnapshots.java index dff176d6c7fa..67970862a416 100644 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/stream/TestExpireSnapshots.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/api/TestExpireSnapshots.java @@ -16,12 +16,12 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.iceberg.flink.maintenance.stream; +package org.apache.iceberg.flink.maintenance.api; import static org.apache.iceberg.flink.SimpleDataUtil.createRecord; +import static org.apache.iceberg.flink.maintenance.api.ExpireSnapshots.DELETE_FILES_OPERATOR_NAME; import static org.apache.iceberg.flink.maintenance.operator.TableMaintenanceMetrics.DELETE_FILE_FAILED_COUNTER; import static org.apache.iceberg.flink.maintenance.operator.TableMaintenanceMetrics.DELETE_FILE_SUCCEEDED_COUNTER; -import static org.apache.iceberg.flink.maintenance.stream.ExpireSnapshots.DELETE_FILES_TASK_NAME; import static org.assertj.core.api.Assertions.assertThat; import java.time.Duration; @@ -31,9 +31,8 @@ import org.apache.iceberg.SerializableTable; import org.apache.iceberg.Snapshot; import org.apache.iceberg.Table; -import org.apache.iceberg.catalog.TableIdentifier; import org.apache.iceberg.flink.SimpleDataUtil; -import org.apache.iceberg.flink.TableLoader; +import org.apache.iceberg.flink.TestFixtures; import org.apache.iceberg.flink.maintenance.operator.MetricsReporterFactoryForTests; import org.apache.iceberg.flink.maintenance.operator.Trigger; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; @@ -43,30 +42,30 @@ import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; -class TestExpireSnapshots extends ScheduledBuilderTestBase { +class TestExpireSnapshots extends MaintenanceTaskTestBase { + private Table table; + @BeforeEach void before() { MetricsReporterFactoryForTests.reset(); + this.table = createTable(); } @Test void testExpireSnapshots() throws Exception { - sql.exec("CREATE TABLE %s (id int, data varchar)", TABLE_NAME); - sql.exec("INSERT INTO %s VALUES (1, 'a')", TABLE_NAME); - sql.exec("INSERT INTO %s VALUES (2, 'b')", TABLE_NAME); - sql.exec("INSERT INTO %s VALUES (3, 'c')", TABLE_NAME); - sql.exec("INSERT INTO %s VALUES (4, 'd')", TABLE_NAME); - - TableLoader tableLoader = sql.tableLoader(TABLE_NAME); - Table table = tableLoader.loadTable(); + insert(table, 1, "a"); + insert(table, 2, "b"); + insert(table, 3, "c"); + insert(table, 4, "d"); + Set snapshots = Sets.newHashSet(table.snapshots()); assertThat(snapshots).hasSize(4); ExpireSnapshots.builder() .parallelism(1) .planningWorkerPoolSize(2) - .deleteAttemptNum(2) - .deleteWorkerPoolSize(5) + .deleteBatchSize(3) + .deleteParallelism(1) .maxSnapshotAge(Duration.ZERO) .retainLast(1) .uidSuffix(UID_SUFFIX) @@ -74,7 +73,7 @@ void testExpireSnapshots() throws Exception { infra.triggerStream(), 0, DUMMY_NAME, - tableLoader, + tableLoader(), "OTHER", StreamGraphGenerator.DEFAULT_SLOT_SHARING_GROUP, 1) @@ -83,9 +82,9 @@ void testExpireSnapshots() throws Exception { runAndWaitForSuccess( infra.env(), infra.source(), infra.sink(), () -> checkDeleteFinished(3L), table); - // Check that the table data not changed table.refresh(); assertThat(Sets.newHashSet(table.snapshots())).hasSize(1); + // Check that the table data not changed SimpleDataUtil.assertTableRecords( table, ImmutableList.of( @@ -97,12 +96,9 @@ void testExpireSnapshots() throws Exception { @Test void testFailure() throws Exception { - sql.exec("CREATE TABLE %s (id int, data varchar)", TABLE_NAME); - sql.exec("INSERT INTO %s VALUES (1, 'a')", TABLE_NAME); - sql.exec("INSERT INTO %s VALUES (2, 'b')", TABLE_NAME); + insert(table, 1, "a"); + insert(table, 2, "b"); - TableLoader tableLoader = sql.tableLoader(TABLE_NAME); - Table table = tableLoader.loadTable(); SerializableTable serializableTable = (SerializableTable) SerializableTable.copyOf(table); ExpireSnapshots.builder() @@ -110,7 +106,7 @@ void testFailure() throws Exception { infra.triggerStream(), 0, DUMMY_NAME, - tableLoader, + tableLoader(), UID_SUFFIX, StreamGraphGenerator.DEFAULT_SLOT_SHARING_GROUP, 1) @@ -130,7 +126,7 @@ void testFailure() throws Exception { assertThat(infra.sink().poll(Duration.ofSeconds(5)).success()).isTrue(); // Drop the table, so it will cause an exception - sql.catalogLoader().loadCatalog().dropTable(TableIdentifier.of(DB_NAME, TABLE_NAME)); + CATALOG_EXTENSION.catalogLoader().loadCatalog().dropTable(TestFixtures.TABLE_IDENTIFIER); // Failed run infra.source().sendRecord(Trigger.create(time + 1, serializableTable, 1), time + 1); @@ -143,17 +139,17 @@ void testFailure() throws Exception { // Check the metrics MetricsReporterFactoryForTests.assertCounters( new ImmutableMap.Builder() - .put(DELETE_FILES_TASK_NAME + "." + DUMMY_NAME + "." + DELETE_FILE_FAILED_COUNTER, 0L) .put( - DELETE_FILES_TASK_NAME + "." + DUMMY_NAME + "." + DELETE_FILE_SUCCEEDED_COUNTER, 0L) + DELETE_FILES_OPERATOR_NAME + "." + DUMMY_NAME + "." + DELETE_FILE_FAILED_COUNTER, + 0L) + .put( + DELETE_FILES_OPERATOR_NAME + "." + DUMMY_NAME + "." + DELETE_FILE_SUCCEEDED_COUNTER, + 0L) .build()); } @Test void testUidAndSlotSharingGroup() { - sql.exec("CREATE TABLE %s (id int, data varchar)", TABLE_NAME); - TableLoader tableLoader = sql.tableLoader(TABLE_NAME); - ExpireSnapshots.builder() .slotSharingGroup(SLOT_SHARING_GROUP) .uidSuffix(UID_SUFFIX) @@ -161,7 +157,7 @@ void testUidAndSlotSharingGroup() { infra.triggerStream(), 0, DUMMY_NAME, - tableLoader, + tableLoader(), UID_SUFFIX, StreamGraphGenerator.DEFAULT_SLOT_SHARING_GROUP, 1) @@ -173,15 +169,12 @@ void testUidAndSlotSharingGroup() { @Test void testUidAndSlotSharingGroupUnset() { - sql.exec("CREATE TABLE %s (id int, data varchar)", TABLE_NAME); - TableLoader tableLoader = sql.tableLoader(TABLE_NAME); - ExpireSnapshots.builder() .append( infra.triggerStream(), 0, DUMMY_NAME, - tableLoader, + tableLoader(), UID_SUFFIX, StreamGraphGenerator.DEFAULT_SLOT_SHARING_GROUP, 1) @@ -193,12 +186,8 @@ void testUidAndSlotSharingGroupUnset() { @Test void testMetrics() throws Exception { - sql.exec("CREATE TABLE %s (id int, data varchar)", TABLE_NAME); - sql.exec("INSERT INTO %s VALUES (1, 'a')", TABLE_NAME); - sql.exec("INSERT INTO %s VALUES (2, 'b')", TABLE_NAME); - - TableLoader tableLoader = sql.tableLoader(TABLE_NAME); - Table table = tableLoader.loadTable(); + insert(table, 1, "a"); + insert(table, 2, "b"); ExpireSnapshots.builder() .maxSnapshotAge(Duration.ZERO) @@ -208,7 +197,7 @@ void testMetrics() throws Exception { infra.triggerStream(), 0, DUMMY_NAME, - tableLoader, + tableLoader(), UID_SUFFIX, StreamGraphGenerator.DEFAULT_SLOT_SHARING_GROUP, 1) @@ -224,14 +213,14 @@ void testMetrics() throws Exception { MetricsReporterFactoryForTests.assertCounters( new ImmutableMap.Builder() .put( - DELETE_FILES_TASK_NAME + DELETE_FILES_OPERATOR_NAME + "." + DUMMY_NAME + "." + DELETE_FILE_FAILED_COUNTER, 0L) .put( - DELETE_FILES_TASK_NAME + DELETE_FILES_OPERATOR_NAME + "." + DUMMY_NAME + "." @@ -243,6 +232,6 @@ void testMetrics() throws Exception { private static boolean checkDeleteFinished(Long expectedDeleteNum) { return expectedDeleteNum.equals( MetricsReporterFactoryForTests.counter( - DELETE_FILES_TASK_NAME + "." + DUMMY_NAME + "." + DELETE_FILE_SUCCEEDED_COUNTER)); + DELETE_FILES_OPERATOR_NAME + "." + DUMMY_NAME + "." + DELETE_FILE_SUCCEEDED_COUNTER)); } } diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/stream/TestMaintenanceE2E.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/api/TestMaintenanceE2E.java similarity index 80% rename from flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/stream/TestMaintenanceE2E.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/api/TestMaintenanceE2E.java index 3a6a7909eca2..467ad2d8ced9 100644 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/stream/TestMaintenanceE2E.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/api/TestMaintenanceE2E.java @@ -16,14 +16,15 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.iceberg.flink.maintenance.stream; +package org.apache.iceberg.flink.maintenance.api; import static org.assertj.core.api.Assertions.assertThat; +import java.io.IOException; import java.time.Duration; import org.apache.flink.core.execution.JobClient; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.iceberg.flink.TableLoader; +import org.apache.iceberg.Table; import org.apache.iceberg.flink.maintenance.operator.OperatorTestBase; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; @@ -32,18 +33,15 @@ class TestMaintenanceE2E extends OperatorTestBase { private StreamExecutionEnvironment env; @BeforeEach - public void beforeEach() { - env = StreamExecutionEnvironment.getExecutionEnvironment(); + public void beforeEach() throws IOException { + this.env = StreamExecutionEnvironment.getExecutionEnvironment(); + Table table = createTable(); + insert(table, 1, "a"); } @Test void testE2e() 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); - - TableMaintenance.forTable(env, tableLoader, LOCK_FACTORY) + TableMaintenance.forTable(env, tableLoader(), LOCK_FACTORY) .uidSuffix("E2eTestUID") .rateLimit(Duration.ofMinutes(10)) .lockCheckDelay(Duration.ofSeconds(10)) @@ -52,7 +50,7 @@ void testE2e() throws Exception { .scheduleOnCommitCount(10) .maxSnapshotAge(Duration.ofMinutes(10)) .retainLast(5) - .deleteWorkerPoolSize(5) + .deleteBatchSize(5) .parallelism(8)) .append(); diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/stream/TestTableMaintenance.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/api/TestTableMaintenance.java similarity index 80% rename from flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/stream/TestTableMaintenance.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/api/TestTableMaintenance.java index 181fa1e6dc87..0b94effada80 100644 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/stream/TestTableMaintenance.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/api/TestTableMaintenance.java @@ -16,9 +16,12 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.iceberg.flink.maintenance.stream; +package org.apache.iceberg.flink.maintenance.api; import static org.apache.iceberg.flink.SimpleDataUtil.createRowData; +import static org.apache.iceberg.flink.maintenance.api.TableMaintenance.LOCK_REMOVER_OPERATOR_NAME; +import static org.apache.iceberg.flink.maintenance.api.TableMaintenance.SOURCE_OPERATOR_NAME; +import static org.apache.iceberg.flink.maintenance.api.TableMaintenance.TRIGGER_MANAGER_OPERATOR_NAME; 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; @@ -26,12 +29,10 @@ 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.IOException; import java.io.Serializable; import java.time.Duration; import java.util.Collections; @@ -79,32 +80,30 @@ class TestTableMaintenance extends OperatorTestBase { Collections.synchronizedList(Lists.newArrayListWithCapacity(1)); private StreamExecutionEnvironment env; + private Table table; @TempDir private File checkpointDir; @BeforeEach - public void beforeEach() { + public void beforeEach() throws IOException { Configuration config = new Configuration(); config.set(CheckpointingOptions.CHECKPOINT_STORAGE, "filesystem"); config.set(CheckpointingOptions.CHECKPOINTS_DIRECTORY, "file://" + checkpointDir.getPath()); - env = StreamExecutionEnvironment.getExecutionEnvironment(config); + this.env = StreamExecutionEnvironment.getExecutionEnvironment(config); + this.table = createTable(); + insert(table, 1, "a"); + 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(); - + void testForChangeStream() throws Exception { ManualSource schedulerSource = new ManualSource<>(env, TypeInformation.of(TableChange.class)); TableMaintenance.Builder streamBuilder = - TableMaintenance.forChangeStream(schedulerSource.dataStream(), tableLoader, LOCK_FACTORY) + TableMaintenance.forChangeStream(schedulerSource.dataStream(), tableLoader(), LOCK_FACTORY) .rateLimit(Duration.ofMillis(2)) .lockCheckDelay(Duration.ofSeconds(3)) .add( @@ -122,16 +121,8 @@ void testFromStream() throws Exception { } @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(); + void testForTable() throws Exception { + TableLoader tableLoader = tableLoader(); env.enableCheckpointing(10); @@ -163,19 +154,13 @@ void testFromEnv() throws Exception { @Test void testLocking() 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(); - TriggerLockFactory.Lock lock = LOCK_FACTORY.createLock(); ManualSource schedulerSource = new ManualSource<>(env, TypeInformation.of(TableChange.class)); TableMaintenance.Builder streamBuilder = - TableMaintenance.forChangeStream(schedulerSource.dataStream(), tableLoader, LOCK_FACTORY) + TableMaintenance.forChangeStream(schedulerSource.dataStream(), tableLoader(), LOCK_FACTORY) .rateLimit(Duration.ofMillis(2)) .add(new MaintenanceTaskBuilderForTest(true).scheduleOnCommitCount(1)); @@ -187,17 +172,11 @@ void testLocking() throws Exception { @Test void testMetrics() 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 schedulerSource = new ManualSource<>(env, TypeInformation.of(TableChange.class)); TableMaintenance.Builder streamBuilder = - TableMaintenance.forChangeStream(schedulerSource.dataStream(), tableLoader, LOCK_FACTORY) + TableMaintenance.forChangeStream(schedulerSource.dataStream(), tableLoader(), LOCK_FACTORY) .rateLimit(Duration.ofMillis(2)) .lockCheckDelay(Duration.ofMillis(2)) .add(new MaintenanceTaskBuilderForTest(true).scheduleOnCommitCount(1)) @@ -212,29 +191,33 @@ void testMetrics() throws Exception { .until( () -> MetricsReporterFactoryForTests.counter( - LOCK_REMOVER_TASK_NAME + "." + TASKS[0] + "." + SUCCEEDED_TASK_COUNTER) + LOCK_REMOVER_OPERATOR_NAME + "." + TASKS[0] + "." + SUCCEEDED_TASK_COUNTER) .equals(2L)); MetricsReporterFactoryForTests.assertCounters( new ImmutableMap.Builder() - .put(LOCK_REMOVER_TASK_NAME + "." + TASKS[0] + "." + SUCCEEDED_TASK_COUNTER, 2L) - .put(LOCK_REMOVER_TASK_NAME + "." + TASKS[0] + "." + FAILED_TASK_COUNTER, 0L) - .put(TRIGGER_MANAGER_TASK_NAME + "." + TASKS[0] + "." + TRIGGERED, 2L) - .put(LOCK_REMOVER_TASK_NAME + "." + TASKS[1] + "." + SUCCEEDED_TASK_COUNTER, 0L) - .put(LOCK_REMOVER_TASK_NAME + "." + TASKS[1] + "." + FAILED_TASK_COUNTER, 1L) - .put(TRIGGER_MANAGER_TASK_NAME + "." + TASKS[1] + "." + TRIGGERED, 1L) + .put(LOCK_REMOVER_OPERATOR_NAME + "." + TASKS[0] + "." + SUCCEEDED_TASK_COUNTER, 2L) + .put(LOCK_REMOVER_OPERATOR_NAME + "." + TASKS[0] + "." + FAILED_TASK_COUNTER, 0L) + .put(TRIGGER_MANAGER_OPERATOR_NAME + "." + TASKS[0] + "." + TRIGGERED, 2L) + .put(LOCK_REMOVER_OPERATOR_NAME + "." + TASKS[1] + "." + SUCCEEDED_TASK_COUNTER, 0L) + .put(LOCK_REMOVER_OPERATOR_NAME + "." + TASKS[1] + "." + FAILED_TASK_COUNTER, 1L) + .put(TRIGGER_MANAGER_OPERATOR_NAME + "." + TASKS[1] + "." + TRIGGERED, 1L) .put( - TRIGGER_MANAGER_TASK_NAME + "." + GROUP_VALUE_DEFAULT + "." + NOTHING_TO_TRIGGER, + TRIGGER_MANAGER_OPERATOR_NAME + + "." + + GROUP_VALUE_DEFAULT + + "." + + NOTHING_TO_TRIGGER, -1L) .put( - TRIGGER_MANAGER_TASK_NAME + TRIGGER_MANAGER_OPERATOR_NAME + "." + GROUP_VALUE_DEFAULT + "." + CONCURRENT_RUN_THROTTLED, -1L) .put( - TRIGGER_MANAGER_TASK_NAME + TRIGGER_MANAGER_OPERATOR_NAME + "." + GROUP_VALUE_DEFAULT + "." @@ -245,15 +228,9 @@ void testMetrics() throws Exception { @Test void testUidAndSlotSharingGroup() { - 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(); - TableMaintenance.forChangeStream( new ManualSource<>(env, TypeInformation.of(TableChange.class)).dataStream(), - tableLoader, + tableLoader(), LOCK_FACTORY) .uidSuffix(UID_SUFFIX) .slotSharingGroup(SLOT_SHARING_GROUP) @@ -270,15 +247,9 @@ void testUidAndSlotSharingGroup() { @Test void testUidAndSlotSharingGroupUnset() { - 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(); - TableMaintenance.forChangeStream( new ManualSource<>(env, TypeInformation.of(TableChange.class)).dataStream(), - tableLoader, + tableLoader(), LOCK_FACTORY) .add(new MaintenanceTaskBuilderForTest(true).scheduleOnCommitCount(1)) .append(); @@ -289,15 +260,9 @@ void testUidAndSlotSharingGroupUnset() { @Test void testUidAndSlotSharingGroupInherit() { - 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(); - TableMaintenance.forChangeStream( new ManualSource<>(env, TypeInformation.of(TableChange.class)).dataStream(), - tableLoader, + tableLoader(), LOCK_FACTORY) .uidSuffix(UID_SUFFIX) .slotSharingGroup(SLOT_SHARING_GROUP) @@ -312,15 +277,9 @@ void testUidAndSlotSharingGroupInherit() { void testUidAndSlotSharingGroupOverWrite() { String anotherUid = "Another-UID"; String anotherSlotSharingGroup = "Another-SlotSharingGroup"; - 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(); - TableMaintenance.forChangeStream( new ManualSource<>(env, TypeInformation.of(TableChange.class)).dataStream(), - tableLoader, + tableLoader(), LOCK_FACTORY) .uidSuffix(UID_SUFFIX) .slotSharingGroup(SLOT_SHARING_GROUP) @@ -331,7 +290,7 @@ void testUidAndSlotSharingGroupOverWrite() { .slotSharingGroup(anotherSlotSharingGroup)) .append(); - // Something from the scheduler + // Choose an operator from the scheduler part of the graph Transformation schedulerTransformation = env.getTransformations().stream() .filter(t -> t.getName().equals("Trigger manager")) @@ -342,7 +301,7 @@ void testUidAndSlotSharingGroupOverWrite() { assertThat(schedulerTransformation.getSlotSharingGroup().get().getName()) .isEqualTo(SLOT_SHARING_GROUP); - // Something from the scheduled stream + // Choose an operator from the maintenance task part of the graph Transformation scheduledTransformation = env.getTransformations().stream() .filter( @@ -356,14 +315,8 @@ void testUidAndSlotSharingGroupOverWrite() { } @Test - void testUidAndSlotSharingGroupForMonitor() { - 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(); - - TableMaintenance.forTable(env, tableLoader, LOCK_FACTORY) + void testUidAndSlotSharingGroupForMonitorSource() { + TableMaintenance.forTable(env, tableLoader(), LOCK_FACTORY) .uidSuffix(UID_SUFFIX) .slotSharingGroup(SLOT_SHARING_GROUP) .add( @@ -416,6 +369,27 @@ private void sendEvents( } } + /** + * Finds the {@link org.apache.iceberg.flink.maintenance.operator.MonitorSource} for testing + * purposes by parsing the transformation tree. + * + * @return The monitor source if we found it + */ + private Transformation monitorSource() { + assertThat(env.getTransformations()).isNotEmpty(); + assertThat(env.getTransformations().get(0).getInputs()).isNotEmpty(); + assertThat(env.getTransformations().get(0).getInputs().get(0).getInputs()).isNotEmpty(); + + Transformation result = + env.getTransformations().get(0).getInputs().get(0).getInputs().get(0); + + // Some checks to make sure this is the transformation we are looking for + assertThat(result).isInstanceOf(SourceTransformation.class); + assertThat(result.getName()).isEqualTo(SOURCE_OPERATOR_NAME); + + return result; + } + private static class MaintenanceTaskBuilderForTest extends MaintenanceTaskBuilder { private final boolean success; @@ -440,27 +414,6 @@ DataStream append(DataStream trigger) { } } - /** - * Finds the {@link org.apache.iceberg.flink.maintenance.operator.MonitorSource} for testing - * purposes by parsing the transformation tree. - * - * @return The monitor source if we found it - */ - private Transformation monitorSource() { - assertThat(env.getTransformations()).isNotEmpty(); - assertThat(env.getTransformations().get(0).getInputs()).isNotEmpty(); - assertThat(env.getTransformations().get(0).getInputs().get(0).getInputs()).isNotEmpty(); - - Transformation result = - env.getTransformations().get(0).getInputs().get(0).getInputs().get(0); - - // Some checks to make sure this is the transformation we are looking for - assertThat(result).isInstanceOf(SourceTransformation.class); - assertThat(result.getName()).isEqualTo(SOURCE_NAME); - - return result; - } - private static class DummyMaintenanceTask implements MapFunction, ResultTypeQueryable, Serializable { private final boolean success; diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/FlinkSqlExtension.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/FlinkSqlExtension.java deleted file mode 100644 index 1e8ebcdeb256..000000000000 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/FlinkSqlExtension.java +++ /dev/null @@ -1,139 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.iceberg.flink.maintenance.operator; - -import java.io.File; -import java.io.IOException; -import java.nio.file.Files; -import java.nio.file.Path; -import java.util.Comparator; -import java.util.List; -import java.util.Map; -import java.util.stream.Collectors; -import java.util.stream.Stream; -import org.apache.flink.table.api.EnvironmentSettings; -import org.apache.flink.table.api.TableEnvironment; -import org.apache.flink.table.api.TableResult; -import org.apache.flink.types.Row; -import org.apache.flink.util.CloseableIterator; -import org.apache.hadoop.conf.Configuration; -import org.apache.iceberg.CatalogProperties; -import org.apache.iceberg.catalog.TableIdentifier; -import org.apache.iceberg.flink.CatalogLoader; -import org.apache.iceberg.flink.TableLoader; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.apache.iceberg.relocated.com.google.common.collect.Maps; -import org.junit.jupiter.api.extension.AfterEachCallback; -import org.junit.jupiter.api.extension.BeforeEachCallback; -import org.junit.jupiter.api.extension.ExtensionContext; - -/** - * Junit 5 extension for running Flink SQL queries. {@link - * org.apache.flink.test.junit5.MiniClusterExtension} is used for executing the SQL batch jobs. - */ -public class FlinkSqlExtension implements BeforeEachCallback, AfterEachCallback { - private final String catalogName; - private final Map catalogProperties; - private final String databaseName; - private final Path warehouse; - private final CatalogLoader catalogLoader; - private TableEnvironment tableEnvironment; - - public FlinkSqlExtension( - String catalogName, Map catalogProperties, String databaseName) { - this.catalogName = catalogName; - this.catalogProperties = Maps.newHashMap(catalogProperties); - this.databaseName = databaseName; - - // Add temporary dir as a warehouse location - try { - this.warehouse = Files.createTempDirectory("warehouse"); - } catch (IOException e) { - throw new RuntimeException(e); - } - this.catalogProperties.put( - CatalogProperties.WAREHOUSE_LOCATION, String.format("file://%s", warehouse)); - this.catalogLoader = - CatalogLoader.hadoop(catalogName, new Configuration(), this.catalogProperties); - } - - @Override - public void beforeEach(ExtensionContext context) { - // We need to recreate the tableEnvironment for every test as the minicluster is recreated - this.tableEnvironment = - TableEnvironment.create(EnvironmentSettings.newInstance().inBatchMode().build()); - exec("CREATE CATALOG %s WITH %s", catalogName, toWithClause(catalogProperties)); - exec("CREATE DATABASE %s.%s", catalogName, databaseName); - exec("USE CATALOG %s", catalogName); - exec("USE %s", databaseName); - } - - @Override - public void afterEach(ExtensionContext context) throws IOException { - List tables = exec("SHOW TABLES"); - tables.forEach(t -> exec("DROP TABLE IF EXISTS %s", t.getField(0))); - exec("USE CATALOG default_catalog"); - exec("DROP CATALOG IF EXISTS %s", catalogName); - try (Stream files = Files.walk(warehouse)) { - files.sorted(Comparator.reverseOrder()).map(Path::toFile).forEach(File::delete); - } - } - - /** - * Executes an SQL query with the given parameters. The parameter substitution is done by {@link - * String#format(String, Object...)}. - * - * @param query to run - * @param parameters to substitute to the query - * @return The {@link Row}s returned by the query - */ - public List exec(String query, Object... parameters) { - TableResult tableResult = tableEnvironment.executeSql(String.format(query, parameters)); - try (CloseableIterator iter = tableResult.collect()) { - return Lists.newArrayList(iter); - } catch (Exception e) { - throw new RuntimeException("Failed to collect table result", e); - } - } - - /** - * Returns the {@link TableLoader} which could be used to access the given table. - * - * @param tableName of the table - * @return the {@link TableLoader} for the table - */ - public TableLoader tableLoader(String tableName) { - TableLoader tableLoader = - TableLoader.fromCatalog(catalogLoader, TableIdentifier.of(databaseName, tableName)); - tableLoader.open(); - return tableLoader; - } - - public CatalogLoader catalogLoader() { - return catalogLoader; - } - - private static String toWithClause(Map props) { - return String.format( - "(%s)", - props.entrySet().stream() - .map(e -> String.format("'%s'='%s'", e.getKey(), e.getValue())) - .collect(Collectors.joining(","))); - } -} diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/OperatorTestBase.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/OperatorTestBase.java index 1cae679b2587..e251625dd2c1 100644 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/OperatorTestBase.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/OperatorTestBase.java @@ -23,6 +23,7 @@ import java.io.File; import java.io.IOException; +import java.nio.file.Path; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.MetricOptions; import org.apache.flink.core.execution.JobClient; @@ -33,20 +34,38 @@ import org.apache.flink.streaming.api.graph.StreamGraphGenerator; import org.apache.flink.streaming.api.transformations.SinkTransformation; import org.apache.flink.test.junit5.MiniClusterExtension; -import org.apache.iceberg.flink.FlinkCatalogFactory; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.data.GenericAppenderHelper; +import org.apache.iceberg.flink.HadoopCatalogExtension; +import org.apache.iceberg.flink.SimpleDataUtil; +import org.apache.iceberg.flink.TableLoader; +import org.apache.iceberg.flink.TestFixtures; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.types.Types; import org.awaitility.Awaitility; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.extension.RegisterExtension; +import org.junit.jupiter.api.io.TempDir; public class OperatorTestBase { private static final int NUMBER_TASK_MANAGERS = 1; private static final int SLOTS_PER_TASK_MANAGER = 8; + private static final Schema SCHEMA_WITH_PRIMARY_KEY = + new Schema( + Lists.newArrayList( + Types.NestedField.required(1, "id", Types.IntegerType.get()), + Types.NestedField.optional(2, "data", Types.StringType.get())), + ImmutableMap.of(), + ImmutableSet.of(SimpleDataUtil.SCHEMA.columns().get(0).fieldId())); protected static final String UID_SUFFIX = "UID-Dummy"; protected static final String SLOT_SHARING_GROUP = "SlotSharingGroup"; - protected static final String TABLE_NAME = "test_table"; protected static final TriggerLockFactory LOCK_FACTORY = new MemoryLockFactory(); public static final String IGNORED_OPERATOR_NAME = "Ignore"; @@ -64,15 +83,14 @@ public class OperatorTestBase { .setConfiguration(config()) .build()); + @TempDir private Path warehouseDir; + @RegisterExtension - public final FlinkSqlExtension sql = - new FlinkSqlExtension( - "catalog", - ImmutableMap.of("type", "iceberg", FlinkCatalogFactory.ICEBERG_CATALOG_TYPE, "hadoop"), - "db"); + private static final HadoopCatalogExtension CATALOG_EXTENSION = + new HadoopCatalogExtension(TestFixtures.DATABASE, TestFixtures.TABLE); @BeforeEach - void before() { + void before() throws IOException { LOCK_FACTORY.open(); MetricsReporterFactoryForTests.reset(); } @@ -82,6 +100,38 @@ void after() throws IOException { LOCK_FACTORY.close(); } + protected static Table createTable() { + return CATALOG_EXTENSION + .catalog() + .createTable( + TestFixtures.TABLE_IDENTIFIER, + SimpleDataUtil.SCHEMA, + PartitionSpec.unpartitioned(), + null, + ImmutableMap.of("flink.max-continuous-empty-commits", "100000")); + } + + protected static Table createTableWithDelete() { + return CATALOG_EXTENSION + .catalog() + .createTable( + TestFixtures.TABLE_IDENTIFIER, + SCHEMA_WITH_PRIMARY_KEY, + PartitionSpec.unpartitioned(), + null, + ImmutableMap.of("format-version", "2", "write.upsert.enabled", "true")); + } + + protected void insert(Table table, Integer id, String data) throws IOException { + new GenericAppenderHelper(table, FileFormat.PARQUET, warehouseDir) + .appendToTable(Lists.newArrayList(SimpleDataUtil.createRecord(id, data))); + table.refresh(); + } + + protected TableLoader tableLoader() { + return CATALOG_EXTENSION.tableLoader(); + } + /** * Close the {@link JobClient} and wait for the job closure. If the savepointDir is specified, it * stops the job with a savepoint. @@ -123,14 +173,14 @@ protected static void closeJobClient(JobClient jobClient) { closeJobClient(jobClient, null); } - protected static void checkUidsAreSet(StreamExecutionEnvironment env, String uidPrefix) { + protected static void checkUidsAreSet(StreamExecutionEnvironment env, String uidSuffix) { env.getTransformations().stream() .filter( t -> !(t instanceof SinkTransformation) && !(t.getName().equals(IGNORED_OPERATOR_NAME))) .forEach( transformation -> { assertThat(transformation.getUid()).isNotNull(); - if (uidPrefix != null) { + if (uidSuffix != null) { assertThat(transformation.getUid()).contains(UID_SUFFIX); } }); diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestAsyncDeleteFiles.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestAsyncDeleteFiles.java deleted file mode 100644 index 3e443fdbbc31..000000000000 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestAsyncDeleteFiles.java +++ /dev/null @@ -1,154 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.iceberg.flink.maintenance.operator; - -import static org.assertj.core.api.Assertions.assertThat; - -import java.io.Serializable; -import java.nio.charset.StandardCharsets; -import java.nio.file.FileSystems; -import java.nio.file.Files; -import java.nio.file.Path; -import java.util.Collection; -import java.util.List; -import java.util.function.Predicate; -import java.util.stream.Collectors; -import org.apache.flink.api.common.typeinfo.BasicTypeInfo; -import org.apache.flink.api.common.typeutils.base.StringSerializer; -import org.apache.flink.streaming.api.datastream.AsyncDataStream; -import org.apache.flink.streaming.api.operators.async.AsyncWaitOperatorFactory; -import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; -import org.apache.flink.streaming.runtime.tasks.OneInputStreamTask; -import org.apache.flink.streaming.runtime.tasks.StreamTaskMailboxTestHarness; -import org.apache.flink.streaming.runtime.tasks.StreamTaskMailboxTestHarnessBuilder; -import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness; -import org.apache.flink.streaming.util.retryable.AsyncRetryStrategies; -import org.apache.iceberg.Table; -import org.apache.iceberg.flink.TableLoader; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; -import org.awaitility.Awaitility; -import org.junit.jupiter.api.Test; - -class TestAsyncDeleteFiles extends OperatorTestBase { - private static final String DUMMY_FILE_NAME = "dummy"; - - @Test - void testDelete() throws Exception { - sql.exec("CREATE TABLE %s (id int, data varchar)", TABLE_NAME); - - TableLoader tableLoader = sql.tableLoader(TABLE_NAME); - Table table = tableLoader.loadTable(); - - // Write an extra files - Path dummyFile = - FileSystems.getDefault().getPath(table.location().substring(5), DUMMY_FILE_NAME); - Files.write(dummyFile, "DUMMY".getBytes(StandardCharsets.UTF_8)); - - List actual = deleteDummyFileAndWait(tableLoader); - - assertThat(actual).isEqualTo(ImmutableList.of(Boolean.TRUE)); - assertThat(Files.exists(dummyFile)).isFalse(); - } - - @Test - void testDeleteMissingFile() throws Exception { - sql.exec("CREATE TABLE %s (id int, data varchar)", TABLE_NAME); - - List actual = deleteDummyFileAndWait(sql.tableLoader(TABLE_NAME)); - - assertThat(actual).isEqualTo(ImmutableList.of(Boolean.TRUE)); - } - - @Test - void testWrongFile() throws Exception { - sql.exec("CREATE TABLE %s (id int, data varchar)", TABLE_NAME); - - StreamTaskMailboxTestHarnessBuilder builder = - new StreamTaskMailboxTestHarnessBuilder<>( - OneInputStreamTask::new, BasicTypeInfo.BOOLEAN_TYPE_INFO) - .addInput(BasicTypeInfo.STRING_TYPE_INFO); - CountingPredicateChecker predicate = new CountingPredicateChecker(); - - try (StreamTaskMailboxTestHarness testHarness = - builder - .setupOutputForSingletonOperatorChain( - asyncWaitOperatorFactory(sql.tableLoader(TABLE_NAME), predicate)) - .build()) { - testHarness.processElement(new StreamRecord<>("wrong://", System.currentTimeMillis())); - - while (testHarness.getOutput().isEmpty()) { - Awaitility.await().until(() -> testHarness.getOutput().isEmpty()); - testHarness.processAll(); - } - - // Make sure that we do not complete immediately - assertThat(CountingPredicateChecker.calls).isEqualTo(3); - - // The result still should be fail - assertThat( - testHarness.getOutput().stream() - .map(r -> ((StreamRecord) r).getValue()) - .collect(Collectors.toList())) - .isEqualTo(ImmutableList.of(Boolean.FALSE)); - } - } - - private List deleteDummyFileAndWait(TableLoader tableLoader) throws Exception { - Table table = tableLoader.loadTable(); - - try (OneInputStreamOperatorTestHarness testHarness = - new OneInputStreamOperatorTestHarness<>( - asyncWaitOperatorFactory(tableLoader, new CountingPredicateChecker()), - StringSerializer.INSTANCE)) { - testHarness.open(); - testHarness.processElement( - table.location() + "/" + DUMMY_FILE_NAME, System.currentTimeMillis()); - - // wait until all async collectors in the buffer have been emitted out. - testHarness.endInput(); - testHarness.close(); - - return testHarness.extractOutputValues(); - } - } - - private AsyncWaitOperatorFactory asyncWaitOperatorFactory( - TableLoader tableLoader, Predicate> predicate) { - return new AsyncWaitOperatorFactory<>( - new AsyncDeleteFiles(DUMMY_NAME, tableLoader, 10), - 1000000, - 10, - AsyncDataStream.OutputMode.ORDERED, - new AsyncRetryStrategies.ExponentialBackoffDelayRetryStrategyBuilder( - 2, 10, 1000, 1.5) - .ifResult(predicate) - .build()); - } - - private static class CountingPredicateChecker - implements Predicate>, Serializable { - private static int calls = 0; - - @Override - public boolean test(Collection param) { - ++calls; - return AsyncDeleteFiles.FAILED_PREDICATE.test(param); - } - } -} diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestDeleteFilesProcessor.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestDeleteFilesProcessor.java new file mode 100644 index 000000000000..9d6ea438ad49 --- /dev/null +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestDeleteFilesProcessor.java @@ -0,0 +1,116 @@ +/* + * 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.operator; + +import static org.assertj.core.api.Assertions.assertThat; + +import com.google.common.collect.ImmutableSet; +import java.io.IOException; +import java.nio.charset.StandardCharsets; +import java.nio.file.FileSystems; +import java.nio.file.Files; +import java.nio.file.Path; +import java.util.Set; +import java.util.stream.Collectors; +import org.apache.flink.api.common.typeutils.base.StringSerializer; +import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness; +import org.apache.iceberg.Table; +import org.apache.iceberg.flink.TableLoader; +import org.apache.iceberg.flink.TestFixtures; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +class TestDeleteFilesProcessor extends OperatorTestBase { + private static final String DUMMY_FILE_NAME = "dummy"; + private static final Set TABLE_FILES = + ImmutableSet.of( + "metadata/v1.metadata.json", + "metadata/version-hint.text", + "metadata/.version-hint.text.crc", + "metadata/.v1.metadata.json.crc"); + + private Table table; + private TableLoader tableLoader; + + @BeforeEach + void before() throws IOException { + this.table = createTable(); + this.tableLoader = tableLoader(); + } + + @Test + void testDelete() throws Exception { + // Write an extra file + Path dummyFile = Path.of(tablePath(table).toString(), DUMMY_FILE_NAME); + Files.write(dummyFile, "DUMMY".getBytes(StandardCharsets.UTF_8)); + + Set files = listFiles(table); + assertThat(files) + .containsAll(TABLE_FILES) + .contains(DUMMY_FILE_NAME) + .hasSize(TABLE_FILES.size() + 1); + + deleteFile(tableLoader, dummyFile.toString()); + + assertThat(listFiles(table)).isEqualTo(TABLE_FILES); + } + + @Test + void testDeleteMissingFile() throws Exception { + Path dummyFile = + FileSystems.getDefault().getPath(table.location().substring(5), DUMMY_FILE_NAME); + + deleteFile(tableLoader, dummyFile.toString()); + + assertThat(listFiles(table)).isEqualTo(TABLE_FILES); + } + + @Test + void testInvalidURIScheme() throws Exception { + deleteFile(tableLoader, "wrong://"); + + assertThat(listFiles(table)).isEqualTo(TABLE_FILES); + } + + private void deleteFile(TableLoader tableLoader, String fileName) throws Exception { + try (OneInputStreamOperatorTestHarness testHarness = + new OneInputStreamOperatorTestHarness<>( + new DeleteFilesProcessor(DUMMY_NAME, tableLoader, 10), StringSerializer.INSTANCE)) { + testHarness.open(); + testHarness.processElement(fileName, System.currentTimeMillis()); + testHarness.processWatermark(EVENT_TIME); + testHarness.endInput(); + } + } + + private static Path tablePath(Table table) { + return FileSystems.getDefault().getPath(table.location().substring(5)); + } + + private static Set listFiles(Table table) throws IOException { + String tableRootPath = TestFixtures.TABLE_IDENTIFIER.toString().replace(".", "/"); + return Files.find( + tablePath(table), Integer.MAX_VALUE, (filePath, fileAttr) -> fileAttr.isRegularFile()) + .map( + p -> + p.toString() + .substring(p.toString().indexOf(tableRootPath) + tableRootPath.length() + 1)) + .collect(Collectors.toSet()); + } +} diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestExpireSnapshotsProcessor.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestExpireSnapshotsProcessor.java index 79af082a0d44..f357924f5aa2 100644 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestExpireSnapshotsProcessor.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestExpireSnapshotsProcessor.java @@ -29,7 +29,7 @@ import org.apache.iceberg.SerializableTable; import org.apache.iceberg.Snapshot; import org.apache.iceberg.Table; -import org.apache.iceberg.flink.TableLoader; +import org.apache.iceberg.flink.TestFixtures; import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.ValueSource; @@ -38,24 +38,21 @@ class TestExpireSnapshotsProcessor extends OperatorTestBase { @ParameterizedTest @ValueSource(booleans = {true, false}) void testExpire(boolean success) throws Exception { - sql.exec("CREATE TABLE %s (id int, data varchar, spec varchar)", TABLE_NAME); - sql.exec("INSERT INTO %s VALUES (1, 'a', 'p1')", TABLE_NAME); - sql.exec("INSERT INTO %s VALUES (2, 'b', 'p2')", TABLE_NAME); - - TableLoader tableLoader = sql.tableLoader(TABLE_NAME); - Table table = tableLoader.loadTable(); + Table table = createTable(); + insert(table, 1, "a"); + insert(table, 2, "b"); SerializableTable serializableTable = (SerializableTable) SerializableTable.copyOf(table); List actual; Queue> deletes; try (OneInputStreamOperatorTestHarness testHarness = ProcessFunctionTestHarnesses.forProcessFunction( - new ExpireSnapshotsProcessor(tableLoader, 0L, 1, 10))) { + new ExpireSnapshotsProcessor(tableLoader(), 0L, 1, 10))) { testHarness.open(); if (!success) { // Cause an exception - sql.exec("DROP TABLE IF EXISTS %s", TABLE_NAME); + CATALOG_EXTENSION.catalogLoader().loadCatalog().dropTable(TestFixtures.TABLE_IDENTIFIER); } testHarness.processElement( @@ -79,7 +76,6 @@ void testExpire(boolean success) throws Exception { assertThat(deletes).hasSize(1); } else { assertThat(result.exceptions()).isNotNull().hasSize(1); - assertThat(deletes).isNull(); } } diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestMonitorSource.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestMonitorSource.java index 3aee05322561..c561c7054eae 100644 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestMonitorSource.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestMonitorSource.java @@ -22,6 +22,7 @@ import static org.assertj.core.api.Assertions.assertThatThrownBy; import java.io.File; +import java.io.IOException; import java.time.Duration; import java.util.List; import java.util.concurrent.atomic.AtomicReference; @@ -60,40 +61,27 @@ class TestMonitorSource extends OperatorTestBase { @ParameterizedTest @ValueSource(booleans = {true, false}) - void testChangeReaderIterator(boolean withDelete) { - if (withDelete) { - sql.exec( - "CREATE TABLE %s (id int, data varchar, PRIMARY KEY(`id`) NOT ENFORCED) WITH ('format-version'='2', 'write.upsert.enabled'='true')", - TABLE_NAME); - } else { - sql.exec("CREATE TABLE %s (id int, data varchar)", TABLE_NAME); - } - - TableLoader tableLoader = sql.tableLoader(TABLE_NAME); - tableLoader.open(); - Table table = tableLoader.loadTable(); + void testChangeReaderIterator(boolean withDelete) throws IOException { + Table table = withDelete ? createTableWithDelete() : createTable(); MonitorSource.TableChangeIterator iterator = - new MonitorSource.TableChangeIterator(tableLoader, null, Long.MAX_VALUE); + new MonitorSource.TableChangeIterator(tableLoader(), null, Long.MAX_VALUE); // For an empty table we get an empty result assertThat(iterator.next()).isEqualTo(EMPTY_EVENT); // Add a single commit and get back the commit data in the event - sql.exec("INSERT INTO %s VALUES (1, 'a')", TABLE_NAME); - table.refresh(); + insert(table, 1, "a"); TableChange expected = tableChangeWithLastSnapshot(table, TableChange.empty()); assertThat(iterator.next()).isEqualTo(expected); // Make sure that consecutive calls do not return the data again assertThat(iterator.next()).isEqualTo(EMPTY_EVENT); // Add two more commits, but fetch the data in one loop - sql.exec("INSERT INTO %s VALUES (2, 'b')", TABLE_NAME); - table.refresh(); + insert(table, 2, "b"); expected = tableChangeWithLastSnapshot(table, TableChange.empty()); - sql.exec("INSERT INTO %s VALUES (3, 'c')", TABLE_NAME); - table.refresh(); + insert(table, 3, "c"); expected = tableChangeWithLastSnapshot(table, expected); assertThat(iterator.next()).isEqualTo(expected); @@ -106,17 +94,11 @@ void testChangeReaderIterator(boolean withDelete) { */ @Test void testSource() throws Exception { - sql.exec( - "CREATE TABLE %s (id int, data varchar) " - + "WITH ('flink.max-continuous-empty-commits'='100000')", - TABLE_NAME); StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); - TableLoader tableLoader = sql.tableLoader(TABLE_NAME); - tableLoader.open(); - Table table = tableLoader.loadTable(); + Table table = createTable(); DataStream events = env.fromSource( - new MonitorSource(tableLoader, HIGH_RATE, Long.MAX_VALUE), + new MonitorSource(tableLoader(), HIGH_RATE, Long.MAX_VALUE), WatermarkStrategy.noWatermarks(), "TableChangeSource") .forceNonParallel(); @@ -176,8 +158,9 @@ void testSource() throws Exception { /** Check that the {@link MonitorSource} operator state is restored correctly. */ @Test void testStateRestore(@TempDir File savepointDir) throws Exception { - sql.exec("CREATE TABLE %s (id int, data varchar)", TABLE_NAME); - sql.exec("INSERT INTO %s VALUES (1, 'a')", TABLE_NAME); + Table table = createTable(); + insert(table, 1, "a"); + TableLoader tableLoader = tableLoader(); Configuration config = new Configuration(); config.set(CheckpointingOptions.CHECKPOINT_STORAGE, "filesystem"); @@ -185,8 +168,6 @@ void testStateRestore(@TempDir File savepointDir) throws Exception { StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(config); env.enableCheckpointing(1000); - TableLoader tableLoader = sql.tableLoader(TABLE_NAME); - tableLoader.open(); DataStream events = env.fromSource( new MonitorSource(tableLoader, HIGH_RATE, Long.MAX_VALUE), @@ -268,14 +249,12 @@ void testStateRestore(@TempDir File savepointDir) throws Exception { @Test void testNotOneParallelismThrows() { - sql.exec("CREATE TABLE %s (id int, data varchar)", TABLE_NAME); + createTable(); StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); - TableLoader tableLoader = sql.tableLoader(TABLE_NAME); - tableLoader.open(); env.fromSource( - new MonitorSource(tableLoader, HIGH_RATE, Long.MAX_VALUE), + new MonitorSource(tableLoader(), HIGH_RATE, Long.MAX_VALUE), WatermarkStrategy.noWatermarks(), "TableChangeSource") .setParallelism(2) @@ -289,14 +268,13 @@ void testNotOneParallelismThrows() { } @Test - void testMaxReadBack() { - sql.exec("CREATE TABLE %s (id int, data varchar)", TABLE_NAME); - sql.exec("INSERT INTO %s VALUES (1, 'a')", TABLE_NAME); - sql.exec("INSERT INTO %s VALUES (2, 'b')", TABLE_NAME); - sql.exec("INSERT INTO %s VALUES (3, 'c')", TABLE_NAME); + void testMaxReadBack() throws IOException { + Table table = createTable(); + insert(table, 1, "a"); + insert(table, 2, "b"); + insert(table, 3, "c"); - TableLoader tableLoader = sql.tableLoader(TABLE_NAME); - tableLoader.open(); + TableLoader tableLoader = tableLoader(); MonitorSource.TableChangeIterator iterator = new MonitorSource.TableChangeIterator(tableLoader, null, 1); @@ -316,12 +294,11 @@ void testMaxReadBack() { } @Test - void testSkipReplace() { - sql.exec("CREATE TABLE %s (id int, data varchar)", TABLE_NAME); - sql.exec("INSERT INTO %s VALUES (1, 'a')", TABLE_NAME); + void testSkipReplace() throws IOException { + Table table = createTable(); + insert(table, 1, "a"); - TableLoader tableLoader = sql.tableLoader(TABLE_NAME); - tableLoader.open(); + TableLoader tableLoader = tableLoader(); MonitorSource.TableChangeIterator iterator = new MonitorSource.TableChangeIterator(tableLoader, null, Long.MAX_VALUE); @@ -330,7 +307,6 @@ void testSkipReplace() { assertThat(iterator.next().commitCount()).isEqualTo(1); // Create a DataOperations.REPLACE snapshot - Table table = tableLoader.loadTable(); DataFile dataFile = table.snapshots().iterator().next().addedDataFiles(table.io()).iterator().next(); RewriteFiles rewrite = tableLoader.loadTable().newRewrite(); diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestTriggerManager.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestTriggerManager.java index c761c2904186..40f229ca244b 100644 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestTriggerManager.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestTriggerManager.java @@ -58,7 +58,7 @@ class TestTriggerManager extends OperatorTestBase { @BeforeEach void before() { - sql.exec("CREATE TABLE %s (id int, data varchar)", TABLE_NAME); + createTable(); this.lock = LOCK_FACTORY.createLock(); this.recoveringLock = LOCK_FACTORY.createRecoveryLock(); } @@ -66,7 +66,7 @@ void before() { @Test void testCommitCount() throws Exception { TriggerManager manager = - manager(sql.tableLoader(TABLE_NAME), new TriggerEvaluator.Builder().commitCount(3).build()); + manager(tableLoader(), new TriggerEvaluator.Builder().commitCount(3).build()); try (KeyedOneInputStreamOperatorTestHarness testHarness = harness(manager)) { testHarness.open(); @@ -87,8 +87,7 @@ void testCommitCount() throws Exception { @Test void testDataFileCount() throws Exception { TriggerManager manager = - manager( - sql.tableLoader(TABLE_NAME), new TriggerEvaluator.Builder().dataFileCount(3).build()); + manager(tableLoader(), new TriggerEvaluator.Builder().dataFileCount(3).build()); try (KeyedOneInputStreamOperatorTestHarness testHarness = harness(manager)) { testHarness.open(); @@ -109,9 +108,7 @@ void testDataFileCount() throws Exception { @Test void testDataFileSizeInBytes() throws Exception { TriggerManager manager = - manager( - sql.tableLoader(TABLE_NAME), - new TriggerEvaluator.Builder().dataFileSizeInBytes(3).build()); + manager(tableLoader(), new TriggerEvaluator.Builder().dataFileSizeInBytes(3).build()); try (KeyedOneInputStreamOperatorTestHarness testHarness = harness(manager)) { testHarness.open(); @@ -130,9 +127,7 @@ void testDataFileSizeInBytes() throws Exception { @Test void testPosDeleteFileCount() throws Exception { TriggerManager manager = - manager( - sql.tableLoader(TABLE_NAME), - new TriggerEvaluator.Builder().posDeleteFileCount(3).build()); + manager(tableLoader(), new TriggerEvaluator.Builder().posDeleteFileCount(3).build()); try (KeyedOneInputStreamOperatorTestHarness testHarness = harness(manager)) { testHarness.open(); @@ -153,9 +148,7 @@ void testPosDeleteFileCount() throws Exception { @Test void testPosDeleteRecordCount() throws Exception { TriggerManager manager = - manager( - sql.tableLoader(TABLE_NAME), - new TriggerEvaluator.Builder().posDeleteRecordCount(3).build()); + manager(tableLoader(), new TriggerEvaluator.Builder().posDeleteRecordCount(3).build()); try (KeyedOneInputStreamOperatorTestHarness testHarness = harness(manager)) { testHarness.open(); @@ -179,9 +172,7 @@ void testPosDeleteRecordCount() throws Exception { @Test void testEqDeleteFileCount() throws Exception { TriggerManager manager = - manager( - sql.tableLoader(TABLE_NAME), - new TriggerEvaluator.Builder().eqDeleteFileCount(3).build()); + manager(tableLoader(), new TriggerEvaluator.Builder().eqDeleteFileCount(3).build()); try (KeyedOneInputStreamOperatorTestHarness testHarness = harness(manager)) { testHarness.open(); @@ -202,9 +193,7 @@ void testEqDeleteFileCount() throws Exception { @Test void testEqDeleteRecordCount() throws Exception { TriggerManager manager = - manager( - sql.tableLoader(TABLE_NAME), - new TriggerEvaluator.Builder().eqDeleteRecordCount(3).build()); + manager(tableLoader(), new TriggerEvaluator.Builder().eqDeleteRecordCount(3).build()); try (KeyedOneInputStreamOperatorTestHarness testHarness = harness(manager)) { testHarness.open(); @@ -224,8 +213,7 @@ void testEqDeleteRecordCount() throws Exception { void testTimeout() throws Exception { TriggerManager manager = manager( - sql.tableLoader(TABLE_NAME), - new TriggerEvaluator.Builder().timeout(Duration.ofSeconds(1)).build()); + tableLoader(), new TriggerEvaluator.Builder().timeout(Duration.ofSeconds(1)).build()); try (KeyedOneInputStreamOperatorTestHarness testHarness = harness(manager)) { testHarness.open(); @@ -264,7 +252,7 @@ void testTimeout() throws Exception { @Test void testStateRestore() throws Exception { - TableLoader tableLoader = sql.tableLoader(TABLE_NAME); + TableLoader tableLoader = tableLoader(); TriggerManager manager = manager(tableLoader); OperatorSubtaskState state; try (KeyedOneInputStreamOperatorTestHarness testHarness = @@ -302,7 +290,7 @@ void testStateRestore() throws Exception { @Test void testMinFireDelay() throws Exception { - TableLoader tableLoader = sql.tableLoader(TABLE_NAME); + TableLoader tableLoader = tableLoader(); TriggerManager manager = manager(tableLoader, DELAY, 1); try (KeyedOneInputStreamOperatorTestHarness testHarness = harness(manager)) { @@ -322,7 +310,7 @@ void testMinFireDelay() throws Exception { @Test void testLockCheckDelay() throws Exception { - TableLoader tableLoader = sql.tableLoader(TABLE_NAME); + TableLoader tableLoader = tableLoader(); TriggerManager manager = manager(tableLoader, 1, DELAY); try (KeyedOneInputStreamOperatorTestHarness testHarness = harness(manager)) { @@ -355,7 +343,7 @@ void testLockCheckDelay() throws Exception { @ParameterizedTest @MethodSource("parametersForTestRecovery") void testRecovery(boolean locked, boolean runningTask) throws Exception { - TableLoader tableLoader = sql.tableLoader(TABLE_NAME); + TableLoader tableLoader = tableLoader(); TriggerManager manager = manager(tableLoader); OperatorSubtaskState state; try (KeyedOneInputStreamOperatorTestHarness testHarness = @@ -417,7 +405,7 @@ void testRecovery(boolean locked, boolean runningTask) throws Exception { @Test void testTriggerMetrics() throws Exception { - TableLoader tableLoader = sql.tableLoader(TABLE_NAME); + TableLoader tableLoader = tableLoader(); StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); ManualSource source = @@ -497,7 +485,7 @@ void testTriggerMetrics() throws Exception { @Test void testRateLimiterMetrics() throws Exception { - TableLoader tableLoader = sql.tableLoader(TABLE_NAME); + TableLoader tableLoader = tableLoader(); StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); ManualSource source = @@ -543,7 +531,7 @@ void testRateLimiterMetrics() throws Exception { @Test void testConcurrentRunMetrics() throws Exception { - TableLoader tableLoader = sql.tableLoader(TABLE_NAME); + TableLoader tableLoader = tableLoader(); StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); ManualSource source = From 3952792feff2d8e7e0f25e30211facd14a2a4be4 Mon Sep 17 00:00:00 2001 From: Peter Vary Date: Wed, 18 Sep 2024 20:47:24 +0200 Subject: [PATCH 04/12] Small fixes --- .../flink/maintenance/api/ExpireSnapshots.java | 2 +- .../maintenance/api/MaintenanceTaskBuilder.java | 3 ++- .../flink/maintenance/api/TableMaintenance.java | 8 ++++---- .../maintenance/operator/DeleteFilesProcessor.java | 7 ++++--- .../flink/maintenance/operator/TaskResult.java | 3 ++- .../flink/maintenance/api/TestExpireSnapshots.java | 3 +-- .../flink/maintenance/operator/OperatorTestBase.java | 4 ++++ .../operator/TestDeleteFilesProcessor.java | 12 +++++------- .../operator/TestExpireSnapshotsProcessor.java | 3 +-- 9 files changed, 24 insertions(+), 21 deletions(-) diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/ExpireSnapshots.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/ExpireSnapshots.java index 9109a9f1a97f..a640e6f2188d 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/ExpireSnapshots.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/ExpireSnapshots.java @@ -33,7 +33,7 @@ /** Deletes expired snapshots and the corresponding files. */ public class ExpireSnapshots { - private static final int DELETE_BATCH_SIZE_DEFAULT = 10; + private static final int DELETE_BATCH_SIZE_DEFAULT = 1000; private static final String EXECUTOR_OPERATOR_NAME = "Expire Snapshot"; @VisibleForTesting static final String DELETE_FILES_OPERATOR_NAME = "Delete file"; diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/MaintenanceTaskBuilder.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/MaintenanceTaskBuilder.java index f286ef4bd898..96d5c97625f6 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/MaintenanceTaskBuilder.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/MaintenanceTaskBuilder.java @@ -21,6 +21,7 @@ import java.time.Duration; import org.apache.flink.annotation.Internal; import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.api.common.operators.util.OperatorValidationUtils; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator; import org.apache.iceberg.flink.TableLoader; @@ -152,7 +153,7 @@ public T slotSharingGroup(String newSlotSharingGroup) { * @param newParallelism the required parallelism */ public T parallelism(int newParallelism) { - Preconditions.checkArgument(newParallelism > 0, "Parallelism should be greater than 0"); + OperatorValidationUtils.validateParallelism(newParallelism); this.parallelism = newParallelism; return (T) this; } diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/TableMaintenance.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/TableMaintenance.java index 6a0909c85040..e8b698d0cbba 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/TableMaintenance.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/TableMaintenance.java @@ -30,6 +30,7 @@ import org.apache.flink.api.common.eventtime.WatermarkGeneratorSupplier; import org.apache.flink.api.common.eventtime.WatermarkOutput; import org.apache.flink.api.common.eventtime.WatermarkStrategy; +import org.apache.flink.api.common.operators.util.OperatorValidationUtils; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.connector.source.util.ratelimit.RateLimiterStrategy; import org.apache.flink.api.dag.Transformation; @@ -53,7 +54,6 @@ /** Creates the table maintenance graph. */ public class TableMaintenance { - private static final String TASK_NAME_FORMAT = "%s [%d]"; static final String SOURCE_OPERATOR_NAME = "Monitor source"; static final String TRIGGER_MANAGER_OPERATOR_NAME = "Trigger manager"; static final String WATERMARK_ASSIGNER_OPERATOR_NAME = "Watermark Assigner"; @@ -113,7 +113,7 @@ public static class Builder { private String slotSharingGroup = StreamGraphGenerator.DEFAULT_SLOT_SHARING_GROUP; private Duration rateLimit = Duration.ofMinutes(1); private Duration lockCheckDelay = Duration.ofSeconds(30); - private Integer parallelism = ExecutionConfig.PARALLELISM_DEFAULT; + private int parallelism = ExecutionConfig.PARALLELISM_DEFAULT; private int maxReadBack = 100; private Builder( @@ -185,7 +185,7 @@ public Builder lockCheckDelay(Duration newLockCheckDelay) { * @param newParallelism task parallelism */ public Builder parallelism(int newParallelism) { - Preconditions.checkArgument(newParallelism > 0, "Parallelism should be greater than 0"); + OperatorValidationUtils.validateParallelism(newParallelism); this.parallelism = newParallelism; return this; } @@ -305,7 +305,7 @@ private DataStream changeStream() { } private static String nameFor(MaintenanceTaskBuilder streamBuilder, int taskId) { - return String.format(TASK_NAME_FORMAT, streamBuilder.getClass().getSimpleName(), taskId); + return String.format("%s [%d]", streamBuilder.getClass().getSimpleName(), taskId); } @Internal diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/DeleteFilesProcessor.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/DeleteFilesProcessor.java index 6f01f8ac46be..f2705c3727c7 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/DeleteFilesProcessor.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/DeleteFilesProcessor.java @@ -21,7 +21,6 @@ import java.util.Set; import org.apache.flink.annotation.Internal; import org.apache.flink.metrics.Counter; -import org.apache.flink.shaded.guava31.com.google.common.collect.Sets; import org.apache.flink.streaming.api.operators.AbstractStreamOperator; import org.apache.flink.streaming.api.operators.OneInputStreamOperator; import org.apache.flink.streaming.api.watermark.Watermark; @@ -32,10 +31,11 @@ import org.apache.iceberg.io.FileIO; import org.apache.iceberg.io.SupportsBulkOperations; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -/** Delete the files using the {@link FileIO}. */ +/** Delete the files using the {@link FileIO} which implements {@link SupportsBulkOperations}. */ @Internal public class DeleteFilesProcessor extends AbstractStreamOperator implements OneInputStreamOperator { @@ -116,7 +116,8 @@ private void deleteFiles() { "Deleted only {} of {} files from table {} using bulk deletes", deletedFilesCount, filesToDelete.size(), - tableName); + tableName, + e); succeededCounter.inc(deletedFilesCount); failedCounter.inc(e.numberFailedObjects()); } diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TaskResult.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TaskResult.java index 06f10f1c1d68..1568b6965e1c 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TaskResult.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TaskResult.java @@ -18,13 +18,14 @@ */ package org.apache.iceberg.flink.maintenance.operator; +import java.io.Serializable; import java.util.List; import org.apache.flink.annotation.Internal; import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; /** The result of a single Maintenance Task. */ @Internal -public class TaskResult { +public class TaskResult implements Serializable { private final int taskIndex; private final long startEpoch; private final boolean success; diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/api/TestExpireSnapshots.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/api/TestExpireSnapshots.java index 67970862a416..5fe027a3e73d 100644 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/api/TestExpireSnapshots.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/api/TestExpireSnapshots.java @@ -32,7 +32,6 @@ import org.apache.iceberg.Snapshot; import org.apache.iceberg.Table; import org.apache.iceberg.flink.SimpleDataUtil; -import org.apache.iceberg.flink.TestFixtures; import org.apache.iceberg.flink.maintenance.operator.MetricsReporterFactoryForTests; import org.apache.iceberg.flink.maintenance.operator.Trigger; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; @@ -126,7 +125,7 @@ void testFailure() throws Exception { assertThat(infra.sink().poll(Duration.ofSeconds(5)).success()).isTrue(); // Drop the table, so it will cause an exception - CATALOG_EXTENSION.catalogLoader().loadCatalog().dropTable(TestFixtures.TABLE_IDENTIFIER); + dropTable(); // Failed run infra.source().sendRecord(Trigger.create(time + 1, serializableTable, 1), time + 1); diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/OperatorTestBase.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/OperatorTestBase.java index e251625dd2c1..067bd3962fe2 100644 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/OperatorTestBase.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/OperatorTestBase.java @@ -128,6 +128,10 @@ protected void insert(Table table, Integer id, String data) throws IOException { table.refresh(); } + protected void dropTable() { + CATALOG_EXTENSION.catalogLoader().loadCatalog().dropTable(TestFixtures.TABLE_IDENTIFIER); + } + protected TableLoader tableLoader() { return CATALOG_EXTENSION.tableLoader(); } diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestDeleteFilesProcessor.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestDeleteFilesProcessor.java index 9d6ea438ad49..3f0cccf08718 100644 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestDeleteFilesProcessor.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestDeleteFilesProcessor.java @@ -20,7 +20,6 @@ import static org.assertj.core.api.Assertions.assertThat; -import com.google.common.collect.ImmutableSet; import java.io.IOException; import java.nio.charset.StandardCharsets; import java.nio.file.FileSystems; @@ -33,6 +32,7 @@ import org.apache.iceberg.Table; import org.apache.iceberg.flink.TableLoader; import org.apache.iceberg.flink.TestFixtures; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; @@ -46,12 +46,10 @@ class TestDeleteFilesProcessor extends OperatorTestBase { "metadata/.v1.metadata.json.crc"); private Table table; - private TableLoader tableLoader; @BeforeEach - void before() throws IOException { + void before() { this.table = createTable(); - this.tableLoader = tableLoader(); } @Test @@ -66,7 +64,7 @@ void testDelete() throws Exception { .contains(DUMMY_FILE_NAME) .hasSize(TABLE_FILES.size() + 1); - deleteFile(tableLoader, dummyFile.toString()); + deleteFile(tableLoader(), dummyFile.toString()); assertThat(listFiles(table)).isEqualTo(TABLE_FILES); } @@ -76,14 +74,14 @@ void testDeleteMissingFile() throws Exception { Path dummyFile = FileSystems.getDefault().getPath(table.location().substring(5), DUMMY_FILE_NAME); - deleteFile(tableLoader, dummyFile.toString()); + deleteFile(tableLoader(), dummyFile.toString()); assertThat(listFiles(table)).isEqualTo(TABLE_FILES); } @Test void testInvalidURIScheme() throws Exception { - deleteFile(tableLoader, "wrong://"); + deleteFile(tableLoader(), "wrong://"); assertThat(listFiles(table)).isEqualTo(TABLE_FILES); } diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestExpireSnapshotsProcessor.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestExpireSnapshotsProcessor.java index f357924f5aa2..4167a4010de9 100644 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestExpireSnapshotsProcessor.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestExpireSnapshotsProcessor.java @@ -29,7 +29,6 @@ import org.apache.iceberg.SerializableTable; import org.apache.iceberg.Snapshot; import org.apache.iceberg.Table; -import org.apache.iceberg.flink.TestFixtures; import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.ValueSource; @@ -52,7 +51,7 @@ void testExpire(boolean success) throws Exception { if (!success) { // Cause an exception - CATALOG_EXTENSION.catalogLoader().loadCatalog().dropTable(TestFixtures.TABLE_IDENTIFIER); + dropTable(); } testHarness.processElement( From fa03a71f22fd9d39b7bbec8f4f45b37d2ce09ad3 Mon Sep 17 00:00:00 2001 From: Peter Vary Date: Wed, 25 Sep 2024 14:41:07 +0200 Subject: [PATCH 05/12] move every publicly accessible class to the api package --- .../flink/maintenance/api/ExpireSnapshots.java | 2 -- .../{operator => api}/JdbcLockFactory.java | 7 ++----- .../flink/maintenance/api/MaintenanceTaskBuilder.java | 2 -- .../flink/maintenance/api/TableMaintenance.java | 4 +--- .../maintenance/{operator => api}/TaskResult.java | 4 +--- .../flink/maintenance/{operator => api}/Trigger.java | 11 ++++++----- .../{operator => api}/TriggerLockFactory.java | 4 +++- .../operator/ExpireSnapshotsProcessor.java | 2 ++ .../flink/maintenance/operator/LockRemover.java | 3 +++ .../flink/maintenance/operator/TableChange.java | 2 +- .../flink/maintenance/operator/TriggerManager.java | 2 ++ .../api/MaintenanceTaskInfraExtension.java | 2 -- .../maintenance/api/MaintenanceTaskTestBase.java | 2 -- .../flink/maintenance/api/TestExpireSnapshots.java | 1 - .../{operator => api}/TestJdbcLockFactory.java | 4 ++-- .../{operator => api}/TestLockFactoryBase.java | 2 +- .../flink/maintenance/api/TestTableMaintenance.java | 3 --- .../flink/maintenance/operator/OperatorTestBase.java | 1 + .../operator/TestExpireSnapshotsProcessor.java | 2 ++ .../flink/maintenance/operator/TestLockRemover.java | 2 ++ .../maintenance/operator/TestTriggerManager.java | 2 ++ 21 files changed, 31 insertions(+), 33 deletions(-) rename flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/{operator => api}/JdbcLockFactory.java (98%) rename flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/{operator => api}/TaskResult.java (94%) rename flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/{operator => api}/Trigger.java (91%) rename flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/{operator => api}/TriggerLockFactory.java (91%) rename flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/{operator => api}/TestJdbcLockFactory.java (92%) rename flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/{operator => api}/TestLockFactoryBase.java (97%) diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/ExpireSnapshots.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/ExpireSnapshots.java index a640e6f2188d..e69fa00616a5 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/ExpireSnapshots.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/ExpireSnapshots.java @@ -26,8 +26,6 @@ import org.apache.iceberg.SystemConfigs; import org.apache.iceberg.flink.maintenance.operator.DeleteFilesProcessor; import org.apache.iceberg.flink.maintenance.operator.ExpireSnapshotsProcessor; -import org.apache.iceberg.flink.maintenance.operator.TaskResult; -import org.apache.iceberg.flink.maintenance.operator.Trigger; import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/JdbcLockFactory.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/JdbcLockFactory.java similarity index 98% rename from flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/JdbcLockFactory.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/JdbcLockFactory.java index 085fbfecd270..9f4fb069aae8 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/JdbcLockFactory.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/JdbcLockFactory.java @@ -16,7 +16,7 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.iceberg.flink.maintenance.operator; +package org.apache.iceberg.flink.maintenance.api; import java.io.IOException; import java.sql.DatabaseMetaData; @@ -38,10 +38,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -/** - * JDBC table backed implementation of the {@link - * org.apache.iceberg.flink.maintenance.operator.TriggerLockFactory}. - */ +/** JDBC table backed implementation of the {@link TriggerLockFactory}. */ public class JdbcLockFactory implements TriggerLockFactory { private static final Logger LOG = LoggerFactory.getLogger(JdbcLockFactory.class); diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/MaintenanceTaskBuilder.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/MaintenanceTaskBuilder.java index 96d5c97625f6..b2383fc93c74 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/MaintenanceTaskBuilder.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/MaintenanceTaskBuilder.java @@ -25,8 +25,6 @@ import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator; import org.apache.iceberg.flink.TableLoader; -import org.apache.iceberg.flink.maintenance.operator.TaskResult; -import org.apache.iceberg.flink.maintenance.operator.Trigger; import org.apache.iceberg.flink.maintenance.operator.TriggerEvaluator; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/TableMaintenance.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/TableMaintenance.java index e8b698d0cbba..25d4754783b9 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/TableMaintenance.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/TableMaintenance.java @@ -43,10 +43,7 @@ import org.apache.iceberg.flink.maintenance.operator.LockRemover; import org.apache.iceberg.flink.maintenance.operator.MonitorSource; 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.TriggerEvaluator; -import org.apache.iceberg.flink.maintenance.operator.TriggerLockFactory; import org.apache.iceberg.flink.maintenance.operator.TriggerManager; import org.apache.iceberg.flink.sink.IcebergSink; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; @@ -73,6 +70,7 @@ private TableMaintenance() { * @param lockFactory used for preventing concurrent task runs * @return builder for the maintenance stream */ + @Internal public static Builder forChangeStream( DataStream changeStream, TableLoader tableLoader, diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TaskResult.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/TaskResult.java similarity index 94% rename from flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TaskResult.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/TaskResult.java index 1568b6965e1c..ca1462526f13 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TaskResult.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/TaskResult.java @@ -16,15 +16,13 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.iceberg.flink.maintenance.operator; +package org.apache.iceberg.flink.maintenance.api; import java.io.Serializable; import java.util.List; -import org.apache.flink.annotation.Internal; import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; /** The result of a single Maintenance Task. */ -@Internal public class TaskResult implements Serializable { private final int taskIndex; private final long startEpoch; diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/Trigger.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/Trigger.java similarity index 91% rename from flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/Trigger.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/Trigger.java index b4556dc1beab..b09edfa256eb 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/Trigger.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/Trigger.java @@ -16,13 +16,12 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.iceberg.flink.maintenance.operator; +package org.apache.iceberg.flink.maintenance.api; import org.apache.flink.annotation.Internal; import org.apache.iceberg.SerializableTable; import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; -@Internal public class Trigger { private final long timestamp; private final SerializableTable table; @@ -36,11 +35,13 @@ private Trigger(long timestamp, SerializableTable table, Integer taskId, boolean this.isRecovery = isRecovery; } + @Internal public static Trigger create(long timestamp, SerializableTable table, int taskId) { return new Trigger(timestamp, table, taskId, false); } - static Trigger recovery(long timestamp) { + @Internal + public static Trigger recovery(long timestamp) { return new Trigger(timestamp, null, null, true); } @@ -48,7 +49,7 @@ public long timestamp() { return timestamp; } - SerializableTable table() { + public SerializableTable table() { return table; } @@ -56,7 +57,7 @@ public Integer taskId() { return taskId; } - boolean isRecovery() { + public boolean isRecovery() { return isRecovery; } diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TriggerLockFactory.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/TriggerLockFactory.java similarity index 91% rename from flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TriggerLockFactory.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/TriggerLockFactory.java index 446e8ce2f2a8..c31381355efe 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TriggerLockFactory.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/TriggerLockFactory.java @@ -16,11 +16,13 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.iceberg.flink.maintenance.operator; +package org.apache.iceberg.flink.maintenance.api; import java.io.Closeable; import java.io.Serializable; import org.apache.flink.annotation.Experimental; +import org.apache.iceberg.flink.maintenance.operator.LockRemover; +import org.apache.iceberg.flink.maintenance.operator.TriggerManager; /** Lock interface for handling locks for the Flink Table Maintenance jobs. */ @Experimental diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/ExpireSnapshotsProcessor.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/ExpireSnapshotsProcessor.java index 52499572a522..50bea6a89e93 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/ExpireSnapshotsProcessor.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/ExpireSnapshotsProcessor.java @@ -30,6 +30,8 @@ import org.apache.iceberg.ExpireSnapshots; import org.apache.iceberg.Table; import org.apache.iceberg.flink.TableLoader; +import org.apache.iceberg.flink.maintenance.api.TaskResult; +import org.apache.iceberg.flink.maintenance.api.Trigger; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.util.ThreadPools; diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/LockRemover.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/LockRemover.java index 3c3761ef2f4d..f4cc0e8a0158 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/LockRemover.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/LockRemover.java @@ -26,6 +26,9 @@ import org.apache.flink.streaming.api.operators.OneInputStreamOperator; import org.apache.flink.streaming.api.watermark.Watermark; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.apache.iceberg.flink.maintenance.api.TaskResult; +import org.apache.iceberg.flink.maintenance.api.Trigger; +import org.apache.iceberg.flink.maintenance.api.TriggerLockFactory; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.slf4j.Logger; diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TableChange.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TableChange.java index 25c3c1028113..78be0a9c15d6 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TableChange.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TableChange.java @@ -37,7 +37,7 @@ public class TableChange { private long eqDeleteRecordCount; private int commitCount; - TableChange( + private TableChange( int dataFileCount, long dataFileSizeInBytes, int posDeleteFileCount, diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TriggerManager.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TriggerManager.java index 6b658d122734..fa52918a71f1 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TriggerManager.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TriggerManager.java @@ -38,6 +38,8 @@ import org.apache.flink.util.Collector; import org.apache.iceberg.SerializableTable; import org.apache.iceberg.flink.TableLoader; +import org.apache.iceberg.flink.maintenance.api.Trigger; +import org.apache.iceberg.flink.maintenance.api.TriggerLockFactory; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.slf4j.Logger; diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/api/MaintenanceTaskInfraExtension.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/api/MaintenanceTaskInfraExtension.java index 392407c8a047..1b69be1fa304 100644 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/api/MaintenanceTaskInfraExtension.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/api/MaintenanceTaskInfraExtension.java @@ -25,8 +25,6 @@ import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.iceberg.flink.maintenance.operator.CollectingSink; import org.apache.iceberg.flink.maintenance.operator.ManualSource; -import org.apache.iceberg.flink.maintenance.operator.TaskResult; -import org.apache.iceberg.flink.maintenance.operator.Trigger; import org.junit.jupiter.api.extension.BeforeEachCallback; import org.junit.jupiter.api.extension.ExtensionContext; diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/api/MaintenanceTaskTestBase.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/api/MaintenanceTaskTestBase.java index e42f77a7ddb7..9fbe34d9be0a 100644 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/api/MaintenanceTaskTestBase.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/api/MaintenanceTaskTestBase.java @@ -29,8 +29,6 @@ import org.apache.iceberg.flink.maintenance.operator.CollectingSink; import org.apache.iceberg.flink.maintenance.operator.ManualSource; import org.apache.iceberg.flink.maintenance.operator.OperatorTestBase; -import org.apache.iceberg.flink.maintenance.operator.TaskResult; -import org.apache.iceberg.flink.maintenance.operator.Trigger; import org.awaitility.Awaitility; import org.junit.jupiter.api.extension.RegisterExtension; diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/api/TestExpireSnapshots.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/api/TestExpireSnapshots.java index 5fe027a3e73d..27cfc9d421fd 100644 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/api/TestExpireSnapshots.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/api/TestExpireSnapshots.java @@ -33,7 +33,6 @@ import org.apache.iceberg.Table; import org.apache.iceberg.flink.SimpleDataUtil; import org.apache.iceberg.flink.maintenance.operator.MetricsReporterFactoryForTests; -import org.apache.iceberg.flink.maintenance.operator.Trigger; 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.Sets; diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestJdbcLockFactory.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/api/TestJdbcLockFactory.java similarity index 92% rename from flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestJdbcLockFactory.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/api/TestJdbcLockFactory.java index 051d09d92bad..c8fa2a7d362a 100644 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestJdbcLockFactory.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/api/TestJdbcLockFactory.java @@ -16,9 +16,9 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.iceberg.flink.maintenance.operator; +package org.apache.iceberg.flink.maintenance.api; -import static org.apache.iceberg.flink.maintenance.operator.JdbcLockFactory.INIT_LOCK_TABLES_PROPERTY; +import static org.apache.iceberg.flink.maintenance.api.JdbcLockFactory.INIT_LOCK_TABLES_PROPERTY; import static org.assertj.core.api.Assertions.assertThat; import java.util.Map; diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestLockFactoryBase.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/api/TestLockFactoryBase.java similarity index 97% rename from flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestLockFactoryBase.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/api/TestLockFactoryBase.java index bf9e86f2534d..c06bef9087d5 100644 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestLockFactoryBase.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/api/TestLockFactoryBase.java @@ -16,7 +16,7 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.iceberg.flink.maintenance.operator; +package org.apache.iceberg.flink.maintenance.api; import static org.assertj.core.api.Assertions.assertThat; diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/api/TestTableMaintenance.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/api/TestTableMaintenance.java index 0b94effada80..fee0b65a3754 100644 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/api/TestTableMaintenance.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/api/TestTableMaintenance.java @@ -57,9 +57,6 @@ 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; diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/OperatorTestBase.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/OperatorTestBase.java index 067bd3962fe2..94c6b4c45500 100644 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/OperatorTestBase.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/OperatorTestBase.java @@ -43,6 +43,7 @@ import org.apache.iceberg.flink.SimpleDataUtil; import org.apache.iceberg.flink.TableLoader; import org.apache.iceberg.flink.TestFixtures; +import org.apache.iceberg.flink.maintenance.api.TriggerLockFactory; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet; import org.apache.iceberg.relocated.com.google.common.collect.Lists; diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestExpireSnapshotsProcessor.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestExpireSnapshotsProcessor.java index 4167a4010de9..6c99fc5a68a6 100644 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestExpireSnapshotsProcessor.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestExpireSnapshotsProcessor.java @@ -29,6 +29,8 @@ import org.apache.iceberg.SerializableTable; import org.apache.iceberg.Snapshot; import org.apache.iceberg.Table; +import org.apache.iceberg.flink.maintenance.api.TaskResult; +import org.apache.iceberg.flink.maintenance.api.Trigger; import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.ValueSource; diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestLockRemover.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestLockRemover.java index f6f6f932e9d0..a6d78ed93682 100644 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestLockRemover.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestLockRemover.java @@ -44,6 +44,8 @@ import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.util.Collector; +import org.apache.iceberg.flink.maintenance.api.TaskResult; +import org.apache.iceberg.flink.maintenance.api.TriggerLockFactory; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.awaitility.Awaitility; diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestTriggerManager.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestTriggerManager.java index 40f229ca244b..ff004671fa9f 100644 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestTriggerManager.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestTriggerManager.java @@ -40,6 +40,8 @@ import org.apache.iceberg.SerializableTable; import org.apache.iceberg.Snapshot; import org.apache.iceberg.flink.TableLoader; +import org.apache.iceberg.flink.maintenance.api.Trigger; +import org.apache.iceberg.flink.maintenance.api.TriggerLockFactory; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.awaitility.Awaitility; From b6dae52c1c0632af08447aecc227d823590e07b1 Mon Sep 17 00:00:00 2001 From: Peter Vary Date: Thu, 26 Sep 2024 14:32:00 +0200 Subject: [PATCH 06/12] Steven's comments --- .../maintenance/api/ExpireSnapshots.java | 28 ++++------------ .../api/MaintenanceTaskBuilder.java | 33 +++++++++---------- .../operator/DeleteFilesProcessor.java | 4 +-- .../operator/ExpireSnapshotsProcessor.java | 21 ++++++++++-- .../maintenance/api/TestExpireSnapshots.java | 23 ++++++++----- 5 files changed, 57 insertions(+), 52 deletions(-) diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/ExpireSnapshots.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/ExpireSnapshots.java index e69fa00616a5..da464f2215d8 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/ExpireSnapshots.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/ExpireSnapshots.java @@ -23,7 +23,6 @@ import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator; import org.apache.iceberg.Snapshot; -import org.apache.iceberg.SystemConfigs; import org.apache.iceberg.flink.maintenance.operator.DeleteFilesProcessor; import org.apache.iceberg.flink.maintenance.operator.ExpireSnapshotsProcessor; import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; @@ -35,9 +34,7 @@ public class ExpireSnapshots { private static final String EXECUTOR_OPERATOR_NAME = "Expire Snapshot"; @VisibleForTesting static final String DELETE_FILES_OPERATOR_NAME = "Delete file"; - private ExpireSnapshots() { - // Do not instantiate directly - } + private ExpireSnapshots() {} /** Creates the builder for creating a stream which expires snapshots for the table. */ public static Builder builder() { @@ -47,9 +44,8 @@ public static Builder builder() { public static class Builder extends MaintenanceTaskBuilder { private Duration maxSnapshotAge = null; private Integer numSnapshots = null; - private int planningWorkerPoolSize = SystemConfigs.WORKER_THREAD_POOL_SIZE.value(); + private Integer planningWorkerPoolSize; private int deleteBatchSize = DELETE_BATCH_SIZE_DEFAULT; - private int deleteParallelism = 1; /** * The snapshots older than this age will be removed. @@ -73,7 +69,8 @@ public Builder retainLast(int newNumSnapshots) { } /** - * The worker pool size used to calculate the files to delete. + * The worker pool size used to calculate the files to delete. If not set, the shared worker + * pool is used. * * @param newPlanningWorkerPoolSize for planning files to delete */ @@ -92,16 +89,6 @@ public Builder deleteBatchSize(int newDeleteBatchSize) { return this; } - /** - * The number of subtasks which are doing the deletes. - * - * @param newDeleteParallelism used for deleting - */ - public Builder deleteParallelism(int newDeleteParallelism) { - this.deleteParallelism = newDeleteParallelism; - return this; - } - @Override DataStream append(DataStream trigger) { Preconditions.checkNotNull(tableLoader(), "TableLoader should not be null"); @@ -114,7 +101,7 @@ DataStream append(DataStream trigger) { maxSnapshotAge == null ? null : maxSnapshotAge.toMillis(), numSnapshots, planningWorkerPoolSize)) - .name(EXECUTOR_OPERATOR_NAME) + .name(operatorName(EXECUTOR_OPERATOR_NAME)) .uid(EXECUTOR_OPERATOR_NAME + uidSuffix()) .slotSharingGroup(slotSharingGroup()) .forceNonParallel(); @@ -123,13 +110,12 @@ DataStream append(DataStream trigger) { .getSideOutput(ExpireSnapshotsProcessor.DELETE_STREAM) .rebalance() .transform( - DELETE_FILES_OPERATOR_NAME, + operatorName(DELETE_FILES_OPERATOR_NAME), TypeInformation.of(Void.class), new DeleteFilesProcessor(name(), tableLoader(), deleteBatchSize)) - .name(DELETE_FILES_OPERATOR_NAME) .uid(DELETE_FILES_OPERATOR_NAME + uidSuffix()) .slotSharingGroup(slotSharingGroup()) - .setParallelism(deleteParallelism); + .setParallelism(parallelism()); // Ignore the file deletion result and return the DataStream directly return result; diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/MaintenanceTaskBuilder.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/MaintenanceTaskBuilder.java index b2383fc93c74..bfee1a2adfd9 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/MaintenanceTaskBuilder.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/MaintenanceTaskBuilder.java @@ -29,14 +29,14 @@ import org.apache.iceberg.relocated.com.google.common.base.Preconditions; @PublicEvolving -abstract class MaintenanceTaskBuilder { +public abstract class MaintenanceTaskBuilder> { private int index; private String name; private TableLoader tableLoader; private String uidSuffix = null; private String slotSharingGroup = null; private Integer parallelism = null; - private TriggerEvaluator.Builder triggerEvaluator = new TriggerEvaluator.Builder(); + private final TriggerEvaluator.Builder triggerEvaluator = new TriggerEvaluator.Builder(); abstract DataStream append(DataStream sourceStream); @@ -156,28 +156,23 @@ public T parallelism(int newParallelism) { return (T) this; } - @Internal - int index() { + protected int index() { return index; } - @Internal - String name() { + protected String name() { return name; } - @Internal - TableLoader tableLoader() { + protected TableLoader tableLoader() { return tableLoader; } - @Internal - String uidSuffix() { + protected String uidSuffix() { return uidSuffix; } - @Internal - String slotSharingGroup() { + protected String slotSharingGroup() { return slotSharingGroup; } @@ -185,6 +180,10 @@ protected Integer parallelism() { return parallelism; } + protected String operatorName(String operatorNameBase) { + return operatorNameBase + "[" + index() + "]"; + } + @Internal TriggerEvaluator evaluator() { return triggerEvaluator.build(); @@ -193,17 +192,17 @@ TriggerEvaluator evaluator() { @Internal DataStream append( DataStream sourceStream, - int maintenanceTaskIndex, - String maintenanceTaskName, + int defaultTaskIndex, + String defaultTaskName, TableLoader newTableLoader, String mainUidSuffix, String mainSlotSharingGroup, int mainParallelism) { - Preconditions.checkNotNull(maintenanceTaskName, "Name should not be null"); + Preconditions.checkNotNull(defaultTaskName, "Name should not be null"); Preconditions.checkNotNull(newTableLoader, "TableLoader should not be null"); - this.index = maintenanceTaskIndex; - this.name = maintenanceTaskName; + this.index = defaultTaskIndex; + this.name = defaultTaskName; this.tableLoader = newTableLoader; if (uidSuffix == null) { diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/DeleteFilesProcessor.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/DeleteFilesProcessor.java index f2705c3727c7..8cf290708481 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/DeleteFilesProcessor.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/DeleteFilesProcessor.java @@ -59,8 +59,8 @@ public DeleteFilesProcessor(String name, TableLoader tableLoader, int batchSize) FileIO fileIO = table.io(); Preconditions.checkArgument( fileIO instanceof SupportsBulkOperations, - "Unsupported FileIO. %s should support bulk delete", - fileIO); + "%s doesn't support bulk delete", + fileIO.getClass().getSimpleName()); this.name = name; this.io = (SupportsBulkOperations) fileIO; diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/ExpireSnapshotsProcessor.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/ExpireSnapshotsProcessor.java index 50bea6a89e93..58581e2a7e6c 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/ExpireSnapshotsProcessor.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/ExpireSnapshotsProcessor.java @@ -51,12 +51,15 @@ public class ExpireSnapshotsProcessor extends ProcessFunction o new TaskResult(trigger.taskId(), trigger.timestamp(), false, Lists.newArrayList(e))); } } + + @Override + public void close() throws Exception { + super.close(); + + if (plannerPoolSize != null) { + plannerPool.shutdown(); + } + } } diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/api/TestExpireSnapshots.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/api/TestExpireSnapshots.java index 27cfc9d421fd..b5f83d890890 100644 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/api/TestExpireSnapshots.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/api/TestExpireSnapshots.java @@ -63,7 +63,6 @@ void testExpireSnapshots() throws Exception { .parallelism(1) .planningWorkerPoolSize(2) .deleteBatchSize(3) - .deleteParallelism(1) .maxSnapshotAge(Duration.ZERO) .retainLast(1) .uidSuffix(UID_SUFFIX) @@ -116,9 +115,7 @@ void testFailure() throws Exception { // Do a single task run long time = System.currentTimeMillis(); - infra - .source() - .sendRecord(Trigger.create(time, serializableTable, 1), System.currentTimeMillis()); + infra.source().sendRecord(Trigger.create(time, serializableTable, 1), time); // First successful run (ensure that the operators are loaded/opened etc.) assertThat(infra.sink().poll(Duration.ofSeconds(5)).success()).isTrue(); @@ -138,10 +135,14 @@ void testFailure() throws Exception { MetricsReporterFactoryForTests.assertCounters( new ImmutableMap.Builder() .put( - DELETE_FILES_OPERATOR_NAME + "." + DUMMY_NAME + "." + DELETE_FILE_FAILED_COUNTER, + DELETE_FILES_OPERATOR_NAME + "[0]." + DUMMY_NAME + "." + DELETE_FILE_FAILED_COUNTER, 0L) .put( - DELETE_FILES_OPERATOR_NAME + "." + DUMMY_NAME + "." + DELETE_FILE_SUCCEEDED_COUNTER, + DELETE_FILES_OPERATOR_NAME + + "[0]." + + DUMMY_NAME + + "." + + DELETE_FILE_SUCCEEDED_COUNTER, 0L) .build()); } @@ -212,14 +213,14 @@ void testMetrics() throws Exception { new ImmutableMap.Builder() .put( DELETE_FILES_OPERATOR_NAME - + "." + + "[0]." + DUMMY_NAME + "." + DELETE_FILE_FAILED_COUNTER, 0L) .put( DELETE_FILES_OPERATOR_NAME - + "." + + "[0]." + DUMMY_NAME + "." + DELETE_FILE_SUCCEEDED_COUNTER, @@ -230,6 +231,10 @@ void testMetrics() throws Exception { private static boolean checkDeleteFinished(Long expectedDeleteNum) { return expectedDeleteNum.equals( MetricsReporterFactoryForTests.counter( - DELETE_FILES_OPERATOR_NAME + "." + DUMMY_NAME + "." + DELETE_FILE_SUCCEEDED_COUNTER)); + DELETE_FILES_OPERATOR_NAME + + "[0]." + + DUMMY_NAME + + "." + + DELETE_FILE_SUCCEEDED_COUNTER)); } } From 415dc906639fed025295f5790ad2cea0debfab1a Mon Sep 17 00:00:00 2001 From: Peter Vary Date: Thu, 26 Sep 2024 14:52:02 +0200 Subject: [PATCH 07/12] Remove Table from the Trigger --- .../api/MaintenanceTaskBuilder.java | 1 + .../flink/maintenance/api/Trigger.java | 16 +++--------- .../maintenance/operator/TriggerManager.java | 15 ++++------- .../api/MaintenanceTaskTestBase.java | 25 +++---------------- .../maintenance/api/TestExpireSnapshots.java | 10 +++----- .../TestExpireSnapshotsProcessor.java | 5 +--- .../operator/TestTriggerManager.java | 20 +-------------- 7 files changed, 19 insertions(+), 73 deletions(-) diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/MaintenanceTaskBuilder.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/MaintenanceTaskBuilder.java index bfee1a2adfd9..ac4ba7642b5d 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/MaintenanceTaskBuilder.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/MaintenanceTaskBuilder.java @@ -29,6 +29,7 @@ import org.apache.iceberg.relocated.com.google.common.base.Preconditions; @PublicEvolving +@SuppressWarnings("unchecked") public abstract class MaintenanceTaskBuilder> { private int index; private String name; diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/Trigger.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/Trigger.java index b09edfa256eb..09209ba15153 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/Trigger.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/Trigger.java @@ -19,40 +19,33 @@ package org.apache.iceberg.flink.maintenance.api; import org.apache.flink.annotation.Internal; -import org.apache.iceberg.SerializableTable; import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; public class Trigger { private final long timestamp; - private final SerializableTable table; private final Integer taskId; private final boolean isRecovery; - private Trigger(long timestamp, SerializableTable table, Integer taskId, boolean isRecovery) { + private Trigger(long timestamp, Integer taskId, boolean isRecovery) { this.timestamp = timestamp; - this.table = table; this.taskId = taskId; this.isRecovery = isRecovery; } @Internal - public static Trigger create(long timestamp, SerializableTable table, int taskId) { - return new Trigger(timestamp, table, taskId, false); + public static Trigger create(long timestamp, int taskId) { + return new Trigger(timestamp, taskId, false); } @Internal public static Trigger recovery(long timestamp) { - return new Trigger(timestamp, null, null, true); + return new Trigger(timestamp, null, true); } public long timestamp() { return timestamp; } - public SerializableTable table() { - return table; - } - public Integer taskId() { return taskId; } @@ -65,7 +58,6 @@ public boolean isRecovery() { public String toString() { return MoreObjects.toStringHelper(this) .add("timestamp", timestamp) - .add("table", table == null ? null : table.name()) .add("taskId", taskId) .add("isRecovery", isRecovery) .toString(); diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TriggerManager.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TriggerManager.java index fa52918a71f1..cd20c6e011a3 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TriggerManager.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TriggerManager.java @@ -36,7 +36,6 @@ import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction; import org.apache.flink.streaming.api.functions.KeyedProcessFunction; import org.apache.flink.util.Collector; -import org.apache.iceberg.SerializableTable; import org.apache.iceberg.flink.TableLoader; import org.apache.iceberg.flink.maintenance.api.Trigger; import org.apache.iceberg.flink.maintenance.api.TriggerLockFactory; @@ -63,7 +62,7 @@ public class TriggerManager extends KeyedProcessFunction maintenanceTaskNames; private final List evaluators; @@ -112,7 +111,8 @@ public TriggerManager( Preconditions.checkArgument( lockCheckDelayMs > 0, "Minimum lock delay rate should be at least 1 ms."); - this.tableLoader = tableLoader; + tableLoader.open(); + this.tableName = tableLoader.loadTable().name(); this.lockFactory = lockFactory; this.maintenanceTaskNames = maintenanceTaskNames; this.evaluators = evaluators; @@ -161,8 +161,6 @@ public void open(Configuration parameters) throws Exception { this.lastTriggerTimesState = getRuntimeContext() .getListState(new ListStateDescriptor<>("triggerManagerLastTriggerTime", Types.LONG)); - - tableLoader.open(); } @Override @@ -222,7 +220,6 @@ public void onTimer(long timestamp, OnTimerContext ctx, Collector out) @Override public void close() throws IOException { - tableLoader.close(); lockFactory.close(); } @@ -258,10 +255,8 @@ private void checkAndFire(long current, TimerService timerService, Collector triggerSource, CollectingSink collectingSink, - Supplier waitForCondition, - Table table) + Supplier waitForCondition) throws Exception { - table.refresh(); - SerializableTable payload = (SerializableTable) SerializableTable.copyOf(table); - JobClient jobClient = null; try { jobClient = env.executeAsync(); // Do a single task run long time = System.currentTimeMillis(); - triggerSource.sendRecord(Trigger.create(time, payload, TESTING_TASK_ID), time); + triggerSource.sendRecord(Trigger.create(time, TESTING_TASK_ID), time); TaskResult result = collectingSink.poll(POLL_DURATION); @@ -73,11 +56,9 @@ void runAndWaitForSuccess( assertThat(result.success()).isTrue(); assertThat(result.taskIndex()).isEqualTo(TESTING_TASK_ID); - Awaitility.await().until(() -> waitForCondition.get()); + Awaitility.await().until(waitForCondition::get); } finally { closeJobClient(jobClient); } - - table.refresh(); } } diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/api/TestExpireSnapshots.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/api/TestExpireSnapshots.java index b5f83d890890..31273f9d7d0d 100644 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/api/TestExpireSnapshots.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/api/TestExpireSnapshots.java @@ -76,8 +76,7 @@ void testExpireSnapshots() throws Exception { 1) .sinkTo(infra.sink()); - runAndWaitForSuccess( - infra.env(), infra.source(), infra.sink(), () -> checkDeleteFinished(3L), table); + runAndWaitForSuccess(infra.env(), infra.source(), infra.sink(), () -> checkDeleteFinished(3L)); table.refresh(); assertThat(Sets.newHashSet(table.snapshots())).hasSize(1); @@ -115,7 +114,7 @@ void testFailure() throws Exception { // Do a single task run long time = System.currentTimeMillis(); - infra.source().sendRecord(Trigger.create(time, serializableTable, 1), time); + infra.source().sendRecord(Trigger.create(time, 1), time); // First successful run (ensure that the operators are loaded/opened etc.) assertThat(infra.sink().poll(Duration.ofSeconds(5)).success()).isTrue(); @@ -124,7 +123,7 @@ void testFailure() throws Exception { dropTable(); // Failed run - infra.source().sendRecord(Trigger.create(time + 1, serializableTable, 1), time + 1); + infra.source().sendRecord(Trigger.create(time + 1, 1), time + 1); assertThat(infra.sink().poll(Duration.ofSeconds(5)).success()).isFalse(); } finally { @@ -202,8 +201,7 @@ void testMetrics() throws Exception { 1) .sinkTo(infra.sink()); - runAndWaitForSuccess( - infra.env(), infra.source(), infra.sink(), () -> checkDeleteFinished(1L), table); + runAndWaitForSuccess(infra.env(), infra.source(), infra.sink(), () -> checkDeleteFinished(1L)); // Check the metrics Awaitility.await() diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestExpireSnapshotsProcessor.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestExpireSnapshotsProcessor.java index 6c99fc5a68a6..d312fc312c99 100644 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestExpireSnapshotsProcessor.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestExpireSnapshotsProcessor.java @@ -26,7 +26,6 @@ import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness; import org.apache.flink.streaming.util.ProcessFunctionTestHarnesses; -import org.apache.iceberg.SerializableTable; import org.apache.iceberg.Snapshot; import org.apache.iceberg.Table; import org.apache.iceberg.flink.maintenance.api.TaskResult; @@ -42,7 +41,6 @@ void testExpire(boolean success) throws Exception { Table table = createTable(); insert(table, 1, "a"); insert(table, 2, "b"); - SerializableTable serializableTable = (SerializableTable) SerializableTable.copyOf(table); List actual; Queue> deletes; @@ -56,8 +54,7 @@ void testExpire(boolean success) throws Exception { dropTable(); } - testHarness.processElement( - Trigger.create(10, serializableTable, 11), System.currentTimeMillis()); + testHarness.processElement(Trigger.create(10, 11), System.currentTimeMillis()); deletes = testHarness.getSideOutput(ExpireSnapshotsProcessor.DELETE_STREAM); actual = testHarness.extractOutputValues(); } diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestTriggerManager.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestTriggerManager.java index ff004671fa9f..7c3418f942ec 100644 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestTriggerManager.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestTriggerManager.java @@ -26,7 +26,6 @@ import static org.assertj.core.api.Assertions.assertThat; import java.time.Duration; -import java.util.Iterator; import java.util.List; import java.util.stream.Stream; import org.apache.flink.api.common.typeinfo.TypeInformation; @@ -37,8 +36,6 @@ import org.apache.flink.streaming.api.operators.KeyedProcessOperator; import org.apache.flink.streaming.util.KeyedOneInputStreamOperatorTestHarness; import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness; -import org.apache.iceberg.SerializableTable; -import org.apache.iceberg.Snapshot; import org.apache.iceberg.flink.TableLoader; import org.apache.iceberg.flink.maintenance.api.Trigger; import org.apache.iceberg.flink.maintenance.api.TriggerLockFactory; @@ -396,11 +393,7 @@ void testRecovery(boolean locked, boolean runningTask) throws Exception { ++processingTime; testHarness.setProcessingTime(processingTime); // Releasing lock will create a new snapshot, and we receive this in the trigger - expected.add( - Trigger.create( - processingTime, - (SerializableTable) SerializableTable.copyOf(tableLoader.loadTable()), - 0)); + expected.add(Trigger.create(processingTime, 0)); assertTriggers(testHarness.extractOutputValues(), expected); } } @@ -648,17 +641,6 @@ private static void assertTriggers(List expected, List actual) assertThat(actualTrigger.timestamp()).isEqualTo(expectedTrigger.timestamp()); assertThat(actualTrigger.taskId()).isEqualTo(expectedTrigger.taskId()); assertThat(actualTrigger.isRecovery()).isEqualTo(expectedTrigger.isRecovery()); - if (expectedTrigger.table() == null) { - assertThat(actualTrigger.table()).isNull(); - } else { - Iterator expectedSnapshots = expectedTrigger.table().snapshots().iterator(); - Iterator actualSnapshots = actualTrigger.table().snapshots().iterator(); - while (expectedSnapshots.hasNext()) { - assertThat(actualSnapshots.hasNext()).isTrue(); - assertThat(expectedSnapshots.next().snapshotId()) - .isEqualTo(actualSnapshots.next().snapshotId()); - } - } } } } From e3dd6f45c3e6d43d4ea3df73f664edbb4c723bed Mon Sep 17 00:00:00 2001 From: Peter Vary Date: Thu, 26 Sep 2024 22:57:38 +0200 Subject: [PATCH 08/12] Steven's new comments --- .../maintenance/api/ExpireSnapshots.java | 2 +- .../api/MaintenanceTaskBuilder.java | 2 - .../maintenance/api/TableMaintenance.java | 117 ++++++++---------- .../operator/DeleteFilesProcessor.java | 7 +- .../operator/ExpireSnapshotsProcessor.java | 1 + .../maintenance/api/TestExpireSnapshots.java | 7 +- .../maintenance/api/TestTableMaintenance.java | 10 +- .../operator/TestDeleteFilesProcessor.java | 4 +- 8 files changed, 70 insertions(+), 80 deletions(-) diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/ExpireSnapshots.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/ExpireSnapshots.java index da464f2215d8..c42195f9d6e8 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/ExpireSnapshots.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/ExpireSnapshots.java @@ -112,7 +112,7 @@ DataStream append(DataStream trigger) { .transform( operatorName(DELETE_FILES_OPERATOR_NAME), TypeInformation.of(Void.class), - new DeleteFilesProcessor(name(), tableLoader(), deleteBatchSize)) + new DeleteFilesProcessor(name(), tableLoader().loadTable(), deleteBatchSize)) .uid(DELETE_FILES_OPERATOR_NAME + uidSuffix()) .slotSharingGroup(slotSharingGroup()) .setParallelism(parallelism()); diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/MaintenanceTaskBuilder.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/MaintenanceTaskBuilder.java index ac4ba7642b5d..9c20067e9c59 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/MaintenanceTaskBuilder.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/MaintenanceTaskBuilder.java @@ -218,8 +218,6 @@ DataStream append( slotSharingGroup = mainSlotSharingGroup; } - tableLoader.open(); - return append(sourceStream); } } diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/TableMaintenance.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/TableMaintenance.java index 25d4754783b9..fab7325d168b 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/TableMaintenance.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/TableMaintenance.java @@ -18,6 +18,7 @@ */ package org.apache.iceberg.flink.maintenance.api; +import java.io.IOException; import java.time.Duration; import java.util.List; import java.util.UUID; @@ -57,9 +58,7 @@ public class TableMaintenance { static final String FILTER_OPERATOR_NAME_PREFIX = "Filter "; static final String LOCK_REMOVER_OPERATOR_NAME = "Lock remover"; - private TableMaintenance() { - // Do not instantiate directly - } + private TableMaintenance() {} /** * Use when the change stream is already provided, like in the {@link @@ -214,7 +213,7 @@ public Builder add(MaintenanceTaskBuilder task) { } /** Builds the task graph for the maintenance tasks. */ - public void append() { + public void append() throws IOException { Preconditions.checkArgument(!taskBuilders.isEmpty(), "Provide at least one task"); Preconditions.checkNotNull(uidSuffix, "Uid suffix should no be null"); @@ -225,73 +224,67 @@ public void append() { evaluators.add(taskBuilders.get(i).evaluator()); } - DataStream triggers = - DataStreamUtils.reinterpretAsKeyedStream(changeStream(), unused -> true) - .process( - new TriggerManager( - tableLoader, - lockFactory, - taskNames, - evaluators, - rateLimit.toMillis(), - lockCheckDelay.toMillis())) - .name(TRIGGER_MANAGER_OPERATOR_NAME) - .uid(TRIGGER_MANAGER_OPERATOR_NAME + uidSuffix) - .slotSharingGroup(slotSharingGroup) - .forceNonParallel() - .assignTimestampsAndWatermarks(new PunctuatedWatermarkStrategy()) - .name(WATERMARK_ASSIGNER_OPERATOR_NAME) - .uid(WATERMARK_ASSIGNER_OPERATOR_NAME + uidSuffix) - .slotSharingGroup(slotSharingGroup) - .forceNonParallel(); - - // Add the specific tasks - DataStream unioned = null; - for (int i = 0; i < taskBuilders.size(); ++i) { - int finalIndex = i; - DataStream filtered = - triggers - .filter(t -> t.taskId() != null && t.taskId() == finalIndex) - .name(FILTER_OPERATOR_NAME_PREFIX + i) + try (TableLoader loader = tableLoader.clone()) { + DataStream triggers = + DataStreamUtils.reinterpretAsKeyedStream(changeStream(loader), unused -> true) + .process( + new TriggerManager( + loader, + lockFactory, + taskNames, + evaluators, + rateLimit.toMillis(), + lockCheckDelay.toMillis())) + .name(TRIGGER_MANAGER_OPERATOR_NAME) + .uid(TRIGGER_MANAGER_OPERATOR_NAME + uidSuffix) + .slotSharingGroup(slotSharingGroup) .forceNonParallel() - .uid(FILTER_OPERATOR_NAME_PREFIX + i + "-" + uidSuffix) - .slotSharingGroup(slotSharingGroup); - MaintenanceTaskBuilder builder = taskBuilders.get(i); - DataStream result = - builder.append( - filtered, - i, - taskNames.get(i), - tableLoader, - uidSuffix, - slotSharingGroup, - parallelism); - if (unioned == null) { - unioned = result; - } else { - unioned = unioned.union(result); + .assignTimestampsAndWatermarks(new PunctuatedWatermarkStrategy()) + .name(WATERMARK_ASSIGNER_OPERATOR_NAME) + .uid(WATERMARK_ASSIGNER_OPERATOR_NAME + uidSuffix) + .slotSharingGroup(slotSharingGroup) + .forceNonParallel(); + + // Add the specific tasks + DataStream unioned = null; + for (int i = 0; i < taskBuilders.size(); ++i) { + int finalIndex = i; + DataStream filtered = + triggers + .filter(t -> t.taskId() != null && t.taskId() == finalIndex) + .name(FILTER_OPERATOR_NAME_PREFIX + i) + .forceNonParallel() + .uid(FILTER_OPERATOR_NAME_PREFIX + i + "-" + uidSuffix) + .slotSharingGroup(slotSharingGroup); + MaintenanceTaskBuilder builder = taskBuilders.get(i); + DataStream result = + builder.append( + filtered, i, taskNames.get(i), loader, uidSuffix, slotSharingGroup, parallelism); + if (unioned == null) { + unioned = result; + } else { + unioned = unioned.union(result); + } } - } - // Add the LockRemover to the end - unioned - .transform( - LOCK_REMOVER_OPERATOR_NAME, - TypeInformation.of(Void.class), - new LockRemover(lockFactory, taskNames)) - .forceNonParallel() - .uid("lock-remover-" + uidSuffix) - .slotSharingGroup(slotSharingGroup); + // Add the LockRemover to the end + unioned + .transform( + LOCK_REMOVER_OPERATOR_NAME, + TypeInformation.of(Void.class), + new LockRemover(lockFactory, taskNames)) + .forceNonParallel() + .uid("lock-remover-" + uidSuffix) + .slotSharingGroup(slotSharingGroup); + } } - private DataStream changeStream() { + private DataStream changeStream(TableLoader loader) { if (inputStream == null) { // Create a monitor source to provide the TableChange stream MonitorSource source = new MonitorSource( - tableLoader, - RateLimiterStrategy.perSecond(1.0 / rateLimit.getSeconds()), - maxReadBack); + loader, RateLimiterStrategy.perSecond(1.0 / rateLimit.getSeconds()), maxReadBack); return env.fromSource(source, WatermarkStrategy.noWatermarks(), SOURCE_OPERATOR_NAME) .uid(SOURCE_OPERATOR_NAME + uidSuffix) .slotSharingGroup(slotSharingGroup) diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/DeleteFilesProcessor.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/DeleteFilesProcessor.java index 8cf290708481..c3ef059e9c46 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/DeleteFilesProcessor.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/DeleteFilesProcessor.java @@ -26,7 +26,6 @@ import org.apache.flink.streaming.api.watermark.Watermark; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import org.apache.iceberg.Table; -import org.apache.iceberg.flink.TableLoader; import org.apache.iceberg.io.BulkDeletionFailureException; import org.apache.iceberg.io.FileIO; import org.apache.iceberg.io.SupportsBulkOperations; @@ -50,12 +49,10 @@ public class DeleteFilesProcessor extends AbstractStreamOperator private transient Counter failedCounter; private transient Counter succeededCounter; - public DeleteFilesProcessor(String name, TableLoader tableLoader, int batchSize) { + public DeleteFilesProcessor(String name, Table table, int batchSize) { Preconditions.checkNotNull(name, "Name should no be null"); - Preconditions.checkNotNull(tableLoader, "Table loader should no be null"); + Preconditions.checkNotNull(table, "Table should no be null"); - tableLoader.open(); - Table table = tableLoader.loadTable(); FileIO fileIO = table.io(); Preconditions.checkArgument( fileIO instanceof SupportsBulkOperations, diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/ExpireSnapshotsProcessor.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/ExpireSnapshotsProcessor.java index 58581e2a7e6c..a09d0244e95d 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/ExpireSnapshotsProcessor.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/ExpireSnapshotsProcessor.java @@ -121,6 +121,7 @@ public void processElement(Trigger trigger, Context ctx, Collector o public void close() throws Exception { super.close(); + tableLoader.close(); if (plannerPoolSize != null) { plannerPool.shutdown(); } diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/api/TestExpireSnapshots.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/api/TestExpireSnapshots.java index 31273f9d7d0d..a105f840adc9 100644 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/api/TestExpireSnapshots.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/api/TestExpireSnapshots.java @@ -28,7 +28,6 @@ import java.util.Set; import org.apache.flink.core.execution.JobClient; import org.apache.flink.streaming.api.graph.StreamGraphGenerator; -import org.apache.iceberg.SerializableTable; import org.apache.iceberg.Snapshot; import org.apache.iceberg.Table; import org.apache.iceberg.flink.SimpleDataUtil; @@ -47,6 +46,7 @@ class TestExpireSnapshots extends MaintenanceTaskTestBase { void before() { MetricsReporterFactoryForTests.reset(); this.table = createTable(); + tableLoader().open(); } @Test @@ -95,8 +95,6 @@ void testFailure() throws Exception { insert(table, 1, "a"); insert(table, 2, "b"); - SerializableTable serializableTable = (SerializableTable) SerializableTable.copyOf(table); - ExpireSnapshots.builder() .append( infra.triggerStream(), @@ -130,7 +128,8 @@ void testFailure() throws Exception { closeJobClient(jobClient); } - // Check the metrics + // Check the metrics. There are no expired snapshots or data files because ExpireSnapshots has + // no max age of number of snapshots set, so no files are removed. MetricsReporterFactoryForTests.assertCounters( new ImmutableMap.Builder() .put( diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/api/TestTableMaintenance.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/api/TestTableMaintenance.java index fee0b65a3754..f4c1f8380e89 100644 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/api/TestTableMaintenance.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/api/TestTableMaintenance.java @@ -224,7 +224,7 @@ void testMetrics() throws Exception { } @Test - void testUidAndSlotSharingGroup() { + void testUidAndSlotSharingGroup() throws IOException { TableMaintenance.forChangeStream( new ManualSource<>(env, TypeInformation.of(TableChange.class)).dataStream(), tableLoader(), @@ -243,7 +243,7 @@ void testUidAndSlotSharingGroup() { } @Test - void testUidAndSlotSharingGroupUnset() { + void testUidAndSlotSharingGroupUnset() throws IOException { TableMaintenance.forChangeStream( new ManualSource<>(env, TypeInformation.of(TableChange.class)).dataStream(), tableLoader(), @@ -256,7 +256,7 @@ void testUidAndSlotSharingGroupUnset() { } @Test - void testUidAndSlotSharingGroupInherit() { + void testUidAndSlotSharingGroupInherit() throws IOException { TableMaintenance.forChangeStream( new ManualSource<>(env, TypeInformation.of(TableChange.class)).dataStream(), tableLoader(), @@ -271,7 +271,7 @@ void testUidAndSlotSharingGroupInherit() { } @Test - void testUidAndSlotSharingGroupOverWrite() { + void testUidAndSlotSharingGroupOverWrite() throws IOException { String anotherUid = "Another-UID"; String anotherSlotSharingGroup = "Another-SlotSharingGroup"; TableMaintenance.forChangeStream( @@ -312,7 +312,7 @@ void testUidAndSlotSharingGroupOverWrite() { } @Test - void testUidAndSlotSharingGroupForMonitorSource() { + void testUidAndSlotSharingGroupForMonitorSource() throws IOException { TableMaintenance.forTable(env, tableLoader(), LOCK_FACTORY) .uidSuffix(UID_SUFFIX) .slotSharingGroup(SLOT_SHARING_GROUP) diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestDeleteFilesProcessor.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestDeleteFilesProcessor.java index 3f0cccf08718..1160f6bff601 100644 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestDeleteFilesProcessor.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestDeleteFilesProcessor.java @@ -87,9 +87,11 @@ void testInvalidURIScheme() throws Exception { } private void deleteFile(TableLoader tableLoader, String fileName) throws Exception { + tableLoader().open(); try (OneInputStreamOperatorTestHarness testHarness = new OneInputStreamOperatorTestHarness<>( - new DeleteFilesProcessor(DUMMY_NAME, tableLoader, 10), StringSerializer.INSTANCE)) { + new DeleteFilesProcessor(DUMMY_NAME, tableLoader.loadTable(), 10), + StringSerializer.INSTANCE)) { testHarness.open(); testHarness.processElement(fileName, System.currentTimeMillis()); testHarness.processWatermark(EVENT_TIME); From 1c6bcda5d4cc8705ea749cb914b62246f0ec78ca Mon Sep 17 00:00:00 2001 From: Peter Vary Date: Mon, 30 Sep 2024 11:19:10 +0200 Subject: [PATCH 09/12] Rod's comments --- .../api/MaintenanceTaskBuilder.java | 6 +++--- .../maintenance/api/TableMaintenance.java | 18 +++++------------- 2 files changed, 8 insertions(+), 16 deletions(-) diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/MaintenanceTaskBuilder.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/MaintenanceTaskBuilder.java index 9c20067e9c59..63076876e74d 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/MaintenanceTaskBuilder.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/MaintenanceTaskBuilder.java @@ -19,8 +19,8 @@ package org.apache.iceberg.flink.maintenance.api; import java.time.Duration; +import org.apache.flink.annotation.Experimental; import org.apache.flink.annotation.Internal; -import org.apache.flink.annotation.PublicEvolving; import org.apache.flink.api.common.operators.util.OperatorValidationUtils; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator; @@ -28,7 +28,7 @@ import org.apache.iceberg.flink.maintenance.operator.TriggerEvaluator; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; -@PublicEvolving +@Experimental @SuppressWarnings("unchecked") public abstract class MaintenanceTaskBuilder> { private int index; @@ -199,7 +199,7 @@ DataStream append( String mainUidSuffix, String mainSlotSharingGroup, int mainParallelism) { - Preconditions.checkNotNull(defaultTaskName, "Name should not be null"); + Preconditions.checkNotNull(defaultTaskName, "Task name should not be null"); Preconditions.checkNotNull(newTableLoader, "TableLoader should not be null"); this.index = defaultTaskIndex; diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/TableMaintenance.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/TableMaintenance.java index fab7325d168b..149ebdb0f813 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/TableMaintenance.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/TableMaintenance.java @@ -78,7 +78,7 @@ public static Builder forChangeStream( Preconditions.checkNotNull(tableLoader, "TableLoader should not be null"); Preconditions.checkNotNull(lockFactory, "LockFactory should not be null"); - return new Builder(changeStream, tableLoader, lockFactory); + return new Builder(null, changeStream, tableLoader, lockFactory); } /** @@ -96,7 +96,7 @@ public static Builder forTable( Preconditions.checkNotNull(tableLoader, "TableLoader should not be null"); Preconditions.checkNotNull(lockFactory, "LockFactory should not be null"); - return new Builder(env, tableLoader, lockFactory); + return new Builder(env, null, tableLoader, lockFactory); } public static class Builder { @@ -114,19 +114,11 @@ public static class Builder { private int maxReadBack = 100; private Builder( - StreamExecutionEnvironment env, TableLoader tableLoader, TriggerLockFactory lockFactory) { - this.env = env; - this.inputStream = null; - this.tableLoader = tableLoader; - this.lockFactory = lockFactory; - this.taskBuilders = Lists.newArrayListWithCapacity(4); - } - - private Builder( + StreamExecutionEnvironment env, DataStream inputStream, TableLoader tableLoader, TriggerLockFactory lockFactory) { - this.env = null; + this.env = env; this.inputStream = inputStream; this.tableLoader = tableLoader; this.lockFactory = lockFactory; @@ -134,7 +126,7 @@ private Builder( } /** - * The prefix used for the generated {@link Transformation}'s uid. + * The suffix used for the generated {@link Transformation}'s uid. * * @param newUidSuffix for the transformations */ From 1b6daac4fd5cbbf1ff648795d2f875bf4584e177 Mon Sep 17 00:00:00 2001 From: Peter Vary Date: Tue, 1 Oct 2024 15:52:21 +0200 Subject: [PATCH 10/12] Remove Internal annotation --- .../iceberg/flink/maintenance/api/MaintenanceTaskBuilder.java | 3 --- 1 file changed, 3 deletions(-) diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/MaintenanceTaskBuilder.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/MaintenanceTaskBuilder.java index 63076876e74d..647a29223020 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/MaintenanceTaskBuilder.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/MaintenanceTaskBuilder.java @@ -20,7 +20,6 @@ import java.time.Duration; import org.apache.flink.annotation.Experimental; -import org.apache.flink.annotation.Internal; import org.apache.flink.api.common.operators.util.OperatorValidationUtils; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator; @@ -185,12 +184,10 @@ protected String operatorName(String operatorNameBase) { return operatorNameBase + "[" + index() + "]"; } - @Internal TriggerEvaluator evaluator() { return triggerEvaluator.build(); } - @Internal DataStream append( DataStream sourceStream, int defaultTaskIndex, From 5a5c8ee9b6e6a430f8f9b7b40460c7d9babe384f Mon Sep 17 00:00:00 2001 From: Peter Vary Date: Thu, 24 Oct 2024 17:18:53 +0200 Subject: [PATCH 11/12] Steven's comments --- .../maintenance/api/ExpireSnapshots.java | 3 +- .../api/MaintenanceTaskBuilder.java | 31 +++--- .../maintenance/api/TableMaintenance.java | 42 +++++--- .../operator/DeleteFilesProcessor.java | 20 ++-- .../maintenance/operator/LockRemover.java | 22 ++++- .../operator/TableMaintenanceMetrics.java | 6 +- .../maintenance/operator/TriggerManager.java | 33 +++---- .../maintenance/api/TestExpireSnapshots.java | 79 +++++++++------ .../maintenance/api/TestTableMaintenance.java | 74 ++++++++------ .../MetricsReporterFactoryForTests.java | 65 ++++++++++--- .../operator/OperatorTestBase.java | 5 +- .../operator/TestDeleteFilesProcessor.java | 2 +- .../maintenance/operator/TestLockRemover.java | 96 +++++++++++++------ .../operator/TestTriggerManager.java | 50 +++++----- 14 files changed, 345 insertions(+), 183 deletions(-) diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/ExpireSnapshots.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/ExpireSnapshots.java index c42195f9d6e8..9cde5cb173e1 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/ExpireSnapshots.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/ExpireSnapshots.java @@ -112,7 +112,8 @@ DataStream append(DataStream trigger) { .transform( operatorName(DELETE_FILES_OPERATOR_NAME), TypeInformation.of(Void.class), - new DeleteFilesProcessor(name(), tableLoader().loadTable(), deleteBatchSize)) + new DeleteFilesProcessor( + index(), taskName(), tableLoader().loadTable(), deleteBatchSize)) .uid(DELETE_FILES_OPERATOR_NAME + uidSuffix()) .slotSharingGroup(slotSharingGroup()) .setParallelism(parallelism()); diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/MaintenanceTaskBuilder.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/MaintenanceTaskBuilder.java index 647a29223020..3dffb59171e3 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/MaintenanceTaskBuilder.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/MaintenanceTaskBuilder.java @@ -31,7 +31,8 @@ @SuppressWarnings("unchecked") public abstract class MaintenanceTaskBuilder> { private int index; - private String name; + private String taskName; + private String tableName; private TableLoader tableLoader; private String uidSuffix = null; private String slotSharingGroup = null; @@ -160,8 +161,12 @@ protected int index() { return index; } - protected String name() { - return name; + protected String taskName() { + return taskName; + } + + protected String tableName() { + return tableName; } protected TableLoader tableLoader() { @@ -190,21 +195,23 @@ TriggerEvaluator evaluator() { DataStream append( DataStream sourceStream, - int defaultTaskIndex, - String defaultTaskName, + int taskIndex, + String taskName, + String tableName, TableLoader newTableLoader, - String mainUidSuffix, - String mainSlotSharingGroup, + String defaultUidSuffix, + String defaultSlotSharingGroup, int mainParallelism) { - Preconditions.checkNotNull(defaultTaskName, "Task name should not be null"); + Preconditions.checkNotNull(taskName, "Task name should not be null"); Preconditions.checkNotNull(newTableLoader, "TableLoader should not be null"); - this.index = defaultTaskIndex; - this.name = defaultTaskName; + this.index = taskIndex; + this.taskName = taskName; + this.tableName = tableName; this.tableLoader = newTableLoader; if (uidSuffix == null) { - uidSuffix = name + "_" + index + "_" + mainUidSuffix; + uidSuffix = this.taskName + "_" + index + "_" + defaultUidSuffix; } if (parallelism == null) { @@ -212,7 +219,7 @@ DataStream append( } if (slotSharingGroup == null) { - slotSharingGroup = mainSlotSharingGroup; + slotSharingGroup = defaultSlotSharingGroup; } return append(sourceStream); diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/TableMaintenance.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/TableMaintenance.java index 149ebdb0f813..f931a9127137 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/TableMaintenance.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/TableMaintenance.java @@ -52,7 +52,7 @@ /** Creates the table maintenance graph. */ public class TableMaintenance { - static final String SOURCE_OPERATOR_NAME = "Monitor source"; + static final String SOURCE_OPERATOR_NAME_PREFIX = "Monitor source for "; static final String TRIGGER_MANAGER_OPERATOR_NAME = "Trigger manager"; static final String WATERMARK_ASSIGNER_OPERATOR_NAME = "Watermark Assigner"; static final String FILTER_OPERATOR_NAME_PREFIX = "Filter "; @@ -217,8 +217,11 @@ public void append() throws IOException { } try (TableLoader loader = tableLoader.clone()) { + loader.open(); + String tableName = loader.loadTable().name(); DataStream triggers = - DataStreamUtils.reinterpretAsKeyedStream(changeStream(loader), unused -> true) + DataStreamUtils.reinterpretAsKeyedStream( + changeStream(tableName, loader), unused -> true) .process( new TriggerManager( loader, @@ -240,18 +243,25 @@ public void append() throws IOException { // Add the specific tasks DataStream unioned = null; for (int i = 0; i < taskBuilders.size(); ++i) { - int finalIndex = i; + int taskIndex = i; DataStream filtered = triggers - .filter(t -> t.taskId() != null && t.taskId() == finalIndex) - .name(FILTER_OPERATOR_NAME_PREFIX + i) + .filter(t -> t.taskId() != null && t.taskId() == taskIndex) + .name(FILTER_OPERATOR_NAME_PREFIX + taskIndex) .forceNonParallel() - .uid(FILTER_OPERATOR_NAME_PREFIX + i + "-" + uidSuffix) + .uid(FILTER_OPERATOR_NAME_PREFIX + taskIndex + "-" + uidSuffix) .slotSharingGroup(slotSharingGroup); - MaintenanceTaskBuilder builder = taskBuilders.get(i); + MaintenanceTaskBuilder builder = taskBuilders.get(taskIndex); DataStream result = builder.append( - filtered, i, taskNames.get(i), loader, uidSuffix, slotSharingGroup, parallelism); + filtered, + taskIndex, + taskNames.get(taskIndex), + tableName, + loader, + uidSuffix, + slotSharingGroup, + parallelism); if (unioned == null) { unioned = result; } else { @@ -264,31 +274,33 @@ public void append() throws IOException { .transform( LOCK_REMOVER_OPERATOR_NAME, TypeInformation.of(Void.class), - new LockRemover(lockFactory, taskNames)) + new LockRemover(tableName, lockFactory, taskNames)) .forceNonParallel() .uid("lock-remover-" + uidSuffix) .slotSharingGroup(slotSharingGroup); } } - private DataStream changeStream(TableLoader loader) { + private DataStream changeStream(String tableName, TableLoader loader) { if (inputStream == null) { // Create a monitor source to provide the TableChange stream MonitorSource source = new MonitorSource( loader, RateLimiterStrategy.perSecond(1.0 / rateLimit.getSeconds()), maxReadBack); - return env.fromSource(source, WatermarkStrategy.noWatermarks(), SOURCE_OPERATOR_NAME) - .uid(SOURCE_OPERATOR_NAME + uidSuffix) + return env.fromSource( + source, WatermarkStrategy.noWatermarks(), SOURCE_OPERATOR_NAME_PREFIX + tableName) + .uid(SOURCE_OPERATOR_NAME_PREFIX + uidSuffix) .slotSharingGroup(slotSharingGroup) .forceNonParallel(); } else { return inputStream.global(); } } - } - private static String nameFor(MaintenanceTaskBuilder streamBuilder, int taskId) { - return String.format("%s [%d]", streamBuilder.getClass().getSimpleName(), taskId); + private static String nameFor(MaintenanceTaskBuilder streamBuilder, int taskIndex) { + return String.format( + "%s [%s]", streamBuilder.getClass().getSimpleName(), String.valueOf(taskIndex)); + } } @Internal diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/DeleteFilesProcessor.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/DeleteFilesProcessor.java index c3ef059e9c46..dc7846c4c4d3 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/DeleteFilesProcessor.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/DeleteFilesProcessor.java @@ -40,7 +40,8 @@ public class DeleteFilesProcessor extends AbstractStreamOperator implements OneInputStreamOperator { private static final Logger LOG = LoggerFactory.getLogger(DeleteFilesProcessor.class); - private final String name; + private final String taskIndex; + private final String taskName; private final SupportsBulkOperations io; private final String tableName; private final Set filesToDelete = Sets.newHashSet(); @@ -49,8 +50,8 @@ public class DeleteFilesProcessor extends AbstractStreamOperator private transient Counter failedCounter; private transient Counter succeededCounter; - public DeleteFilesProcessor(String name, Table table, int batchSize) { - Preconditions.checkNotNull(name, "Name should no be null"); + public DeleteFilesProcessor(int taskIndex, String taskName, Table table, int batchSize) { + Preconditions.checkNotNull(taskName, "Task name should no be null"); Preconditions.checkNotNull(table, "Table should no be null"); FileIO fileIO = table.io(); @@ -59,7 +60,8 @@ public DeleteFilesProcessor(String name, Table table, int batchSize) { "%s doesn't support bulk delete", fileIO.getClass().getSimpleName()); - this.name = name; + this.taskIndex = String.valueOf(taskIndex); + this.taskName = taskName; this.io = (SupportsBulkOperations) fileIO; this.tableName = table.name(); this.batchSize = batchSize; @@ -70,12 +72,18 @@ public void open() throws Exception { this.failedCounter = getRuntimeContext() .getMetricGroup() - .addGroup(TableMaintenanceMetrics.GROUP_KEY, name) + .addGroup(TableMaintenanceMetrics.GROUP_KEY) + .addGroup(TableMaintenanceMetrics.TABLE_NAME_KEY, tableName) + .addGroup(TableMaintenanceMetrics.TASK_NAME_KEY, taskName) + .addGroup(TableMaintenanceMetrics.TASK_INDEX_KEY, taskIndex) .counter(TableMaintenanceMetrics.DELETE_FILE_FAILED_COUNTER); this.succeededCounter = getRuntimeContext() .getMetricGroup() - .addGroup(TableMaintenanceMetrics.GROUP_KEY, name) + .addGroup(TableMaintenanceMetrics.GROUP_KEY) + .addGroup(TableMaintenanceMetrics.TABLE_NAME_KEY, tableName) + .addGroup(TableMaintenanceMetrics.TASK_NAME_KEY, taskName) + .addGroup(TableMaintenanceMetrics.TASK_INDEX_KEY, taskIndex) .counter(TableMaintenanceMetrics.DELETE_FILE_SUCCEEDED_COUNTER); } diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/LockRemover.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/LockRemover.java index f4cc0e8a0158..14d590162c8b 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/LockRemover.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/LockRemover.java @@ -67,6 +67,7 @@ public class LockRemover extends AbstractStreamOperator implements OneInputStreamOperator { private static final Logger LOG = LoggerFactory.getLogger(LockRemover.class); + private final String tableName; private final TriggerLockFactory lockFactory; private final List maintenanceTaskNames; @@ -77,12 +78,14 @@ public class LockRemover extends AbstractStreamOperator private transient TriggerLockFactory.Lock recoveryLock; private transient long lastProcessedTaskStartEpoch = 0L; - public LockRemover(TriggerLockFactory lockFactory, List maintenanceTaskNames) { + public LockRemover( + String tableName, TriggerLockFactory lockFactory, List maintenanceTaskNames) { Preconditions.checkNotNull(lockFactory, "Lock factory should no be null"); Preconditions.checkArgument( maintenanceTaskNames != null && !maintenanceTaskNames.isEmpty(), "Invalid maintenance task names: null or empty"); + this.tableName = tableName; this.lockFactory = lockFactory; this.maintenanceTaskNames = maintenanceTaskNames; } @@ -94,22 +97,31 @@ public void open() throws Exception { Lists.newArrayListWithExpectedSize(maintenanceTaskNames.size()); this.failedTaskResultCounters = Lists.newArrayListWithExpectedSize(maintenanceTaskNames.size()); this.taskLastRunDurationMs = Lists.newArrayListWithExpectedSize(maintenanceTaskNames.size()); - for (String name : maintenanceTaskNames) { + for (int taskIndex = 0; taskIndex < maintenanceTaskNames.size(); ++taskIndex) { succeededTaskResultCounters.add( getRuntimeContext() .getMetricGroup() - .addGroup(TableMaintenanceMetrics.GROUP_KEY, name) + .addGroup(TableMaintenanceMetrics.GROUP_KEY) + .addGroup(TableMaintenanceMetrics.TABLE_NAME_KEY, tableName) + .addGroup(TableMaintenanceMetrics.TASK_NAME_KEY, maintenanceTaskNames.get(taskIndex)) + .addGroup(TableMaintenanceMetrics.TASK_INDEX_KEY, String.valueOf(taskIndex)) .counter(TableMaintenanceMetrics.SUCCEEDED_TASK_COUNTER)); failedTaskResultCounters.add( getRuntimeContext() .getMetricGroup() - .addGroup(TableMaintenanceMetrics.GROUP_KEY, name) + .addGroup(TableMaintenanceMetrics.GROUP_KEY) + .addGroup(TableMaintenanceMetrics.TABLE_NAME_KEY, tableName) + .addGroup(TableMaintenanceMetrics.TASK_NAME_KEY, maintenanceTaskNames.get(taskIndex)) + .addGroup(TableMaintenanceMetrics.TASK_INDEX_KEY, String.valueOf(taskIndex)) .counter(TableMaintenanceMetrics.FAILED_TASK_COUNTER)); AtomicLong duration = new AtomicLong(0); taskLastRunDurationMs.add(duration); getRuntimeContext() .getMetricGroup() - .addGroup(TableMaintenanceMetrics.GROUP_KEY, name) + .addGroup(TableMaintenanceMetrics.GROUP_KEY) + .addGroup(TableMaintenanceMetrics.TABLE_NAME_KEY, tableName) + .addGroup(TableMaintenanceMetrics.TASK_NAME_KEY, maintenanceTaskNames.get(taskIndex)) + .addGroup(TableMaintenanceMetrics.TASK_INDEX_KEY, String.valueOf(taskIndex)) .gauge(TableMaintenanceMetrics.LAST_RUN_DURATION_MS, duration::get); } diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TableMaintenanceMetrics.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TableMaintenanceMetrics.java index c57ed5092504..6147c3a5fd16 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TableMaintenanceMetrics.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TableMaintenanceMetrics.java @@ -19,8 +19,10 @@ package org.apache.iceberg.flink.maintenance.operator; public class TableMaintenanceMetrics { - public static final String GROUP_KEY = "maintenanceTask"; - public static final String GROUP_VALUE_DEFAULT = "maintenanceTask"; + public static final String GROUP_KEY = "maintenance"; + public static final String TASK_NAME_KEY = "taskName"; + public static final String TASK_INDEX_KEY = "taskIndex"; + public static final String TABLE_NAME_KEY = "tableName"; // TriggerManager metrics public static final String RATE_LIMITER_TRIGGERED = "rateLimiterTriggered"; diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TriggerManager.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TriggerManager.java index cd20c6e011a3..a96e99d94299 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TriggerManager.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TriggerManager.java @@ -20,7 +20,6 @@ import java.io.IOException; import java.util.List; -import java.util.stream.Collectors; import org.apache.flink.annotation.Internal; import org.apache.flink.api.common.state.ListState; import org.apache.flink.api.common.state.ListStateDescriptor; @@ -125,30 +124,32 @@ public void open(Configuration parameters) throws Exception { this.rateLimiterTriggeredCounter = getRuntimeContext() .getMetricGroup() - .addGroup( - TableMaintenanceMetrics.GROUP_KEY, TableMaintenanceMetrics.GROUP_VALUE_DEFAULT) + .addGroup(TableMaintenanceMetrics.GROUP_KEY) + .addGroup(TableMaintenanceMetrics.TABLE_NAME_KEY, tableName) .counter(TableMaintenanceMetrics.RATE_LIMITER_TRIGGERED); this.concurrentRunThrottledCounter = getRuntimeContext() .getMetricGroup() - .addGroup( - TableMaintenanceMetrics.GROUP_KEY, TableMaintenanceMetrics.GROUP_VALUE_DEFAULT) + .addGroup(TableMaintenanceMetrics.GROUP_KEY) + .addGroup(TableMaintenanceMetrics.TABLE_NAME_KEY, tableName) .counter(TableMaintenanceMetrics.CONCURRENT_RUN_THROTTLED); this.nothingToTriggerCounter = getRuntimeContext() .getMetricGroup() - .addGroup( - TableMaintenanceMetrics.GROUP_KEY, TableMaintenanceMetrics.GROUP_VALUE_DEFAULT) + .addGroup(TableMaintenanceMetrics.GROUP_KEY) + .addGroup(TableMaintenanceMetrics.TABLE_NAME_KEY, tableName) .counter(TableMaintenanceMetrics.NOTHING_TO_TRIGGER); - this.triggerCounters = - maintenanceTaskNames.stream() - .map( - name -> - getRuntimeContext() - .getMetricGroup() - .addGroup(TableMaintenanceMetrics.GROUP_KEY, name) - .counter(TableMaintenanceMetrics.TRIGGERED)) - .collect(Collectors.toList()); + this.triggerCounters = Lists.newArrayListWithExpectedSize(maintenanceTaskNames.size()); + for (int taskIndex = 0; taskIndex < maintenanceTaskNames.size(); ++taskIndex) { + triggerCounters.add( + getRuntimeContext() + .getMetricGroup() + .addGroup(TableMaintenanceMetrics.GROUP_KEY) + .addGroup(TableMaintenanceMetrics.TABLE_NAME_KEY, tableName) + .addGroup(TableMaintenanceMetrics.TASK_NAME_KEY, maintenanceTaskNames.get(taskIndex)) + .addGroup(TableMaintenanceMetrics.TASK_INDEX_KEY, String.valueOf(taskIndex)) + .counter(TableMaintenanceMetrics.TRIGGERED)); + } this.nextEvaluationTimeState = getRuntimeContext() diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/api/TestExpireSnapshots.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/api/TestExpireSnapshots.java index a105f840adc9..f80129f966e1 100644 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/api/TestExpireSnapshots.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/api/TestExpireSnapshots.java @@ -25,6 +25,7 @@ import static org.assertj.core.api.Assertions.assertThat; import java.time.Duration; +import java.util.List; import java.util.Set; import org.apache.flink.core.execution.JobClient; import org.apache.flink.streaming.api.graph.StreamGraphGenerator; @@ -69,14 +70,16 @@ void testExpireSnapshots() throws Exception { .append( infra.triggerStream(), 0, - DUMMY_NAME, + DUMMY_TASK_NAME, + DUMMY_TABLE_NAME, tableLoader(), "OTHER", StreamGraphGenerator.DEFAULT_SLOT_SHARING_GROUP, 1) .sinkTo(infra.sink()); - runAndWaitForSuccess(infra.env(), infra.source(), infra.sink(), () -> checkDeleteFinished(3L)); + runAndWaitForSuccess( + infra.env(), infra.source(), infra.sink(), () -> checkDeleteFinished(table.name(), 3L)); table.refresh(); assertThat(Sets.newHashSet(table.snapshots())).hasSize(1); @@ -99,7 +102,8 @@ void testFailure() throws Exception { .append( infra.triggerStream(), 0, - DUMMY_NAME, + DUMMY_TASK_NAME, + DUMMY_TABLE_NAME, tableLoader(), UID_SUFFIX, StreamGraphGenerator.DEFAULT_SLOT_SHARING_GROUP, @@ -131,16 +135,22 @@ void testFailure() throws Exception { // Check the metrics. There are no expired snapshots or data files because ExpireSnapshots has // no max age of number of snapshots set, so no files are removed. MetricsReporterFactoryForTests.assertCounters( - new ImmutableMap.Builder() + new ImmutableMap.Builder, Long>() .put( - DELETE_FILES_OPERATOR_NAME + "[0]." + DUMMY_NAME + "." + DELETE_FILE_FAILED_COUNTER, + ImmutableList.of( + DELETE_FILES_OPERATOR_NAME + "[0]", + table.name(), + DUMMY_TASK_NAME, + "0", + DELETE_FILE_FAILED_COUNTER), 0L) .put( - DELETE_FILES_OPERATOR_NAME - + "[0]." - + DUMMY_NAME - + "." - + DELETE_FILE_SUCCEEDED_COUNTER, + ImmutableList.of( + DELETE_FILES_OPERATOR_NAME + "[0]", + table.name(), + DUMMY_TASK_NAME, + "0", + DELETE_FILE_SUCCEEDED_COUNTER), 0L) .build()); } @@ -153,7 +163,8 @@ void testUidAndSlotSharingGroup() { .append( infra.triggerStream(), 0, - DUMMY_NAME, + DUMMY_TASK_NAME, + DUMMY_TABLE_NAME, tableLoader(), UID_SUFFIX, StreamGraphGenerator.DEFAULT_SLOT_SHARING_GROUP, @@ -170,7 +181,8 @@ void testUidAndSlotSharingGroupUnset() { .append( infra.triggerStream(), 0, - DUMMY_NAME, + DUMMY_TASK_NAME, + DUMMY_TABLE_NAME, tableLoader(), UID_SUFFIX, StreamGraphGenerator.DEFAULT_SLOT_SHARING_GROUP, @@ -193,45 +205,50 @@ void testMetrics() throws Exception { .append( infra.triggerStream(), 0, - DUMMY_NAME, + DUMMY_TASK_NAME, + DUMMY_TABLE_NAME, tableLoader(), UID_SUFFIX, StreamGraphGenerator.DEFAULT_SLOT_SHARING_GROUP, 1) .sinkTo(infra.sink()); - runAndWaitForSuccess(infra.env(), infra.source(), infra.sink(), () -> checkDeleteFinished(1L)); + runAndWaitForSuccess( + infra.env(), infra.source(), infra.sink(), () -> checkDeleteFinished(table.name(), 1L)); // Check the metrics Awaitility.await() .untilAsserted( () -> MetricsReporterFactoryForTests.assertCounters( - new ImmutableMap.Builder() + new ImmutableMap.Builder, Long>() .put( - DELETE_FILES_OPERATOR_NAME - + "[0]." - + DUMMY_NAME - + "." - + DELETE_FILE_FAILED_COUNTER, + ImmutableList.of( + DELETE_FILES_OPERATOR_NAME + "[0]", + table.name(), + DUMMY_TASK_NAME, + "0", + DELETE_FILE_FAILED_COUNTER), 0L) .put( - DELETE_FILES_OPERATOR_NAME - + "[0]." - + DUMMY_NAME - + "." - + DELETE_FILE_SUCCEEDED_COUNTER, + ImmutableList.of( + DELETE_FILES_OPERATOR_NAME + "[0]", + table.name(), + DUMMY_TASK_NAME, + "0", + DELETE_FILE_SUCCEEDED_COUNTER), 1L) .build())); } - private static boolean checkDeleteFinished(Long expectedDeleteNum) { + private static boolean checkDeleteFinished(String tableName, Long expectedDeleteNum) { return expectedDeleteNum.equals( MetricsReporterFactoryForTests.counter( - DELETE_FILES_OPERATOR_NAME - + "[0]." - + DUMMY_NAME - + "." - + DELETE_FILE_SUCCEEDED_COUNTER)); + ImmutableList.of( + DELETE_FILES_OPERATOR_NAME + "[0]", + tableName, + DUMMY_TASK_NAME, + "0", + DELETE_FILE_SUCCEEDED_COUNTER))); } } diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/api/TestTableMaintenance.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/api/TestTableMaintenance.java index f4c1f8380e89..0e4a72bd16f8 100644 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/api/TestTableMaintenance.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/api/TestTableMaintenance.java @@ -20,11 +20,10 @@ import static org.apache.iceberg.flink.SimpleDataUtil.createRowData; import static org.apache.iceberg.flink.maintenance.api.TableMaintenance.LOCK_REMOVER_OPERATOR_NAME; -import static org.apache.iceberg.flink.maintenance.api.TableMaintenance.SOURCE_OPERATOR_NAME; +import static org.apache.iceberg.flink.maintenance.api.TableMaintenance.SOURCE_OPERATOR_NAME_PREFIX; import static org.apache.iceberg.flink.maintenance.api.TableMaintenance.TRIGGER_MANAGER_OPERATOR_NAME; 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; @@ -188,37 +187,58 @@ void testMetrics() throws Exception { .until( () -> MetricsReporterFactoryForTests.counter( - LOCK_REMOVER_OPERATOR_NAME + "." + TASKS[0] + "." + SUCCEEDED_TASK_COUNTER) + ImmutableList.of( + LOCK_REMOVER_OPERATOR_NAME, + table.name(), + TASKS[0], + "0", + SUCCEEDED_TASK_COUNTER)) .equals(2L)); MetricsReporterFactoryForTests.assertCounters( - new ImmutableMap.Builder() - .put(LOCK_REMOVER_OPERATOR_NAME + "." + TASKS[0] + "." + SUCCEEDED_TASK_COUNTER, 2L) - .put(LOCK_REMOVER_OPERATOR_NAME + "." + TASKS[0] + "." + FAILED_TASK_COUNTER, 0L) - .put(TRIGGER_MANAGER_OPERATOR_NAME + "." + TASKS[0] + "." + TRIGGERED, 2L) - .put(LOCK_REMOVER_OPERATOR_NAME + "." + TASKS[1] + "." + SUCCEEDED_TASK_COUNTER, 0L) - .put(LOCK_REMOVER_OPERATOR_NAME + "." + TASKS[1] + "." + FAILED_TASK_COUNTER, 1L) - .put(TRIGGER_MANAGER_OPERATOR_NAME + "." + TASKS[1] + "." + TRIGGERED, 1L) + new ImmutableMap.Builder, Long>() .put( - TRIGGER_MANAGER_OPERATOR_NAME - + "." - + GROUP_VALUE_DEFAULT - + "." - + NOTHING_TO_TRIGGER, + ImmutableList.of( + LOCK_REMOVER_OPERATOR_NAME, + table.name(), + TASKS[0], + "0", + SUCCEEDED_TASK_COUNTER), + 2L) + .put( + ImmutableList.of( + LOCK_REMOVER_OPERATOR_NAME, table.name(), TASKS[0], "0", FAILED_TASK_COUNTER), + 0L) + .put( + ImmutableList.of( + TRIGGER_MANAGER_OPERATOR_NAME, table.name(), TASKS[0], "0", TRIGGERED), + 2L) + .put( + ImmutableList.of( + LOCK_REMOVER_OPERATOR_NAME, + table.name(), + TASKS[1], + "1", + SUCCEEDED_TASK_COUNTER), + 0L) + .put( + ImmutableList.of( + LOCK_REMOVER_OPERATOR_NAME, table.name(), TASKS[1], "1", FAILED_TASK_COUNTER), + 1L) + .put( + ImmutableList.of( + TRIGGER_MANAGER_OPERATOR_NAME, table.name(), TASKS[1], "1", TRIGGERED), + 1L) + .put( + ImmutableList.of(TRIGGER_MANAGER_OPERATOR_NAME, table.name(), NOTHING_TO_TRIGGER), -1L) .put( - TRIGGER_MANAGER_OPERATOR_NAME - + "." - + GROUP_VALUE_DEFAULT - + "." - + CONCURRENT_RUN_THROTTLED, + ImmutableList.of( + TRIGGER_MANAGER_OPERATOR_NAME, table.name(), CONCURRENT_RUN_THROTTLED), -1L) .put( - TRIGGER_MANAGER_OPERATOR_NAME - + "." - + GROUP_VALUE_DEFAULT - + "." - + RATE_LIMITER_TRIGGERED, + ImmutableList.of( + TRIGGER_MANAGER_OPERATOR_NAME, table.name(), RATE_LIMITER_TRIGGERED), -1L) .build()); } @@ -382,7 +402,7 @@ private Transformation monitorSource() { // Some checks to make sure this is the transformation we are looking for assertThat(result).isInstanceOf(SourceTransformation.class); - assertThat(result.getName()).isEqualTo(SOURCE_OPERATOR_NAME); + assertThat(result.getName()).startsWith(SOURCE_OPERATOR_NAME_PREFIX); return result; } @@ -405,7 +425,7 @@ DataStream append(DataStream trigger) { return trigger .map(new DummyMaintenanceTask(success)) .name(name) - .uid(uidSuffix() + "-test-mapper-" + name) + .uid(uidSuffix() + "-test-mapper-" + name + "-" + id) .slotSharingGroup(slotSharingGroup()) .forceNonParallel(); } diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/MetricsReporterFactoryForTests.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/MetricsReporterFactoryForTests.java index 7a523035b7fb..ed66ff3df076 100644 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/MetricsReporterFactoryForTests.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/MetricsReporterFactoryForTests.java @@ -21,6 +21,7 @@ import static org.assertj.core.api.Assertions.assertThat; import java.util.Arrays; +import java.util.List; import java.util.Map; import java.util.Properties; import java.util.Set; @@ -38,10 +39,24 @@ public class MetricsReporterFactoryForTests implements MetricReporterFactory { private static final TestMetricsReporter INSTANCE = new TestMetricsReporter(); - private static final Pattern FULL_METRIC_NAME = + private static final Pattern TASK_METRIC_NAME = Pattern.compile( "\\.taskmanager\\.[^.]+\\.[^.]+\\.([^.]+)\\.\\d+\\." + TableMaintenanceMetrics.GROUP_KEY + + "\\." + + TableMaintenanceMetrics.TABLE_NAME_KEY + + "\\.([^.]+)\\." + + TableMaintenanceMetrics.TASK_NAME_KEY + + "\\.([^.]+)\\." + + TableMaintenanceMetrics.TASK_INDEX_KEY + + "\\.([^.]+)\\.([^.]+)"); + + private static final Pattern MAIN_METRIC_NAME = + Pattern.compile( + "\\.taskmanager\\.[^.]+\\.[^.]+\\.([^.]+)\\.\\d+\\." + + TableMaintenanceMetrics.GROUP_KEY + + "\\." + + TableMaintenanceMetrics.TABLE_NAME_KEY + "\\.([^.]+)\\.([^.]+)"); private static Map counters = Maps.newConcurrentMap(); @@ -72,20 +87,26 @@ public static void reset() { gauges = Maps.newConcurrentMap(); } - public static Long counter(String name) { - return counterValues().get(name); + public static Long counter(List parts) { + return counterValues().get(longName(parts)); } - public static Long gauge(String name) { - return gaugeValues().get(name); + public static Long gauge(List parts) { + return gaugeValues().get(longName(parts)); } - public static void assertGauges(Map expected) { - assertThat(filter(gaugeValues(), expected)).isEqualTo(filter(expected, expected)); + public static void assertGauges(Map, Long> expected) { + Map transformed = + expected.entrySet().stream() + .collect(Collectors.toMap(k -> longName(k.getKey()), Map.Entry::getValue)); + assertThat(filter(gaugeValues(), transformed)).isEqualTo(filter(transformed, transformed)); } - public static void assertCounters(Map expected) { - assertThat(filter(counterValues(), expected)).isEqualTo(filter(expected, expected)); + public static void assertCounters(Map, Long> expected) { + Map transformed = + expected.entrySet().stream() + .collect(Collectors.toMap(k -> longName(k.getKey()), Map.Entry::getValue)); + assertThat(filter(counterValues(), transformed)).isEqualTo(filter(transformed, transformed)); } private static Map gaugeValues() { @@ -113,12 +134,30 @@ private static Map filter(Map original, Map parts) { + return parts.stream().map(s -> s.replaceAll("\\.", "_")).collect(Collectors.joining(".")); } private static class TestMetricsReporter implements MetricReporter { diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/OperatorTestBase.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/OperatorTestBase.java index 94c6b4c45500..5e05f40e53cf 100644 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/OperatorTestBase.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/OperatorTestBase.java @@ -73,7 +73,8 @@ public class OperatorTestBase { static final long EVENT_TIME = 10L; static final long EVENT_TIME_2 = 11L; - protected static final String DUMMY_NAME = "dummy"; + protected static final String DUMMY_TASK_NAME = "dummyTask"; + protected static final String DUMMY_TABLE_NAME = "dummyTable"; @RegisterExtension protected static final MiniClusterExtension MINI_CLUSTER_EXTENSION = @@ -91,7 +92,7 @@ public class OperatorTestBase { new HadoopCatalogExtension(TestFixtures.DATABASE, TestFixtures.TABLE); @BeforeEach - void before() throws IOException { + void before() { LOCK_FACTORY.open(); MetricsReporterFactoryForTests.reset(); } diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestDeleteFilesProcessor.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestDeleteFilesProcessor.java index 1160f6bff601..d70c4aafd59a 100644 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestDeleteFilesProcessor.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestDeleteFilesProcessor.java @@ -90,7 +90,7 @@ private void deleteFile(TableLoader tableLoader, String fileName) throws Excepti tableLoader().open(); try (OneInputStreamOperatorTestHarness testHarness = new OneInputStreamOperatorTestHarness<>( - new DeleteFilesProcessor(DUMMY_NAME, tableLoader.loadTable(), 10), + new DeleteFilesProcessor(0, DUMMY_TASK_NAME, tableLoader.loadTable(), 10), StringSerializer.INSTANCE)) { testHarness.open(); testHarness.processElement(fileName, System.currentTimeMillis()); diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestLockRemover.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestLockRemover.java index a6d78ed93682..f75c765df967 100644 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestLockRemover.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestLockRemover.java @@ -26,6 +26,7 @@ import java.io.File; import java.time.Duration; import java.util.Collection; +import java.util.List; import org.apache.flink.api.common.functions.FlatMapFunction; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.connector.sink2.Committer; @@ -46,6 +47,7 @@ import org.apache.flink.util.Collector; import org.apache.iceberg.flink.maintenance.api.TaskResult; import org.apache.iceberg.flink.maintenance.api.TriggerLockFactory; +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; @@ -74,9 +76,9 @@ void testProcess() throws Exception { source .dataStream() .transform( - DUMMY_NAME, + DUMMY_TASK_NAME, TypeInformation.of(Void.class), - new LockRemover(new TestingLockFactory(), Lists.newArrayList(TASKS))) + new LockRemover(DUMMY_TABLE_NAME, new TestingLockFactory(), Lists.newArrayList(TASKS))) .setParallelism(1); JobClient jobClient = null; @@ -131,9 +133,9 @@ void testMetrics() throws Exception { source .dataStream() .transform( - DUMMY_NAME, + DUMMY_TASK_NAME, TypeInformation.of(Void.class), - new LockRemover(new TestingLockFactory(), Lists.newArrayList(TASKS))) + new LockRemover(DUMMY_TABLE_NAME, new TestingLockFactory(), Lists.newArrayList(TASKS))) .setParallelism(1); JobClient jobClient = null; @@ -152,31 +154,57 @@ void testMetrics() throws Exception { .until( () -> MetricsReporterFactoryForTests.counter( - DUMMY_NAME + "." + TASKS[1] + "." + SUCCEEDED_TASK_COUNTER) + ImmutableList.of( + DUMMY_TASK_NAME, + DUMMY_TABLE_NAME, + TASKS[1], + "1", + SUCCEEDED_TASK_COUNTER)) .equals(3L)); // Final check all the counters MetricsReporterFactoryForTests.assertCounters( - new ImmutableMap.Builder() - .put(DUMMY_NAME + "." + TASKS[0] + "." + SUCCEEDED_TASK_COUNTER, 2L) - .put(DUMMY_NAME + "." + TASKS[0] + "." + FAILED_TASK_COUNTER, 1L) - .put(DUMMY_NAME + "." + TASKS[1] + "." + SUCCEEDED_TASK_COUNTER, 3L) - .put(DUMMY_NAME + "." + TASKS[1] + "." + FAILED_TASK_COUNTER, 0L) - .put(DUMMY_NAME + "." + TASKS[2] + "." + SUCCEEDED_TASK_COUNTER, 0L) - .put(DUMMY_NAME + "." + TASKS[2] + "." + FAILED_TASK_COUNTER, 0L) + new ImmutableMap.Builder, Long>() + .put( + ImmutableList.of( + DUMMY_TASK_NAME, DUMMY_TABLE_NAME, TASKS[0], "0", SUCCEEDED_TASK_COUNTER), + 2L) + .put( + ImmutableList.of( + DUMMY_TASK_NAME, DUMMY_TABLE_NAME, TASKS[0], "0", FAILED_TASK_COUNTER), + 1L) + .put( + ImmutableList.of( + DUMMY_TASK_NAME, DUMMY_TABLE_NAME, TASKS[1], "1", SUCCEEDED_TASK_COUNTER), + 3L) + .put( + ImmutableList.of( + DUMMY_TASK_NAME, DUMMY_TABLE_NAME, TASKS[1], "1", FAILED_TASK_COUNTER), + 0L) + .put( + ImmutableList.of( + DUMMY_TASK_NAME, DUMMY_TABLE_NAME, TASKS[2], "2", SUCCEEDED_TASK_COUNTER), + 0L) + .put( + ImmutableList.of( + DUMMY_TASK_NAME, DUMMY_TABLE_NAME, TASKS[2], "2", FAILED_TASK_COUNTER), + 0L) .build()); assertThat( MetricsReporterFactoryForTests.gauge( - DUMMY_NAME + "." + TASKS[0] + "." + LAST_RUN_DURATION_MS)) + ImmutableList.of( + DUMMY_TASK_NAME, DUMMY_TABLE_NAME, TASKS[0], "0", LAST_RUN_DURATION_MS))) .isPositive(); assertThat( MetricsReporterFactoryForTests.gauge( - DUMMY_NAME + "." + TASKS[1] + "." + LAST_RUN_DURATION_MS)) + ImmutableList.of( + DUMMY_TASK_NAME, DUMMY_TABLE_NAME, TASKS[1], "1", LAST_RUN_DURATION_MS))) .isGreaterThan(time); assertThat( MetricsReporterFactoryForTests.gauge( - DUMMY_NAME + "." + TASKS[2] + "." + LAST_RUN_DURATION_MS)) + ImmutableList.of( + DUMMY_TASK_NAME, DUMMY_TABLE_NAME, TASKS[2], "2", LAST_RUN_DURATION_MS))) .isZero(); } finally { closeJobClient(jobClient); @@ -200,9 +228,10 @@ void testRecovery() throws Exception { .dataStream() .union(source2.dataStream()) .transform( - DUMMY_NAME, + DUMMY_TASK_NAME, TypeInformation.of(Void.class), - new LockRemover(new TestingLockFactory(), Lists.newArrayList(TASKS[0]))) + new LockRemover( + DUMMY_TABLE_NAME, new TestingLockFactory(), Lists.newArrayList(TASKS[0]))) .setParallelism(1); JobClient jobClient = null; @@ -220,7 +249,12 @@ void testRecovery() throws Exception { .until( () -> MetricsReporterFactoryForTests.counter( - DUMMY_NAME + "." + TASKS[0] + "." + SUCCEEDED_TASK_COUNTER) + ImmutableList.of( + DUMMY_TASK_NAME, + DUMMY_TABLE_NAME, + TASKS[0], + "0", + SUCCEEDED_TASK_COUNTER)) .equals(2L)); // We did not remove the recovery lock, as no watermark received from the other source @@ -242,20 +276,21 @@ private void processAndCheck(ManualSource source, TaskResult input) private void processAndCheck( ManualSource source, TaskResult input, String counterPrefix) { + List counterKey = + ImmutableList.of( + (counterPrefix != null ? counterPrefix : "") + DUMMY_TASK_NAME, + DUMMY_TABLE_NAME, + TASKS[input.taskIndex()], + String.valueOf(input.taskIndex()), + input.success() ? SUCCEEDED_TASK_COUNTER : FAILED_TASK_COUNTER); + Long counterValue = MetricsReporterFactoryForTests.counter(counterKey); + Long expected = counterValue != null ? counterValue + 1 : 1L; + source.sendRecord(input); source.sendWatermark(input.startEpoch()); - String counterName = - (counterPrefix != null ? counterPrefix : "") - .concat( - input.success() - ? DUMMY_NAME + "." + TASKS[input.taskIndex()] + "." + SUCCEEDED_TASK_COUNTER - : DUMMY_NAME + "." + TASKS[input.taskIndex()] + "." + FAILED_TASK_COUNTER); - Long counterValue = MetricsReporterFactoryForTests.counter(counterName); - Long expected = counterValue != null ? counterValue + 1 : 1L; - Awaitility.await() - .until(() -> expected.equals(MetricsReporterFactoryForTests.counter(counterName))); + .until(() -> expected.equals(MetricsReporterFactoryForTests.counter(counterKey))); } private static class TestingLockFactory implements TriggerLockFactory { @@ -389,9 +424,10 @@ public void flatMap( } }) .transform( - DUMMY_NAME, + DUMMY_TASK_NAME, TypeInformation.of(Void.class), - new LockRemover(new TestingLockFactory(), Lists.newArrayList(TASKS[0]))); + new LockRemover( + DUMMY_TABLE_NAME, new TestingLockFactory(), Lists.newArrayList(TASKS[0]))); } } } diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestTriggerManager.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestTriggerManager.java index 7c3418f942ec..a70d27279460 100644 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestTriggerManager.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestTriggerManager.java @@ -19,7 +19,6 @@ package org.apache.iceberg.flink.maintenance.operator; import static org.apache.iceberg.flink.maintenance.operator.TableMaintenanceMetrics.CONCURRENT_RUN_THROTTLED; -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.TRIGGERED; @@ -36,9 +35,11 @@ import org.apache.flink.streaming.api.operators.KeyedProcessOperator; import org.apache.flink.streaming.util.KeyedOneInputStreamOperatorTestHarness; import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness; +import org.apache.iceberg.Table; import org.apache.iceberg.flink.TableLoader; import org.apache.iceberg.flink.maintenance.api.Trigger; import org.apache.iceberg.flink.maintenance.api.TriggerLockFactory; +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; @@ -54,12 +55,14 @@ class TestTriggerManager extends OperatorTestBase { private long processingTime = 0L; private TriggerLockFactory.Lock lock; private TriggerLockFactory.Lock recoveringLock; + private String tableName; @BeforeEach void before() { - createTable(); + Table table = createTable(); this.lock = LOCK_FACTORY.createLock(); this.recoveringLock = LOCK_FACTORY.createRecoveryLock(); + this.tableName = table.name(); } @Test @@ -421,7 +424,7 @@ void testTriggerMetrics() throws Exception { .dataStream() .keyBy(unused -> true) .process(manager) - .name(DUMMY_NAME) + .name(DUMMY_TASK_NAME) .forceNonParallel() .sinkTo(sink); @@ -437,7 +440,7 @@ void testTriggerMetrics() throws Exception { () -> { Long notingCounter = MetricsReporterFactoryForTests.counter( - DUMMY_NAME + "." + GROUP_VALUE_DEFAULT + "." + NOTHING_TO_TRIGGER); + ImmutableList.of(DUMMY_TASK_NAME, tableName, NOTHING_TO_TRIGGER)); return notingCounter != null && notingCounter.equals(1L); }); @@ -446,7 +449,8 @@ void testTriggerMetrics() throws Exception { // Wait until we receive the trigger assertThat(sink.poll(Duration.ofSeconds(5))).isNotNull(); assertThat( - MetricsReporterFactoryForTests.counter(DUMMY_NAME + "." + TASKS[0] + "." + TRIGGERED)) + MetricsReporterFactoryForTests.counter( + ImmutableList.of(DUMMY_TASK_NAME, tableName, TASKS[0], "0", TRIGGERED))) .isEqualTo(1L); lock.unlock(); @@ -458,20 +462,22 @@ void testTriggerMetrics() throws Exception { assertThat(sink.poll(Duration.ofSeconds(5))).isNotNull(); lock.unlock(); assertThat( - MetricsReporterFactoryForTests.counter(DUMMY_NAME + "." + TASKS[0] + "." + TRIGGERED)) + MetricsReporterFactoryForTests.counter( + ImmutableList.of(DUMMY_TASK_NAME, tableName, TASKS[0], "0", TRIGGERED))) .isEqualTo(2L); assertThat( - MetricsReporterFactoryForTests.counter(DUMMY_NAME + "." + TASKS[1] + "." + TRIGGERED)) + MetricsReporterFactoryForTests.counter( + ImmutableList.of(DUMMY_TASK_NAME, tableName, TASKS[1], "1", TRIGGERED))) .isEqualTo(1L); // Final check all the counters MetricsReporterFactoryForTests.assertCounters( - new ImmutableMap.Builder() - .put(DUMMY_NAME + "." + GROUP_VALUE_DEFAULT + "." + RATE_LIMITER_TRIGGERED, -1L) - .put(DUMMY_NAME + "." + GROUP_VALUE_DEFAULT + "." + CONCURRENT_RUN_THROTTLED, -1L) - .put(DUMMY_NAME + "." + TASKS[0] + "." + TRIGGERED, 2L) - .put(DUMMY_NAME + "." + TASKS[1] + "." + TRIGGERED, 1L) - .put(DUMMY_NAME + "." + GROUP_VALUE_DEFAULT + "." + NOTHING_TO_TRIGGER, 1L) + new ImmutableMap.Builder, Long>() + .put(ImmutableList.of(DUMMY_TASK_NAME, tableName, RATE_LIMITER_TRIGGERED), -1L) + .put(ImmutableList.of(DUMMY_TASK_NAME, tableName, CONCURRENT_RUN_THROTTLED), -1L) + .put(ImmutableList.of(DUMMY_TASK_NAME, tableName, TASKS[0], "0", TRIGGERED), 2L) + .put(ImmutableList.of(DUMMY_TASK_NAME, tableName, TASKS[1], "1", TRIGGERED), 1L) + .put(ImmutableList.of(DUMMY_TASK_NAME, tableName, NOTHING_TO_TRIGGER), 1L) .build()); } finally { closeJobClient(jobClient); @@ -493,7 +499,7 @@ void testRateLimiterMetrics() throws Exception { .dataStream() .keyBy(unused -> true) .process(manager) - .name(DUMMY_NAME) + .name(DUMMY_TASK_NAME) .forceNonParallel() .sinkTo(sink); @@ -514,7 +520,7 @@ void testRateLimiterMetrics() throws Exception { .until( () -> MetricsReporterFactoryForTests.counter( - DUMMY_NAME + "." + GROUP_VALUE_DEFAULT + "." + RATE_LIMITER_TRIGGERED) + ImmutableList.of(DUMMY_TASK_NAME, tableName, RATE_LIMITER_TRIGGERED)) .equals(1L)); // Final check all the counters @@ -539,7 +545,7 @@ void testConcurrentRunMetrics() throws Exception { .dataStream() .keyBy(unused -> true) .process(manager) - .name(DUMMY_NAME) + .name(DUMMY_TASK_NAME) .forceNonParallel() .sinkTo(sink); @@ -557,7 +563,7 @@ void testConcurrentRunMetrics() throws Exception { .until( () -> MetricsReporterFactoryForTests.counter( - DUMMY_NAME + "." + GROUP_VALUE_DEFAULT + "." + CONCURRENT_RUN_THROTTLED) + ImmutableList.of(DUMMY_TASK_NAME, tableName, CONCURRENT_RUN_THROTTLED)) .equals(1L)); // Final check all the counters @@ -577,15 +583,15 @@ private static Stream parametersForTestRecovery() { private void assertCounters(long rateLimiterTrigger, long concurrentRunTrigger) { MetricsReporterFactoryForTests.assertCounters( - new ImmutableMap.Builder() + new ImmutableMap.Builder, Long>() .put( - DUMMY_NAME + "." + GROUP_VALUE_DEFAULT + "." + RATE_LIMITER_TRIGGERED, + ImmutableList.of(DUMMY_TASK_NAME, tableName, RATE_LIMITER_TRIGGERED), rateLimiterTrigger) .put( - DUMMY_NAME + "." + GROUP_VALUE_DEFAULT + "." + CONCURRENT_RUN_THROTTLED, + ImmutableList.of(DUMMY_TASK_NAME, tableName, CONCURRENT_RUN_THROTTLED), concurrentRunTrigger) - .put(DUMMY_NAME + "." + TASKS[0] + "." + TRIGGERED, 1L) - .put(DUMMY_NAME + "." + GROUP_VALUE_DEFAULT + "." + NOTHING_TO_TRIGGER, 0L) + .put(ImmutableList.of(DUMMY_TASK_NAME, tableName, TASKS[0], "0", TRIGGERED), 1L) + .put(ImmutableList.of(DUMMY_TASK_NAME, tableName, NOTHING_TO_TRIGGER), 0L) .build()); } From 2f5f127e9beaabe906446cff69e730d015a53212 Mon Sep 17 00:00:00 2001 From: Peter Vary Date: Fri, 25 Oct 2024 08:33:34 +0200 Subject: [PATCH 12/12] Fix and review --- .../maintenance/api/MaintenanceTaskBuilder.java | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/MaintenanceTaskBuilder.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/MaintenanceTaskBuilder.java index 3dffb59171e3..3fc431d02547 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/MaintenanceTaskBuilder.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/MaintenanceTaskBuilder.java @@ -196,18 +196,18 @@ TriggerEvaluator evaluator() { DataStream append( DataStream sourceStream, int taskIndex, - String taskName, - String tableName, + String newTaskName, + String newTableName, TableLoader newTableLoader, String defaultUidSuffix, String defaultSlotSharingGroup, - int mainParallelism) { - Preconditions.checkNotNull(taskName, "Task name should not be null"); + int defaultParallelism) { + Preconditions.checkNotNull(newTaskName, "Task name should not be null"); Preconditions.checkNotNull(newTableLoader, "TableLoader should not be null"); this.index = taskIndex; - this.taskName = taskName; - this.tableName = tableName; + this.taskName = newTaskName; + this.tableName = newTableName; this.tableLoader = newTableLoader; if (uidSuffix == null) { @@ -215,7 +215,7 @@ DataStream append( } if (parallelism == null) { - parallelism = mainParallelism; + parallelism = defaultParallelism; } if (slotSharingGroup == null) {