mxm commented on code in PR #15151: URL: https://github.com/apache/iceberg/pull/15151#discussion_r2735788672
########## flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TableMaintenanceCoordinator.java: ########## @@ -0,0 +1,371 @@ +/* + * 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 java.util.Locale; +import java.util.Map; +import java.util.concurrent.Callable; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.ThreadFactory; +import java.util.function.Consumer; +import javax.annotation.Nonnull; +import org.apache.flink.annotation.Experimental; +import org.apache.flink.annotation.Internal; +import org.apache.flink.runtime.operators.coordination.OperatorCoordinator; +import org.apache.flink.runtime.operators.coordination.OperatorEvent; +import org.apache.flink.util.ExceptionUtils; +import org.apache.flink.util.FatalExitExceptionHandler; +import org.apache.flink.util.FlinkRuntimeException; +import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@Experimental +@Internal +public class TableMaintenanceCoordinator implements OperatorCoordinator { + + private static final Logger LOG = LoggerFactory.getLogger(TableMaintenanceCoordinator.class); + + private final String operatorName; + private final Context context; + + private final ExecutorService coordinatorExecutor; + private final CoordinatorExecutorThreadFactory coordinatorThreadFactory; + private transient boolean started; + private final transient SubtaskGateways subtaskGateways; + private static final Map<String, Consumer<LockReleasedEvent>> LOCK_RELEASE_CONSUMERS = + Maps.newConcurrentMap(); + private transient List<LockReleasedEvent> pendingReleaseEvents = Lists.newArrayList(); + + public TableMaintenanceCoordinator(String operatorName, Context context) { + this.operatorName = operatorName; + this.context = context; + + this.coordinatorThreadFactory = + new CoordinatorExecutorThreadFactory( + "TableMaintenanceCoordinator-" + operatorName, context.getUserCodeClassloader()); + this.coordinatorExecutor = Executors.newSingleThreadExecutor(coordinatorThreadFactory); + this.subtaskGateways = new SubtaskGateways(operatorName, context.currentParallelism()); + LOG.info("Created TableMaintenanceCoordinator: {}", operatorName); + } + + @Override + public void start() throws Exception { + LOG.info("Starting TableMaintenanceCoordinator: {}", operatorName); + this.started = true; + } + + @Override + public void close() throws Exception { + coordinatorExecutor.shutdown(); + this.started = false; + LOG.info("Closed TableMaintenanceCoordinator: {}", operatorName); + LOCK_RELEASE_CONSUMERS.clear(); + pendingReleaseEvents.clear(); + } + + @Override + public void handleEventFromOperator(int subtask, int attemptNumber, OperatorEvent event) { + runInCoordinatorThread( + () -> { + LOG.debug( + "Handling event from subtask {} (#{}) of {}: {}", + subtask, + attemptNumber, + operatorName, + event); + if (event instanceof LockRegisterEvent) { + registerTriggerManagerReceiveReleaseEvent((LockRegisterEvent) event); + } else if (event instanceof LockReleasedEvent) { + handleReleaseLock((LockReleasedEvent) event); + } else { + throw new IllegalArgumentException( + "Invalid operator event type: " + event.getClass().getCanonicalName()); + } + }, + String.format( + Locale.ROOT, + "handling operator event %s from subtask %d (#%d)", + event.getClass(), + subtask, + attemptNumber)); + } + + @SuppressWarnings("FutureReturnValueIgnored") + private void registerTriggerManagerReceiveReleaseEvent(LockRegisterEvent lockRegisterEvent) { + LOCK_RELEASE_CONSUMERS.put( + lockRegisterEvent.lockId(), + lock -> { + LOG.info( + "Send release event for lock id {}, timestamp: {} to Operator {}", + lock.lockId(), + lock.timestamp(), + operatorName); + this.subtaskGateways.getSubtaskGateway(0).sendEvent(lock); + }); + + if (!pendingReleaseEvents.isEmpty()) { + pendingReleaseEvents.forEach(this::handleReleaseLock); + pendingReleaseEvents.clear(); + } + } + + /** Release the lock and optionally trigger the next pending task. */ + @VisibleForTesting + void handleReleaseLock(LockReleasedEvent lockReleasedEvent) { + if (LOCK_RELEASE_CONSUMERS.containsKey(lockReleasedEvent.lockId())) { + LOCK_RELEASE_CONSUMERS.get(lockReleasedEvent.lockId()).accept(lockReleasedEvent); + LOG.info( + "Send release event for lock id {}, timestamp: {}", + lockReleasedEvent.lockId(), + lockReleasedEvent.timestamp()); + } else { + pendingReleaseEvents.add(lockReleasedEvent); + LOG.info( + "No consumer for lock id {}, timestamp: {}", + lockReleasedEvent.lockId(), + lockReleasedEvent.timestamp()); + } + } + + @Override + public void checkpointCoordinator(long checkpointId, CompletableFuture<byte[]> resultFuture) { + // We don’t need to track how many locks are currently held, because when recovering from state, + // a `recover lock` will be issued to ensure all tasks finish running and then release all + // locks. + // The `TriggerManagerOperator` already keeps the `TableChange` state and related information, + // so there’s no need to store additional state here. + runInCoordinatorThread( + () -> { + resultFuture.complete(new byte[0]); + }, + String.format(Locale.ROOT, "taking checkpoint %d", checkpointId)); + } Review Comment: Why don't we checkpoint all the locks? This will give us a precise snapshot of the current locking state. That way, we could also get rid of the recovery lock. ########## flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TableMaintenanceCoordinator.java: ########## @@ -0,0 +1,371 @@ +/* + * 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 java.util.Locale; +import java.util.Map; +import java.util.concurrent.Callable; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.ThreadFactory; +import java.util.function.Consumer; +import javax.annotation.Nonnull; +import org.apache.flink.annotation.Experimental; +import org.apache.flink.annotation.Internal; +import org.apache.flink.runtime.operators.coordination.OperatorCoordinator; +import org.apache.flink.runtime.operators.coordination.OperatorEvent; +import org.apache.flink.util.ExceptionUtils; +import org.apache.flink.util.FatalExitExceptionHandler; +import org.apache.flink.util.FlinkRuntimeException; +import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@Experimental +@Internal +public class TableMaintenanceCoordinator implements OperatorCoordinator { + + private static final Logger LOG = LoggerFactory.getLogger(TableMaintenanceCoordinator.class); + + private final String operatorName; + private final Context context; + + private final ExecutorService coordinatorExecutor; + private final CoordinatorExecutorThreadFactory coordinatorThreadFactory; + private transient boolean started; Review Comment: ```suggestion private boolean started; ``` Same here, `transient` isn't required. ########## flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TableMaintenanceCoordinator.java: ########## @@ -0,0 +1,371 @@ +/* + * 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 java.util.Locale; +import java.util.Map; +import java.util.concurrent.Callable; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.ThreadFactory; +import java.util.function.Consumer; +import javax.annotation.Nonnull; +import org.apache.flink.annotation.Experimental; +import org.apache.flink.annotation.Internal; +import org.apache.flink.runtime.operators.coordination.OperatorCoordinator; +import org.apache.flink.runtime.operators.coordination.OperatorEvent; +import org.apache.flink.util.ExceptionUtils; +import org.apache.flink.util.FatalExitExceptionHandler; +import org.apache.flink.util.FlinkRuntimeException; +import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@Experimental +@Internal +public class TableMaintenanceCoordinator implements OperatorCoordinator { + + private static final Logger LOG = LoggerFactory.getLogger(TableMaintenanceCoordinator.class); + + private final String operatorName; + private final Context context; + + private final ExecutorService coordinatorExecutor; + private final CoordinatorExecutorThreadFactory coordinatorThreadFactory; + private transient boolean started; + private final transient SubtaskGateways subtaskGateways; + private static final Map<String, Consumer<LockReleasedEvent>> LOCK_RELEASE_CONSUMERS = + Maps.newConcurrentMap(); + private transient List<LockReleasedEvent> pendingReleaseEvents = Lists.newArrayList(); Review Comment: I think we should use `final` here instead. ```suggestion private final List<LockReleasedEvent> pendingReleaseEvents = Lists.newArrayList(); ``` ########## flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TableMaintenanceCoordinator.java: ########## @@ -0,0 +1,371 @@ +/* + * 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 java.util.Locale; +import java.util.Map; +import java.util.concurrent.Callable; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.ThreadFactory; +import java.util.function.Consumer; +import javax.annotation.Nonnull; +import org.apache.flink.annotation.Experimental; +import org.apache.flink.annotation.Internal; +import org.apache.flink.runtime.operators.coordination.OperatorCoordinator; +import org.apache.flink.runtime.operators.coordination.OperatorEvent; +import org.apache.flink.util.ExceptionUtils; +import org.apache.flink.util.FatalExitExceptionHandler; +import org.apache.flink.util.FlinkRuntimeException; +import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@Experimental +@Internal +public class TableMaintenanceCoordinator implements OperatorCoordinator { + + private static final Logger LOG = LoggerFactory.getLogger(TableMaintenanceCoordinator.class); + + private final String operatorName; + private final Context context; + + private final ExecutorService coordinatorExecutor; + private final CoordinatorExecutorThreadFactory coordinatorThreadFactory; + private transient boolean started; + private final transient SubtaskGateways subtaskGateways; + private static final Map<String, Consumer<LockReleasedEvent>> LOCK_RELEASE_CONSUMERS = + Maps.newConcurrentMap(); + private transient List<LockReleasedEvent> pendingReleaseEvents = Lists.newArrayList(); + + public TableMaintenanceCoordinator(String operatorName, Context context) { + this.operatorName = operatorName; + this.context = context; + + this.coordinatorThreadFactory = + new CoordinatorExecutorThreadFactory( + "TableMaintenanceCoordinator-" + operatorName, context.getUserCodeClassloader()); + this.coordinatorExecutor = Executors.newSingleThreadExecutor(coordinatorThreadFactory); + this.subtaskGateways = new SubtaskGateways(operatorName, context.currentParallelism()); + LOG.info("Created TableMaintenanceCoordinator: {}", operatorName); + } + + @Override + public void start() throws Exception { + LOG.info("Starting TableMaintenanceCoordinator: {}", operatorName); + this.started = true; + } + + @Override + public void close() throws Exception { + coordinatorExecutor.shutdown(); + this.started = false; + LOG.info("Closed TableMaintenanceCoordinator: {}", operatorName); + LOCK_RELEASE_CONSUMERS.clear(); + pendingReleaseEvents.clear(); + } + + @Override + public void handleEventFromOperator(int subtask, int attemptNumber, OperatorEvent event) { + runInCoordinatorThread( + () -> { + LOG.debug( + "Handling event from subtask {} (#{}) of {}: {}", + subtask, + attemptNumber, + operatorName, + event); + if (event instanceof LockRegisterEvent) { + registerTriggerManagerReceiveReleaseEvent((LockRegisterEvent) event); + } else if (event instanceof LockReleasedEvent) { + handleReleaseLock((LockReleasedEvent) event); + } else { + throw new IllegalArgumentException( + "Invalid operator event type: " + event.getClass().getCanonicalName()); + } + }, + String.format( + Locale.ROOT, + "handling operator event %s from subtask %d (#%d)", + event.getClass(), + subtask, + attemptNumber)); + } + + @SuppressWarnings("FutureReturnValueIgnored") + private void registerTriggerManagerReceiveReleaseEvent(LockRegisterEvent lockRegisterEvent) { + LOCK_RELEASE_CONSUMERS.put( Review Comment: Have we considered a different flow, where we confirm the lock registration? Why are we assuming that every lock register event succeeds? ########## flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TriggerManagerOperator.java: ########## @@ -0,0 +1,328 @@ +/* + * 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.annotation.VisibleForTesting; +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.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 List<String> maintenanceTaskNames; + private final List<TriggerEvaluator> evaluators; + private transient Long nextEvaluationTime; + private final long minFireDelayMs; + private final OperatorEventGateway operatorEventGateway; + 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 final String tableName; + private transient boolean restoreTasks = false; + private transient boolean lockHeld = false; + + public TriggerManagerOperator( + StreamOperatorParameters<Trigger> parameters, + OperatorEventGateway operatorEventGateway, + List<String> maintenanceTaskNames, + List<TriggerEvaluator> evaluators, + long minFireDelayMs, + long lockCheckDelayMs, + String tableName) { + 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.maintenanceTaskNames = maintenanceTaskNames; + this.evaluators = evaluators; + this.minFireDelayMs = minFireDelayMs; + this.lockCheckDelayMs = lockCheckDelayMs; + this.tableName = tableName; + this.operatorEventGateway = operatorEventGateway; + } + + @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)); + + 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); + } + } + + // register the lock register event + operatorEventGateway.sendEventToCoordinator(new LockRegisterEvent(tableName, current)); Review Comment: AFAIK We register once on state restore, but never again, even after the lock has been released again. Am I missing something here? ########## flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TableMaintenanceCoordinator.java: ########## @@ -0,0 +1,371 @@ +/* + * 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 java.util.Locale; +import java.util.Map; +import java.util.concurrent.Callable; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.ThreadFactory; +import java.util.function.Consumer; +import javax.annotation.Nonnull; +import org.apache.flink.annotation.Experimental; +import org.apache.flink.annotation.Internal; +import org.apache.flink.runtime.operators.coordination.OperatorCoordinator; +import org.apache.flink.runtime.operators.coordination.OperatorEvent; +import org.apache.flink.util.ExceptionUtils; +import org.apache.flink.util.FatalExitExceptionHandler; +import org.apache.flink.util.FlinkRuntimeException; +import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@Experimental +@Internal +public class TableMaintenanceCoordinator implements OperatorCoordinator { + + private static final Logger LOG = LoggerFactory.getLogger(TableMaintenanceCoordinator.class); + + private final String operatorName; + private final Context context; + + private final ExecutorService coordinatorExecutor; + private final CoordinatorExecutorThreadFactory coordinatorThreadFactory; + private transient boolean started; + private final transient SubtaskGateways subtaskGateways; + private static final Map<String, Consumer<LockReleasedEvent>> LOCK_RELEASE_CONSUMERS = + Maps.newConcurrentMap(); + private transient List<LockReleasedEvent> pendingReleaseEvents = Lists.newArrayList(); + + public TableMaintenanceCoordinator(String operatorName, Context context) { + this.operatorName = operatorName; + this.context = context; + + this.coordinatorThreadFactory = + new CoordinatorExecutorThreadFactory( + "TableMaintenanceCoordinator-" + operatorName, context.getUserCodeClassloader()); + this.coordinatorExecutor = Executors.newSingleThreadExecutor(coordinatorThreadFactory); + this.subtaskGateways = new SubtaskGateways(operatorName, context.currentParallelism()); + LOG.info("Created TableMaintenanceCoordinator: {}", operatorName); + } + + @Override + public void start() throws Exception { + LOG.info("Starting TableMaintenanceCoordinator: {}", operatorName); + this.started = true; + } + + @Override + public void close() throws Exception { + coordinatorExecutor.shutdown(); + this.started = false; + LOG.info("Closed TableMaintenanceCoordinator: {}", operatorName); + LOCK_RELEASE_CONSUMERS.clear(); + pendingReleaseEvents.clear(); + } + + @Override + public void handleEventFromOperator(int subtask, int attemptNumber, OperatorEvent event) { + runInCoordinatorThread( + () -> { + LOG.debug( + "Handling event from subtask {} (#{}) of {}: {}", + subtask, + attemptNumber, + operatorName, + event); + if (event instanceof LockRegisterEvent) { + registerTriggerManagerReceiveReleaseEvent((LockRegisterEvent) event); + } else if (event instanceof LockReleasedEvent) { + handleReleaseLock((LockReleasedEvent) event); + } else { + throw new IllegalArgumentException( + "Invalid operator event type: " + event.getClass().getCanonicalName()); + } + }, + String.format( + Locale.ROOT, + "handling operator event %s from subtask %d (#%d)", + event.getClass(), + subtask, + attemptNumber)); + } + + @SuppressWarnings("FutureReturnValueIgnored") + private void registerTriggerManagerReceiveReleaseEvent(LockRegisterEvent lockRegisterEvent) { + LOCK_RELEASE_CONSUMERS.put( Review Comment: There is classloader-level isolation between coordinators from different jobs, which means that there will be multiple instances of the static field because every job will load this class through its user class loader. +1 for checking for existing consumers though. Is the locking meant to be re-entrant? Even so, we should check that its actually re-entrant and not coming from a different subtask. ########## flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TableMaintenanceCoordinator.java: ########## @@ -0,0 +1,371 @@ +/* + * 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 java.util.Locale; +import java.util.Map; +import java.util.concurrent.Callable; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.ThreadFactory; +import java.util.function.Consumer; +import javax.annotation.Nonnull; +import org.apache.flink.annotation.Experimental; +import org.apache.flink.annotation.Internal; +import org.apache.flink.runtime.operators.coordination.OperatorCoordinator; +import org.apache.flink.runtime.operators.coordination.OperatorEvent; +import org.apache.flink.util.ExceptionUtils; +import org.apache.flink.util.FatalExitExceptionHandler; +import org.apache.flink.util.FlinkRuntimeException; +import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@Experimental +@Internal +public class TableMaintenanceCoordinator implements OperatorCoordinator { + + private static final Logger LOG = LoggerFactory.getLogger(TableMaintenanceCoordinator.class); + + private final String operatorName; + private final Context context; + + private final ExecutorService coordinatorExecutor; + private final CoordinatorExecutorThreadFactory coordinatorThreadFactory; + private transient boolean started; + private final transient SubtaskGateways subtaskGateways; + private static final Map<String, Consumer<LockReleasedEvent>> LOCK_RELEASE_CONSUMERS = + Maps.newConcurrentMap(); + private transient List<LockReleasedEvent> pendingReleaseEvents = Lists.newArrayList(); + + public TableMaintenanceCoordinator(String operatorName, Context context) { + this.operatorName = operatorName; + this.context = context; + + this.coordinatorThreadFactory = + new CoordinatorExecutorThreadFactory( + "TableMaintenanceCoordinator-" + operatorName, context.getUserCodeClassloader()); + this.coordinatorExecutor = Executors.newSingleThreadExecutor(coordinatorThreadFactory); + this.subtaskGateways = new SubtaskGateways(operatorName, context.currentParallelism()); + LOG.info("Created TableMaintenanceCoordinator: {}", operatorName); + } + + @Override + public void start() throws Exception { + LOG.info("Starting TableMaintenanceCoordinator: {}", operatorName); + this.started = true; + } + + @Override + public void close() throws Exception { + coordinatorExecutor.shutdown(); + this.started = false; + LOG.info("Closed TableMaintenanceCoordinator: {}", operatorName); + LOCK_RELEASE_CONSUMERS.clear(); + pendingReleaseEvents.clear(); + } + + @Override + public void handleEventFromOperator(int subtask, int attemptNumber, OperatorEvent event) { + runInCoordinatorThread( + () -> { + LOG.debug( + "Handling event from subtask {} (#{}) of {}: {}", + subtask, + attemptNumber, + operatorName, + event); + if (event instanceof LockRegisterEvent) { + registerTriggerManagerReceiveReleaseEvent((LockRegisterEvent) event); + } else if (event instanceof LockReleasedEvent) { + handleReleaseLock((LockReleasedEvent) event); + } else { + throw new IllegalArgumentException( + "Invalid operator event type: " + event.getClass().getCanonicalName()); + } + }, + String.format( + Locale.ROOT, + "handling operator event %s from subtask %d (#%d)", + event.getClass(), + subtask, + attemptNumber)); + } + + @SuppressWarnings("FutureReturnValueIgnored") + private void registerTriggerManagerReceiveReleaseEvent(LockRegisterEvent lockRegisterEvent) { Review Comment: I don't understand the method name. This method handles `lockRegisterEvent`. Would something like `registerLock(LockRegisterEvent event)` be better? ########## flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TableMaintenanceCoordinator.java: ########## @@ -0,0 +1,371 @@ +/* + * 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 java.util.Locale; +import java.util.Map; +import java.util.concurrent.Callable; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.ThreadFactory; +import java.util.function.Consumer; +import javax.annotation.Nonnull; +import org.apache.flink.annotation.Experimental; +import org.apache.flink.annotation.Internal; +import org.apache.flink.runtime.operators.coordination.OperatorCoordinator; +import org.apache.flink.runtime.operators.coordination.OperatorEvent; +import org.apache.flink.util.ExceptionUtils; +import org.apache.flink.util.FatalExitExceptionHandler; +import org.apache.flink.util.FlinkRuntimeException; +import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@Experimental +@Internal +public class TableMaintenanceCoordinator implements OperatorCoordinator { + + private static final Logger LOG = LoggerFactory.getLogger(TableMaintenanceCoordinator.class); + + private final String operatorName; + private final Context context; + + private final ExecutorService coordinatorExecutor; + private final CoordinatorExecutorThreadFactory coordinatorThreadFactory; + private transient boolean started; + private final transient SubtaskGateways subtaskGateways; + private static final Map<String, Consumer<LockReleasedEvent>> LOCK_RELEASE_CONSUMERS = + Maps.newConcurrentMap(); + private transient List<LockReleasedEvent> pendingReleaseEvents = Lists.newArrayList(); + + public TableMaintenanceCoordinator(String operatorName, Context context) { + this.operatorName = operatorName; + this.context = context; + + this.coordinatorThreadFactory = + new CoordinatorExecutorThreadFactory( + "TableMaintenanceCoordinator-" + operatorName, context.getUserCodeClassloader()); + this.coordinatorExecutor = Executors.newSingleThreadExecutor(coordinatorThreadFactory); + this.subtaskGateways = new SubtaskGateways(operatorName, context.currentParallelism()); + LOG.info("Created TableMaintenanceCoordinator: {}", operatorName); + } + + @Override + public void start() throws Exception { + LOG.info("Starting TableMaintenanceCoordinator: {}", operatorName); + this.started = true; + } + + @Override + public void close() throws Exception { + coordinatorExecutor.shutdown(); + this.started = false; + LOG.info("Closed TableMaintenanceCoordinator: {}", operatorName); + LOCK_RELEASE_CONSUMERS.clear(); + pendingReleaseEvents.clear(); + } + + @Override + public void handleEventFromOperator(int subtask, int attemptNumber, OperatorEvent event) { + runInCoordinatorThread( + () -> { + LOG.debug( + "Handling event from subtask {} (#{}) of {}: {}", + subtask, + attemptNumber, + operatorName, + event); + if (event instanceof LockRegisterEvent) { + registerTriggerManagerReceiveReleaseEvent((LockRegisterEvent) event); + } else if (event instanceof LockReleasedEvent) { + handleReleaseLock((LockReleasedEvent) event); + } else { + throw new IllegalArgumentException( + "Invalid operator event type: " + event.getClass().getCanonicalName()); + } + }, + String.format( + Locale.ROOT, + "handling operator event %s from subtask %d (#%d)", + event.getClass(), + subtask, + attemptNumber)); + } + + @SuppressWarnings("FutureReturnValueIgnored") + private void registerTriggerManagerReceiveReleaseEvent(LockRegisterEvent lockRegisterEvent) { + LOCK_RELEASE_CONSUMERS.put( + lockRegisterEvent.lockId(), + lock -> { + LOG.info( + "Send release event for lock id {}, timestamp: {} to Operator {}", + lock.lockId(), + lock.timestamp(), + operatorName); + this.subtaskGateways.getSubtaskGateway(0).sendEvent(lock); + }); + + if (!pendingReleaseEvents.isEmpty()) { + pendingReleaseEvents.forEach(this::handleReleaseLock); + pendingReleaseEvents.clear(); + } Review Comment: AFAIK The check isn't required. ```suggestion pendingReleaseEvents.forEach(this::handleReleaseLock); pendingReleaseEvents.clear(); ``` ########## flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TableMaintenanceCoordinator.java: ########## @@ -0,0 +1,371 @@ +/* + * 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 java.util.Locale; +import java.util.Map; +import java.util.concurrent.Callable; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.ThreadFactory; +import java.util.function.Consumer; +import javax.annotation.Nonnull; +import org.apache.flink.annotation.Experimental; +import org.apache.flink.annotation.Internal; +import org.apache.flink.runtime.operators.coordination.OperatorCoordinator; +import org.apache.flink.runtime.operators.coordination.OperatorEvent; +import org.apache.flink.util.ExceptionUtils; +import org.apache.flink.util.FatalExitExceptionHandler; +import org.apache.flink.util.FlinkRuntimeException; +import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@Experimental +@Internal +public class TableMaintenanceCoordinator implements OperatorCoordinator { + + private static final Logger LOG = LoggerFactory.getLogger(TableMaintenanceCoordinator.class); + + private final String operatorName; + private final Context context; + + private final ExecutorService coordinatorExecutor; + private final CoordinatorExecutorThreadFactory coordinatorThreadFactory; + private transient boolean started; + private final transient SubtaskGateways subtaskGateways; + private static final Map<String, Consumer<LockReleasedEvent>> LOCK_RELEASE_CONSUMERS = + Maps.newConcurrentMap(); + private transient List<LockReleasedEvent> pendingReleaseEvents = Lists.newArrayList(); Review Comment: ```suggestion private List<LockReleasedEvent> pendingReleaseEvents = Lists.newArrayList(); ``` `transient` doesn't make sense in this context. This class is not serializable. -- 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]
