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 2022/03/15 02:22:18 UTC

[GitHub] [pulsar] Jason918 commented on a change in pull request #13130: [Issue 13129] [pulsar-metadata] Add watchdog thread in metadata store and track long running tasks.

Jason918 commented on a change in pull request #13130:
URL: https://github.com/apache/pulsar/pull/13130#discussion_r826523864



##########
File path: pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/ExecutorMonitor.java
##########
@@ -0,0 +1,73 @@
+/**
+ * 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.metadata.impl;
+
+import io.netty.util.concurrent.DefaultThreadFactory;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Consumer;
+import lombok.extern.slf4j.Slf4j;
+
+@Slf4j
+public class ExecutorMonitor implements AutoCloseable {
+
+    private final ExecutorService executorToMonitor;
+    private final ScheduledExecutorService scheduler;
+    private final ScheduledFuture<?> task;
+    private final long timeoutSeconds;
+    private final Consumer<Long> consumer;
+
+    public ExecutorMonitor(ExecutorService executor, long timeoutSeconds, Consumer<Long> consumer) {
+        this.executorToMonitor = executor;
+        this.scheduler = Executors
+                .newSingleThreadScheduledExecutor(new DefaultThreadFactory("metadata-store-executor-monitor"));
+        this.task = scheduler.scheduleAtFixedRate(this::executeTask, 1000, 5000, TimeUnit.MILLISECONDS);
+        this.timeoutSeconds = timeoutSeconds;
+        this.consumer = consumer;
+    }
+
+    private synchronized void executeTask() {
+        Future<?> f = executorToMonitor.submit(() -> {
+            if (log.isDebugEnabled()) {
+                log.debug("Executor callback invoked on thread {}, {}", Thread.currentThread().getId(),
+                  Thread.currentThread().getName());
+                return;
+            }
+        });
+        long start = System.currentTimeMillis();
+        try {
+            f.get(timeoutSeconds, TimeUnit.SECONDS);
+        } catch (Exception e) {
+            log.warn("Got exception waiting for executor callback", e);
+            long elapsed = System.currentTimeMillis() - start;
+            consumer.accept(elapsed);

Review comment:
       Is it ok to call `consumer.accept(elapsed);` multi times?
   Maybe we should just stop this monitor once timeout is detected.

##########
File path: pulsar-common/src/main/java/org/apache/pulsar/common/util/ThreadDumpUtil.java
##########
@@ -39,51 +39,73 @@
 public class ThreadDumpUtil {
     private static final String INDENT = "    ";
 
+    public interface DiagnosticProcessor {

Review comment:
       Can you explains a little about the modification in this class? 




-- 
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: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org