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/12/03 18:24:59 UTC

[GitHub] [pulsar] bharanic-dev opened a new pull request #13130: [Issue 13129] [pulsar-metadata] Add watchdog thread in metadata store and track long running tasks.

bharanic-dev opened a new pull request #13130:
URL: https://github.com/apache/pulsar/pull/13130


   Fixes #13129 
   
   *(or if this PR is one task of a github issue, please add `Master Issue: #<xyz>` to link to the master issue.)*
   
   Master Issue: #12929
   
   ### Motivation
   
   Add a watchdog monitor to monitor and log the stack trace of long running metadata-store callback tasks.
   
   ### Modifications
   
   *Describe the modifications you've done.*
   
   ### Verifying this change
   
   - [x] Make sure that the change passes the CI checks.
   
   This change is a trivial rework / code cleanup without any test coverage.
   
   ### Documentation
   
   - [x] `no-need-doc` 
     
   Code enhancement, not user visible.


-- 
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



[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.

Posted by GitBox <gi...@apache.org>.
Jason918 commented on a change in pull request #13130:
URL: https://github.com/apache/pulsar/pull/13130#discussion_r828700964



##########
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:
       > I think it should be OK to call `consumer.accept()` multiple times.
   
   But it's not necessary. And it would log all stack info multiple times. It's confusing.




-- 
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



[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.

Posted by GitBox <gi...@apache.org>.
Jason918 commented on a change in pull request #13130:
URL: https://github.com/apache/pulsar/pull/13130#discussion_r840264507



##########
File path: pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/AbstractMetadataStore.java
##########
@@ -77,6 +78,8 @@
     protected AbstractMetadataStore() {
         this.executor = Executors
                 .newSingleThreadScheduledExecutor(new DefaultThreadFactory("metadata-store"));
+        this.executorMonitor = new ExecutorMonitor(this.executor, TimeUnit.MINUTES.toSeconds(5),

Review comment:
       @bharanic-dev I think you can start a discussion about this change in the mail list so that more people can get involved.




-- 
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



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

Posted by GitBox <gi...@apache.org>.
bharanic-dev commented on a change in pull request #13130:
URL: https://github.com/apache/pulsar/pull/13130#discussion_r762336742



##########
File path: pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/AbstractMetadataStore.java
##########
@@ -317,7 +346,33 @@ public void invalidateAll() {
      */
     protected void execute(Runnable task, CompletableFuture<?> future) {
         try {
-            executor.execute(task);
+            // Wrap the original task, so we can record the thread on which it is running
+            TaskWrapper taskWrapper = new TaskWrapper(task);
+            executorWatchDog.execute(() -> {

Review comment:
       What you propose also works, except that it is not easy to figure out which code path(task) is the one that is actually blocked. Dumping the stack trace of all threads to logs could result in large volume of logs that one has to wade through to get to the task that is blocked. We could dump the stack trace of all threads to a file, but need alternate mechanisms to extract the file from the server.
   
   One advantage with monitoring each task is that you get the dump of the actual task that is potentially blocked. The 5 seconds is an arbitrary interval, we can change it to 10s or more (whatever we feel is reasonable). Also, note that at this time, we are only logging the stack trace. We could kill the broker if the task exceeds more than say 30s, but I did not do it because I thought that one might want to do that optionally (I can add a knob if we agree that that is what we want to do).
   
   Can you please explain what is the problem with tracking each task? The callback executor is already a single-threaded, which can only execute one task at a time. We can make the monitoring executor multi-threaded but that doesn't help as long as the callback thread is single-threaded.




-- 
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



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

Posted by GitBox <gi...@apache.org>.
bharanic-dev commented on a change in pull request #13130:
URL: https://github.com/apache/pulsar/pull/13130#discussion_r833577748



##########
File path: pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/AbstractMetadataStore.java
##########
@@ -77,6 +78,8 @@
     protected AbstractMetadataStore() {
         this.executor = Executors
                 .newSingleThreadScheduledExecutor(new DefaultThreadFactory("metadata-store"));
+        this.executorMonitor = new ExecutorMonitor(this.executor, TimeUnit.MINUTES.toSeconds(5),

Review comment:
       @Jason918 PTAL when you get a chance.




-- 
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



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

Posted by GitBox <gi...@apache.org>.
bharanic-dev commented on a change in pull request #13130:
URL: https://github.com/apache/pulsar/pull/13130#discussion_r836933914



##########
File path: pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/AbstractMetadataStore.java
##########
@@ -77,6 +78,8 @@
     protected AbstractMetadataStore() {
         this.executor = Executors
                 .newSingleThreadScheduledExecutor(new DefaultThreadFactory("metadata-store"));
+        this.executorMonitor = new ExecutorMonitor(this.executor, TimeUnit.MINUTES.toSeconds(5),

Review comment:
       @Jason918 @merlimat @lhotari please comment when you get a chance. Thank you!




-- 
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



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

Posted by GitBox <gi...@apache.org>.
lhotari commented on a change in pull request #13130:
URL: https://github.com/apache/pulsar/pull/13130#discussion_r788366446



##########
File path: pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/AbstractMetadataStore.java
##########
@@ -317,7 +346,33 @@ public void invalidateAll() {
      */
     protected void execute(Runnable task, CompletableFuture<?> future) {
         try {
-            executor.execute(task);
+            // Wrap the original task, so we can record the thread on which it is running
+            TaskWrapper taskWrapper = new TaskWrapper(task);
+            executorWatchDog.execute(() -> {

Review comment:
       @bharanic-dev #13124 has been merged so you could use the added ThreadDumpUtil in this PR.




-- 
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



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

Posted by GitBox <gi...@apache.org>.
merlimat commented on a change in pull request #13130:
URL: https://github.com/apache/pulsar/pull/13130#discussion_r762164290



##########
File path: pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/AbstractMetadataStore.java
##########
@@ -317,7 +346,33 @@ public void invalidateAll() {
      */
     protected void execute(Runnable task, CompletableFuture<?> future) {
         try {
-            executor.execute(task);
+            // Wrap the original task, so we can record the thread on which it is running
+            TaskWrapper taskWrapper = new TaskWrapper(task);
+            executorWatchDog.execute(() -> {

Review comment:
       I think that having a wrapper on each task is not the best option here, since we have a single thread where we're making all the blocking calls on the future. 
   
   Also, a 5 delay in a metadata operation does not necessary indicate that there is a deadlock (eg: a particular request could be slow while there is a leader election happening).
   
   Instead, we could have the watchdog thread to execute a dummy task in the `metadata-store` executor. If that task cannot be scheduled for, say, 30sec, then we can be quite confident that there is indeed a deadlock, in which case we should dump the stack traces and kill the broker, since there's no way for it to fix itself up.




-- 
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



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

Posted by GitBox <gi...@apache.org>.
bharanic-dev commented on pull request #13130:
URL: https://github.com/apache/pulsar/pull/13130#issuecomment-1069703362


   @Jason918 PTAL when you get a chance.


-- 
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



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

Posted by GitBox <gi...@apache.org>.
bharanic-dev commented on a change in pull request #13130:
URL: https://github.com/apache/pulsar/pull/13130#discussion_r827189740



##########
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:
       The ThreadDumpUtil. buildThreadDiagnosticString() returns a huge string that has the stack traces of all the threads as well as the lock info and deadlock info. The size of the buffer could be several kilobytes to a megabyte. Some third-party logging services have limitations on the maximum size of a message that they can ingest. The idea is to break the log output to several messages so that each individual message can be easily ingested.
   
   In the changes that I implemented, the default implementation is left as-is. But the client code has the option to register a callback for individual thread stack traces and lock info. The callback can choose to either build one huge buffer that has everything or break the individual stack traces and log it.




-- 
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



[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.

Posted by GitBox <gi...@apache.org>.
Jason918 commented on a change in pull request #13130:
URL: https://github.com/apache/pulsar/pull/13130#discussion_r833859225



##########
File path: pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/AbstractMetadataStore.java
##########
@@ -77,6 +78,8 @@
     protected AbstractMetadataStore() {
         this.executor = Executors
                 .newSingleThreadScheduledExecutor(new DefaultThreadFactory("metadata-store"));
+        this.executorMonitor = new ExecutorMonitor(this.executor, TimeUnit.MINUTES.toSeconds(5),

Review comment:
       > why would anyone not want to have the monitoring disabled?
   
   1. Debug purpose, so that we can attach into the process and dig deeper.
   2. IMO, currently we want to add this because there are some potential deadlock bug. Not some known issue that can not be fixed. Once we are confident these kind of issue won't happen again, we won't need this.
   




-- 
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



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

Posted by GitBox <gi...@apache.org>.
bharanic-dev commented on a change in pull request #13130:
URL: https://github.com/apache/pulsar/pull/13130#discussion_r835388358



##########
File path: pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/AbstractMetadataStore.java
##########
@@ -77,6 +78,8 @@
     protected AbstractMetadataStore() {
         this.executor = Executors
                 .newSingleThreadScheduledExecutor(new DefaultThreadFactory("metadata-store"));
+        this.executorMonitor = new ExecutorMonitor(this.executor, TimeUnit.MINUTES.toSeconds(5),

Review comment:
       1. If one is able to attach to the process (like a developer), then they should have access to the code and be able to comment out the code. I don't see the knob adding much value to a "user of the Pulsar cluster". The worry I have is the blowing up of plethora of knobs, which over time becomes unmanageable and untestable. This is a small enhancement for which knob does not make sense, IMO. @merlimat @lhotari please chime in if you have an opinion. If others agree, I will go ahead and add the knob.
   
   2. Even if we fix all known deadlock issues, one may still want to have the monitoring enabled to make sure that new code changes don't introduce new deadlock issues.
   
   




-- 
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



[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.

Posted by GitBox <gi...@apache.org>.
Jason918 commented on a change in pull request #13130:
URL: https://github.com/apache/pulsar/pull/13130#discussion_r840267135



##########
File path: pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/AbstractMetadataStore.java
##########
@@ -77,6 +78,8 @@
     protected AbstractMetadataStore() {
         this.executor = Executors
                 .newSingleThreadScheduledExecutor(new DefaultThreadFactory("metadata-store"));
+        this.executorMonitor = new ExecutorMonitor(this.executor, TimeUnit.MINUTES.toSeconds(5),

Review comment:
       Personally, I am agreed with this watchdog but just need a switch in case something goes wrong with this self-killing feature. 




-- 
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



[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.

Posted by GitBox <gi...@apache.org>.
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



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

Posted by GitBox <gi...@apache.org>.
bharanic-dev commented on a change in pull request #13130:
URL: https://github.com/apache/pulsar/pull/13130#discussion_r830397108



##########
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:
       The monitoring task schedules a dummy job once in 5 seconds. And the default timeout for the dummy task is 5 minutes. Once the deadlock is detected, shutdown() is called. The shutdown() better finish within 5 minutes, otherwise something is really wrong. So, it is very unlikely for the stack info to be logged multiple times.




-- 
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



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

Posted by GitBox <gi...@apache.org>.
bharanic-dev commented on a change in pull request #13130:
URL: https://github.com/apache/pulsar/pull/13130#discussion_r763194732



##########
File path: pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/AbstractMetadataStore.java
##########
@@ -317,7 +346,33 @@ public void invalidateAll() {
      */
     protected void execute(Runnable task, CompletableFuture<?> future) {
         try {
-            executor.execute(task);
+            // Wrap the original task, so we can record the thread on which it is running
+            TaskWrapper taskWrapper = new TaskWrapper(task);
+            executorWatchDog.execute(() -> {

Review comment:
       w.r.t your comment: "even if a write operation takes 30seconds, the thread wouldn't normally get blocked for such long time".
   
   Just want to clarify my own understanding. Today, the metadata-store callbacks are invoked by the call to execute method:
   
   https://github.com/apache/pulsar/blob/master/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/AbstractMetadataStore.java#L318
   
   And the executor is a single-threaded executor. So, the callbacks can only be executed one at a time. Like you point out there could be 10s or 100s of such tasks.
   
   With my changes, the additional overhead is that the task being submitted to the watchdog thread, which in turn delegates to the metadata-store executor. So, there is a "wrapper task creation overhead" and additional latency latency. But that sholuld not be too bad, compared to the current code? The write operation could take as long, but the watchdog executor is not blocked. It only gets blocked if the callback task gets blocked (which is the current code as well). Is my understanding correct?
   
   And regarding this comment: "When you get the stack trace for the task that is blocked, you're only seeing the stack trace of a healthy code path, while you won't see the other threads that have caused the deadlock."
   
   While I can't comment on all possible code paths, the specific one in the issue 
   
   https://github.com/apache/pulsar/issues/12929
   
   The metadata-store thread stack trace is the most relevant one and that is the only one that gets logged in the code I added.
   
   If you still think that "submitting a dummy task periodically" is the way to go, I will go work on those changes. But please help clarify the above.




-- 
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



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

Posted by GitBox <gi...@apache.org>.
bharanic-dev commented on a change in pull request #13130:
URL: https://github.com/apache/pulsar/pull/13130#discussion_r763194732



##########
File path: pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/AbstractMetadataStore.java
##########
@@ -317,7 +346,33 @@ public void invalidateAll() {
      */
     protected void execute(Runnable task, CompletableFuture<?> future) {
         try {
-            executor.execute(task);
+            // Wrap the original task, so we can record the thread on which it is running
+            TaskWrapper taskWrapper = new TaskWrapper(task);
+            executorWatchDog.execute(() -> {

Review comment:
       w.r.t your comment: "even if a write operation takes 30seconds, the thread wouldn't normally get blocked for such long time".
   
   Just want to clarify my own understanding. Today, the metadata-store callbacks are invoked by the call to execute method:
   
   https://github.com/apache/pulsar/blob/master/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/AbstractMetadataStore.java#L318
   
   And the executor is a single-threaded executor. So, the callbacks can only be executed one at a time. Like you point out there could be 10s or 100s of such tasks.
   
   With my changes, the additional overhead is that the task being submitted to the watchdog thread, which in turn delegates to the metadata-store executor. So, there is a "wrapper task creation overhead" and additional latency. But that sholuld not be too bad, compared to the current code? The write operation could take as long as it does, but the watchdog executor is not blocked. It only gets blocked if the callback task gets blocked (which is the case in existing code as well). Is my understanding correct?
   
   And regarding this comment: "When you get the stack trace for the task that is blocked, you're only seeing the stack trace of a healthy code path, while you won't see the other threads that have caused the deadlock."
   
   While I can't comment on all possible code paths, the specific one in the issue 
   
   https://github.com/apache/pulsar/issues/12929
   
   The metadata-store thread stack trace is the most relevant one and that is the only one that gets logged in the code I added.
   
   If you still think that "submitting a dummy task periodically" is the way to go, I will go work on those changes. But please help clarify the above.




-- 
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



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

Posted by GitBox <gi...@apache.org>.
bharanic-dev commented on a change in pull request #13130:
URL: https://github.com/apache/pulsar/pull/13130#discussion_r763507842



##########
File path: pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/AbstractMetadataStore.java
##########
@@ -317,7 +346,33 @@ public void invalidateAll() {
      */
     protected void execute(Runnable task, CompletableFuture<?> future) {
         try {
-            executor.execute(task);
+            // Wrap the original task, so we can record the thread on which it is running
+            TaskWrapper taskWrapper = new TaskWrapper(task);
+            executorWatchDog.execute(() -> {

Review comment:
       Thank you. Makes sense, somehow I missed thinking about transitive dependencies.




-- 
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



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

Posted by GitBox <gi...@apache.org>.
bharanic-dev commented on a change in pull request #13130:
URL: https://github.com/apache/pulsar/pull/13130#discussion_r830143180



##########
File path: pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/AbstractMetadataStore.java
##########
@@ -77,6 +78,8 @@
     protected AbstractMetadataStore() {
         this.executor = Executors
                 .newSingleThreadScheduledExecutor(new DefaultThreadFactory("metadata-store"));
+        this.executorMonitor = new ExecutorMonitor(this.executor, TimeUnit.MINUTES.toSeconds(5),

Review comment:
       Can you please elaborate why one would want a config knob for this - why would anyone not want to have the monitoring disabled? Once there is a deadlock, the broker will no longer be functional.
   
   Assuming we still want to add a knob, which part are you suggesting we have config for:
   1) monitoring the executor.
   2) action to take when deadlock is detected (just log and keep chugging along vs restart the broker).
   




-- 
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



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

Posted by GitBox <gi...@apache.org>.
bharanic-dev commented on a change in pull request #13130:
URL: https://github.com/apache/pulsar/pull/13130#discussion_r839763540



##########
File path: pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/AbstractMetadataStore.java
##########
@@ -77,6 +78,8 @@
     protected AbstractMetadataStore() {
         this.executor = Executors
                 .newSingleThreadScheduledExecutor(new DefaultThreadFactory("metadata-store"));
+        this.executorMonitor = new ExecutorMonitor(this.executor, TimeUnit.MINUTES.toSeconds(5),

Review comment:
       @Jason918 can you comment please?  I have stated my concerns above, but if the consensus from the community is to add the knob, I will go ahead and do it.




-- 
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



[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.

Posted by GitBox <gi...@apache.org>.
Jason918 commented on a change in pull request #13130:
URL: https://github.com/apache/pulsar/pull/13130#discussion_r828703163



##########
File path: pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/AbstractMetadataStore.java
##########
@@ -77,6 +78,8 @@
     protected AbstractMetadataStore() {
         this.executor = Executors
                 .newSingleThreadScheduledExecutor(new DefaultThreadFactory("metadata-store"));
+        this.executorMonitor = new ExecutorMonitor(this.executor, TimeUnit.MINUTES.toSeconds(5),

Review comment:
       I suggest a config to enable this feature, and default as false.




-- 
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



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

Posted by GitBox <gi...@apache.org>.
bharanic-dev commented on pull request #13130:
URL: https://github.com/apache/pulsar/pull/13130#issuecomment-1067376658


   @merlimat, @Jason918, @lhotari Updated the PR with the changes suggested. PTAL when you get a chance. Thank you!


-- 
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



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

Posted by GitBox <gi...@apache.org>.
bharanic-dev commented on a change in pull request #13130:
URL: https://github.com/apache/pulsar/pull/13130#discussion_r827184767



##########
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:
       From the monitor perspective, the callback is invoked when it detects that the dummy task has been blocked for the duration of monitoring time. It is up to the client implementation to decide what to do with it. The client could decide to invoke the close() call to terminate the monitoring.
   
   In the case of metadata-store, the monitor is used to detect potential deadlocks, so it calls shutdown(). I think it should be OK to call `consumer.accept()` multiple times.




-- 
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



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

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on pull request #13130:
URL: https://github.com/apache/pulsar/pull/13130#issuecomment-1051440166


   The pr had no activity for 30 days, mark with Stale label.


-- 
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



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

Posted by GitBox <gi...@apache.org>.
merlimat commented on a change in pull request #13130:
URL: https://github.com/apache/pulsar/pull/13130#discussion_r762358130



##########
File path: pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/AbstractMetadataStore.java
##########
@@ -317,7 +346,33 @@ public void invalidateAll() {
      */
     protected void execute(Runnable task, CompletableFuture<?> future) {
         try {
-            executor.execute(task);
+            // Wrap the original task, so we can record the thread on which it is running
+            TaskWrapper taskWrapper = new TaskWrapper(task);
+            executorWatchDog.execute(() -> {

Review comment:
       > Can you please explain what is the problem with tracking each task? 
   
   The problem is that we can have 10s or 100s of thousand of such tasks per each second, each with different durations. This adds an overhead in submitting the tasks to the watcher thread. 
   
   With the other approach, we only need to submit a "dummy" task every few seconds to the executor and check that it can complete. It is also completely independent from the timeout of such operations (eg: even if a write operation takes 30seconds, the thread wouldn't normally get blocked for such long time).
   
   So, I think it's much more efficient to track 1 dummy task every few seconds rather than every single task.
   
   > Dumping the stack trace of all threads to logs could result in large volume of logs
   
   When you get the stack trace for the task that is blocked, you're only seeing the stack trace of a healthy code path, while you won't see the other threads that have caused the deadlock. 
   Most likely the `metadata-store` thread stack trace will be the most relevant one, though not necessarily.
   
   




-- 
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



[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.

Posted by GitBox <gi...@apache.org>.
Jason918 commented on a change in pull request #13130:
URL: https://github.com/apache/pulsar/pull/13130#discussion_r762377508



##########
File path: pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/AbstractMetadataStore.java
##########
@@ -317,7 +346,33 @@ public void invalidateAll() {
      */
     protected void execute(Runnable task, CompletableFuture<?> future) {
         try {
-            executor.execute(task);
+            // Wrap the original task, so we can record the thread on which it is running
+            TaskWrapper taskWrapper = new TaskWrapper(task);
+            executorWatchDog.execute(() -> {

Review comment:
       Agreed on we should dump the stack trace of all threads. 
   In previous deadlock cases, we got deadlock because metadata-store thread depends on it self. But it's possible that, the depend path is like `metadata-store thread` --> async task in other pulsar-executor --> `metadata-store thread`.  So we need to see the whole picture.
   
   Futher more, I think just dumping the stack trace is not enough, We need the lock info of these threads to help locating the deadlock.  We can use `ThreadDumpUtil#buildThreadDiagnosticString` metioned in #13124




-- 
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



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

Posted by GitBox <gi...@apache.org>.
bharanic-dev commented on a change in pull request #13130:
URL: https://github.com/apache/pulsar/pull/13130#discussion_r763194732



##########
File path: pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/AbstractMetadataStore.java
##########
@@ -317,7 +346,33 @@ public void invalidateAll() {
      */
     protected void execute(Runnable task, CompletableFuture<?> future) {
         try {
-            executor.execute(task);
+            // Wrap the original task, so we can record the thread on which it is running
+            TaskWrapper taskWrapper = new TaskWrapper(task);
+            executorWatchDog.execute(() -> {

Review comment:
       w.r.t your comment: "even if a write operation takes 30seconds, the thread wouldn't normally get blocked for such long time".
   
   Just want to clarify my own understanding. Today, the metadata-store callbacks are invoked by the call to execute method:
   
   https://github.com/apache/pulsar/blob/master/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/AbstractMetadataStore.java#L318
   
   And the executor is a single-threaded executor. So, the callbacks can only be executed one at a time. Like you point out there could be 10s or 100s of such tasks.
   
   With my changes, the additional overhead is that the task being submitted to the watchdog thread, which in turn delegates to the metadata-store executor. So, there is a "wrapper task creation overhead" and additional latency. But that sholuld not be too bad, compared to the current code? The write operation could take as long as it does, but the watchdog executor is not blocked. It only gets blocked if the callback task gets blocked (which is the case in existing code as well). Is my understanding correct?
   
   And regarding this comment: "When you get the stack trace for the task that is blocked, you're only seeing the stack trace of a healthy code path, while you won't see the other threads that have caused the deadlock."
   
   While I can't comment on all possible code paths, the specific one in the issue 
   
   https://github.com/apache/pulsar/issues/12929
   
   The metadata-store thread stack trace is the most relevant one and that is the only one that gets logged in the code I added.
   
   If you still think that "submitting a dummy task periodically" is the way to go, I will go work on those changes. But please help clarify the above.




-- 
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



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

Posted by GitBox <gi...@apache.org>.
bharanic-dev commented on a change in pull request #13130:
URL: https://github.com/apache/pulsar/pull/13130#discussion_r763507842



##########
File path: pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/AbstractMetadataStore.java
##########
@@ -317,7 +346,33 @@ public void invalidateAll() {
      */
     protected void execute(Runnable task, CompletableFuture<?> future) {
         try {
-            executor.execute(task);
+            // Wrap the original task, so we can record the thread on which it is running
+            TaskWrapper taskWrapper = new TaskWrapper(task);
+            executorWatchDog.execute(() -> {

Review comment:
       @Jason918 thank you. Makes sense, somehow I missed thinking about transitive dependencies.




-- 
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