Skip to content

Commit

Permalink
[7.17] Add full GC fallback within the current attempt (#102563) (#10…
Browse files Browse the repository at this point in the history
…3505)

* Add full GC fallback within the current attempt (#102563)

* Add full GC fallback within the current attempt
* Try re-acquire the lock if we timed out due to a full GC
* Condition full GC to memory reclaimed only

* Fix: transform record to private static class
  • Loading branch information
ldematte committed Dec 18, 2023
1 parent 41afa78 commit bfe46b4
Show file tree
Hide file tree
Showing 2 changed files with 171 additions and 51 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,7 @@
import org.elasticsearch.common.unit.ByteSizeValue;
import org.elasticsearch.common.util.concurrent.ReleasableLock;
import org.elasticsearch.core.Booleans;
import org.elasticsearch.core.SuppressForbidden;
import org.elasticsearch.core.TimeValue;
import org.elasticsearch.monitor.jvm.GcNames;
import org.elasticsearch.monitor.jvm.JvmInfo;
Expand Down Expand Up @@ -511,17 +512,22 @@ static OverLimitStrategy createOverLimitStrategy(boolean trackRealMemoryUsage) {
if (trackRealMemoryUsage && jvmInfo.useG1GC().equals("true")
// messing with GC is "dangerous" so we apply an escape hatch. Not intended to be used.
&& Booleans.parseBoolean(System.getProperty("es.real_memory_circuit_breaker.g1_over_limit_strategy.enabled"), true)) {
TimeValue lockTimeout = TimeValue.timeValueMillis(
Integer.parseInt(System.getProperty("es.real_memory_circuit_breaker.g1_over_limit_strategy.lock_timeout_ms", "500"))

long lockTimeoutInMillis = Integer.parseInt(
System.getProperty("es.real_memory_circuit_breaker.g1_over_limit_strategy.lock_timeout_ms", "500")
);
TimeValue lockTimeout = TimeValue.timeValueMillis(lockTimeoutInMillis);
TimeValue fullGCLockTimeout = TimeValue.timeValueMillis(lockTimeoutInMillis);
// hardcode interval, do not want any tuning of it outside code changes.
return new G1OverLimitStrategy(
jvmInfo,
HierarchyCircuitBreakerService::realMemoryUsage,
createYoungGcCountSupplier(),
System::currentTimeMillis,
500,
lockTimeout
5000,
lockTimeout,
fullGCLockTimeout
);
} else {
return memoryUsed -> memoryUsed;
Expand Down Expand Up @@ -554,10 +560,18 @@ static class G1OverLimitStrategy implements OverLimitStrategy {
private final LongSupplier gcCountSupplier;
private final LongSupplier timeSupplier;
private final TimeValue lockTimeout;

// The lock acquisition timeout when we are running a full GC
private final TimeValue fullGCLockTimeout;
private final long maxHeap;

private long lastCheckTime = Long.MIN_VALUE;
private long lastFullGCTime = Long.MIN_VALUE;
private final long minimumInterval;
private volatile boolean performingFullGC = false;

// Minimum interval before triggering another full GC
private final long fullGCMinimumInterval;

private long blackHole;
private final ReleasableLock lock = new ReleasableLock(new ReentrantLock());
Expand All @@ -570,14 +584,18 @@ static class G1OverLimitStrategy implements OverLimitStrategy {
LongSupplier gcCountSupplier,
LongSupplier timeSupplier,
long minimumInterval,
TimeValue lockTimeout
long fullGCMinimumInterval,
TimeValue lockTimeout,
TimeValue fullGCLockTimeout
) {
this.lockTimeout = lockTimeout;
this.fullGCLockTimeout = fullGCLockTimeout;
assert minimumInterval > 0;
this.currentMemoryUsageSupplier = currentMemoryUsageSupplier;
this.gcCountSupplier = gcCountSupplier;
this.timeSupplier = timeSupplier;
this.minimumInterval = minimumInterval;
this.fullGCMinimumInterval = fullGCMinimumInterval;
this.maxHeap = jvmInfo.getMem().getHeapMax().getBytes();
long g1RegionSize = jvmInfo.getG1RegionSize();
if (g1RegionSize <= 0) {
Expand All @@ -604,71 +622,69 @@ static long fallbackRegionSize(JvmInfo jvmInfo) {
return regionSize;
}

@SuppressForbidden(reason = "Prefer full GC to OOM or CBE")
private static void performFullGC() {
System.gc();
}

@Override
public MemoryUsage overLimit(MemoryUsage memoryUsed) {
boolean leader = false;
int allocationIndex = 0;
long allocationDuration = 0;
long begin = 0;

TriggerGCResult result = TriggerGCResult.EMPTY;
int attemptNoCopy = 0;

try (ReleasableLock locked = lock.tryAcquire(lockTimeout)) {
if (locked != null) {
attemptNoCopy = ++this.attemptNo;
begin = timeSupplier.getAsLong();
leader = begin >= lastCheckTime + minimumInterval;
overLimitTriggered(leader);
if (leader) {
long initialCollectionCount = gcCountSupplier.getAsLong();
logger.info("attempting to trigger G1GC due to high heap usage [{}]", memoryUsed.baseUsage);
long localBlackHole = 0;
// number of allocations, corresponding to (approximately) number of free regions + 1
int allocationCount = Math.toIntExact((maxHeap - memoryUsed.baseUsage) / g1RegionSize + 1);
// allocations of half-region size becomes single humongous alloc, thus taking up a full region.
int allocationSize = (int) (g1RegionSize >> 1);
long maxUsageObserved = memoryUsed.baseUsage;
for (; allocationIndex < allocationCount; ++allocationIndex) {
long current = currentMemoryUsageSupplier.getAsLong();
if (current >= maxUsageObserved) {
maxUsageObserved = current;
} else {
// we observed a memory drop, so some GC must have occurred
break;
}
if (initialCollectionCount != gcCountSupplier.getAsLong()) {
break;
}
localBlackHole += new byte[allocationSize].hashCode();
}

blackHole += localBlackHole;
logger.trace("black hole [{}]", blackHole);

long now = timeSupplier.getAsLong();
this.lastCheckTime = now;
allocationDuration = now - begin;
this.attemptNo = 0;
}
result = tryTriggerGC(memoryUsed);
} else {
logger.info("could not acquire lock within {} when attempting to trigger G1GC due to high heap usage", lockTimeout);
}
} catch (InterruptedException e) {
logger.info("could not acquire lock when attempting to trigger G1GC due to high heap usage");
Thread.currentThread().interrupt();
// fallthrough
}

if (performingFullGC && attemptNoCopy == 0) {
// Another thread is currently performing a full GC, and we were not able to try (lock acquire timeout)
// Since the full GC thread may hold the lock for longer, try again for an additional timeout
logger.info(
"could not acquire lock within {} while another thread was performing a full GC, waiting again for {}",
lockTimeout,
fullGCLockTimeout
);
try (ReleasableLock locked = lock.tryAcquire(fullGCLockTimeout)) {
if (locked != null) {
attemptNoCopy = ++this.attemptNo;
result = tryTriggerGC(memoryUsed);
} else {
logger.info(
"could not acquire lock within {} when attempting to trigger G1GC due to high heap usage",
fullGCLockTimeout
);
}
} catch (InterruptedException e) {
logger.info("could not acquire lock when attempting to trigger G1GC due to high heap usage");
Thread.currentThread().interrupt();
// fallthrough
}
}

final long current = currentMemoryUsageSupplier.getAsLong();
if (current < memoryUsed.baseUsage) {
if (leader) {
if (result.gcAttempted()) {
logger.info(
"GC did bring memory usage down, before [{}], after [{}], allocations [{}], duration [{}]",
memoryUsed.baseUsage,
current,
allocationIndex,
allocationDuration
result.allocationIndex(),
result.allocationDuration()
);
} else if (attemptNoCopy < 10 || Long.bitCount(attemptNoCopy) == 1) {
logger.info(
"memory usage down after [{}], before [{}], after [{}]",
begin - lastCheckTime,
result.timeSinceLastCheck(),
memoryUsed.baseUsage,
current
);
Expand All @@ -680,18 +696,18 @@ public MemoryUsage overLimit(MemoryUsage memoryUsed) {
memoryUsed.permanentChildUsage
);
} else {
if (leader) {
if (result.gcAttempted()) {
logger.info(
"GC did not bring memory usage down, before [{}], after [{}], allocations [{}], duration [{}]",
memoryUsed.baseUsage,
current,
allocationIndex,
allocationDuration
result.allocationIndex(),
result.allocationDuration()
);
} else if (attemptNoCopy < 10 || Long.bitCount(attemptNoCopy) == 1) {
logger.info(
"memory usage not down after [{}], before [{}], after [{}]",
begin - lastCheckTime,
result.timeSinceLastCheck(),
memoryUsed.baseUsage,
current
);
Expand All @@ -701,6 +717,95 @@ public MemoryUsage overLimit(MemoryUsage memoryUsed) {
}
}

private TriggerGCResult tryTriggerGC(MemoryUsage memoryUsed) {
long begin = timeSupplier.getAsLong();
boolean canPerformGC = begin >= lastCheckTime + minimumInterval;
int allocationIndex = 0;

overLimitTriggered(canPerformGC);

if (canPerformGC) {
long initialCollectionCount = gcCountSupplier.getAsLong();
logger.info("attempting to trigger G1GC due to high heap usage [{}]", memoryUsed.baseUsage);
long localBlackHole = 0;
// number of allocations, corresponding to (approximately) number of free regions + 1
int allocationCount = Math.toIntExact((maxHeap - memoryUsed.baseUsage) / g1RegionSize + 1);
// allocations of half-region size becomes single humongous alloc, thus taking up a full region.
int allocationSize = (int) (g1RegionSize >> 1);
long maxUsageObserved = memoryUsed.baseUsage;
for (; allocationIndex < allocationCount; ++allocationIndex) {
long current = currentMemoryUsageSupplier.getAsLong();
if (current >= maxUsageObserved) {
maxUsageObserved = current;
} else {
// we observed a memory drop, so some GC must have occurred
break;
}
if (initialCollectionCount != gcCountSupplier.getAsLong()) {
break;
}
localBlackHole += new byte[allocationSize].hashCode();
}

blackHole += localBlackHole;
logger.trace("black hole [{}]", blackHole);

this.lastCheckTime = timeSupplier.getAsLong();
this.attemptNo = 0;
}

long reclaimedMemory = memoryUsed.baseUsage - currentMemoryUsageSupplier.getAsLong();
// TODO: use a threshold? Relative to % of memory?
if (reclaimedMemory <= 0) {
long now = timeSupplier.getAsLong();
boolean canPerformFullGC = now >= lastFullGCTime + fullGCMinimumInterval;
if (canPerformFullGC) {
// Enough time passed between 2 full GC fallbacks
performingFullGC = true;
logger.info("attempt to trigger young GC failed to bring memory down, triggering full GC");
performFullGC();
performingFullGC = false;
this.lastFullGCTime = timeSupplier.getAsLong();
}
}

long allocationDuration = timeSupplier.getAsLong() - begin;
return new TriggerGCResult(canPerformGC, allocationIndex, allocationDuration, begin - lastCheckTime);
}

private static final class TriggerGCResult {
private final boolean gcAttempted;
private final int allocationIndex;
private final long allocationDuration;
private final long timeSinceLastCheck;

private static final TriggerGCResult EMPTY = new TriggerGCResult(false, 0, 0, 0);

TriggerGCResult(boolean gcAttempted, int allocationIndex, long allocationDuration, long timeSinceLastCheck) {

this.gcAttempted = gcAttempted;
this.allocationIndex = allocationIndex;
this.allocationDuration = allocationDuration;
this.timeSinceLastCheck = timeSinceLastCheck;
}

public boolean gcAttempted() {
return gcAttempted;
}

public int allocationIndex() {
return allocationIndex;
}

public long allocationDuration() {
return allocationDuration;
}

public long timeSinceLastCheck() {
return timeSinceLastCheck;
}
}

void overLimitTriggered(boolean leader) {
// for tests to override.
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -342,6 +342,7 @@ public void testParentTriggersG1GCBeforeBreaking() throws InterruptedException,
AtomicReference<Consumer<Boolean>> onOverLimit = new AtomicReference<>(leader -> {});
AtomicLong time = new AtomicLong(randomLongBetween(Long.MIN_VALUE / 2, Long.MAX_VALUE / 2));
long interval = randomLongBetween(1, 1000);
long fullGCInterval = randomLongBetween(500, 2000);
final HierarchyCircuitBreakerService service = new HierarchyCircuitBreakerService(
clusterSettings,
Collections.emptyList(),
Expand All @@ -352,6 +353,8 @@ public void testParentTriggersG1GCBeforeBreaking() throws InterruptedException,
HierarchyCircuitBreakerService.createYoungGcCountSupplier(),
time::get,
interval,
fullGCInterval,
TimeValue.timeValueSeconds(30),
TimeValue.timeValueSeconds(30)
) {

Expand Down Expand Up @@ -477,6 +480,7 @@ public void testG1OverLimitStrategyBreakOnMemory() {
AtomicInteger leaderTriggerCount = new AtomicInteger();
AtomicInteger nonLeaderTriggerCount = new AtomicInteger();
long interval = randomLongBetween(1, 1000);
long fullGCInterval = randomLongBetween(500, 2000);
AtomicLong memoryUsage = new AtomicLong();

HierarchyCircuitBreakerService.G1OverLimitStrategy strategy = new HierarchyCircuitBreakerService.G1OverLimitStrategy(
Expand All @@ -485,6 +489,8 @@ public void testG1OverLimitStrategyBreakOnMemory() {
() -> 0,
time::get,
interval,
fullGCInterval,
TimeValue.timeValueSeconds(30),
TimeValue.timeValueSeconds(30)
) {
@Override
Expand Down Expand Up @@ -531,6 +537,7 @@ public void testG1OverLimitStrategyBreakOnGcCount() {
AtomicInteger leaderTriggerCount = new AtomicInteger();
AtomicInteger nonLeaderTriggerCount = new AtomicInteger();
long interval = randomLongBetween(1, 1000);
long fullGCInterval = randomLongBetween(500, 2000);
AtomicLong memoryUsageCounter = new AtomicLong();
AtomicLong gcCounter = new AtomicLong();
LongSupplier memoryUsageSupplier = () -> {
Expand All @@ -543,6 +550,8 @@ public void testG1OverLimitStrategyBreakOnGcCount() {
gcCounter::incrementAndGet,
time::get,
interval,
fullGCInterval,
TimeValue.timeValueSeconds(30),
TimeValue.timeValueSeconds(30)
) {

Expand All @@ -565,20 +574,24 @@ void overLimitTriggered(boolean leader) {
assertThat(strategy.overLimit(input), sameInstance(input));
assertThat(leaderTriggerCount.get(), equalTo(1));
assertThat(gcCounter.get(), equalTo(2L));
assertThat(memoryUsageCounter.get(), equalTo(2L)); // 1 before gc count break and 1 to get resulting memory usage.
// 1 before gc count break, 1 for full GC check and 1 to get resulting memory usage.
assertThat(memoryUsageCounter.get(), equalTo(3L));
}

public void testG1OverLimitStrategyThrottling() throws InterruptedException, BrokenBarrierException, TimeoutException {
AtomicLong time = new AtomicLong(randomLongBetween(Long.MIN_VALUE / 2, Long.MAX_VALUE / 2));
AtomicInteger leaderTriggerCount = new AtomicInteger();
long interval = randomLongBetween(1, 1000);
long fullGCInterval = randomLongBetween(500, 2000);
AtomicLong memoryUsage = new AtomicLong();
HierarchyCircuitBreakerService.G1OverLimitStrategy strategy = new HierarchyCircuitBreakerService.G1OverLimitStrategy(
JvmInfo.jvmInfo(),
memoryUsage::get,
() -> 0,
time::get,
interval,
fullGCInterval,
TimeValue.timeValueSeconds(30),
TimeValue.timeValueSeconds(30)
) {

Expand Down Expand Up @@ -661,6 +674,8 @@ public void testG1LockTimeout() throws Exception {
gcCounter::incrementAndGet,
() -> 0,
1,
1,
TimeValue.timeValueMillis(randomFrom(0, 5, 10)),
TimeValue.timeValueMillis(randomFrom(0, 5, 10))
) {

Expand Down

0 comments on commit bfe46b4

Please sign in to comment.