You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@accumulo.apache.org by GitBox <gi...@apache.org> on 2022/07/26 11:52:36 UTC

[GitHub] [accumulo] dlmarion commented on a diff in pull request #2827: refactor GarbageCollectionLogger so one instance per context

dlmarion commented on code in PR #2827:
URL: https://github.com/apache/accumulo/pull/2827#discussion_r929863244


##########
server/base/src/main/java/org/apache/accumulo/server/AbstractServer.java:
##########
@@ -54,6 +54,8 @@ protected AbstractServer(String appName, ServerOpts opts, String[] args) {
       // Server-side "client" check to make sure we're logged in as a user we expect to be
       context.enforceKerberosLogin();
     }
+    // start jvn gc logging

Review Comment:
   ```suggestion
       // start jvm gc logging
   ```



##########
server/base/src/main/java/org/apache/accumulo/server/GarbageCollectionLogger.java:
##########
@@ -22,109 +22,113 @@
 import java.lang.management.ManagementFactory;
 import java.util.HashMap;
 import java.util.List;
-import java.util.concurrent.locks.Lock;
-import java.util.concurrent.locks.ReentrantLock;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
 
-import org.apache.accumulo.core.conf.AccumuloConfiguration;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.util.Halt;
+import org.apache.accumulo.core.util.threads.ThreadPools;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 public class GarbageCollectionLogger {
   private static final Logger log = LoggerFactory.getLogger(GarbageCollectionLogger.class);
 
+  private final ServerContext context;
+
   private final HashMap<String,Long> prevGcTime = new HashMap<>();
   private long lastMemorySize = 0;
   private long gcTimeIncreasedCount = 0;
-  private static long lastMemoryCheckTime = 0;
-  private static final Lock memCheckTimeLock = new ReentrantLock();
-
-  public void logGCInfo(AccumuloConfiguration conf) {
+  private long lastMemoryCheckTime = 0;
 
-    memCheckTimeLock.lock();
-    try {
-      final long now = System.currentTimeMillis();
+  private static final long TIME_BETWEEN_GC_CHECKS_MIN = 5;
 
-      List<GarbageCollectorMXBean> gcmBeans = ManagementFactory.getGarbageCollectorMXBeans();
-      Runtime rt = Runtime.getRuntime();
+  public GarbageCollectionLogger(final ServerContext context) {
+    this.context = context;
 
-      StringBuilder sb = new StringBuilder("gc");
+    ScheduledFuture<?> future = context.getScheduledExecutor()
+        .scheduleWithFixedDelay(this::logGCInfo, 0, TIME_BETWEEN_GC_CHECKS_MIN, TimeUnit.MINUTES);
+    ThreadPools.watchNonCriticalScheduledTask(future);
+  }
 
-      boolean sawChange = false;
+  public synchronized void logGCInfo() {

Review Comment:
   I'm wondering if this method needs to be synchronized. Do we know what the lock in the previous version of the code is protecting? Is it the JMX MBeans? It looks like the only time this is called is either from the background thread, when the server is stopping, or when Halt is being invoked.



##########
server/base/src/main/java/org/apache/accumulo/server/GarbageCollectionLogger.java:
##########
@@ -22,109 +22,113 @@
 import java.lang.management.ManagementFactory;
 import java.util.HashMap;
 import java.util.List;
-import java.util.concurrent.locks.Lock;
-import java.util.concurrent.locks.ReentrantLock;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
 
-import org.apache.accumulo.core.conf.AccumuloConfiguration;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.util.Halt;
+import org.apache.accumulo.core.util.threads.ThreadPools;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 public class GarbageCollectionLogger {
   private static final Logger log = LoggerFactory.getLogger(GarbageCollectionLogger.class);
 
+  private final ServerContext context;
+
   private final HashMap<String,Long> prevGcTime = new HashMap<>();
   private long lastMemorySize = 0;
   private long gcTimeIncreasedCount = 0;
-  private static long lastMemoryCheckTime = 0;
-  private static final Lock memCheckTimeLock = new ReentrantLock();
-
-  public void logGCInfo(AccumuloConfiguration conf) {
+  private long lastMemoryCheckTime = 0;
 
-    memCheckTimeLock.lock();
-    try {
-      final long now = System.currentTimeMillis();
+  private static final long TIME_BETWEEN_GC_CHECKS_MIN = 5;
 
-      List<GarbageCollectorMXBean> gcmBeans = ManagementFactory.getGarbageCollectorMXBeans();
-      Runtime rt = Runtime.getRuntime();
+  public GarbageCollectionLogger(final ServerContext context) {
+    this.context = context;
 
-      StringBuilder sb = new StringBuilder("gc");
+    ScheduledFuture<?> future = context.getScheduledExecutor()
+        .scheduleWithFixedDelay(this::logGCInfo, 0, TIME_BETWEEN_GC_CHECKS_MIN, TimeUnit.MINUTES);
+    ThreadPools.watchNonCriticalScheduledTask(future);
+  }
 
-      boolean sawChange = false;
+  public synchronized void logGCInfo() {
+    final long now = System.currentTimeMillis();
 
-      long maxIncreaseInCollectionTime = 0;
+    List<GarbageCollectorMXBean> gcmBeans = ManagementFactory.getGarbageCollectorMXBeans();
+    Runtime rt = Runtime.getRuntime();
 
-      for (GarbageCollectorMXBean gcBean : gcmBeans) {
-        Long prevTime = prevGcTime.get(gcBean.getName());
-        long pt = 0;
-        if (prevTime != null) {
-          pt = prevTime;
-        }
+    StringBuilder sb = new StringBuilder("gc");
 
-        long time = gcBean.getCollectionTime();
+    boolean sawChange = false;
 
-        if (time - pt != 0) {
-          sawChange = true;
-        }
+    long maxIncreaseInCollectionTime = 0;
 
-        long increaseInCollectionTime = time - pt;
-        sb.append(String.format(" %s=%,.2f(+%,.2f) secs", gcBean.getName(), time / 1000.0,
-            increaseInCollectionTime / 1000.0));
-        maxIncreaseInCollectionTime =
-            Math.max(increaseInCollectionTime, maxIncreaseInCollectionTime);
-        prevGcTime.put(gcBean.getName(), time);
+    for (GarbageCollectorMXBean gcBean : gcmBeans) {
+      Long prevTime = prevGcTime.get(gcBean.getName());
+      long pt = 0;
+      if (prevTime != null) {
+        pt = prevTime;
       }
 
-      long mem = rt.freeMemory();
-      if (maxIncreaseInCollectionTime == 0) {
-        gcTimeIncreasedCount = 0;
-      } else {
-        gcTimeIncreasedCount++;
-        if (gcTimeIncreasedCount > 3 && mem < rt.maxMemory() * 0.05) {
-          log.warn("Running low on memory");
-          gcTimeIncreasedCount = 0;
-        }
-      }
+      long time = gcBean.getCollectionTime();
 
-      if (mem != lastMemorySize) {
+      if (time - pt != 0) {
         sawChange = true;
       }
 
-      String sign = "+";
-      if (mem - lastMemorySize <= 0) {
-        sign = "";
+      long increaseInCollectionTime = time - pt;
+      sb.append(String.format(" %s=%,.2f(+%,.2f) secs", gcBean.getName(), time / 1000.0,
+          increaseInCollectionTime / 1000.0));
+      maxIncreaseInCollectionTime = Math.max(increaseInCollectionTime, maxIncreaseInCollectionTime);
+      prevGcTime.put(gcBean.getName(), time);
+    }
+
+    long mem = rt.freeMemory();
+    if (maxIncreaseInCollectionTime == 0) {
+      gcTimeIncreasedCount = 0;
+    } else {
+      gcTimeIncreasedCount++;
+      if (gcTimeIncreasedCount > 3 && mem < rt.maxMemory() * 0.05) {
+        log.warn("Running low on memory");
+        gcTimeIncreasedCount = 0;
       }
+    }
+
+    if (mem != lastMemorySize) {
+      sawChange = true;
+    }
 
-      sb.append(String.format(" freemem=%,d(%s%,d) totalmem=%,d", mem, sign, (mem - lastMemorySize),
-          rt.totalMemory()));
+    String sign = "+";
+    if (mem - lastMemorySize <= 0) {
+      sign = "";
+    }
 
-      if (sawChange) {
-        log.debug(sb.toString());
-      }
+    sb.append(String.format(" freemem=%,d(%s%,d) totalmem=%,d", mem, sign, (mem - lastMemorySize),
+        rt.totalMemory()));
 
-      final long keepAliveTimeout = conf.getTimeInMillis(Property.INSTANCE_ZK_TIMEOUT);
-      if (lastMemoryCheckTime > 0 && lastMemoryCheckTime < now) {
-        final long diff = now - lastMemoryCheckTime;
-        if (diff > keepAliveTimeout + 1000) {
-          log.warn(String.format(
-              "GC pause checker not called in a timely"
-                  + " fashion. Expected every %.1f seconds but was %.1f seconds since last check",
-              keepAliveTimeout / 1000., diff / 1000.));
-        }
-        lastMemoryCheckTime = now;
-        return;
-      }
+    if (sawChange) {
+      log.debug(sb.toString());
+    }
 
-      if (maxIncreaseInCollectionTime > keepAliveTimeout) {
-        Halt.halt("Garbage collection may be interfering with lock keep-alive.  Halting.", -1);
+    final long keepAliveTimeout =
+        context.getConfiguration().getTimeInMillis(Property.INSTANCE_ZK_TIMEOUT);
+    if (lastMemoryCheckTime > 0 && lastMemoryCheckTime < now) {
+      final long diff = now - lastMemoryCheckTime;
+      if (diff > keepAliveTimeout + 1000) {
+        log.warn(String.format(
+            "GC pause checker not called in a timely"
+                + " fashion. Expected every %.1f seconds but was %.1f seconds since last check",
+            keepAliveTimeout / 1000., diff / 1000.));
       }
-
-      lastMemorySize = mem;
       lastMemoryCheckTime = now;
-    } finally {
-      memCheckTimeLock.unlock();
+      return;
     }
-  }
 
+    if (maxIncreaseInCollectionTime > keepAliveTimeout) {
+      Halt.halt("Garbage collection may be interfering with lock keep-alive.  Halting.", -1);

Review Comment:
   I'm wondering if we need to still preemptively Halt the server since we have better exception handling in background threads now. If an OOME due to GC Overhead is thrown in a background thread it will halt the VM.



-- 
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: notifications-unsubscribe@accumulo.apache.org

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