mxm commented on code in PR #15042:
URL: https://github.com/apache/iceberg/pull/15042#discussion_r2716706536
##########
flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/TableMaintenance.java:
##########
@@ -81,29 +83,59 @@ public static Builder forChangeStream(
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(null, changeStream, tableLoader, lockFactory);
}
+ /**
+ * Use when the change stream is already provided, like in the {@link
+ * IcebergSink#addPostCommitTopology(DataStream)}.
+ *
+ * @param changeStream the table changes
+ * @param tableLoader used for accessing the table
+ * @return builder for the maintenance stream
+ */
+ @Internal
+ public static Builder forChangeStream(
+ DataStream<TableChange> changeStream, TableLoader tableLoader) {
+ Preconditions.checkNotNull(changeStream, "The change stream should not be
null");
+ Preconditions.checkNotNull(tableLoader, "TableLoader should not be null");
+
+ return new Builder(null, changeStream, tableLoader, null);
+ }
+
/**
* 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
- * @param lockFactory used for preventing concurrent task runs
+ * @param lockFactory used for preventing concurrent task runs,if null, use
coordination lock.
Review Comment:
```suggestion
* @param lockFactory used for preventing concurrent task runs. If null,
use coordination lock.
```
##########
flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/TableMaintenance.java:
##########
@@ -81,29 +83,59 @@ public static Builder forChangeStream(
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(null, changeStream, tableLoader, lockFactory);
}
+ /**
+ * Use when the change stream is already provided, like in the {@link
+ * IcebergSink#addPostCommitTopology(DataStream)}.
+ *
+ * @param changeStream the table changes
+ * @param tableLoader used for accessing the table
+ * @return builder for the maintenance stream
+ */
+ @Internal
+ public static Builder forChangeStream(
+ DataStream<TableChange> changeStream, TableLoader tableLoader) {
+ Preconditions.checkNotNull(changeStream, "The change stream should not be
null");
+ Preconditions.checkNotNull(tableLoader, "TableLoader should not be null");
+
+ return new Builder(null, changeStream, tableLoader, null);
+ }
+
/**
* 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
- * @param lockFactory used for preventing concurrent task runs
+ * @param lockFactory used for preventing concurrent task runs,if null, use
coordination lock.
* @return builder for the maintenance stream
*/
public static Builder forTable(
StreamExecutionEnvironment env, TableLoader tableLoader,
TriggerLockFactory lockFactory) {
Review Comment:
```suggestion
StreamExecutionEnvironment env, TableLoader tableLoader, @Nullable
TriggerLockFactory lockFactory) {
```
##########
flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TriggerManagerOperator.java:
##########
@@ -0,0 +1,318 @@
+/*
+ * 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.List;
+import org.apache.flink.annotation.Experimental;
+import org.apache.flink.annotation.Internal;
+import
org.apache.flink.api.common.operators.ProcessingTimeService.ProcessingTimeCallback;
+import org.apache.flink.api.common.state.ListState;
+import org.apache.flink.api.common.state.ListStateDescriptor;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.common.typeinfo.Types;
+import org.apache.flink.metrics.Counter;
+import org.apache.flink.metrics.MetricGroup;
+import org.apache.flink.runtime.operators.coordination.OperatorEvent;
+import org.apache.flink.runtime.operators.coordination.OperatorEventGateway;
+import org.apache.flink.runtime.operators.coordination.OperatorEventHandler;
+import org.apache.flink.runtime.state.StateInitializationContext;
+import org.apache.flink.runtime.state.StateSnapshotContext;
+import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
+import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
+import org.apache.flink.streaming.api.operators.Output;
+import org.apache.flink.streaming.api.operators.StreamOperatorParameters;
+import org.apache.flink.streaming.api.watermark.Watermark;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.streaming.runtime.tasks.ProcessingTimeService;
+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.Iterables;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * TriggerManagerOperator sends events to the coordinator to acquire a lock,
then waits for the
+ * response. If the response indicates that the lock has been acquired, it
fires a trigger;
+ * otherwise, it schedules the next attempt. When the job recovers from a
failure, tasks from
+ * different execution paths of the previous run may still be running.
Therefore, it first needs to
+ * send a lock with the maximum timestamp, and then send a recovery trigger.
Only after the
+ * downstream removes this lock can we be sure that all tasks have fully
stopped.
+ */
+@Experimental
+@Internal
+public class TriggerManagerOperator extends AbstractStreamOperator<Trigger>
+ implements OneInputStreamOperator<TableChange, Trigger>,
+ OperatorEventHandler,
+ ProcessingTimeCallback {
+ private static final Logger LOG =
LoggerFactory.getLogger(TriggerManagerOperator.class);
+
+ private final OperatorEventGateway operatorEventGateway;
+ private final List<String> maintenanceTaskNames;
+ private final List<TriggerEvaluator> evaluators;
+ private transient Long nextEvaluationTime;
+ private final long minFireDelayMs;
+ private transient List<TableChange> accumulatedChanges;
+ private transient ListState<Long> nextEvaluationTimeState;
+ private transient ListState<TableChange> accumulatedChangesState;
+ private transient ListState<Long> lastTriggerTimesState;
+ private transient Counter rateLimiterTriggeredCounter;
+ private transient Counter concurrentRunThrottledCounter;
+ private transient Counter nothingToTriggerCounter;
+ private transient List<Counter> triggerCounters;
+ private final long lockCheckDelayMs;
+ private transient List<Long> lastTriggerTimes;
+ // To keep the task scheduling fair we keep the last triggered task position
in memory.
+ // If we find a task to trigger, then we run it, but after it is finished,
we start from the given
+ // position to prevent "starvation" of the tasks.
+ // When there is nothing to trigger, we start from the beginning, as the
order of the tasks might
+ // be important (RewriteDataFiles first, and then RewriteManifestFiles later)
+ private transient int startsFrom = 0;
+ private transient boolean triggered = false;
+ private transient Integer taskToStart;
+ private final String lockId;
+ private final String tableName;
+
+ public TriggerManagerOperator(
+ StreamOperatorParameters<Trigger> parameters,
+ OperatorEventGateway operatorEventGateway,
+ List<String> maintenanceTaskNames,
+ List<TriggerEvaluator> evaluators,
+ long minFireDelayMs,
+ long lockCheckDelayMs,
+ String lockId) {
+ super(parameters);
+ Preconditions.checkArgument(
+ maintenanceTaskNames != null && !maintenanceTaskNames.isEmpty(),
+ "Invalid maintenance task names: null or empty");
+ Preconditions.checkArgument(
+ evaluators != null && !evaluators.isEmpty(), "Invalid evaluators: null
or empty");
+ Preconditions.checkArgument(
+ maintenanceTaskNames.size() == evaluators.size(),
+ "Provide a name and evaluator for all of the maintenance tasks");
+ Preconditions.checkArgument(minFireDelayMs > 0, "Minimum fire delay should
be at least 1.");
+ Preconditions.checkArgument(
+ lockCheckDelayMs > 0, "Minimum lock delay rate should be at least 1
ms.");
+
+ this.operatorEventGateway = operatorEventGateway;
+
+ this.maintenanceTaskNames = maintenanceTaskNames;
+ this.evaluators = evaluators;
+ this.minFireDelayMs = minFireDelayMs;
+ this.lockCheckDelayMs = lockCheckDelayMs;
+ this.lockId = lockId;
+ this.tableName = lockId;
+ }
+
+ @Override
+ public void open() throws Exception {
+ super.open();
+ MetricGroup mainGroup =
TableMaintenanceMetrics.groupFor(getRuntimeContext(), tableName);
+ this.rateLimiterTriggeredCounter =
+ mainGroup.counter(TableMaintenanceMetrics.RATE_LIMITER_TRIGGERED);
+ this.concurrentRunThrottledCounter =
+ mainGroup.counter(TableMaintenanceMetrics.CONCURRENT_RUN_THROTTLED);
+ this.nothingToTriggerCounter =
mainGroup.counter(TableMaintenanceMetrics.NOTHING_TO_TRIGGER);
+ this.triggerCounters =
Lists.newArrayListWithExpectedSize(maintenanceTaskNames.size());
+ for (int taskIndex = 0; taskIndex < maintenanceTaskNames.size();
++taskIndex) {
+ triggerCounters.add(
+ TableMaintenanceMetrics.groupFor(
+ mainGroup, maintenanceTaskNames.get(taskIndex), taskIndex)
+ .counter(TableMaintenanceMetrics.TRIGGERED));
+ }
+ }
+
+ @Override
+ public void initializeState(StateInitializationContext context) throws
Exception {
+ super.initializeState(context);
+ this.nextEvaluationTimeState =
+ context
+ .getOperatorStateStore()
+ .getListState(new
ListStateDescriptor<>("triggerManagerNextTriggerTime", Types.LONG));
+
+ this.accumulatedChangesState =
+ context
+ .getOperatorStateStore()
+ .getListState(
+ new ListStateDescriptor<>(
+ "triggerManagerAccumulatedChange",
TypeInformation.of(TableChange.class)));
+
+ this.lastTriggerTimesState =
+ context
+ .getOperatorStateStore()
+ .getListState(new
ListStateDescriptor<>("triggerManagerLastTriggerTime", Types.LONG));
+
+ if (!Iterables.isEmpty(nextEvaluationTimeState.get())) {
+ nextEvaluationTime =
Iterables.getOnlyElement(nextEvaluationTimeState.get());
+ }
+
+ long current = getProcessingTimeService().getCurrentProcessingTime();
+
+ // Initialize from state
+ if (!Iterables.isEmpty(nextEvaluationTimeState.get())) {
+ nextEvaluationTime =
Iterables.getOnlyElement(nextEvaluationTimeState.get());
+ }
+
+ this.accumulatedChanges =
Lists.newArrayList(accumulatedChangesState.get());
+ this.lastTriggerTimes = Lists.newArrayList(lastTriggerTimesState.get());
+
+ // Initialize if the state was empty
+ if (accumulatedChanges.isEmpty()) {
+ for (int i = 0; i < evaluators.size(); ++i) {
+ accumulatedChanges.add(TableChange.empty());
+ lastTriggerTimes.add(current);
+ }
+ }
+
+ if (context.isRestored()) {
+ // When the job state is restored, there could be ongoing tasks.
+ // To prevent collision with the new triggers the following is done:
+ // - add a recovery lock
+ // - fire a recovery trigger
+ // This ensures that the tasks of the previous trigger are executed, and
the lock is removed
+ // in the end. The result of the 'tryLock' is ignored as an already
existing lock prevents
+ // collisions as well.
+ operatorEventGateway.sendEventToCoordinator(new
LockAcquiredEvent(lockId, Long.MAX_VALUE));
Review Comment:
> // When the job state is restored, there could be ongoing tasks.
How exactly is that possible? The maintenance tasks won't be running. Are we
assuming there could be an in-flight trigger message?
##########
flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/TableMaintenance.java:
##########
@@ -226,21 +258,43 @@ public void append() throws IOException {
try (TableLoader loader = tableLoader.clone()) {
loader.open();
String tableName = loader.loadTable().name();
- DataStream<Trigger> triggers =
- DataStreamUtils.reinterpretAsKeyedStream(
- changeStream(tableName, 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()
+ DataStream<Trigger> triggers;
+ if (lockFactory == null) {
+ triggers =
+ DataStreamUtils.reinterpretAsKeyedStream(
+ changeStream(tableName, loader), unused -> true)
+ .transform(
+ TRIGGER_MANAGER_OPERATOR_NAME,
+ TypeInformation.of(Trigger.class),
+ new TriggerManagerOperatorFactory(
+ tableName,
+ taskNames,
+ evaluators,
+ rateLimit.toMillis(),
+ lockCheckDelay.toMillis()))
+ .uid(TRIGGER_MANAGER_OPERATOR_NAME + uidSuffix)
+ .slotSharingGroup(slotSharingGroup)
+ .forceNonParallel();
+ } else {
+ triggers =
+ DataStreamUtils.reinterpretAsKeyedStream(
+ changeStream(tableName, 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();
Review Comment:
Looks like we are duplicating a fair amount of code with this fork. Have we
considered implementing the new functionality with the existing TriggerManager
and LockRemover? Would it be feasible to add coordinators to TriggerManager and
LockRemover and create corresponding LockFactories which interact with the
coordinators?
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]