xintongsong commented on a change in pull request #8952: URL: https://github.com/apache/flink/pull/8952#discussion_r550381861
########## File path: flink-metrics/flink-metrics-core/src/main/java/org/apache/flink/metrics/ThresholdMeter.java ########## @@ -0,0 +1,99 @@ +/* + * 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.flink.metrics; + +import java.time.Duration; +import java.util.ArrayDeque; +import java.util.Queue; +import java.util.function.Supplier; + +/** A timestamp queue based threshold meter. */ +public class ThresholdMeter implements Meter { + private static final Double MILLISECONDS_PER_SECOND = 1000.0; + private final Supplier<Long> currentTimeMillisSupplier; + private final double maxEventsPerInterval; + private final Duration interval; + private final Queue<Long> failureTimestamps; + private long failureCounter = 0; + + public ThresholdMeter(double maximumFailureRate, Duration interval) { + this(maximumFailureRate, interval, System::currentTimeMillis); + } + + public ThresholdMeter( + double maxEventsPerInterval, Duration interval, Supplier<Long> customSupplier) { + this.maxEventsPerInterval = maxEventsPerInterval; + this.interval = interval; + this.failureTimestamps = new ArrayDeque<>(); + this.currentTimeMillisSupplier = customSupplier; + } + + @Override + public void markEvent() { + failureTimestamps.add(currentTimeMillisSupplier.get()); + failureCounter++; + } + + @Override + public void markEvent(long n) { + long timestamp = currentTimeMillisSupplier.get(); + for (int i = 0; i < n; i++) { + failureTimestamps.add(timestamp); + } + failureCounter = failureCounter + n; + } + + @Override + public double getRate() { + return getEventCountsRecentInterval() / (interval.toMillis() / MILLISECONDS_PER_SECOND); + } + + @Override + public long getCount() { + return failureCounter; + } + + public void checkAgainstThreshold() throws ThresholdExceedException { + if (getEventCountsRecentInterval() >= maxEventsPerInterval) { + throw new ThresholdExceedException( + String.format( + "Maximum number of events %f is detected", + getEventCountsRecentInterval())); + } + } + + private double getEventCountsRecentInterval() { Review comment: Return type should not be `double` ########## File path: flink-metrics/flink-metrics-core/src/main/java/org/apache/flink/metrics/ThresholdMeter.java ########## @@ -0,0 +1,99 @@ +/* + * 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.flink.metrics; + +import java.time.Duration; +import java.util.ArrayDeque; +import java.util.Queue; +import java.util.function.Supplier; + +/** A timestamp queue based threshold meter. */ +public class ThresholdMeter implements Meter { + private static final Double MILLISECONDS_PER_SECOND = 1000.0; Review comment: The primitive type `double` is preferred. ########## File path: flink-metrics/flink-metrics-core/src/test/java/org/apache/flink/metrics/ThresholdMeterTest.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.flink.metrics; + +import org.apache.flink.metrics.ThresholdMeter.ThresholdExceedException; +import org.apache.flink.util.TestLogger; + +import org.junit.Test; + +import java.time.Duration; + +/** Test time stamp based threshold meter. */ +public class ThresholdMeterTest extends TestLogger { Review comment: I think we should also test against the contracts of `Meter`. E.g., `markEvent`, `getRate`, `getCount`. ########## File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/active/ActiveResourceManager.java ########## @@ -301,7 +353,25 @@ private boolean clearStateForWorker(ResourceID resourceId) { return true; } - private void requestWorkerIfRequired() { + private void tryResetWorkerCreationCoolDown() { + if (workerCreationCoolDown.isDone()) { + log.info( + "Reaching max start worker failure rate. Will not retry creating worker in {}.", + workerCreationRetryInterval); + workerCreationCoolDown = new CompletableFuture<>(); + getMainThreadExecutor() + .scheduleRunAsync( + () -> workerCreationCoolDown.complete(null), + workerCreationRetryInterval.getSize()); Review comment: There's no guaranteed that `workerCreationRetryInterval.getSize` returns in milliseconds. ########## File path: flink-metrics/flink-metrics-core/src/main/java/org/apache/flink/metrics/ThresholdMeter.java ########## @@ -0,0 +1,99 @@ +/* + * 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.flink.metrics; + +import java.time.Duration; +import java.util.ArrayDeque; +import java.util.Queue; +import java.util.function.Supplier; + +/** A timestamp queue based threshold meter. */ +public class ThresholdMeter implements Meter { + private static final Double MILLISECONDS_PER_SECOND = 1000.0; + private final Supplier<Long> currentTimeMillisSupplier; + private final double maxEventsPerInterval; + private final Duration interval; + private final Queue<Long> failureTimestamps; + private long failureCounter = 0; + + public ThresholdMeter(double maximumFailureRate, Duration interval) { + this(maximumFailureRate, interval, System::currentTimeMillis); + } + + public ThresholdMeter( + double maxEventsPerInterval, Duration interval, Supplier<Long> customSupplier) { + this.maxEventsPerInterval = maxEventsPerInterval; + this.interval = interval; + this.failureTimestamps = new ArrayDeque<>(); + this.currentTimeMillisSupplier = customSupplier; + } + + @Override + public void markEvent() { + failureTimestamps.add(currentTimeMillisSupplier.get()); + failureCounter++; + } + + @Override + public void markEvent(long n) { + long timestamp = currentTimeMillisSupplier.get(); + for (int i = 0; i < n; i++) { + failureTimestamps.add(timestamp); + } + failureCounter = failureCounter + n; + } + + @Override + public double getRate() { + return getEventCountsRecentInterval() / (interval.toMillis() / MILLISECONDS_PER_SECOND); Review comment: Need to make sure `interval` is not `0`. ########## File path: flink-metrics/flink-metrics-core/src/main/java/org/apache/flink/metrics/ThresholdMeter.java ########## @@ -0,0 +1,99 @@ +/* + * 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.flink.metrics; + +import java.time.Duration; +import java.util.ArrayDeque; +import java.util.Queue; +import java.util.function.Supplier; + +/** A timestamp queue based threshold meter. */ +public class ThresholdMeter implements Meter { + private static final Double MILLISECONDS_PER_SECOND = 1000.0; + private final Supplier<Long> currentTimeMillisSupplier; + private final double maxEventsPerInterval; + private final Duration interval; + private final Queue<Long> failureTimestamps; + private long failureCounter = 0; + + public ThresholdMeter(double maximumFailureRate, Duration interval) { + this(maximumFailureRate, interval, System::currentTimeMillis); + } + + public ThresholdMeter( Review comment: Seems this constructor is no longer used. We can remote this constructor, as well as `currentTimeMillisSupplier`. ########## File path: flink-metrics/flink-metrics-core/src/test/java/org/apache/flink/metrics/ThresholdMeterTest.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.flink.metrics; + +import org.apache.flink.metrics.ThresholdMeter.ThresholdExceedException; +import org.apache.flink.util.TestLogger; + +import org.junit.Test; + +import java.time.Duration; + +/** Test time stamp based threshold meter. */ +public class ThresholdMeterTest extends TestLogger { + + @Test(expected = ThresholdExceedException.class) + public void testMaximumFailureCheck() { + ThresholdMeter rater = new ThresholdMeter(5, Duration.ofSeconds(10)); + + for (int i = 0; i < 6; i++) { + rater.markEvent(); + } + + rater.checkAgainstThreshold(); + } + + @Test(expected = ThresholdExceedException.class) + public void testRateRecordMultipleEvents() throws InterruptedException { + ThresholdMeter rater = new ThresholdMeter(5, Duration.ofMillis(500)); + + for (int i = 0; i < 3; i++) { + rater.markEvent(2); + Thread.sleep(150); Review comment: I think we can further decrease the sleep time, to reduce the time needed to run the tests. ########## File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/active/ActiveResourceManager.java ########## @@ -213,14 +235,33 @@ public void onPreviousAttemptWorkersRecovered(Collection<WorkerType> recoveredWo } } + /** + * Record failure number of worker in ResourceManagers. Return whether maximum failure rate is + * detected. + * + * @return whether should acquire new container/worker after the a stop interval + */ + public boolean recordWorkerFailure() { + failureRater.markEvent(); + + try { + failureRater.checkAgainstThreshold(); + } catch (ThresholdExceedException e) { + log.warn(e.getMessage() + " in resource manager failure rater."); + return true; + } + + return false; + } + @Override public void onWorkerTerminated(ResourceID resourceId, String diagnostics) { if (clearStateForWorker(resourceId)) { log.info( "Worker {} is terminated. Diagnostics: {}", resourceId.getStringWithMetadata(), diagnostics); - requestWorkerIfRequired(); + recordWorkerFailureAndPauseWorkerCreationIfNeeded(); Review comment: Incorrect condition. This will also record failures for previous attempt workers. ########## File path: flink-runtime/src/main/java/org/apache/flink/runtime/metrics/MetricNames.java ########## @@ -62,6 +62,8 @@ private MetricNames() {} public static final String CHECKPOINT_ALIGNMENT_TIME = "checkpointAlignmentTime"; public static final String CHECKPOINT_START_DELAY_TIME = "checkpointStartDelayNanos"; + public static final String WORKER_FAILURE_RATE = "startWorkFailure" + SUFFIX_RATE; Review comment: This is never used. Seems the metrics is never registered. ########## File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/active/ActiveResourceManager.java ########## @@ -301,7 +353,25 @@ private boolean clearStateForWorker(ResourceID resourceId) { return true; } - private void requestWorkerIfRequired() { + private void tryResetWorkerCreationCoolDown() { + if (workerCreationCoolDown.isDone()) { + log.info( + "Reaching max start worker failure rate. Will not retry creating worker in {}.", + workerCreationRetryInterval); + workerCreationCoolDown = new CompletableFuture<>(); + getMainThreadExecutor() Review comment: For `ActiveResourceManager`, no need to call `getMainThreadExecutor`. Its super class `RpcEndpoint` already provided methods for scheduling runnable in the main thread. ########## File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/active/ActiveResourceManager.java ########## @@ -230,6 +271,12 @@ public void onError(Throwable exception) { onFatalError(exception); } + public static ThresholdMeter createFailureRater(Configuration configuration) { + double rate = configuration.getDouble(ResourceManagerOptions.MAXIMUM_WORKERS_FAILURE_RATE); + Preconditions.checkArgument(rate > 0, "Failure rate should be larger than 0"); + return new ThresholdMeter(rate, Duration.ofMinutes(1)); + } Review comment: This should be moved to `ActiveResourceManagerFactory`. ---------------------------------------------------------------- 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. For queries about this service, please contact Infrastructure at: [email protected]
