Skip to content

Commit

Permalink
Used data size instead of entry size of compaction queue (apache#5252)
Browse files Browse the repository at this point in the history
Modified the compaction queue limit to use the data size of the
compaction jobs instead of the number of compaction jobs for limiting
the number of compaction jobs buffered.

fixes apache#5186
  • Loading branch information
keith-turner authored Jan 16, 2025
1 parent 0cc90ef commit 2a12b13
Show file tree
Hide file tree
Showing 10 changed files with 285 additions and 56 deletions.
16 changes: 7 additions & 9 deletions core/src/main/java/org/apache/accumulo/core/conf/Property.java
Original file line number Diff line number Diff line change
Expand Up @@ -465,14 +465,12 @@ public enum Property {
"The number of threads used to seed fate split task, the actual split work is done by fate"
+ " threads.",
"4.0.0"),

MANAGER_COMPACTION_SERVICE_PRIORITY_QUEUE_INITIAL_SIZE(
"manager.compaction.major.service.queue.initial.size", "10000", PropertyType.COUNT,
"The initial size of each resource groups compaction job priority queue.", "4.0.0"),
MANAGER_COMPACTION_SERVICE_PRIORITY_QUEUE_SIZE_FACTOR(
"manager.compaction.major.service.queue.size.factor", "3.0", PropertyType.FRACTION,
"The dynamic resizing of the compaction job priority queue is based on"
+ " the number of compactors for the group multiplied by this factor.",
MANAGER_COMPACTION_SERVICE_PRIORITY_QUEUE_SIZE("manager.compaction.major.service.queue.size",
"1M", PropertyType.MEMORY,
"The data size of each resource groups compaction job priority queue. The memory size of "
+ "each compaction job is estimated and the sum of these sizes per resource group will not "
+ "exceed this setting. When the size is exceeded the lowest priority jobs are dropped as "
+ "needed.",
"4.0.0"),
SPLIT_PREFIX("split.", null, PropertyType.PREFIX,
"System wide properties related to splitting tablets.", "3.1.0"),
Expand Down Expand Up @@ -1460,7 +1458,7 @@ public static boolean isValidTablePropertyKey(String key) {
RPC_MAX_MESSAGE_SIZE,

// compaction coordiantor properties
MANAGER_COMPACTION_SERVICE_PRIORITY_QUEUE_INITIAL_SIZE,
MANAGER_COMPACTION_SERVICE_PRIORITY_QUEUE_SIZE,

// block cache options
GENERAL_CACHE_MANAGER_IMPL, TSERV_DATACACHE_SIZE, TSERV_INDEXCACHE_SIZE,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -188,8 +188,8 @@ MiniAccumuloConfigImpl initialize() {

mergeProp(Property.COMPACTOR_PORTSEARCH.getKey(), "true");

mergeProp(Property.MANAGER_COMPACTION_SERVICE_PRIORITY_QUEUE_INITIAL_SIZE.getKey(),
Property.MANAGER_COMPACTION_SERVICE_PRIORITY_QUEUE_INITIAL_SIZE.getDefaultValue());
mergeProp(Property.MANAGER_COMPACTION_SERVICE_PRIORITY_QUEUE_SIZE.getKey(),
Property.MANAGER_COMPACTION_SERVICE_PRIORITY_QUEUE_SIZE.getDefaultValue());
mergeProp(Property.COMPACTION_SERVICE_DEFAULT_PLANNER.getKey(),
Property.COMPACTION_SERVICE_DEFAULT_PLANNER.getDefaultValue());

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -194,7 +194,7 @@ public class CompactionCoordinator
private final Manager manager;

private final LoadingCache<String,Integer> compactorCounts;
private final int jobQueueInitialSize;
private final long jobQueueInitialSize;

private volatile long coordinatorStartTime;

Expand All @@ -208,8 +208,8 @@ public CompactionCoordinator(ServerContext ctx, SecurityOperation security,
this.security = security;
this.manager = Objects.requireNonNull(manager);

this.jobQueueInitialSize = ctx.getConfiguration()
.getCount(Property.MANAGER_COMPACTION_SERVICE_PRIORITY_QUEUE_INITIAL_SIZE);
this.jobQueueInitialSize =
ctx.getConfiguration().getAsBytes(Property.MANAGER_COMPACTION_SERVICE_PRIORITY_QUEUE_SIZE);

this.jobQueues = new CompactionJobQueues(jobQueueInitialSize);

Expand Down Expand Up @@ -1121,8 +1121,6 @@ private void cleanUpEmptyCompactorPathInZK() {
final String compactorQueuesPath = this.ctx.getZooKeeperRoot() + Constants.ZCOMPACTORS;

final var zoorw = this.ctx.getZooSession().asReaderWriter();
final double queueSizeFactor = ctx.getConfiguration()
.getFraction(Property.MANAGER_COMPACTION_SERVICE_PRIORITY_QUEUE_SIZE_FACTOR);

try {
var groups = zoorw.getChildren(compactorQueuesPath);
Expand All @@ -1139,7 +1137,6 @@ private void cleanUpEmptyCompactorPathInZK() {
CompactionJobPriorityQueue queue = getJobQueues().getQueue(cgid);
if (queue != null) {
queue.clearIfInactive(Duration.ofMinutes(10));
queue.setMaxSize(this.jobQueueInitialSize);
}
} else {
int aliveCompactorsForGroup = 0;
Expand All @@ -1152,16 +1149,8 @@ private void cleanUpEmptyCompactorPathInZK() {
aliveCompactorsForGroup++;
}
}
CompactionJobPriorityQueue queue = getJobQueues().getQueue(cgid);
if (queue != null) {
queue.setMaxSize(Math.min(
Math.max(1, (int) (aliveCompactorsForGroup * queueSizeFactor)), Integer.MAX_VALUE));
}

}

}

} catch (KeeperException | RuntimeException e) {
LOG.warn("Failed to clean up compactors", e);
} catch (InterruptedException e) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -31,11 +31,9 @@
import java.util.Objects;
import java.util.Optional;
import java.util.Set;
import java.util.TreeMap;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicLong;
import java.util.concurrent.atomic.AtomicReference;

Expand Down Expand Up @@ -116,8 +114,8 @@ public boolean equals(Object o) {
// behavior is not supported with a PriorityQueue. Second a PriorityQueue does not support
// efficiently removing entries from anywhere in the queue. Efficient removal is needed for the
// case where tablets decided to issues different compaction jobs than what is currently queued.
private final TreeMap<CjpqKey,CompactionJobQueues.MetaJob> jobQueue;
private final AtomicInteger maxSize;
private final SizeTrackingTreeMap<CjpqKey,CompactionJobQueues.MetaJob> jobQueue;
private final AtomicLong maxSize;
private final AtomicLong rejectedJobs;
private final AtomicLong dequeuedJobs;
private final ArrayDeque<CompletableFuture<CompactionJobQueues.MetaJob>> futures;
Expand All @@ -142,9 +140,10 @@ private TabletJobs(long generation, HashSet<CjpqKey> jobs) {

private final AtomicLong nextSeq = new AtomicLong(0);

public CompactionJobPriorityQueue(CompactorGroupId groupId, int maxSize) {
this.jobQueue = new TreeMap<>();
this.maxSize = new AtomicInteger(maxSize);
public CompactionJobPriorityQueue(CompactorGroupId groupId, long maxSize,
SizeTrackingTreeMap.Weigher<CompactionJobQueues.MetaJob> weigher) {
this.jobQueue = new SizeTrackingTreeMap<>(weigher);
this.maxSize = new AtomicLong(maxSize);
this.tabletJobs = new HashMap<>();
this.groupId = groupId;
this.rejectedJobs = new AtomicLong(0);
Expand Down Expand Up @@ -230,11 +229,11 @@ public synchronized int add(TabletMetadata tabletMetadata, Collection<Compaction
return jobsAdded;
}

public synchronized int getMaxSize() {
public synchronized long getMaxSize() {
return maxSize.get();
}

public synchronized void setMaxSize(int maxSize) {
public synchronized void setMaxSize(long maxSize) {
Preconditions.checkArgument(maxSize > 0,
"Maximum size of the Compaction job priority queue must be greater than 0");
this.maxSize.set(maxSize);
Expand All @@ -249,7 +248,7 @@ public long getDequeuedJobs() {
}

public synchronized long getQueuedJobs() {
return jobQueue.size();
return jobQueue.entrySize();
}

public synchronized long getLowestPriority() {
Expand Down Expand Up @@ -332,7 +331,7 @@ private void removePreviousSubmissions(KeyExtent extent, boolean removeJobAges)
}

private CjpqKey addJobToQueue(TabletMetadata tabletMetadata, CompactionJob job) {
if (jobQueue.size() >= maxSize.get()) {
if (jobQueue.dataSize() >= maxSize.get()) {
var lastEntry = jobQueue.lastKey();
if (job.getPriority() <= lastEntry.job.getPriority()) {
// the queue is full and this job has a lower or same priority than the lowest job in the
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -48,18 +48,20 @@ public class CompactionJobQueues {
private final ConcurrentHashMap<CompactorGroupId,CompactionJobPriorityQueue> priorityQueues =
new ConcurrentHashMap<>();

private final int queueSize;
private final long queueSize;

private final Map<DataLevel,AtomicLong> currentGenerations;

public CompactionJobQueues(int queueSize) {
private SizeTrackingTreeMap.Weigher<MetaJob> weigher =
val -> val.getTabletMetadata().toString().length() + val.getJob().toString().length();

public CompactionJobQueues(long queueSize) {
this.queueSize = queueSize;
Map<DataLevel,AtomicLong> cg = new EnumMap<>(DataLevel.class);
for (var level : DataLevel.values()) {
cg.put(level, new AtomicLong());
}
currentGenerations = Collections.unmodifiableMap(cg);

}

public void beginFullScan(DataLevel level) {
Expand Down Expand Up @@ -164,7 +166,7 @@ public TabletMetadata getTabletMetadata() {
*/
public CompletableFuture<MetaJob> getAsync(CompactorGroupId groupId) {
var pq = priorityQueues.computeIfAbsent(groupId,
gid -> new CompactionJobPriorityQueue(gid, queueSize));
gid -> new CompactionJobPriorityQueue(gid, queueSize, weigher));
return pq.getAsync();
}

Expand All @@ -187,7 +189,7 @@ private void add(TabletMetadata tabletMetadata, CompactorGroupId groupId,
}

var pq = priorityQueues.computeIfAbsent(groupId,
gid -> new CompactionJobPriorityQueue(gid, queueSize));
gid -> new CompactionJobPriorityQueue(gid, queueSize, weigher));
pq.add(tabletMetadata, jobs,
currentGenerations.get(DataLevel.of(tabletMetadata.getTableId())).get());
}
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,129 @@
/*
* 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.manager.compaction.queue;

import java.util.AbstractMap;
import java.util.Map;
import java.util.TreeMap;

import com.google.common.base.Preconditions;

/**
* This class wraps a treemap and tracks the data size of everything added and removed from the
* treemap.
*/
class SizeTrackingTreeMap<K,V> {

private static class ValueWrapper<V2> {
final V2 val;
final long computedSize;

private ValueWrapper(V2 val, long computedSize) {
this.val = val;
this.computedSize = computedSize;
}
}

private final TreeMap<K,ValueWrapper<V>> map = new TreeMap<>();
private long dataSize = 0;
private Weigher<V> weigher;

private Map.Entry<K,V> unwrap(Map.Entry<K,ValueWrapper<V>> wrapperEntry) {
if (wrapperEntry == null) {
return null;
}
return new AbstractMap.SimpleImmutableEntry<>(wrapperEntry.getKey(),
wrapperEntry.getValue().val);
}

private void incrementDataSize(ValueWrapper<V> val) {
Preconditions.checkState(dataSize >= 0);
dataSize += val.computedSize;
}

private void decrementDataSize(Map.Entry<K,ValueWrapper<V>> entry) {
if (entry != null) {
decrementDataSize(entry.getValue());
}
}

private void decrementDataSize(ValueWrapper<V> val) {
if (val != null) {
Preconditions.checkState(dataSize >= val.computedSize);
dataSize -= val.computedSize;
}
}

interface Weigher<V2> {
long weigh(V2 val);
}

public SizeTrackingTreeMap(Weigher<V> weigher) {
this.weigher = weigher;
}

public boolean isEmpty() {
return map.isEmpty();
}

public long dataSize() {
return dataSize;
}

public int entrySize() {
return map.size();
}

public K lastKey() {
return map.lastKey();
}

public Map.Entry<K,V> firstEntry() {
return unwrap(map.firstEntry());
}

public void remove(K key) {
var prev = map.remove(key);
decrementDataSize(prev);
}

public Map.Entry<K,V> pollFirstEntry() {
var first = map.pollFirstEntry();
decrementDataSize(first);
return unwrap(first);
}

public Map.Entry<K,V> pollLastEntry() {
var last = map.pollLastEntry();
decrementDataSize(last);
return unwrap(last);
}

public void put(K key, V val) {
var wrapped = new ValueWrapper<>(val, weigher.weigh(val));
var prev = map.put(key, wrapped);
decrementDataSize(prev);
incrementDataSize(wrapped);
}

public void clear() {
map.clear();
dataSize = 0;
}
}
Loading

0 comments on commit 2a12b13

Please sign in to comment.