pvary commented on code in PR #11144: URL: https://github.com/apache/iceberg/pull/11144#discussion_r1762551328
########## 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<TableChange> 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<TableChange> changeStream; + private final TableLoader tableLoader; + private final List<MaintenanceTaskBuilder<?>> 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<TableChange> 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) { Review Comment: Doesn't really makes sense to check the table more frequently than the scheduling rate limit. So I think they should be the same. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org --------------------------------------------------------------------- To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org For additional commands, e-mail: issues-h...@iceberg.apache.org