This is an automated email from the ASF dual-hosted git repository. ddanielr pushed a commit to branch main in repository https://gitbox.apache.org/repos/asf/accumulo.git
The following commit(s) were added to refs/heads/main by this push: new f2c779ea56 Warn when tablet needs compacting and is not (#4143) f2c779ea56 is described below commit f2c779ea561dcc5b42b00ccb3afddfe5ded51bb0 Author: Daniel Roberts <ddani...@gmail.com> AuthorDate: Tue Jan 9 15:26:05 2024 -0500 Warn when tablet needs compacting and is not (#4143) * Warn when tablet needs compacting and is not * Adds error logging suppression Adds a 5 minute suppression for the empty compaction plan error message that occures when tablet files exceed TSERV_SCAN_MAX_OPEN_FILES. --------- Co-authored-by: Keith Turner <ktur...@apache.org> --- .../tserver/compactions/CompactionService.java | 31 ++++++++++++++++++++++ 1 file changed, 31 insertions(+) diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/compactions/CompactionService.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/compactions/CompactionService.java index d696c7f826..0e18f17df3 100644 --- a/server/tserver/src/main/java/org/apache/accumulo/tserver/compactions/CompactionService.java +++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/compactions/CompactionService.java @@ -18,6 +18,8 @@ */ package org.apache.accumulo.tserver.compactions; +import static java.util.concurrent.TimeUnit.MINUTES; + import java.util.Collection; import java.util.Collections; import java.util.EnumMap; @@ -40,6 +42,7 @@ import java.util.function.Function; import org.apache.accumulo.core.client.admin.compaction.CompactableFile; import org.apache.accumulo.core.conf.ConfigurationTypeHelper; +import org.apache.accumulo.core.conf.Property; import org.apache.accumulo.core.data.TableId; import org.apache.accumulo.core.dataImpl.KeyExtent; import org.apache.accumulo.core.spi.common.ServiceEnvironment; @@ -65,6 +68,8 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; import com.google.common.base.Preconditions; +import com.google.common.cache.Cache; +import com.google.common.cache.CacheBuilder; import com.google.common.collect.Sets; public class CompactionService { @@ -84,6 +89,9 @@ public class CompactionService { private AtomicLong rateLimit = new AtomicLong(0); private Function<CompactionExecutorId,ExternalCompactionExecutor> externExecutorSupplier; + // use to limit logging of max scan files exceeded + private final Cache<TableId,Long> maxScanFilesExceededErrorCache; + private static final Logger log = LoggerFactory.getLogger(CompactionService.class); public CompactionService(String serviceName, String plannerClass, String plannerPrefix, @@ -132,6 +140,8 @@ public class CompactionService { queuedForPlanning.put(kind, new ConcurrentHashMap<KeyExtent,Compactable>()); } + maxScanFilesExceededErrorCache = CacheBuilder.newBuilder().expireAfterWrite(5, MINUTES).build(); + log.debug("Created new compaction service id:{} rate limit:{} planner:{} planner options:{}", myId, maxRate, plannerClass, plannerOptions); } @@ -283,6 +293,27 @@ public class CompactionService { CompactionPlan plan; try { plan = planner.makePlan(params); + var tableId = compactable.getTableId(); + + if (plan.getJobs().isEmpty()) { + int maxScanFiles = + context.getTableConfiguration(tableId).getCount(Property.TSERV_SCAN_MAX_OPENFILES); + + if (files.allFiles.size() >= maxScanFiles && files.compacting.isEmpty()) { + var last = maxScanFilesExceededErrorCache.getIfPresent(tableId); + + if (last == null) { + log.warn( + "The tablet {} has {} files and the max files for scan is {}. No compactions are " + + "running and none were planned for this tablet by {}, so the files will " + + "not be reduced by compaction which could cause scans to fail. Please " + + "check your compaction configuration. This log message is temporarily suppressed for the entire table.", + compactable.getExtent(), files.allFiles.size(), maxScanFiles, myId); + maxScanFilesExceededErrorCache.put(tableId, System.currentTimeMillis()); + } + } + } + } catch (RuntimeException e) { log.debug("Planner failed {} {} {} {} {}", myId, planner.getClass().getName(), compactable.getExtent(), kind, files, e);