This is an automated email from the ASF dual-hosted git repository.

dlmarion pushed a commit to branch 2.1
in repository https://gitbox.apache.org/repos/asf/accumulo.git


The following commit(s) were added to refs/heads/2.1 by this push:
     new 221259e12e Log message when Tablet has been unloading for over 15 
minutes (#4558)
221259e12e is described below

commit 221259e12e1cb390dcd5d2dfc93489a27d6e0d25
Author: Dave Marion <dlmar...@apache.org>
AuthorDate: Fri May 24 12:46:44 2024 -0400

    Log message when Tablet has been unloading for over 15 minutes (#4558)
    
    Created an abstract ConditionalLogger class with two implementations. The
    EscalatingLogger will conditionally log at a higher level and the
    deduplicating logger will conditionally suppress log messages. Wired up the
    deduplicating logger in the UnloadTabletHandler to suppress multiple 
invocations
    of unload and wired up the escalating logger in the TabletGroupWatcher when
    the same tablet has been requested to be unloaded.
    
    Closes #4539
---
 core/pom.xml                                       |  16 ++
 .../accumulo/core/logging/ConditionalLogger.java   | 194 +++++++++++++++++++++
 .../core/logging/DeduplicatingLoggerTest.java      |  69 ++++++++
 .../core/logging/EscalatingLoggerTest.java         |  77 ++++++++
 .../accumulo/manager/TabletGroupWatcher.java       |  11 +-
 .../accumulo/tserver/UnloadTabletHandler.java      |   1 -
 .../org/apache/accumulo/tserver/tablet/Tablet.java |  22 ++-
 7 files changed, 385 insertions(+), 5 deletions(-)

diff --git a/core/pom.xml b/core/pom.xml
index 6e4ffc9883..57c8c425b4 100644
--- a/core/pom.xml
+++ b/core/pom.xml
@@ -152,11 +152,27 @@
       <artifactId>hadoop-client-runtime</artifactId>
       <scope>runtime</scope>
     </dependency>
+    <!-- bnd dependency added due to lint issue, see 
https://github.com/apache/logging-log4j2/issues/2232 -->
+    <dependency>
+      <groupId>biz.aQute.bnd</groupId>
+      <artifactId>biz.aQute.bnd.annotation</artifactId>
+      <scope>test</scope>
+    </dependency>
     <dependency>
       <groupId>org.apache.hadoop</groupId>
       <artifactId>hadoop-client-minicluster</artifactId>
       <scope>test</scope>
     </dependency>
+    <dependency>
+      <groupId>org.apache.logging.log4j</groupId>
+      <artifactId>log4j-api</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.logging.log4j</groupId>
+      <artifactId>log4j-core</artifactId>
+      <scope>test</scope>
+    </dependency>
     <dependency>
       <groupId>org.apache.logging.log4j</groupId>
       <artifactId>log4j-slf4j2-impl</artifactId>
diff --git 
a/core/src/main/java/org/apache/accumulo/core/logging/ConditionalLogger.java 
b/core/src/main/java/org/apache/accumulo/core/logging/ConditionalLogger.java
new file mode 100644
index 0000000000..6da6454f06
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/logging/ConditionalLogger.java
@@ -0,0 +1,194 @@
+/*
+ * 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
+ *
+ *   https://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.accumulo.core.logging;
+
+import java.time.Duration;
+import java.util.Arrays;
+import java.util.List;
+import java.util.concurrent.ConcurrentMap;
+import java.util.function.BiFunction;
+
+import org.apache.accumulo.core.util.Pair;
+import org.slf4j.Logger;
+import org.slf4j.Marker;
+import org.slf4j.event.Level;
+import org.slf4j.helpers.AbstractLogger;
+
+import com.github.benmanes.caffeine.cache.Cache;
+import com.github.benmanes.caffeine.cache.Caffeine;
+
+/**
+ * Logger that wraps another Logger and only emits a log message once per the 
supplied duration.
+ *
+ */
+public abstract class ConditionalLogger extends AbstractLogger {
+
+  private static final long serialVersionUID = 1L;
+
+  /**
+   * A Logger implementation that will log a message at the supplied elevated 
level if it has not
+   * been seen in the supplied duration. For repeat occurrences the message 
will be logged at the
+   * level used in code (which is likely a lower level). Note that the first 
log message will be
+   * logged at the elevated level because it has not been seen before.
+   */
+  public static class EscalatingLogger extends DeduplicatingLogger {
+
+    private static final long serialVersionUID = 1L;
+    private final Level elevatedLevel;
+
+    public EscalatingLogger(Logger log, Duration threshold, long 
maxCachedLogMessages,
+        Level elevatedLevel) {
+      super(log, threshold, maxCachedLogMessages);
+      this.elevatedLevel = elevatedLevel;
+    }
+
+    @Override
+    protected void handleNormalizedLoggingCall(Level level, Marker marker, 
String messagePattern,
+        Object[] arguments, Throwable throwable) {
+
+      if (arguments == null) {
+        arguments = new Object[0];
+      }
+      if (!condition.apply(messagePattern, Arrays.asList(arguments))) {
+        
delegate.atLevel(level).addMarker(marker).setCause(throwable).log(messagePattern,
+            arguments);
+      } else {
+        
delegate.atLevel(elevatedLevel).addMarker(marker).setCause(throwable).log(messagePattern,
+            arguments);
+      }
+
+    }
+
+  }
+
+  /**
+   * A Logger implementation that will suppress duplicate messages within the 
supplied duration.
+   */
+  public static class DeduplicatingLogger extends ConditionalLogger {
+
+    private static final long serialVersionUID = 1L;
+
+    public DeduplicatingLogger(Logger log, Duration threshold, long 
maxCachedLogMessages) {
+      super(log, new BiFunction<>() {
+
+        private final Cache<Pair<String,List<Object>>,Boolean> cache = 
Caffeine.newBuilder()
+            
.expireAfterWrite(threshold).maximumSize(maxCachedLogMessages).build();
+
+        private final ConcurrentMap<Pair<String,List<Object>>,Boolean> 
cacheMap = cache.asMap();
+
+        /**
+         * Function that will return true if the message has not been seen in 
the supplied duration.
+         *
+         * @param msg log message
+         * @param args log message arguments
+         * @return true if message has not been seen in duration, else false.
+         */
+        @Override
+        public Boolean apply(String msg, List<Object> args) {
+          return cacheMap.putIfAbsent(new Pair<>(msg, args), true) == null;
+        }
+
+      });
+    }
+
+  }
+
+  protected final Logger delegate;
+  protected final BiFunction<String,List<Object>,Boolean> condition;
+
+  protected ConditionalLogger(Logger log, 
BiFunction<String,List<Object>,Boolean> condition) {
+    // this.delegate = new DelegateWrapper(log);
+    this.delegate = log;
+    this.condition = condition;
+  }
+
+  @Override
+  public boolean isTraceEnabled() {
+    return this.delegate.isTraceEnabled();
+  }
+
+  @Override
+  public boolean isTraceEnabled(Marker marker) {
+    return this.delegate.isTraceEnabled(marker);
+  }
+
+  @Override
+  public boolean isDebugEnabled() {
+    return this.delegate.isDebugEnabled();
+  }
+
+  @Override
+  public boolean isDebugEnabled(Marker marker) {
+    return this.delegate.isDebugEnabled(marker);
+  }
+
+  @Override
+  public boolean isInfoEnabled() {
+    return this.delegate.isInfoEnabled();
+  }
+
+  @Override
+  public boolean isInfoEnabled(Marker marker) {
+    return this.delegate.isInfoEnabled(marker);
+  }
+
+  @Override
+  public boolean isWarnEnabled() {
+    return this.delegate.isWarnEnabled();
+  }
+
+  @Override
+  public boolean isWarnEnabled(Marker marker) {
+    return this.delegate.isWarnEnabled(marker);
+  }
+
+  @Override
+  public boolean isErrorEnabled() {
+    return this.delegate.isErrorEnabled();
+  }
+
+  @Override
+  public boolean isErrorEnabled(Marker marker) {
+    return this.delegate.isErrorEnabled(marker);
+  }
+
+  @Override
+  public String getName() {
+    return this.delegate.getName();
+  }
+
+  @Override
+  protected String getFullyQualifiedCallerName() {
+    return this.delegate.getName();
+  }
+
+  @Override
+  protected void handleNormalizedLoggingCall(Level level, Marker marker, 
String messagePattern,
+      Object[] arguments, Throwable throwable) {
+
+    if (arguments == null) {
+      arguments = new Object[0];
+    }
+    if (condition.apply(messagePattern, Arrays.asList(arguments))) {
+      
delegate.atLevel(level).addMarker(marker).setCause(throwable).log(messagePattern,
 arguments);
+    }
+
+  }
+
+}
diff --git 
a/core/src/test/java/org/apache/accumulo/core/logging/DeduplicatingLoggerTest.java
 
b/core/src/test/java/org/apache/accumulo/core/logging/DeduplicatingLoggerTest.java
new file mode 100644
index 0000000000..826dfed144
--- /dev/null
+++ 
b/core/src/test/java/org/apache/accumulo/core/logging/DeduplicatingLoggerTest.java
@@ -0,0 +1,69 @@
+/*
+ * 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
+ *
+ *   https://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.accumulo.core.logging;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+
+import java.io.StringWriter;
+import java.time.Duration;
+
+import org.apache.accumulo.core.logging.ConditionalLogger.DeduplicatingLogger;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.logging.log4j.core.Appender;
+import org.apache.logging.log4j.core.LoggerContext;
+import org.apache.logging.log4j.core.appender.WriterAppender;
+import org.apache.logging.log4j.core.config.Configuration;
+import org.apache.logging.log4j.core.layout.PatternLayout;
+import org.junit.jupiter.api.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class DeduplicatingLoggerTest {
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(DeduplicatingLoggerTest.class);
+  private static final Logger TEST_LOGGER =
+      new DeduplicatingLogger(LOG, Duration.ofMinutes(1), 100);
+
+  @Test
+  public void test() {
+
+    StringWriter writer = new StringWriter();
+
+    // Programatically modify the Log4j2 Logging configuration to add an 
appender
+    LoggerContext ctx = LoggerContext.getContext(false);
+    Configuration cfg = ctx.getConfiguration();
+    PatternLayout layout = PatternLayout.createDefaultLayout(cfg);
+    Appender appender = WriterAppender.createAppender(layout, null, writer,
+        "DeduplicatingLoggerTestAppender", false, true);
+    appender.start();
+    cfg.addAppender(appender);
+    
cfg.getLoggerConfig(DeduplicatingLoggerTest.class.getName()).addAppender(appender,
 null, null);
+
+    TEST_LOGGER.error("ERROR TEST");
+    TEST_LOGGER.warn("WARN TEST");
+    assertEquals(1, StringUtils.countMatches(writer.toString(), "ERROR TEST"));
+    assertEquals(1, StringUtils.countMatches(writer.toString(), "WARN TEST"));
+    TEST_LOGGER.error("ERROR TEST");
+    TEST_LOGGER.warn("WARN TEST");
+    assertEquals(1, StringUtils.countMatches(writer.toString(), "ERROR TEST"));
+    assertEquals(1, StringUtils.countMatches(writer.toString(), "WARN TEST"));
+
+  }
+
+}
diff --git 
a/core/src/test/java/org/apache/accumulo/core/logging/EscalatingLoggerTest.java 
b/core/src/test/java/org/apache/accumulo/core/logging/EscalatingLoggerTest.java
new file mode 100644
index 0000000000..cd368e38a4
--- /dev/null
+++ 
b/core/src/test/java/org/apache/accumulo/core/logging/EscalatingLoggerTest.java
@@ -0,0 +1,77 @@
+/*
+ * 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
+ *
+ *   https://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.accumulo.core.logging;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+
+import java.io.StringWriter;
+import java.time.Duration;
+
+import org.apache.accumulo.core.logging.ConditionalLogger.EscalatingLogger;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.logging.log4j.core.Appender;
+import org.apache.logging.log4j.core.LoggerContext;
+import org.apache.logging.log4j.core.appender.WriterAppender;
+import org.apache.logging.log4j.core.config.Configuration;
+import org.apache.logging.log4j.core.layout.PatternLayout;
+import org.junit.jupiter.api.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.slf4j.event.Level;
+
+public class EscalatingLoggerTest {
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(EscalatingLoggerTest.class);
+  private static final Logger TEST_LOGGER =
+      new EscalatingLogger(LOG, Duration.ofSeconds(3), 100, Level.WARN);
+
+  @Test
+  public void test() throws InterruptedException {
+
+    StringWriter writer = new StringWriter();
+
+    // Programatically modify the Log4j2 Logging configuration to add an 
appender
+    LoggerContext ctx = LoggerContext.getContext(false);
+    Configuration cfg = ctx.getConfiguration();
+    PatternLayout layout = PatternLayout.newBuilder().withConfiguration(cfg)
+        .withPattern(PatternLayout.SIMPLE_CONVERSION_PATTERN).build();
+    Appender appender = WriterAppender.createAppender(layout, null, writer,
+        "EscalatingLoggerTestAppender", false, true);
+    appender.start();
+    cfg.addAppender(appender);
+    
cfg.getLoggerConfig(EscalatingLoggerTest.class.getName()).addAppender(appender, 
null, null);
+
+    TEST_LOGGER.info("TEST MESSAGE");
+    TEST_LOGGER.info("TEST MESSAGE");
+    TEST_LOGGER.info("TEST MESSAGE");
+    TEST_LOGGER.info("TEST MESSAGE");
+
+    assertEquals(1, StringUtils.countMatches(writer.toString(), "WARN"));
+    assertEquals(3, StringUtils.countMatches(writer.toString(), "INFO"));
+
+    Thread.sleep(5000);
+
+    TEST_LOGGER.info("TEST MESSAGE");
+
+    assertEquals(2, StringUtils.countMatches(writer.toString(), "WARN"));
+    assertEquals(3, StringUtils.countMatches(writer.toString(), "INFO"));
+
+  }
+
+}
diff --git 
a/server/manager/src/main/java/org/apache/accumulo/manager/TabletGroupWatcher.java
 
b/server/manager/src/main/java/org/apache/accumulo/manager/TabletGroupWatcher.java
index 216526d328..662fca40d4 100644
--- 
a/server/manager/src/main/java/org/apache/accumulo/manager/TabletGroupWatcher.java
+++ 
b/server/manager/src/main/java/org/apache/accumulo/manager/TabletGroupWatcher.java
@@ -22,6 +22,7 @@ import static java.lang.Math.min;
 import static 
org.apache.accumulo.core.util.UtilWaitThread.sleepUninterruptibly;
 
 import java.io.IOException;
+import java.time.Duration;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.HashMap;
@@ -52,6 +53,7 @@ import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.dataImpl.KeyExtent;
 import org.apache.accumulo.core.gc.ReferenceFile;
+import org.apache.accumulo.core.logging.ConditionalLogger.EscalatingLogger;
 import org.apache.accumulo.core.logging.TabletLogger;
 import org.apache.accumulo.core.manager.state.tables.TableState;
 import org.apache.accumulo.core.manager.thrift.ManagerState;
@@ -100,14 +102,17 @@ import org.apache.accumulo.server.tablets.TabletTime;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.Text;
 import org.apache.thrift.TException;
+import org.slf4j.Logger;
+import org.slf4j.event.Level;
 
 import com.google.common.base.Preconditions;
 import com.google.common.collect.ImmutableSortedSet;
 import com.google.common.collect.Iterators;
 
 abstract class TabletGroupWatcher extends AccumuloDaemonThread {
-  // Constants used to make sure assignment logging isn't excessive in 
quantity or size
 
+  private static final Logger TABLET_UNLOAD_LOGGER =
+      new EscalatingLogger(Manager.log, Duration.ofMinutes(5), 1000, 
Level.INFO);
   private final Manager manager;
   private final TabletStateStore store;
   private final TabletGroupWatcher dependentWatcher;
@@ -345,8 +350,8 @@ abstract class TabletGroupWatcher extends 
AccumuloDaemonThread {
                     
manager.tserverSet.getConnection(location.getServerInstance());
                 if (client != null) {
                   try {
-                    Manager.log.trace("[{}] Requesting TabletServer {} unload 
{} {}", store.name(),
-                        location.getServerInstance(), tls.extent, 
goal.howUnload());
+                    TABLET_UNLOAD_LOGGER.trace("[{}] Requesting TabletServer 
{} unload {} {}",
+                        store.name(), location.getServerInstance(), 
tls.extent, goal.howUnload());
                     client.unloadTablet(manager.managerLock, tls.extent, 
goal.howUnload(),
                         manager.getSteadyTime());
                     unloaded++;
diff --git 
a/server/tserver/src/main/java/org/apache/accumulo/tserver/UnloadTabletHandler.java
 
b/server/tserver/src/main/java/org/apache/accumulo/tserver/UnloadTabletHandler.java
index ee941803b9..d454d0d79c 100644
--- 
a/server/tserver/src/main/java/org/apache/accumulo/tserver/UnloadTabletHandler.java
+++ 
b/server/tserver/src/main/java/org/apache/accumulo/tserver/UnloadTabletHandler.java
@@ -59,7 +59,6 @@ class UnloadTabletHandler implements Runnable {
     synchronized (server.unopenedTablets) {
       if (server.unopenedTablets.contains(extent)) {
         server.unopenedTablets.remove(extent);
-        // enqueueManagerMessage(new TabletUnloadedMessage(extent));
         return;
       }
     }
diff --git 
a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Tablet.java 
b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Tablet.java
index 1fac59303c..b6ad6150cb 100644
--- 
a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Tablet.java
+++ 
b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Tablet.java
@@ -27,6 +27,7 @@ import java.io.DataInputStream;
 import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.lang.ref.SoftReference;
+import java.time.Duration;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
@@ -67,6 +68,7 @@ import org.apache.accumulo.core.dataImpl.thrift.MapFileInfo;
 import org.apache.accumulo.core.file.FileOperations;
 import org.apache.accumulo.core.iterators.SortedKeyValueIterator;
 import org.apache.accumulo.core.iteratorsImpl.system.SourceSwitchingIterator;
+import org.apache.accumulo.core.logging.ConditionalLogger.DeduplicatingLogger;
 import org.apache.accumulo.core.logging.TabletLogger;
 import org.apache.accumulo.core.manager.state.tables.TableState;
 import org.apache.accumulo.core.master.thrift.BulkImportState;
@@ -141,6 +143,8 @@ import io.opentelemetry.context.Scope;
  */
 public class Tablet extends TabletBase {
   private static final Logger log = LoggerFactory.getLogger(Tablet.class);
+  private static final Logger CLOSING_STUCK_LOGGER =
+      new DeduplicatingLogger(log, Duration.ofMinutes(5), 1000);
 
   private final TabletServer tabletServer;
   private final TabletResourceManager tabletResources;
@@ -164,9 +168,10 @@ public class Tablet extends TabletBase {
   }
 
   private enum CloseState {
-    OPEN, CLOSING, CLOSED, COMPLETE
+    OPEN, REQUESTED, CLOSING, CLOSED, COMPLETE
   }
 
+  private long closeRequestTime = 0;
   private volatile CloseState closeState = CloseState.OPEN;
 
   private boolean updatingFlushID = false;
@@ -905,6 +910,21 @@ public class Tablet extends TabletBase {
   void initiateClose(boolean saveState) {
     log.trace("initiateClose(saveState={}) {}", saveState, getExtent());
 
+    synchronized (this) {
+      if (closeState == CloseState.OPEN) {
+        closeRequestTime = System.nanoTime();
+        closeState = CloseState.REQUESTED;
+      } else {
+        Preconditions.checkState(closeRequestTime != 0);
+        long runningTime = Duration.ofNanos(System.nanoTime() - 
closeRequestTime).toMinutes();
+        if (runningTime >= 15) {
+          CLOSING_STUCK_LOGGER.info(
+              "Tablet {} close requested again, but has been closing for {} 
minutes", this.extent,
+              runningTime);
+        }
+      }
+    }
+
     MinorCompactionTask mct = null;
     if (saveState) {
       try {

Reply via email to