You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@geode.apache.org by GitBox <gi...@apache.org> on 2020/11/19 21:13:24 UTC

[GitHub] [geode] dschneider-pivotal opened a new pull request #5763: GEODE-8521: detect if a p2p reader thread is hung

dschneider-pivotal opened a new pull request #5763:
URL: https://github.com/apache/geode/pull/5763


   Each p2p reader thread now creates a single instance of AbstractExecutor
   and registers it with the ThreadsMonitoring instance while it is "dispatching"
   the message. It will not be monitored while it is waiting on the socket to read
   a message. So this will only report stuck p2p readers that are stuck dispatch
   which for inline processing includes processing the message and reading the
   direct ack.
   
   Thank you for submitting a contribution to Apache Geode.
   
   In order to streamline the review of the contribution we ask you
   to ensure the following steps have been taken:
   
   ### For all changes:
   - [ ] Is there a JIRA ticket associated with this PR? Is it referenced in the commit message?
   
   - [ ] Has your PR been rebased against the latest commit within the target branch (typically `develop`)?
   
   - [ ] Is your initial contribution a single, squashed commit?
   
   - [ ] Does `gradlew build` run cleanly?
   
   - [ ] Have you written or updated unit tests to verify your changes?
   
   - [ ] If adding new dependencies to the code, are these dependencies licensed in a way that is compatible for inclusion under [ASF 2.0](http://www.apache.org/legal/resolved.html#category-a)?
   
   ### Note:
   Please ensure that once the PR is submitted, check Concourse for build issues and
   submit an update to your PR as soon as possible. If you need help, please send an
   email to dev@geode.apache.org.
   


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



[GitHub] [geode] bschuchardt commented on a change in pull request #5763: GEODE-8521: detect if a p2p reader thread is hung

Posted by GitBox <gi...@apache.org>.
bschuchardt commented on a change in pull request #5763:
URL: https://github.com/apache/geode/pull/5763#discussion_r527801127



##########
File path: geode-core/src/main/java/org/apache/geode/internal/monitoring/ThreadsMonitoringImpl.java
##########
@@ -96,39 +98,60 @@ public void updateThreadStatus() {
 
   @Override
   public boolean startMonitor(Mode mode) {
-    AbstractExecutor absExtgroup;
+    return startMonitoring(createAbstractExecutor(mode));
+  }
+
+  @Override
+  public void endMonitor() {
+    this.monitorMap.remove(Thread.currentThread().getId());
+  }
+
+  @VisibleForTesting
+  boolean isMonitoring() {
+    return monitorMap.containsKey(Thread.currentThread().getId());
+  }
+
+  @Override
+  public AbstractExecutor createAbstractExecutor(Mode mode) {
     switch (mode) {
       case FunctionExecutor:
-        absExtgroup = new FunctionExecutionPooledExecutorGroup(this);
-        break;
+        return new FunctionExecutionPooledExecutorGroup();
       case PooledExecutor:
-        absExtgroup = new PooledExecutorGroup(this);
-        break;
+        return new PooledExecutorGroup();
       case SerialQueuedExecutor:
-        absExtgroup = new SerialQueuedExecutorGroup(this);
-        break;
+        return new SerialQueuedExecutorGroup();
       case OneTaskOnlyExecutor:
-        absExtgroup = new OneTaskOnlyExecutorGroup(this);
-        break;
+        return new OneTaskOnlyExecutorGroup();
       case ScheduledThreadExecutor:
-        absExtgroup = new ScheduledThreadPoolExecutorWKAGroup(this);
-        break;
+        return new ScheduledThreadPoolExecutorWKAGroup();
       case AGSExecutor:
-        absExtgroup = new GatewaySenderEventProcessorGroup(this);
-        break;
+        return new GatewaySenderEventProcessorGroup();
+      case P2PReaderExecutor:
+        return new P2PReaderExecutorGroup();
       default:
-        return false;
+        throw new IllegalStateException("Unhandled mode=" + mode);
     }
-    this.monitorMap.put(Thread.currentThread().getId(), absExtgroup);
+  }
+
+  @Override
+  public boolean startMonitoring(AbstractExecutor executor) {
+    executor.setStartTime(System.currentTimeMillis());

Review comment:
       I'm a little concerned about the impact of taking yet another millisecond clock reading on every message that we dispatch.  This should be run through extensive performance testing before merging it to develop.




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



[GitHub] [geode] dschneider-pivotal commented on a change in pull request #5763: GEODE-8521: detect if a p2p reader thread is hung

Posted by GitBox <gi...@apache.org>.
dschneider-pivotal commented on a change in pull request #5763:
URL: https://github.com/apache/geode/pull/5763#discussion_r527838255



##########
File path: geode-core/src/main/java/org/apache/geode/internal/monitoring/ThreadsMonitoringImpl.java
##########
@@ -96,39 +98,60 @@ public void updateThreadStatus() {
 
   @Override
   public boolean startMonitor(Mode mode) {
-    AbstractExecutor absExtgroup;
+    return startMonitoring(createAbstractExecutor(mode));
+  }
+
+  @Override
+  public void endMonitor() {
+    this.monitorMap.remove(Thread.currentThread().getId());
+  }
+
+  @VisibleForTesting
+  boolean isMonitoring() {
+    return monitorMap.containsKey(Thread.currentThread().getId());
+  }
+
+  @Override
+  public AbstractExecutor createAbstractExecutor(Mode mode) {
     switch (mode) {
       case FunctionExecutor:
-        absExtgroup = new FunctionExecutionPooledExecutorGroup(this);
-        break;
+        return new FunctionExecutionPooledExecutorGroup();
       case PooledExecutor:
-        absExtgroup = new PooledExecutorGroup(this);
-        break;
+        return new PooledExecutorGroup();
       case SerialQueuedExecutor:
-        absExtgroup = new SerialQueuedExecutorGroup(this);
-        break;
+        return new SerialQueuedExecutorGroup();
       case OneTaskOnlyExecutor:
-        absExtgroup = new OneTaskOnlyExecutorGroup(this);
-        break;
+        return new OneTaskOnlyExecutorGroup();
       case ScheduledThreadExecutor:
-        absExtgroup = new ScheduledThreadPoolExecutorWKAGroup(this);
-        break;
+        return new ScheduledThreadPoolExecutorWKAGroup();
       case AGSExecutor:
-        absExtgroup = new GatewaySenderEventProcessorGroup(this);
-        break;
+        return new GatewaySenderEventProcessorGroup();
+      case P2PReaderExecutor:
+        return new P2PReaderExecutorGroup();
       default:
-        return false;
+        throw new IllegalStateException("Unhandled mode=" + mode);
     }
-    this.monitorMap.put(Thread.currentThread().getId(), absExtgroup);
+  }
+
+  @Override
+  public boolean startMonitoring(AbstractExecutor executor) {
+    executor.setStartTime(System.currentTimeMillis());

Review comment:
       I had the same thought. I was even concerned about constantly adding and removing from the concurrent map. We could just use the frequency of the monitor thread to do the time work. For example if it finds an AbstractExecutor in the map that it has not seen before then the monitor thread could set the time it first saw it. If later monitor samples see it again it could use this time (i.e. the one the monitor set) to determine if it is stuck. This could even be done without timestamps. Just a simple counter the monitor incs and the p2p reader thread clears. If the monitor thread is waking up at a fixed interval then you know the thread has been stuck for at least that much time. I like your idea of having custom code in P2PReaderExecutorGroup for this. When I was thinking about it I thought I would need to change the behavior of all the existing threads being monitored. Thanks for the feedback!




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



[GitHub] [geode] bschuchardt commented on a change in pull request #5763: GEODE-8521: detect if a p2p reader thread is hung

Posted by GitBox <gi...@apache.org>.
bschuchardt commented on a change in pull request #5763:
URL: https://github.com/apache/geode/pull/5763#discussion_r527913185



##########
File path: geode-core/src/main/java/org/apache/geode/internal/monitoring/ThreadsMonitoringImpl.java
##########
@@ -96,39 +98,60 @@ public void updateThreadStatus() {
 
   @Override
   public boolean startMonitor(Mode mode) {
-    AbstractExecutor absExtgroup;
+    return startMonitoring(createAbstractExecutor(mode));
+  }
+
+  @Override
+  public void endMonitor() {
+    this.monitorMap.remove(Thread.currentThread().getId());
+  }
+
+  @VisibleForTesting
+  boolean isMonitoring() {
+    return monitorMap.containsKey(Thread.currentThread().getId());
+  }
+
+  @Override
+  public AbstractExecutor createAbstractExecutor(Mode mode) {
     switch (mode) {
       case FunctionExecutor:
-        absExtgroup = new FunctionExecutionPooledExecutorGroup(this);
-        break;
+        return new FunctionExecutionPooledExecutorGroup();
       case PooledExecutor:
-        absExtgroup = new PooledExecutorGroup(this);
-        break;
+        return new PooledExecutorGroup();
       case SerialQueuedExecutor:
-        absExtgroup = new SerialQueuedExecutorGroup(this);
-        break;
+        return new SerialQueuedExecutorGroup();
       case OneTaskOnlyExecutor:
-        absExtgroup = new OneTaskOnlyExecutorGroup(this);
-        break;
+        return new OneTaskOnlyExecutorGroup();
       case ScheduledThreadExecutor:
-        absExtgroup = new ScheduledThreadPoolExecutorWKAGroup(this);
-        break;
+        return new ScheduledThreadPoolExecutorWKAGroup();
       case AGSExecutor:
-        absExtgroup = new GatewaySenderEventProcessorGroup(this);
-        break;
+        return new GatewaySenderEventProcessorGroup();
+      case P2PReaderExecutor:
+        return new P2PReaderExecutorGroup();
       default:
-        return false;
+        throw new IllegalStateException("Unhandled mode=" + mode);
     }
-    this.monitorMap.put(Thread.currentThread().getId(), absExtgroup);
+  }
+
+  @Override
+  public boolean startMonitoring(AbstractExecutor executor) {
+    executor.setStartTime(System.currentTimeMillis());

Review comment:
       having the monitor thread set the time is a good idea.  Another thing you could do is have the Monitor thread record the current time when it wakes up and use that for new entries in the map.  GMSHealthMonitor's monitor thread does something like that.




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



[GitHub] [geode] dschneider-pivotal commented on a change in pull request #5763: GEODE-8521: detect if a p2p reader thread is hung

Posted by GitBox <gi...@apache.org>.
dschneider-pivotal commented on a change in pull request #5763:
URL: https://github.com/apache/geode/pull/5763#discussion_r532929942



##########
File path: geode-core/src/test/java/org/apache/geode/internal/monitoring/executor/P2PReaderExecutorGroupJUnitTest.java
##########
@@ -0,0 +1,53 @@
+/*
+ * 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.geode.internal.monitoring.executor;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+import org.junit.Test;
+
+public class P2PReaderExecutorGroupJUnitTest {

Review comment:
       done




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



[GitHub] [geode] dschneider-pivotal commented on a change in pull request #5763: GEODE-8521: detect if a p2p reader thread is hung

Posted by GitBox <gi...@apache.org>.
dschneider-pivotal commented on a change in pull request #5763:
URL: https://github.com/apache/geode/pull/5763#discussion_r528011618



##########
File path: geode-core/src/main/java/org/apache/geode/internal/monitoring/ThreadsMonitoringImpl.java
##########
@@ -96,39 +98,60 @@ public void updateThreadStatus() {
 
   @Override
   public boolean startMonitor(Mode mode) {
-    AbstractExecutor absExtgroup;
+    return startMonitoring(createAbstractExecutor(mode));
+  }
+
+  @Override
+  public void endMonitor() {
+    this.monitorMap.remove(Thread.currentThread().getId());
+  }
+
+  @VisibleForTesting
+  boolean isMonitoring() {
+    return monitorMap.containsKey(Thread.currentThread().getId());
+  }
+
+  @Override
+  public AbstractExecutor createAbstractExecutor(Mode mode) {
     switch (mode) {
       case FunctionExecutor:
-        absExtgroup = new FunctionExecutionPooledExecutorGroup(this);
-        break;
+        return new FunctionExecutionPooledExecutorGroup();
       case PooledExecutor:
-        absExtgroup = new PooledExecutorGroup(this);
-        break;
+        return new PooledExecutorGroup();
       case SerialQueuedExecutor:
-        absExtgroup = new SerialQueuedExecutorGroup(this);
-        break;
+        return new SerialQueuedExecutorGroup();
       case OneTaskOnlyExecutor:
-        absExtgroup = new OneTaskOnlyExecutorGroup(this);
-        break;
+        return new OneTaskOnlyExecutorGroup();
       case ScheduledThreadExecutor:
-        absExtgroup = new ScheduledThreadPoolExecutorWKAGroup(this);
-        break;
+        return new ScheduledThreadPoolExecutorWKAGroup();
       case AGSExecutor:
-        absExtgroup = new GatewaySenderEventProcessorGroup(this);
-        break;
+        return new GatewaySenderEventProcessorGroup();
+      case P2PReaderExecutor:
+        return new P2PReaderExecutorGroup();
       default:
-        return false;
+        throw new IllegalStateException("Unhandled mode=" + mode);
     }
-    this.monitorMap.put(Thread.currentThread().getId(), absExtgroup);
+  }
+
+  @Override
+  public boolean startMonitoring(AbstractExecutor executor) {
+    executor.setStartTime(System.currentTimeMillis());

Review comment:
       the p2p reader now simply sets a volatile boolean before and after messageDispatch.
   the startTime for a p2p reader will now be set by the monitor thread.
   Let me know what you think




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



[GitHub] [geode] dschneider-pivotal commented on a change in pull request #5763: GEODE-8521: detect if a p2p reader thread is hung

Posted by GitBox <gi...@apache.org>.
dschneider-pivotal commented on a change in pull request #5763:
URL: https://github.com/apache/geode/pull/5763#discussion_r532930752



##########
File path: geode-core/src/main/java/org/apache/geode/internal/monitoring/ThreadsMonitoringProcess.java
##########
@@ -33,73 +31,86 @@
 
 public class ThreadsMonitoringProcess extends TimerTask {
 
-  private final ThreadsMonitoring threadsMonitoring;
-  private ResourceManagerStats resourceManagerStats = null;
   private static final Logger logger = LogService.getLogger();
+
+  private final ThreadsMonitoring threadsMonitoring;
   private final int timeLimit;
   private final InternalDistributedSystem internalDistributedSystem;
 
-  private final Properties nonDefault = new Properties();
-  private final DistributionConfigImpl distributionConfigImpl =
-      new DistributionConfigImpl(nonDefault);
+  private ResourceManagerStats resourceManagerStats = null;
 
   protected ThreadsMonitoringProcess(ThreadsMonitoring tMonitoring,
-      InternalDistributedSystem iDistributedSystem) {
-    this.timeLimit = this.distributionConfigImpl.getThreadMonitorTimeLimit();
+      InternalDistributedSystem iDistributedSystem, int timeLimit) {

Review comment:
       I changed the name to timeLimitMillis. This unit is determined by an external geode property so it will always be milliseconds

##########
File path: geode-core/src/test/java/org/apache/geode/internal/monitoring/ThreadsMonitoringProcessJUnitTest.java
##########
@@ -34,11 +31,13 @@
  */
 public class ThreadsMonitoringProcessJUnitTest {
 
+  private static final int TIME_LIMIT = 1000;

Review comment:
       done




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



[GitHub] [geode] dschneider-pivotal commented on a change in pull request #5763: GEODE-8521: detect if a p2p reader thread is hung

Posted by GitBox <gi...@apache.org>.
dschneider-pivotal commented on a change in pull request #5763:
URL: https://github.com/apache/geode/pull/5763#discussion_r527859411



##########
File path: geode-core/src/main/java/org/apache/geode/internal/monitoring/ThreadsMonitoringImpl.java
##########
@@ -96,39 +98,60 @@ public void updateThreadStatus() {
 
   @Override
   public boolean startMonitor(Mode mode) {
-    AbstractExecutor absExtgroup;
+    return startMonitoring(createAbstractExecutor(mode));
+  }
+
+  @Override
+  public void endMonitor() {
+    this.monitorMap.remove(Thread.currentThread().getId());
+  }
+
+  @VisibleForTesting
+  boolean isMonitoring() {
+    return monitorMap.containsKey(Thread.currentThread().getId());
+  }
+
+  @Override
+  public AbstractExecutor createAbstractExecutor(Mode mode) {
     switch (mode) {
       case FunctionExecutor:
-        absExtgroup = new FunctionExecutionPooledExecutorGroup(this);
-        break;
+        return new FunctionExecutionPooledExecutorGroup();
       case PooledExecutor:
-        absExtgroup = new PooledExecutorGroup(this);
-        break;
+        return new PooledExecutorGroup();
       case SerialQueuedExecutor:
-        absExtgroup = new SerialQueuedExecutorGroup(this);
-        break;
+        return new SerialQueuedExecutorGroup();
       case OneTaskOnlyExecutor:
-        absExtgroup = new OneTaskOnlyExecutorGroup(this);
-        break;
+        return new OneTaskOnlyExecutorGroup();
       case ScheduledThreadExecutor:
-        absExtgroup = new ScheduledThreadPoolExecutorWKAGroup(this);
-        break;
+        return new ScheduledThreadPoolExecutorWKAGroup();
       case AGSExecutor:
-        absExtgroup = new GatewaySenderEventProcessorGroup(this);
-        break;
+        return new GatewaySenderEventProcessorGroup();
+      case P2PReaderExecutor:
+        return new P2PReaderExecutorGroup();
       default:
-        return false;
+        throw new IllegalStateException("Unhandled mode=" + mode);
     }
-    this.monitorMap.put(Thread.currentThread().getId(), absExtgroup);
+  }
+
+  @Override
+  public boolean startMonitoring(AbstractExecutor executor) {
+    executor.setStartTime(System.currentTimeMillis());

Review comment:
       I converted back to draft mode. Will rework this and submit it again.




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



[GitHub] [geode] dschneider-pivotal commented on pull request #5763: GEODE-8521: detect if a p2p reader thread is hung

Posted by GitBox <gi...@apache.org>.
dschneider-pivotal commented on pull request #5763:
URL: https://github.com/apache/geode/pull/5763#issuecomment-738499171


   the dunit failure is the known issue: GEODE-8573


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



[GitHub] [geode] kirklund commented on a change in pull request #5763: GEODE-8521: detect if a p2p reader thread is hung

Posted by GitBox <gi...@apache.org>.
kirklund commented on a change in pull request #5763:
URL: https://github.com/apache/geode/pull/5763#discussion_r532766936



##########
File path: geode-core/src/test/java/org/apache/geode/internal/monitoring/executor/P2PReaderExecutorGroupJUnitTest.java
##########
@@ -0,0 +1,53 @@
+/*
+ * 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.geode.internal.monitoring.executor;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+import org.junit.Test;
+
+public class P2PReaderExecutorGroupJUnitTest {

Review comment:
       I recommend just naming unit tests as P2PReaderExecutorGroupTest (no need to include JUnit)

##########
File path: geode-core/src/test/java/org/apache/geode/internal/monitoring/ThreadsMonitoringProcessJUnitTest.java
##########
@@ -34,11 +31,13 @@
  */
 public class ThreadsMonitoringProcessJUnitTest {
 
+  private static final int TIME_LIMIT = 1000;

Review comment:
       Please indicate the units of measure in the name or an additional parameter.

##########
File path: geode-core/src/main/java/org/apache/geode/internal/monitoring/ThreadsMonitoringProcess.java
##########
@@ -33,73 +31,86 @@
 
 public class ThreadsMonitoringProcess extends TimerTask {
 
-  private final ThreadsMonitoring threadsMonitoring;
-  private ResourceManagerStats resourceManagerStats = null;
   private static final Logger logger = LogService.getLogger();
+
+  private final ThreadsMonitoring threadsMonitoring;
   private final int timeLimit;
   private final InternalDistributedSystem internalDistributedSystem;
 
-  private final Properties nonDefault = new Properties();
-  private final DistributionConfigImpl distributionConfigImpl =
-      new DistributionConfigImpl(nonDefault);
+  private ResourceManagerStats resourceManagerStats = null;
 
   protected ThreadsMonitoringProcess(ThreadsMonitoring tMonitoring,
-      InternalDistributedSystem iDistributedSystem) {
-    this.timeLimit = this.distributionConfigImpl.getThreadMonitorTimeLimit();
+      InternalDistributedSystem iDistributedSystem, int timeLimit) {

Review comment:
       What unit is timeLimit? I recommend either adding a TimeUnit parameter or renaming timeLimit to include Millis or something as a prefix.




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



[GitHub] [geode] dschneider-pivotal merged pull request #5763: GEODE-8521: detect if a p2p reader thread is hung

Posted by GitBox <gi...@apache.org>.
dschneider-pivotal merged pull request #5763:
URL: https://github.com/apache/geode/pull/5763


   


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