You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pulsar.apache.org by GitBox <gi...@apache.org> on 2021/04/15 17:46:35 UTC

[GitHub] [pulsar] merlimat commented on a change in pull request #10199: Shutdown Broker gracefully, but forcefully after brokerShutdownTimeoutMs

merlimat commented on a change in pull request #10199:
URL: https://github.com/apache/pulsar/pull/10199#discussion_r614240424



##########
File path: pulsar-broker/src/main/java/org/apache/pulsar/broker/service/GracefulExecutorServicesShutdownHandler.java
##########
@@ -0,0 +1,116 @@
+/**
+ * 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.pulsar.broker.service;
+
+import io.netty.util.concurrent.DefaultThreadFactory;
+import java.util.List;
+import java.util.Objects;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.RejectedExecutionException;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.pulsar.common.util.FutureUtil;
+
+/**
+ * Shuts down one or many {@link ExecutorService}s in a graceful way.
+ *
+ * The executors will be terminated forcefully after the timeout or when the future is cancelled.
+ *
+ * Designed to be used via the API in {@link GracefulExecutorServicesShutdown}
+ */
+@Slf4j
+class GracefulExecutorServicesShutdownHandler {
+    private final ScheduledExecutorService shutdownScheduler = Executors.newSingleThreadScheduledExecutor(
+            new DefaultThreadFactory(getClass().getSimpleName()));
+    private final List<ExecutorService> executors;
+    private final CompletableFuture<Void> future;
+    private final long timeoutMs;
+    private final long shutdownStatusPollingInterval;
+
+    GracefulExecutorServicesShutdownHandler(long timeoutMs, List<ExecutorService> executorServices) {
+        this.timeoutMs = Math.max(timeoutMs, 1L);
+        this.shutdownStatusPollingInterval = Math.min(Math.max(timeoutMs / 100, 10), timeoutMs);
+        executors = executorServices.stream()
+                .filter(Objects::nonNull)
+                .collect(Collectors.toList());
+        future = new CompletableFuture<>();
+    }
+
+    CompletableFuture<Void> startShutdownHandler() {
+        log.info("Shutting down {} executors.", executors.size());
+        executors.forEach(ExecutorService::shutdown);

Review comment:
       Isn't the shutdown already triggered when the executors are added to `GracefulExecutorServicesShutdown` ? 

##########
File path: pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java
##########
@@ -480,6 +487,20 @@ public void close() throws PulsarServerException {
         }
     }
 
+    private CompletableFuture<Void> addTimeoutHandling(CompletableFuture<Void> future) {
+        ScheduledExecutorService shutdownExecutor = Executors.newSingleThreadScheduledExecutor(
+                new DefaultThreadFactory(getClass().getSimpleName() + "-shutdown"));
+        FutureUtil.addTimeoutHandling(future,
+                Duration.ofMillis(Math.max(1L, getConfiguration().getBrokerShutdownTimeoutMs())),
+                shutdownExecutor, () -> FutureUtil.createTimeoutException("Timeout in close", getClass(), "close"));
+        future.handle((v, t) -> {
+            // shutdown the shutdown executor
+            shutdownExecutor.shutdown();

Review comment:
       Should we do `shutdownExecutor.shutdownDown() && shutdownExecutor.await()` here?

##########
File path: pulsar-broker/src/main/java/org/apache/pulsar/broker/service/GracefulExecutorServicesShutdownHandler.java
##########
@@ -0,0 +1,116 @@
+/**
+ * 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.pulsar.broker.service;
+
+import io.netty.util.concurrent.DefaultThreadFactory;
+import java.util.List;
+import java.util.Objects;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.RejectedExecutionException;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.pulsar.common.util.FutureUtil;
+
+/**
+ * Shuts down one or many {@link ExecutorService}s in a graceful way.
+ *
+ * The executors will be terminated forcefully after the timeout or when the future is cancelled.
+ *
+ * Designed to be used via the API in {@link GracefulExecutorServicesShutdown}
+ */
+@Slf4j
+class GracefulExecutorServicesShutdownHandler {
+    private final ScheduledExecutorService shutdownScheduler = Executors.newSingleThreadScheduledExecutor(
+            new DefaultThreadFactory(getClass().getSimpleName()));
+    private final List<ExecutorService> executors;
+    private final CompletableFuture<Void> future;
+    private final long timeoutMs;
+    private final long shutdownStatusPollingInterval;
+
+    GracefulExecutorServicesShutdownHandler(long timeoutMs, List<ExecutorService> executorServices) {
+        this.timeoutMs = Math.max(timeoutMs, 1L);
+        this.shutdownStatusPollingInterval = Math.min(Math.max(timeoutMs / 100, 10), timeoutMs);
+        executors = executorServices.stream()
+                .filter(Objects::nonNull)
+                .collect(Collectors.toList());
+        future = new CompletableFuture<>();
+    }
+
+    CompletableFuture<Void> startShutdownHandler() {
+        log.info("Shutting down {} executors.", executors.size());
+        executors.forEach(ExecutorService::shutdown);
+        FutureUtil.whenCancelledOrTimedOut(future, () -> {
+            terminate();
+        });
+        checkIfExecutorsHaveBeenShutdown();
+        if (!shutdownScheduler.isShutdown()) {
+            try {
+                shutdownScheduler.schedule(this::terminate, timeoutMs, TimeUnit.MILLISECONDS);
+            } catch (RejectedExecutionException e) {
+                // ignore
+            }
+        }
+        return future;
+    }
+
+    private void terminate() {
+        for (ExecutorService executor : executors) {
+            if (!executor.isTerminated()) {
+                log.info("Shutting down forcefully executor {}", executor);
+                for (Runnable runnable : executor.shutdownNow()) {
+                    log.info("Execution in progress for runnable instance of {}: {}", runnable.getClass(),
+                            runnable);

Review comment:
       Shouldn't we do a `executor.awaitTermination()` here to ensure the thread is really gone before we proceed? (Or at least we give it the chance to exit) 
   

##########
File path: pulsar-broker/src/main/java/org/apache/pulsar/broker/service/GracefulExecutorServicesShutdownHandler.java
##########
@@ -0,0 +1,116 @@
+/**
+ * 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.pulsar.broker.service;
+
+import io.netty.util.concurrent.DefaultThreadFactory;
+import java.util.List;
+import java.util.Objects;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.RejectedExecutionException;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.pulsar.common.util.FutureUtil;
+
+/**
+ * Shuts down one or many {@link ExecutorService}s in a graceful way.
+ *
+ * The executors will be terminated forcefully after the timeout or when the future is cancelled.
+ *
+ * Designed to be used via the API in {@link GracefulExecutorServicesShutdown}
+ */
+@Slf4j
+class GracefulExecutorServicesShutdownHandler {
+    private final ScheduledExecutorService shutdownScheduler = Executors.newSingleThreadScheduledExecutor(
+            new DefaultThreadFactory(getClass().getSimpleName()));
+    private final List<ExecutorService> executors;
+    private final CompletableFuture<Void> future;
+    private final long timeoutMs;
+    private final long shutdownStatusPollingInterval;
+
+    GracefulExecutorServicesShutdownHandler(long timeoutMs, List<ExecutorService> executorServices) {
+        this.timeoutMs = Math.max(timeoutMs, 1L);
+        this.shutdownStatusPollingInterval = Math.min(Math.max(timeoutMs / 100, 10), timeoutMs);
+        executors = executorServices.stream()
+                .filter(Objects::nonNull)
+                .collect(Collectors.toList());
+        future = new CompletableFuture<>();
+    }
+
+    CompletableFuture<Void> startShutdownHandler() {
+        log.info("Shutting down {} executors.", executors.size());
+        executors.forEach(ExecutorService::shutdown);
+        FutureUtil.whenCancelledOrTimedOut(future, () -> {
+            terminate();
+        });
+        checkIfExecutorsHaveBeenShutdown();
+        if (!shutdownScheduler.isShutdown()) {
+            try {
+                shutdownScheduler.schedule(this::terminate, timeoutMs, TimeUnit.MILLISECONDS);
+            } catch (RejectedExecutionException e) {
+                // ignore
+            }
+        }
+        return future;
+    }
+
+    private void terminate() {
+        for (ExecutorService executor : executors) {
+            if (!executor.isTerminated()) {
+                log.info("Shutting down forcefully executor {}", executor);
+                for (Runnable runnable : executor.shutdownNow()) {
+                    log.info("Execution in progress for runnable instance of {}: {}", runnable.getClass(),
+                            runnable);
+                }
+            }
+        }
+        shutdown();
+    }
+
+    private void shutdown() {
+        if (!shutdownScheduler.isShutdown()) {
+            log.info("Shutting down scheduler.");
+            shutdownScheduler.shutdown();

Review comment:
       Also for the scheduler thread, shouldn't we await for the thread termination?




-- 
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:
users@infra.apache.org