Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

PIP-211 [feat][tiered-storage] Introduce offload throttling, Part 1 #17485

Closed
wants to merge 18 commits into from
Closed
Original file line number Diff line number Diff line change
Expand Up @@ -85,6 +85,7 @@ public class ManagedLedgerConfig {
private int minimumBacklogCursorsForCaching = 0;
private int minimumBacklogEntriesForCaching = 1000;
private int maxBacklogBetweenCursorsForCaching = 1000;
private long managedLedgerOffloadFlowPermitsPerSecond = -1;

public boolean isCreateIfMissing() {
return createIfMissing;
Expand Down Expand Up @@ -749,4 +750,23 @@ public String getShadowSource() {
}

public static final String PROPERTY_SOURCE_TOPIC_KEY = "PULSAR.SHADOW_SOURCE";

/**
* Set permitted size to offload on the broker.
*
* @param managedLedgerOffloadBrokerFlowPermit
*/
public void setManagedLedgerOffloadFlowPermitsPerSecond(long managedLedgerOffloadBrokerFlowPermit) {
this.managedLedgerOffloadFlowPermitsPerSecond = managedLedgerOffloadBrokerFlowPermit;
}

/**
* Get permitted size to offload on the broker.
*
* @return
*/
public long getManagedLedgerOffloadFlowPermitsPerSecond() {
return managedLedgerOffloadFlowPermitsPerSecond;
}

}
Original file line number Diff line number Diff line change
@@ -0,0 +1,213 @@
/*
* 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
*
* http://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.bookkeeper.mledger;

import com.google.common.annotations.VisibleForTesting;
import java.util.Objects;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicLong;
import org.apache.bookkeeper.client.api.LastConfirmedAndEntry;
import org.apache.bookkeeper.client.api.LedgerEntries;
import org.apache.bookkeeper.client.api.LedgerMetadata;
import org.apache.bookkeeper.client.api.ReadHandle;
import org.apache.bookkeeper.common.util.OrderedScheduler;
import org.apache.bookkeeper.mledger.util.TimeWindow;
import org.apache.bookkeeper.mledger.util.WindowWrap;

public class OffloadReadHandle implements ReadHandle {
private static final AtomicBoolean INITIALIZED = new AtomicBoolean(false);
private static volatile long flowPermits = -1L;
private static volatile TimeWindow<AtomicLong> window;

private final ReadHandle delegator;
private final OrderedScheduler scheduler;

private OffloadReadHandle(ReadHandle handle, ManagedLedgerConfig config,
OrderedScheduler scheduler) {
initialize(config);
this.delegator = handle;
this.scheduler = Objects.requireNonNull(scheduler);
}

private static void initialize(ManagedLedgerConfig config) {
if (INITIALIZED.compareAndSet(false, true)) {
flowPermits = config.getManagedLedgerOffloadFlowPermitsPerSecond();
window = new TimeWindow<>(2, 1000);
}
}

public static CompletableFuture<ReadHandle> create(ReadHandle handle, ManagedLedgerConfig config,
OrderedScheduler scheduler) {
return CompletableFuture.completedFuture(new OffloadReadHandle(handle, config, scheduler));
}

@Override
public CompletableFuture<LedgerEntries> readAsync(long firstEntry, long lastEntry) {
final long delayMills = calculateDelayMillis();
if (delayMills > 0) {
CompletableFuture<LedgerEntries> f = new CompletableFuture<>();
Runnable cmd = new ReadAsyncCommand(firstEntry, lastEntry, f);
scheduler.schedule(cmd, delayMills, TimeUnit.MILLISECONDS);
return f;
}

return this.delegator
.readAsync(firstEntry, lastEntry)
.whenComplete((v, t) -> {
if (t == null) {
recordReadBytes(v);
}
});
}

@Override
public CompletableFuture<LedgerEntries> readUnconfirmedAsync(long firstEntry, long lastEntry) {
return this.delegator.readUnconfirmedAsync(firstEntry, lastEntry);
}

@Override
public CompletableFuture<Long> readLastAddConfirmedAsync() {
return this.delegator.readLastAddConfirmedAsync();
}

@Override
public CompletableFuture<Long> tryReadLastAddConfirmedAsync() {
return this.delegator.tryReadLastAddConfirmedAsync();
}

@Override
public long getLastAddConfirmed() {
return this.delegator.getLastAddConfirmed();
}

@Override
public long getLength() {
return this.delegator.getLength();
}

@Override
public boolean isClosed() {
return this.delegator.isClosed();
}

@Override
public CompletableFuture<LastConfirmedAndEntry> readLastAddConfirmedAndEntryAsync(
long entryId, long timeOutInMillis, boolean parallel) {
return this.delegator.readLastAddConfirmedAndEntryAsync(entryId, timeOutInMillis, parallel);
}

@Override
public long getId() {
return this.delegator.getId();
}

@Override
public CompletableFuture<Void> closeAsync() {
return this.delegator.closeAsync();
}

@Override
public LedgerMetadata getLedgerMetadata() {
return this.delegator.getLedgerMetadata();
}


private long calculateDelayMillis() {
if (flowPermits <= 0) {
return 0;
}

WindowWrap<AtomicLong> wrap = window.current(__ -> new AtomicLong(0));
if (wrap == null) {
// it should never goes here
return 0;
}

if (wrap.value().get() >= flowPermits) {
// park until next window start
long end = wrap.start() + wrap.interval();
return end - System.currentTimeMillis();
}

return 0;
}

private void recordReadBytes(LedgerEntries entries) {
if (flowPermits <= 0) {
return;
}

if (entries == null) {
return;
}

AtomicLong num = new AtomicLong(0);
entries.forEach(en -> num.addAndGet(en.getLength()));

WindowWrap<AtomicLong> wrap = window.current(__ -> new AtomicLong(0));
if (wrap == null) {
// it should never goes here
return;
}

wrap.value().addAndGet(num.get());
}


private final class ReadAsyncCommand implements Runnable {

private final long firstEntry;
private final long lastEntry;
private final CompletableFuture<LedgerEntries> f;

ReadAsyncCommand(long firstEntry, long lastEntry, CompletableFuture<LedgerEntries> f) {
this.firstEntry = firstEntry;
this.lastEntry = lastEntry;
this.f = f;
}

@Override
public void run() {
long delayMillis = calculateDelayMillis();
if (delayMillis > 0) {
scheduler.schedule(this, delayMillis, TimeUnit.MILLISECONDS);
return;
}

delegator.readAsync(firstEntry, lastEntry)
.whenComplete((entries, e) -> {
if (e != null) {
f.completeExceptionally(e);
} else {
f.complete(entries);
recordReadBytes(entries);
}
});
}
}

@VisibleForTesting
public void reset() {
INITIALIZED.set(false);
flowPermits = -1L;
window = null;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -112,6 +112,7 @@
import org.apache.bookkeeper.mledger.ManagedLedgerException.NonRecoverableLedgerException;
import org.apache.bookkeeper.mledger.ManagedLedgerException.TooManyRequestsException;
import org.apache.bookkeeper.mledger.ManagedLedgerMXBean;
import org.apache.bookkeeper.mledger.OffloadReadHandle;
import org.apache.bookkeeper.mledger.Position;
import org.apache.bookkeeper.mledger.WaitingEntryCallBack;
import org.apache.bookkeeper.mledger.impl.ManagedCursorImpl.VoidCallback;
Expand Down Expand Up @@ -3126,15 +3127,16 @@ private void offloadLoop(CompletableFuture<PositionImpl> promise, Queue<LedgerIn
Map<String, String> driverMetadata = config.getLedgerOffloader().getOffloadDriverMetadata();

prepareLedgerInfoForOffloaded(ledgerId, uuid, driverName, driverMetadata)
.thenCompose((ignore) -> getLedgerHandle(ledgerId))
.thenCompose(readHandle -> config.getLedgerOffloader().offload(readHandle, uuid, extraMetadata))
.thenCompose((ignore) -> {
.thenCompose((ignore) -> getLedgerHandle(ledgerId))
.thenCompose(handle -> OffloadReadHandle.create(handle, config, scheduledExecutor))
.thenCompose(readHandle -> config.getLedgerOffloader().offload(readHandle, uuid, extraMetadata))
.thenCompose((ignore) -> {
return Retries.run(Backoff.exponentialJittered(TimeUnit.SECONDS.toMillis(1),
TimeUnit.SECONDS.toHours(1)).limit(10),
FAIL_ON_CONFLICT,
() -> completeLedgerInfoForOffloaded(ledgerId, uuid),
scheduledExecutor, name)
.whenComplete((ignore2, exception) -> {
TimeUnit.SECONDS.toHours(1)).limit(10),
FAIL_ON_CONFLICT,
() -> completeLedgerInfoForOffloaded(ledgerId, uuid),
scheduledExecutor, name)
.whenComplete((ignore2, exception) -> {
if (exception != null) {
Throwable e = FutureUtil.unwrapCompletionException(exception);
if (e instanceof MetaStoreException) {
Expand All @@ -3157,7 +3159,7 @@ private void offloadLoop(CompletableFuture<PositionImpl> promise, Queue<LedgerIn
}
});
})
.whenComplete((ignore, exception) -> {
.whenComplete((ignore, exception) -> {
if (exception != null) {
lastOffloadFailureTimestamp = System.currentTimeMillis();
log.warn("[{}] Exception occurred for ledgerId {} timestamp {} during offload", name,
Expand All @@ -3174,9 +3176,7 @@ private void offloadLoop(CompletableFuture<PositionImpl> promise, Queue<LedgerIn
}
}

offloadLoop(promise, ledgersToOffload,
newFirstUnoffloaded,
errorToReport);
offloadLoop(promise, ledgersToOffload, newFirstUnoffloaded, errorToReport);
} else {
lastOffloadSuccessTimestamp = System.currentTimeMillis();
log.info("[{}] offload for ledgerId {} timestamp {} succeed", name, ledgerId,
Expand Down
Loading