From 221259e12e1cb390dcd5d2dfc93489a27d6e0d25 Mon Sep 17 00:00:00 2001 From: Dave Marion Date: Fri, 24 May 2024 12:46:44 -0400 Subject: [PATCH] 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 ++ .../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 - .../accumulo/tserver/tablet/Tablet.java | 22 +- 7 files changed, 385 insertions(+), 5 deletions(-) create mode 100644 core/src/main/java/org/apache/accumulo/core/logging/ConditionalLogger.java create mode 100644 core/src/test/java/org/apache/accumulo/core/logging/DeduplicatingLoggerTest.java create mode 100644 core/src/test/java/org/apache/accumulo/core/logging/EscalatingLoggerTest.java diff --git a/core/pom.xml b/core/pom.xml index 6e4ffc98831..57c8c425b41 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -152,11 +152,27 @@ hadoop-client-runtime runtime + + + biz.aQute.bnd + biz.aQute.bnd.annotation + test + org.apache.hadoop hadoop-client-minicluster test + + org.apache.logging.log4j + log4j-api + test + + + org.apache.logging.log4j + log4j-core + test + org.apache.logging.log4j log4j-slf4j2-impl 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 00000000000..6da6454f06c --- /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>,Boolean> cache = Caffeine.newBuilder() + .expireAfterWrite(threshold).maximumSize(maxCachedLogMessages).build(); + + private final ConcurrentMap>,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 args) { + return cacheMap.putIfAbsent(new Pair<>(msg, args), true) == null; + } + + }); + } + + } + + protected final Logger delegate; + protected final BiFunction,Boolean> condition; + + protected ConditionalLogger(Logger log, BiFunction,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 00000000000..826dfed1444 --- /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 00000000000..cd368e38a4a --- /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 216526d328c..662fca40d42 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 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.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.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 @@ public void run() { 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 ee941803b90..d454d0d79cf 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 @@ public void run() { 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 1fac59303c1..b6ad6150cbc 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.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.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 @@ */ 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 long getDataSourceDeletions() { } 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 void close(boolean saveState) throws IOException { 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 {