stevenzwu commented on code in PR #11010: URL: https://github.com/apache/iceberg/pull/11010#discussion_r1731685941
########## flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/LockRemover.java: ########## @@ -0,0 +1,120 @@ +/* + * 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.Map; +import java.util.concurrent.atomic.AtomicLong; +import org.apache.flink.annotation.Internal; +import org.apache.flink.metrics.Counter; +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.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** Manages locks and collect {@link org.apache.flink.metrics.Metric} for the Maintenance Tasks. */ +@Internal +public class LockRemover extends AbstractStreamOperator<MaintenanceResult> + implements OneInputStreamOperator<TaskResult, MaintenanceResult> { + private static final Logger LOG = LoggerFactory.getLogger(LockRemover.class); + + private final TriggerLockFactory lockFactory; + private final List<String> taskNames; + + private transient Map<Integer, Counter> successfulStreamResultCounterMap; + private transient Map<Integer, Counter> failedStreamResultCounterMap; + private transient Map<Integer, AtomicLong> lastRunLength; + private transient TriggerLockFactory.Lock lock; + private transient TriggerLockFactory.Lock recoveryLock; + private transient long lastProcessed = 0L; + + public LockRemover(TriggerLockFactory lockFactory, List<String> taskNames) { + Preconditions.checkNotNull(lockFactory, "Lock factory should no be null"); + Preconditions.checkArgument( + taskNames != null && !taskNames.isEmpty(), "Invalid task names: null or empty"); + + this.lockFactory = lockFactory; + this.taskNames = taskNames; + } + + @Override + public void open() throws Exception { + super.open(); + this.successfulStreamResultCounterMap = Maps.newHashMapWithExpectedSize(taskNames.size()); + this.failedStreamResultCounterMap = Maps.newHashMapWithExpectedSize(taskNames.size()); + this.lastRunLength = Maps.newHashMapWithExpectedSize(taskNames.size()); + for (int i = 0; i < taskNames.size(); ++i) { + String name = taskNames.get(i); + successfulStreamResultCounterMap.put( + i, + getRuntimeContext() + .getMetricGroup() + .addGroup(TableMaintenanceMetrics.GROUP_KEY, name) + .counter(TableMaintenanceMetrics.SUCCESSFUL_STREAM_COUNTER)); + failedStreamResultCounterMap.put( + i, + getRuntimeContext() + .getMetricGroup() + .addGroup(TableMaintenanceMetrics.GROUP_KEY, name) + .counter(TableMaintenanceMetrics.FAILED_STREAM_COUNTER)); + AtomicLong length = new AtomicLong(0); + lastRunLength.put(i, length); + getRuntimeContext() + .getMetricGroup() + .addGroup(TableMaintenanceMetrics.GROUP_KEY, name) + .gauge(TableMaintenanceMetrics.LAST_RUN_LENGTH, length::get); + } + + this.lock = lockFactory.createLock(); + this.recoveryLock = lockFactory.createRecoveryLock(); + } + + @Override + public void processElement(StreamRecord<TaskResult> element) { + TaskResult value = element.getValue(); + LOG.debug("TaskResult {} arrived", value); + long length = System.currentTimeMillis() - value.startEpoch(); + output.collect( + new StreamRecord<>( + new MaintenanceResult( + value.startEpoch(), value.taskId(), length, value.success(), value.exceptions()))); + lock.unlock(); + this.lastProcessed = value.startEpoch(); + + // Update the metrics + lastRunLength.get(value.taskId()).set(length); Review Comment: it seems that `duration` and `lastRunDuration` is more accurate ########## flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/LockRemover.java: ########## @@ -0,0 +1,120 @@ +/* + * 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.Map; +import java.util.concurrent.atomic.AtomicLong; +import org.apache.flink.annotation.Internal; +import org.apache.flink.metrics.Counter; +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.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** Manages locks and collect {@link org.apache.flink.metrics.Metric} for the Maintenance Tasks. */ +@Internal +public class LockRemover extends AbstractStreamOperator<MaintenanceResult> + implements OneInputStreamOperator<TaskResult, MaintenanceResult> { + private static final Logger LOG = LoggerFactory.getLogger(LockRemover.class); + + private final TriggerLockFactory lockFactory; + private final List<String> taskNames; + + private transient Map<Integer, Counter> successfulStreamResultCounterMap; + private transient Map<Integer, Counter> failedStreamResultCounterMap; + private transient Map<Integer, AtomicLong> lastRunLength; + private transient TriggerLockFactory.Lock lock; + private transient TriggerLockFactory.Lock recoveryLock; + private transient long lastProcessed = 0L; + + public LockRemover(TriggerLockFactory lockFactory, List<String> taskNames) { + Preconditions.checkNotNull(lockFactory, "Lock factory should no be null"); + Preconditions.checkArgument( + taskNames != null && !taskNames.isEmpty(), "Invalid task names: null or empty"); + + this.lockFactory = lockFactory; + this.taskNames = taskNames; + } + + @Override + public void open() throws Exception { + super.open(); + this.successfulStreamResultCounterMap = Maps.newHashMapWithExpectedSize(taskNames.size()); + this.failedStreamResultCounterMap = Maps.newHashMapWithExpectedSize(taskNames.size()); + this.lastRunLength = Maps.newHashMapWithExpectedSize(taskNames.size()); + for (int i = 0; i < taskNames.size(); ++i) { + String name = taskNames.get(i); + successfulStreamResultCounterMap.put( + i, + getRuntimeContext() + .getMetricGroup() + .addGroup(TableMaintenanceMetrics.GROUP_KEY, name) + .counter(TableMaintenanceMetrics.SUCCESSFUL_STREAM_COUNTER)); + failedStreamResultCounterMap.put( + i, + getRuntimeContext() + .getMetricGroup() + .addGroup(TableMaintenanceMetrics.GROUP_KEY, name) + .counter(TableMaintenanceMetrics.FAILED_STREAM_COUNTER)); + AtomicLong length = new AtomicLong(0); + lastRunLength.put(i, length); + getRuntimeContext() + .getMetricGroup() + .addGroup(TableMaintenanceMetrics.GROUP_KEY, name) + .gauge(TableMaintenanceMetrics.LAST_RUN_LENGTH, length::get); + } + + this.lock = lockFactory.createLock(); + this.recoveryLock = lockFactory.createRecoveryLock(); + } + + @Override + public void processElement(StreamRecord<TaskResult> element) { + TaskResult value = element.getValue(); + LOG.debug("TaskResult {} arrived", value); + long length = System.currentTimeMillis() - value.startEpoch(); + output.collect( + new StreamRecord<>( + new MaintenanceResult( + value.startEpoch(), value.taskId(), length, value.success(), value.exceptions()))); + lock.unlock(); + this.lastProcessed = value.startEpoch(); + + // Update the metrics + lastRunLength.get(value.taskId()).set(length); + if (value.success()) { + successfulStreamResultCounterMap.get(value.taskId()).inc(); Review Comment: so `taskId` is more like the list index of the `taskNames`? it can easily get confused with Flink subtask/task id. ########## flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/LockRemover.java: ########## @@ -0,0 +1,120 @@ +/* + * 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.Map; +import java.util.concurrent.atomic.AtomicLong; +import org.apache.flink.annotation.Internal; +import org.apache.flink.metrics.Counter; +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.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** Manages locks and collect {@link org.apache.flink.metrics.Metric} for the Maintenance Tasks. */ +@Internal +public class LockRemover extends AbstractStreamOperator<MaintenanceResult> + implements OneInputStreamOperator<TaskResult, MaintenanceResult> { + private static final Logger LOG = LoggerFactory.getLogger(LockRemover.class); + + private final TriggerLockFactory lockFactory; + private final List<String> taskNames; + + private transient Map<Integer, Counter> successfulStreamResultCounterMap; + private transient Map<Integer, Counter> failedStreamResultCounterMap; + private transient Map<Integer, AtomicLong> lastRunLength; + private transient TriggerLockFactory.Lock lock; + private transient TriggerLockFactory.Lock recoveryLock; + private transient long lastProcessed = 0L; + + public LockRemover(TriggerLockFactory lockFactory, List<String> taskNames) { + Preconditions.checkNotNull(lockFactory, "Lock factory should no be null"); + Preconditions.checkArgument( + taskNames != null && !taskNames.isEmpty(), "Invalid task names: null or empty"); + + this.lockFactory = lockFactory; + this.taskNames = taskNames; + } + + @Override + public void open() throws Exception { + super.open(); + this.successfulStreamResultCounterMap = Maps.newHashMapWithExpectedSize(taskNames.size()); + this.failedStreamResultCounterMap = Maps.newHashMapWithExpectedSize(taskNames.size()); + this.lastRunLength = Maps.newHashMapWithExpectedSize(taskNames.size()); + for (int i = 0; i < taskNames.size(); ++i) { + String name = taskNames.get(i); + successfulStreamResultCounterMap.put( + i, + getRuntimeContext() + .getMetricGroup() + .addGroup(TableMaintenanceMetrics.GROUP_KEY, name) + .counter(TableMaintenanceMetrics.SUCCESSFUL_STREAM_COUNTER)); + failedStreamResultCounterMap.put( + i, + getRuntimeContext() + .getMetricGroup() + .addGroup(TableMaintenanceMetrics.GROUP_KEY, name) + .counter(TableMaintenanceMetrics.FAILED_STREAM_COUNTER)); + AtomicLong length = new AtomicLong(0); + lastRunLength.put(i, length); + getRuntimeContext() + .getMetricGroup() + .addGroup(TableMaintenanceMetrics.GROUP_KEY, name) + .gauge(TableMaintenanceMetrics.LAST_RUN_LENGTH, length::get); + } + + this.lock = lockFactory.createLock(); + this.recoveryLock = lockFactory.createRecoveryLock(); + } + + @Override + public void processElement(StreamRecord<TaskResult> element) { + TaskResult value = element.getValue(); + LOG.debug("TaskResult {} arrived", value); + long length = System.currentTimeMillis() - value.startEpoch(); + output.collect( + new StreamRecord<>( + new MaintenanceResult( + value.startEpoch(), value.taskId(), length, value.success(), value.exceptions()))); Review Comment: so `MaintenanceResult` is `TaskResult` plus the run duration? should `MaintenanceResult` extends from `TaskResult`? Also can the upstream maintenance task calculates the `duration` before emitting the `TaskResult` to `LockRemover`? what is the downstream operator of this one? what does it do with the `MaintenanceResult`? ########## flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/MaintenanceResult.java: ########## @@ -0,0 +1,93 @@ +/* + * 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.Objects; +import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; + +/** The output of the Maintenance Flow. */ +public class MaintenanceResult { + private final long startEpoch; + private final int taskId; + private final long length; + private final boolean success; + private final List<Exception> exceptions; + + public MaintenanceResult( + long startEpoch, int taskId, long length, boolean success, List<Exception> exceptions) { + this.startEpoch = startEpoch; + this.taskId = taskId; + this.length = length; + this.success = success; + this.exceptions = exceptions; + } + + public long startEpoch() { + return startEpoch; + } + + public int taskId() { + return taskId; + } + + public long length() { + return length; + } + + public boolean success() { + return success; + } + + public List<Exception> exceptions() { + return exceptions; + } + + @Override + public String toString() { + return MoreObjects.toStringHelper(this) + .add("startEpoch", startEpoch) + .add("taskId", taskId) + .add("length", length) + .add("success", success) + .add("exceptions", exceptions) + .toString(); + } + + @Override + public boolean equals(Object other) { + if (this == other) { + return true; + } else if (other == null || getClass() != other.getClass()) { + return false; + } + + MaintenanceResult that = (MaintenanceResult) other; + return this.startEpoch == that.startEpoch + && this.taskId == that.taskId + && this.length == that.length + && this.success == that.success + && this.exceptions.size() == that.exceptions.size(); + } + + @Override + public int hashCode() { + return Objects.hash(startEpoch, taskId, length, success, exceptions.size()); Review Comment: only check size? ########## flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TableMaintenanceMetrics.java: ########## @@ -28,6 +28,11 @@ public class TableMaintenanceMetrics { public static final String TRIGGERED = "triggered"; public static final String NOTHING_TO_TRIGGER = "nothingToTrigger"; + // LockRemover metrics + public static final String SUCCESSFUL_STREAM_COUNTER = "successfulStream"; Review Comment: nit: `successful` doesn't pair well with `failed`. maybe `succeeded` or `completed`? also `stream` is not intuitive to me. does it mean `tasks` here? ########## flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TaskResult.java: ########## @@ -0,0 +1,65 @@ +/* + * 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.Internal; +import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; + +/** The result of a single Maintenance Task. */ +@Internal +public class TaskResult { + private final int taskId; + private final long startEpoch; + private final boolean success; + private final List<Exception> exceptions; + + public TaskResult(int taskId, long startEpoch, boolean success, List<Exception> exceptions) { + this.taskId = taskId; + this.startEpoch = startEpoch; + this.success = success; + this.exceptions = exceptions; + } + + public int taskId() { + return taskId; + } + + public long startEpoch() { + return startEpoch; + } + + public boolean success() { + return success; + } + + public List<Exception> exceptions() { + return exceptions; + } + + @Override + public String toString() { Review Comment: this class doesn't need to implement `hashCode` and `equals` as the `MaintenanceResult` class? ########## flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/LockRemover.java: ########## @@ -0,0 +1,120 @@ +/* + * 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.Map; +import java.util.concurrent.atomic.AtomicLong; +import org.apache.flink.annotation.Internal; +import org.apache.flink.metrics.Counter; +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.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** Manages locks and collect {@link org.apache.flink.metrics.Metric} for the Maintenance Tasks. */ +@Internal +public class LockRemover extends AbstractStreamOperator<MaintenanceResult> + implements OneInputStreamOperator<TaskResult, MaintenanceResult> { + private static final Logger LOG = LoggerFactory.getLogger(LockRemover.class); + + private final TriggerLockFactory lockFactory; + private final List<String> taskNames; + + private transient Map<Integer, Counter> successfulStreamResultCounterMap; + private transient Map<Integer, Counter> failedStreamResultCounterMap; + private transient Map<Integer, AtomicLong> lastRunLength; + private transient TriggerLockFactory.Lock lock; + private transient TriggerLockFactory.Lock recoveryLock; + private transient long lastProcessed = 0L; + + public LockRemover(TriggerLockFactory lockFactory, List<String> taskNames) { + Preconditions.checkNotNull(lockFactory, "Lock factory should no be null"); + Preconditions.checkArgument( + taskNames != null && !taskNames.isEmpty(), "Invalid task names: null or empty"); + + this.lockFactory = lockFactory; + this.taskNames = taskNames; + } + + @Override + public void open() throws Exception { + super.open(); + this.successfulStreamResultCounterMap = Maps.newHashMapWithExpectedSize(taskNames.size()); Review Comment: if these maps always have the same size as the `taskNames`, why don't we keep them as `List` as well and use list index number to reference the value. ########## flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/LockRemover.java: ########## @@ -0,0 +1,120 @@ +/* + * 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.Map; +import java.util.concurrent.atomic.AtomicLong; +import org.apache.flink.annotation.Internal; +import org.apache.flink.metrics.Counter; +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.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** Manages locks and collect {@link org.apache.flink.metrics.Metric} for the Maintenance Tasks. */ +@Internal +public class LockRemover extends AbstractStreamOperator<MaintenanceResult> + implements OneInputStreamOperator<TaskResult, MaintenanceResult> { + private static final Logger LOG = LoggerFactory.getLogger(LockRemover.class); + + private final TriggerLockFactory lockFactory; + private final List<String> taskNames; + + private transient Map<Integer, Counter> successfulStreamResultCounterMap; + private transient Map<Integer, Counter> failedStreamResultCounterMap; + private transient Map<Integer, AtomicLong> lastRunLength; + private transient TriggerLockFactory.Lock lock; + private transient TriggerLockFactory.Lock recoveryLock; + private transient long lastProcessed = 0L; + + public LockRemover(TriggerLockFactory lockFactory, List<String> taskNames) { + Preconditions.checkNotNull(lockFactory, "Lock factory should no be null"); + Preconditions.checkArgument( + taskNames != null && !taskNames.isEmpty(), "Invalid task names: null or empty"); + + this.lockFactory = lockFactory; + this.taskNames = taskNames; + } + + @Override + public void open() throws Exception { + super.open(); + this.successfulStreamResultCounterMap = Maps.newHashMapWithExpectedSize(taskNames.size()); + this.failedStreamResultCounterMap = Maps.newHashMapWithExpectedSize(taskNames.size()); + this.lastRunLength = Maps.newHashMapWithExpectedSize(taskNames.size()); + for (int i = 0; i < taskNames.size(); ++i) { + String name = taskNames.get(i); + successfulStreamResultCounterMap.put( + i, + getRuntimeContext() + .getMetricGroup() + .addGroup(TableMaintenanceMetrics.GROUP_KEY, name) + .counter(TableMaintenanceMetrics.SUCCESSFUL_STREAM_COUNTER)); + failedStreamResultCounterMap.put( + i, + getRuntimeContext() + .getMetricGroup() + .addGroup(TableMaintenanceMetrics.GROUP_KEY, name) + .counter(TableMaintenanceMetrics.FAILED_STREAM_COUNTER)); + AtomicLong length = new AtomicLong(0); + lastRunLength.put(i, length); + getRuntimeContext() + .getMetricGroup() + .addGroup(TableMaintenanceMetrics.GROUP_KEY, name) + .gauge(TableMaintenanceMetrics.LAST_RUN_LENGTH, length::get); + } + + this.lock = lockFactory.createLock(); + this.recoveryLock = lockFactory.createRecoveryLock(); + } + + @Override + public void processElement(StreamRecord<TaskResult> element) { + TaskResult value = element.getValue(); + LOG.debug("TaskResult {} arrived", value); + long length = System.currentTimeMillis() - value.startEpoch(); + output.collect( + new StreamRecord<>( + new MaintenanceResult( + value.startEpoch(), value.taskId(), length, value.success(), value.exceptions()))); + lock.unlock(); + this.lastProcessed = value.startEpoch(); + + // Update the metrics + lastRunLength.get(value.taskId()).set(length); + if (value.success()) { + successfulStreamResultCounterMap.get(value.taskId()).inc(); + } else { + failedStreamResultCounterMap.get(value.taskId()).inc(); + } + } + + @Override + public void processWatermark(Watermark mark) { Review Comment: is watermark calculated/advanced by the start epoch of the triggered tasks? ########## flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/LockRemover.java: ########## @@ -0,0 +1,120 @@ +/* + * 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.Map; +import java.util.concurrent.atomic.AtomicLong; +import org.apache.flink.annotation.Internal; +import org.apache.flink.metrics.Counter; +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.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** Manages locks and collect {@link org.apache.flink.metrics.Metric} for the Maintenance Tasks. */ +@Internal +public class LockRemover extends AbstractStreamOperator<MaintenanceResult> + implements OneInputStreamOperator<TaskResult, MaintenanceResult> { + private static final Logger LOG = LoggerFactory.getLogger(LockRemover.class); + + private final TriggerLockFactory lockFactory; + private final List<String> taskNames; + + private transient Map<Integer, Counter> successfulStreamResultCounterMap; + private transient Map<Integer, Counter> failedStreamResultCounterMap; + private transient Map<Integer, AtomicLong> lastRunLength; + private transient TriggerLockFactory.Lock lock; + private transient TriggerLockFactory.Lock recoveryLock; + private transient long lastProcessed = 0L; + + public LockRemover(TriggerLockFactory lockFactory, List<String> taskNames) { + Preconditions.checkNotNull(lockFactory, "Lock factory should no be null"); + Preconditions.checkArgument( + taskNames != null && !taskNames.isEmpty(), "Invalid task names: null or empty"); + + this.lockFactory = lockFactory; + this.taskNames = taskNames; + } + + @Override + public void open() throws Exception { + super.open(); + this.successfulStreamResultCounterMap = Maps.newHashMapWithExpectedSize(taskNames.size()); + this.failedStreamResultCounterMap = Maps.newHashMapWithExpectedSize(taskNames.size()); + this.lastRunLength = Maps.newHashMapWithExpectedSize(taskNames.size()); + for (int i = 0; i < taskNames.size(); ++i) { + String name = taskNames.get(i); + successfulStreamResultCounterMap.put( + i, + getRuntimeContext() + .getMetricGroup() + .addGroup(TableMaintenanceMetrics.GROUP_KEY, name) + .counter(TableMaintenanceMetrics.SUCCESSFUL_STREAM_COUNTER)); + failedStreamResultCounterMap.put( + i, + getRuntimeContext() + .getMetricGroup() + .addGroup(TableMaintenanceMetrics.GROUP_KEY, name) + .counter(TableMaintenanceMetrics.FAILED_STREAM_COUNTER)); + AtomicLong length = new AtomicLong(0); + lastRunLength.put(i, length); + getRuntimeContext() + .getMetricGroup() + .addGroup(TableMaintenanceMetrics.GROUP_KEY, name) + .gauge(TableMaintenanceMetrics.LAST_RUN_LENGTH, length::get); + } + + this.lock = lockFactory.createLock(); + this.recoveryLock = lockFactory.createRecoveryLock(); + } + + @Override + public void processElement(StreamRecord<TaskResult> element) { + TaskResult value = element.getValue(); + LOG.debug("TaskResult {} arrived", value); Review Comment: info level? nit: `Processing task result: {}` ########## flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/LockRemover.java: ########## @@ -0,0 +1,120 @@ +/* + * 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.Map; +import java.util.concurrent.atomic.AtomicLong; +import org.apache.flink.annotation.Internal; +import org.apache.flink.metrics.Counter; +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.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** Manages locks and collect {@link org.apache.flink.metrics.Metric} for the Maintenance Tasks. */ +@Internal +public class LockRemover extends AbstractStreamOperator<MaintenanceResult> + implements OneInputStreamOperator<TaskResult, MaintenanceResult> { + private static final Logger LOG = LoggerFactory.getLogger(LockRemover.class); + + private final TriggerLockFactory lockFactory; + private final List<String> taskNames; + + private transient Map<Integer, Counter> successfulStreamResultCounterMap; + private transient Map<Integer, Counter> failedStreamResultCounterMap; + private transient Map<Integer, AtomicLong> lastRunLength; + private transient TriggerLockFactory.Lock lock; + private transient TriggerLockFactory.Lock recoveryLock; + private transient long lastProcessed = 0L; + + public LockRemover(TriggerLockFactory lockFactory, List<String> taskNames) { + Preconditions.checkNotNull(lockFactory, "Lock factory should no be null"); + Preconditions.checkArgument( + taskNames != null && !taskNames.isEmpty(), "Invalid task names: null or empty"); + + this.lockFactory = lockFactory; + this.taskNames = taskNames; + } + + @Override + public void open() throws Exception { + super.open(); + this.successfulStreamResultCounterMap = Maps.newHashMapWithExpectedSize(taskNames.size()); + this.failedStreamResultCounterMap = Maps.newHashMapWithExpectedSize(taskNames.size()); + this.lastRunLength = Maps.newHashMapWithExpectedSize(taskNames.size()); + for (int i = 0; i < taskNames.size(); ++i) { + String name = taskNames.get(i); + successfulStreamResultCounterMap.put( + i, + getRuntimeContext() + .getMetricGroup() + .addGroup(TableMaintenanceMetrics.GROUP_KEY, name) + .counter(TableMaintenanceMetrics.SUCCESSFUL_STREAM_COUNTER)); + failedStreamResultCounterMap.put( + i, + getRuntimeContext() + .getMetricGroup() + .addGroup(TableMaintenanceMetrics.GROUP_KEY, name) + .counter(TableMaintenanceMetrics.FAILED_STREAM_COUNTER)); + AtomicLong length = new AtomicLong(0); + lastRunLength.put(i, length); + getRuntimeContext() + .getMetricGroup() + .addGroup(TableMaintenanceMetrics.GROUP_KEY, name) + .gauge(TableMaintenanceMetrics.LAST_RUN_LENGTH, length::get); + } + + this.lock = lockFactory.createLock(); + this.recoveryLock = lockFactory.createRecoveryLock(); + } + + @Override + public void processElement(StreamRecord<TaskResult> element) { + TaskResult value = element.getValue(); Review Comment: nit: for code readability, maybe name this as `taskResult` ########## flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/LockRemover.java: ########## @@ -0,0 +1,120 @@ +/* + * 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.Map; +import java.util.concurrent.atomic.AtomicLong; +import org.apache.flink.annotation.Internal; +import org.apache.flink.metrics.Counter; +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.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** Manages locks and collect {@link org.apache.flink.metrics.Metric} for the Maintenance Tasks. */ +@Internal +public class LockRemover extends AbstractStreamOperator<MaintenanceResult> + implements OneInputStreamOperator<TaskResult, MaintenanceResult> { + private static final Logger LOG = LoggerFactory.getLogger(LockRemover.class); + + private final TriggerLockFactory lockFactory; + private final List<String> taskNames; + + private transient Map<Integer, Counter> successfulStreamResultCounterMap; + private transient Map<Integer, Counter> failedStreamResultCounterMap; + private transient Map<Integer, AtomicLong> lastRunLength; + private transient TriggerLockFactory.Lock lock; + private transient TriggerLockFactory.Lock recoveryLock; + private transient long lastProcessed = 0L; + + public LockRemover(TriggerLockFactory lockFactory, List<String> taskNames) { + Preconditions.checkNotNull(lockFactory, "Lock factory should no be null"); + Preconditions.checkArgument( + taskNames != null && !taskNames.isEmpty(), "Invalid task names: null or empty"); + + this.lockFactory = lockFactory; + this.taskNames = taskNames; + } + + @Override + public void open() throws Exception { + super.open(); + this.successfulStreamResultCounterMap = Maps.newHashMapWithExpectedSize(taskNames.size()); + this.failedStreamResultCounterMap = Maps.newHashMapWithExpectedSize(taskNames.size()); + this.lastRunLength = Maps.newHashMapWithExpectedSize(taskNames.size()); + for (int i = 0; i < taskNames.size(); ++i) { + String name = taskNames.get(i); + successfulStreamResultCounterMap.put( + i, + getRuntimeContext() + .getMetricGroup() + .addGroup(TableMaintenanceMetrics.GROUP_KEY, name) + .counter(TableMaintenanceMetrics.SUCCESSFUL_STREAM_COUNTER)); + failedStreamResultCounterMap.put( + i, + getRuntimeContext() + .getMetricGroup() + .addGroup(TableMaintenanceMetrics.GROUP_KEY, name) + .counter(TableMaintenanceMetrics.FAILED_STREAM_COUNTER)); + AtomicLong length = new AtomicLong(0); + lastRunLength.put(i, length); + getRuntimeContext() + .getMetricGroup() + .addGroup(TableMaintenanceMetrics.GROUP_KEY, name) + .gauge(TableMaintenanceMetrics.LAST_RUN_LENGTH, length::get); + } + + this.lock = lockFactory.createLock(); + this.recoveryLock = lockFactory.createRecoveryLock(); + } + + @Override + public void processElement(StreamRecord<TaskResult> element) { + TaskResult value = element.getValue(); + LOG.debug("TaskResult {} arrived", value); + long length = System.currentTimeMillis() - value.startEpoch(); + output.collect( + new StreamRecord<>( + new MaintenanceResult( + value.startEpoch(), value.taskId(), length, value.success(), value.exceptions()))); + lock.unlock(); + this.lastProcessed = value.startEpoch(); + + // Update the metrics + lastRunLength.get(value.taskId()).set(length); + if (value.success()) { + successfulStreamResultCounterMap.get(value.taskId()).inc(); + } else { + failedStreamResultCounterMap.get(value.taskId()).inc(); + } + } + + @Override + public void processWatermark(Watermark mark) { + if (mark.getTimestamp() > lastProcessed) { Review Comment: should this be `>` or `>=`? ########## flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/LockRemover.java: ########## @@ -0,0 +1,120 @@ +/* + * 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.Map; +import java.util.concurrent.atomic.AtomicLong; +import org.apache.flink.annotation.Internal; +import org.apache.flink.metrics.Counter; +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.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** Manages locks and collect {@link org.apache.flink.metrics.Metric} for the Maintenance Tasks. */ +@Internal +public class LockRemover extends AbstractStreamOperator<MaintenanceResult> + implements OneInputStreamOperator<TaskResult, MaintenanceResult> { + private static final Logger LOG = LoggerFactory.getLogger(LockRemover.class); + + private final TriggerLockFactory lockFactory; + private final List<String> taskNames; + + private transient Map<Integer, Counter> successfulStreamResultCounterMap; + private transient Map<Integer, Counter> failedStreamResultCounterMap; + private transient Map<Integer, AtomicLong> lastRunLength; + private transient TriggerLockFactory.Lock lock; + private transient TriggerLockFactory.Lock recoveryLock; + private transient long lastProcessed = 0L; + + public LockRemover(TriggerLockFactory lockFactory, List<String> taskNames) { Review Comment: I found it easy to mix the `task` here with the Flink operator task/subtask. should we standarize on `action` for `maintenance action` to avoid potential confusion. ########## flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TableMaintenanceMetrics.java: ########## @@ -28,6 +28,11 @@ public class TableMaintenanceMetrics { public static final String TRIGGERED = "triggered"; public static final String NOTHING_TO_TRIGGER = "nothingToTrigger"; + // LockRemover metrics + public static final String SUCCESSFUL_STREAM_COUNTER = "successfulStream"; + public static final String FAILED_STREAM_COUNTER = "failedStream"; + public static final String LAST_RUN_LENGTH = "lastRunLength"; Review Comment: length -> duration. maybe like `lastRunDurationMs` ########## flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestLockRemover.java: ########## @@ -0,0 +1,234 @@ +/* + * 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.apache.iceberg.flink.maintenance.operator.ConstantsForTests.DUMMY_NAME; +import static org.apache.iceberg.flink.maintenance.operator.TableMaintenanceMetrics.FAILED_STREAM_COUNTER; +import static org.apache.iceberg.flink.maintenance.operator.TableMaintenanceMetrics.SUCCESSFUL_STREAM_COUNTER; +import static org.assertj.core.api.Assertions.assertThat; + +import java.time.Duration; +import java.util.concurrent.TimeoutException; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.core.execution.JobClient; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +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; + +class TestLockRemover extends OperatorTestBase { + private static final String TASK_0 = "task0"; + private static final String TASK_1 = "task1"; + private static final TriggerLockFactory.Lock LOCK = new TestingLock(); + private static final TriggerLockFactory.Lock RECOVERY_LOCK = new TestingLock(); + + @BeforeEach + void before() { + MetricsReporterFactoryForTests.reset(); + } + + @Test + void testProcess() throws Exception { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + ManualSource<TaskResult> source = new ManualSource<>(env, TypeInformation.of(TaskResult.class)); + CollectingSink<MaintenanceResult> result = new CollectingSink<>(); + source + .dataStream() + .transform( + DUMMY_NAME, + TypeInformation.of(MaintenanceResult.class), + new LockRemover(new TestingLockFactory(), Lists.newArrayList(TASK_0, TASK_1))) + .setParallelism(1) + .sinkTo(result); + + JobClient jobClient = null; Review Comment: this pattern may be a little bit simpler ``` try (CloseableIterator<T> iter = stream.executeAndCollect()) { } ``` ########## flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/MaintenanceResult.java: ########## @@ -0,0 +1,93 @@ +/* + * 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.Objects; +import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; + +/** The output of the Maintenance Flow. */ +public class MaintenanceResult { + private final long startEpoch; + private final int taskId; + private final long length; + private final boolean success; + private final List<Exception> exceptions; + + public MaintenanceResult( + long startEpoch, int taskId, long length, boolean success, List<Exception> exceptions) { + this.startEpoch = startEpoch; + this.taskId = taskId; + this.length = length; + this.success = success; + this.exceptions = exceptions; + } + + public long startEpoch() { + return startEpoch; + } + + public int taskId() { + return taskId; + } + + public long length() { + return length; + } + + public boolean success() { + return success; + } + + public List<Exception> exceptions() { + return exceptions; + } + + @Override + public String toString() { + return MoreObjects.toStringHelper(this) + .add("startEpoch", startEpoch) + .add("taskId", taskId) + .add("length", length) + .add("success", success) + .add("exceptions", exceptions) + .toString(); + } + + @Override + public boolean equals(Object other) { + if (this == other) { + return true; + } else if (other == null || getClass() != other.getClass()) { + return false; + } + + MaintenanceResult that = (MaintenanceResult) other; + return this.startEpoch == that.startEpoch + && this.taskId == that.taskId + && this.length == that.length + && this.success == that.success + && this.exceptions.size() == that.exceptions.size(); Review Comment: only checking size? ########## flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestLockRemover.java: ########## @@ -0,0 +1,234 @@ +/* + * 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.apache.iceberg.flink.maintenance.operator.ConstantsForTests.DUMMY_NAME; +import static org.apache.iceberg.flink.maintenance.operator.TableMaintenanceMetrics.FAILED_STREAM_COUNTER; +import static org.apache.iceberg.flink.maintenance.operator.TableMaintenanceMetrics.SUCCESSFUL_STREAM_COUNTER; +import static org.assertj.core.api.Assertions.assertThat; + +import java.time.Duration; +import java.util.concurrent.TimeoutException; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.core.execution.JobClient; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +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; + +class TestLockRemover extends OperatorTestBase { + private static final String TASK_0 = "task0"; + private static final String TASK_1 = "task1"; + private static final TriggerLockFactory.Lock LOCK = new TestingLock(); + private static final TriggerLockFactory.Lock RECOVERY_LOCK = new TestingLock(); + + @BeforeEach + void before() { + MetricsReporterFactoryForTests.reset(); + } + + @Test + void testProcess() throws Exception { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + ManualSource<TaskResult> source = new ManualSource<>(env, TypeInformation.of(TaskResult.class)); + CollectingSink<MaintenanceResult> result = new CollectingSink<>(); + source + .dataStream() + .transform( + DUMMY_NAME, + TypeInformation.of(MaintenanceResult.class), + new LockRemover(new TestingLockFactory(), Lists.newArrayList(TASK_0, TASK_1))) + .setParallelism(1) + .sinkTo(result); + + JobClient jobClient = null; + try { + jobClient = env.executeAsync(); + + LOCK.tryLock(); + assertThat(LOCK.isHeld()).isTrue(); + + // Start a successful trigger for task1 and assert the return value is correct + processAndCheck(source, result, new TaskResult(0, 0L, true, Lists.newArrayList())); + + // Assert that the lock is removed + assertThat(LOCK.isHeld()).isFalse(); + } finally { + closeJobClient(jobClient); + } + } + + @Test + void testMetrics() throws Exception { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + ManualSource<TaskResult> source = new ManualSource<>(env, TypeInformation.of(TaskResult.class)); + CollectingSink<MaintenanceResult> result = new CollectingSink<>(); + source + .dataStream() + .transform( + DUMMY_NAME, + TypeInformation.of(MaintenanceResult.class), + new LockRemover(new TestingLockFactory(), Lists.newArrayList(TASK_0, TASK_1))) + .setParallelism(1) + .sinkTo(result); + + JobClient jobClient = null; + try { + jobClient = env.executeAsync(); + + // Start the 2 successful and one failed result trigger for task1, and 3 successful for task2 + processAndCheck(source, result, new TaskResult(0, 0L, true, Lists.newArrayList())); + processAndCheck(source, result, new TaskResult(1, 1L, true, Lists.newArrayList())); + processAndCheck(source, result, new TaskResult(1, 2L, true, Lists.newArrayList())); + processAndCheck(source, result, new TaskResult(0, 3L, false, Lists.newArrayList())); + processAndCheck(source, result, new TaskResult(0, 4L, true, Lists.newArrayList())); + processAndCheck(source, result, new TaskResult(1, 5L, true, Lists.newArrayList())); + + Awaitility.await() + .until( + () -> + MetricsReporterFactoryForTests.counter( + DUMMY_NAME + "." + TASK_1 + "." + SUCCESSFUL_STREAM_COUNTER) + .equals(3L)); + + // Final check all the counters + MetricsReporterFactoryForTests.assertCounters( + new ImmutableMap.Builder<String, Long>() + .put(DUMMY_NAME + "." + TASK_0 + "." + SUCCESSFUL_STREAM_COUNTER, 2L) + .put(DUMMY_NAME + "." + TASK_0 + "." + FAILED_STREAM_COUNTER, 1L) + .put(DUMMY_NAME + "." + TASK_1 + "." + SUCCESSFUL_STREAM_COUNTER, 3L) + .put(DUMMY_NAME + "." + TASK_1 + "." + FAILED_STREAM_COUNTER, 0L) + .build()); + } finally { + closeJobClient(jobClient); + } + } + + /** + * The test checks if the recovery watermark is only removed if the watermark has arrived from + * both downstream sources. Review Comment: downstream -> upstream -- 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