ctubbsii commented on code in PR #5989: URL: https://github.com/apache/accumulo/pull/5989#discussion_r2558049215
########## server/compaction-coordinator/src/main/java/org/apache/accumulo/coordinator/CoordinatorSummaryLogger.java: ########## @@ -0,0 +1,79 @@ +/* + * 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.coordinator; + +import java.util.HashMap; +import java.util.Map; +import java.util.TreeMap; +import java.util.TreeSet; +import java.util.concurrent.atomic.AtomicLong; + +import org.apache.accumulo.core.data.TableId; +import org.apache.accumulo.core.dataImpl.KeyExtent; +import org.apache.accumulo.core.metadata.TServerInstance; +import org.apache.accumulo.server.ServerContext; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.github.benmanes.caffeine.cache.Cache; + +public class CoordinatorSummaryLogger { + private static final Logger LOG = LoggerFactory.getLogger(CoordinatorSummaryLogger.class); + + private static final TreeMap<Short,TreeSet<TServerInstance>> EMPTY = new TreeMap<>(); + private final ServerContext ctx; + private final Cache<String,Integer> compactorCounts; + + public CoordinatorSummaryLogger(ServerContext ctx, Cache<String,Integer> compactorCounts) { + this.ctx = ctx; + this.compactorCounts = compactorCounts; + } + + public void logSummary() { + + final Map<TableId,String> tableMap = ctx.getTableIdToNameMap(); + final Map<String,AtomicLong> perQueueRunningCount = new HashMap<>(); + final Map<String,AtomicLong> perTableRunningCount = new HashMap<>(); + + CompactionCoordinator.RUNNING_CACHE.values().forEach(rc -> { + TableId tid = KeyExtent.fromThrift(rc.getJob().getExtent()).tableId(); + String tableName = tableMap.getOrDefault(tid, "Unmapped table id: " + tid.canonical()); + perQueueRunningCount.computeIfAbsent(rc.getQueueName(), q -> new AtomicLong(0)) + .incrementAndGet(); + perTableRunningCount.computeIfAbsent(tableName, t -> new AtomicLong(0)).incrementAndGet(); + }); + + perQueueRunningCount.forEach((q, count) -> { + LOG.info( + "Queue {}: compactors: {}, queued majc (minimum, possibly higher): {}, running majc: {}", + q, compactorCounts.asMap().getOrDefault(q, 0), + // This map only contains the highest priority for each tserver. So when tservers have + // other priorities that need to compact or have more than one compaction for a + // priority level this count will be lower than the actual number of queued. + CompactionCoordinator.QUEUE_SUMMARIES.QUEUES.getOrDefault(q, EMPTY).values().stream() Review Comment: If the QUEUES were of type SortedMap instead of TreeMap, you could use Collections.emptySortedMap() instead of creating your own EMPTY one. ########## server/compaction-coordinator/src/main/java/org/apache/accumulo/coordinator/CoordinatorSummaryLogger.java: ########## @@ -0,0 +1,79 @@ +/* + * 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.coordinator; + +import java.util.HashMap; +import java.util.Map; +import java.util.TreeMap; +import java.util.TreeSet; +import java.util.concurrent.atomic.AtomicLong; + +import org.apache.accumulo.core.data.TableId; +import org.apache.accumulo.core.dataImpl.KeyExtent; +import org.apache.accumulo.core.metadata.TServerInstance; +import org.apache.accumulo.server.ServerContext; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.github.benmanes.caffeine.cache.Cache; + +public class CoordinatorSummaryLogger { + private static final Logger LOG = LoggerFactory.getLogger(CoordinatorSummaryLogger.class); + + private static final TreeMap<Short,TreeSet<TServerInstance>> EMPTY = new TreeMap<>(); + private final ServerContext ctx; + private final Cache<String,Integer> compactorCounts; + + public CoordinatorSummaryLogger(ServerContext ctx, Cache<String,Integer> compactorCounts) { + this.ctx = ctx; + this.compactorCounts = compactorCounts; + } + + public void logSummary() { + + final Map<TableId,String> tableMap = ctx.getTableIdToNameMap(); + final Map<String,AtomicLong> perQueueRunningCount = new HashMap<>(); + final Map<String,AtomicLong> perTableRunningCount = new HashMap<>(); + + CompactionCoordinator.RUNNING_CACHE.values().forEach(rc -> { + TableId tid = KeyExtent.fromThrift(rc.getJob().getExtent()).tableId(); + String tableName = tableMap.getOrDefault(tid, "Unmapped table id: " + tid.canonical()); + perQueueRunningCount.computeIfAbsent(rc.getQueueName(), q -> new AtomicLong(0)) + .incrementAndGet(); + perTableRunningCount.computeIfAbsent(tableName, t -> new AtomicLong(0)).incrementAndGet(); + }); + + perQueueRunningCount.forEach((q, count) -> { + LOG.info( + "Queue {}: compactors: {}, queued majc (minimum, possibly higher): {}, running majc: {}", + q, compactorCounts.asMap().getOrDefault(q, 0), + // This map only contains the highest priority for each tserver. So when tservers have + // other priorities that need to compact or have more than one compaction for a + // priority level this count will be lower than the actual number of queued. + CompactionCoordinator.QUEUE_SUMMARIES.QUEUES.getOrDefault(q, EMPTY).values().stream() + .mapToLong(TreeSet::size).sum(), + count.get()); + + }); + perTableRunningCount.forEach((t, count) -> { + LOG.info("Running compactions for table {}: {}", t, count); + }); Review Comment: ```suggestion perTableRunningCount.forEach((t, count) -> LOG.info("Running compactions for table {}: {}", t, count)); ``` ########## server/compaction-coordinator/src/main/java/org/apache/accumulo/coordinator/CompactionCoordinator.java: ########## @@ -187,8 +188,9 @@ protected CompactionCoordinator(ServerOpts opts, String[] args, AccumuloConfigur printStartupMsg(); startCompactionCleaner(); startRunningCleaner(); - compactorCounts = Caffeine.newBuilder().expireAfterWrite(30, TimeUnit.SECONDS) + compactorCounts = Caffeine.newBuilder().expireAfterWrite(120, TimeUnit.SECONDS) Review Comment: Easy math, but this seems easier to read. ```suggestion compactorCounts = Caffeine.newBuilder().expireAfterWrite(2, TimeUnit.MINUTES) ``` ########## server/compaction-coordinator/src/main/java/org/apache/accumulo/coordinator/CompactionCoordinator.java: ########## @@ -685,6 +688,12 @@ private void captureFailure(ExternalCompactionId ecid, KeyExtent extent) { failingTables.compute(extent.tableId(), FailureCounts::incrementFailure); } + protected void startQueueRunningSummaryLogging() { + ScheduledFuture<?> future = getContext().getScheduledExecutor() + .scheduleWithFixedDelay(summaryLogger::logSummary, 0, 60, TimeUnit.SECONDS); Review Comment: You can also statically import these TimeUnits, to make the line more readable. ```suggestion .scheduleWithFixedDelay(summaryLogger::logSummary, 0, 1, TimeUnit.MINUTES); ``` -- 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: [email protected] For queries about this service, please contact Infrastructure at: [email protected]
