pvary commented on code in PR #10308: URL: https://github.com/apache/iceberg/pull/10308#discussion_r1610991960
########## flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/MonitorSource.java: ########## @@ -0,0 +1,200 @@ +/* + * 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.IOException; +import java.util.Iterator; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.connector.source.Boundedness; +import org.apache.flink.api.connector.source.SourceReader; +import org.apache.flink.api.connector.source.SourceReaderContext; +import org.apache.flink.api.connector.source.util.ratelimit.RateLimitedSourceReader; +import org.apache.flink.api.connector.source.util.ratelimit.RateLimiter; +import org.apache.flink.api.connector.source.util.ratelimit.RateLimiterStrategy; +import org.apache.flink.core.io.SimpleVersionedSerializer; +import org.apache.flink.core.memory.DataInputDeserializer; +import org.apache.flink.core.memory.DataOutputSerializer; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.Table; +import org.apache.iceberg.flink.TableLoader; +import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; +import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** Monitors an Iceberg table for changes */ +public class MonitorSource extends SingleThreadedIteratorSource<TableChange> { + private static final Logger LOG = LoggerFactory.getLogger(MonitorSource.class); + + private final TableLoader tableLoader; + private final RateLimiterStrategy rateLimiterStrategy; + private final long maxReadBack; + + /** + * Creates a {@link org.apache.flink.api.connector.source.Source} which monitors an Iceberg table + * for changes. + * + * @param tableLoader used for accessing the table + * @param rateLimiterStrategy limits the frequency the table is checked + * @param maxReadBack sets the number of snapshots read before stopping change collection + */ + 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"); + Preconditions.checkArgument(maxReadBack > 0, "Need to read at least 1 snapshot to work"); + + this.tableLoader = tableLoader; + this.rateLimiterStrategy = rateLimiterStrategy; + this.maxReadBack = maxReadBack; + } + + @Override + public Boundedness getBoundedness() { + return Boundedness.CONTINUOUS_UNBOUNDED; + } + + @Override + public TypeInformation<TableChange> getProducedType() { + return TypeInformation.of(TableChange.class); + } + + @Override + Iterator<TableChange> createIterator() { + return new SchedulerEventIterator(tableLoader, null, maxReadBack); + } + + @Override + SimpleVersionedSerializer<Iterator<TableChange>> getIteratorSerializer() { + return new SchedulerEventIteratorSerializer(tableLoader, maxReadBack); + } + + @Override + public SourceReader<TableChange, GlobalSplit<TableChange>> createReader( + SourceReaderContext readerContext) throws Exception { + RateLimiter rateLimiter = rateLimiterStrategy.createRateLimiter(1); + return new RateLimitedSourceReader<>(super.createReader(readerContext), rateLimiter); Review Comment: Here are the requirements: - We need to emit the `TableChange` events for a given timestamp in a single event record - we want to prevent unnecessary Maintenance Tasks scheduling - We need to have state for the source - we want to avoid emitting duplications, even on state restore I have considered the following: - The split is a `Snapshot`, and the enumerator emits snapshots, reader reads the snapshot and emits `TableChange` based on a given snapshot - this breaks our first requirement, as it will generate multiple small `TableChange` events on startup, or when multiple commits arrive between enumeration intervals - The split is basically a `TableChange` calculated by the enumerator, and the reader just emits the value it has received - I think this also breaks the first requirement on state restore. We might emit an old `TableChange` and a new `TableChange` immediately. This could again double trigger our Maintenance Tasks. How sensitive is the Enumerator for long running `TableChange` calculations? Do I miss something, or do you have another idea? Thanks, Peter -- 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