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

[fix] [broker] Fix system topic can not be loaded up if it contains data offloaded #23279

Merged
Merged
Show file tree
Hide file tree
Changes from 2 commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -230,5 +230,9 @@ default void scanLedgers(OffloadedLedgerMetadataConsumer consumer,
Map<String, String> offloadDriverMetadata) throws ManagedLedgerException {
throw ManagedLedgerException.getManagedLedgerException(new UnsupportedOperationException());
}

default boolean isAppendable() {
return true;
}
}

Original file line number Diff line number Diff line change
Expand Up @@ -94,6 +94,7 @@
import org.apache.bookkeeper.mledger.AsyncCallbacks.TerminateCallback;
import org.apache.bookkeeper.mledger.AsyncCallbacks.UpdatePropertiesCallback;
import org.apache.bookkeeper.mledger.Entry;
import org.apache.bookkeeper.mledger.LedgerOffloader;
import org.apache.bookkeeper.mledger.ManagedCursor;
import org.apache.bookkeeper.mledger.ManagedLedger;
import org.apache.bookkeeper.mledger.ManagedLedgerAttributes;
Expand Down Expand Up @@ -2451,8 +2452,7 @@ private void scheduleDeferredTrimming(boolean isTruncate, CompletableFuture<?> p
}

public void maybeOffloadInBackground(CompletableFuture<Position> promise) {
if (config.getLedgerOffloader() == null || config.getLedgerOffloader() == NullLedgerOffloader.INSTANCE
|| config.getLedgerOffloader().getOffloadPolicies() == null) {
if (getOffloadPoliciesIfAppendable().isEmpty()) {
return;
}

Expand All @@ -2468,8 +2468,7 @@ public void maybeOffloadInBackground(CompletableFuture<Position> promise) {

private void maybeOffload(long offloadThresholdInBytes, long offloadThresholdInSeconds,
CompletableFuture<Position> finalPromise) {
if (config.getLedgerOffloader() == null || config.getLedgerOffloader() == NullLedgerOffloader.INSTANCE
|| config.getLedgerOffloader().getOffloadPolicies() == null) {
if (getOffloadPoliciesIfAppendable().isEmpty()) {
String msg = String.format("[%s] Nothing to offload due to offloader or offloadPolicies is NULL", name);
finalPromise.completeExceptionally(new IllegalArgumentException(msg));
return;
Expand Down Expand Up @@ -2572,6 +2571,16 @@ void internalTrimConsumedLedgers(CompletableFuture<?> promise) {
internalTrimLedgers(false, promise);
}

private Optional<OffloadPolicies> getOffloadPoliciesIfAppendable() {
LedgerOffloader ledgerOffloader = config.getLedgerOffloader();
if (ledgerOffloader == null
|| !ledgerOffloader.isAppendable()
|| ledgerOffloader.getOffloadPolicies() == null) {
return Optional.empty();
}
return Optional.ofNullable(ledgerOffloader.getOffloadPolicies());
}

void internalTrimLedgers(boolean isTruncate, CompletableFuture<?> promise) {
if (!factory.isMetadataServiceAvailable()) {
// Defer trimming of ledger if we cannot connect to metadata service
Expand All @@ -2587,10 +2596,7 @@ void internalTrimLedgers(boolean isTruncate, CompletableFuture<?> promise) {

List<LedgerInfo> ledgersToDelete = new ArrayList<>();
List<LedgerInfo> offloadedLedgersToDelete = new ArrayList<>();
Optional<OffloadPolicies> optionalOffloadPolicies = Optional.ofNullable(config.getLedgerOffloader() != null
&& config.getLedgerOffloader() != NullLedgerOffloader.INSTANCE
? config.getLedgerOffloader().getOffloadPolicies()
: null);
Optional<OffloadPolicies> optionalOffloadPolicies = getOffloadPoliciesIfAppendable();
poorbarcode marked this conversation as resolved.
Show resolved Hide resolved
synchronized (this) {
if (log.isDebugEnabled()) {
log.debug("[{}] Start TrimConsumedLedgers. ledgers={} totalSize={}", name, ledgers.keySet(),
Expand Down Expand Up @@ -3117,8 +3123,10 @@ public void offloadFailed(ManagedLedgerException e, Object ctx) {

@Override
public void asyncOffloadPrefix(Position pos, OffloadCallback callback, Object ctx) {
if (config.getLedgerOffloader() != null && config.getLedgerOffloader() == NullLedgerOffloader.INSTANCE) {
callback.offloadFailed(new ManagedLedgerException("NullLedgerOffloader"), ctx);
LedgerOffloader ledgerOffloader = config.getLedgerOffloader();
if (ledgerOffloader != null && !ledgerOffloader.isAppendable()) {
String msg = String.format("[%s] does not support offload", ledgerOffloader.getClass().getSimpleName());
callback.offloadFailed(new ManagedLedgerException(msg), ctx);
return;
}
Position requestOffloadTo = pos;
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,74 @@
/*
* 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.impl;

import java.util.Map;
import java.util.UUID;
import java.util.concurrent.CompletableFuture;
import org.apache.bookkeeper.client.api.ReadHandle;
import org.apache.bookkeeper.mledger.LedgerOffloader;
import org.apache.pulsar.common.policies.data.OffloadPolicies;
import org.apache.pulsar.common.util.FutureUtil;

public class NonAppendableLedgerOffloader implements LedgerOffloader {
private LedgerOffloader delegate;

public NonAppendableLedgerOffloader(LedgerOffloader delegate) {
this.delegate = delegate;
}

@Override
public String getOffloadDriverName() {
return delegate.getOffloadDriverName();
}

@Override
public CompletableFuture<Void> offload(ReadHandle ledger,
UUID uid,
Map<String, String> extraMetadata) {
return FutureUtil.failedFuture(new UnsupportedOperationException());
}

@Override
public CompletableFuture<ReadHandle> readOffloaded(long ledgerId, UUID uid,
Map<String, String> offloadDriverMetadata) {
return delegate.readOffloaded(ledgerId, uid, offloadDriverMetadata);
}

@Override
public CompletableFuture<Void> deleteOffloaded(long ledgerId, UUID uid,
Map<String, String> offloadDriverMetadata) {
return delegate.deleteOffloaded(ledgerId, uid, offloadDriverMetadata);
}

@Override
public OffloadPolicies getOffloadPolicies() {
return delegate.getOffloadPolicies();
}

@Override
public void close() {
delegate.close();
}

@Override
public boolean isAppendable() {
return false;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -70,4 +70,9 @@ public OffloadPolicies getOffloadPolicies() {
public void close() {

}

@Override
public boolean isAppendable() {
return false;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -29,6 +29,7 @@
import static org.mockito.Mockito.times;
import static org.mockito.Mockito.verify;
import static org.testng.Assert.assertEquals;
import static org.testng.Assert.assertTrue;
import io.netty.buffer.ByteBuf;

import java.util.ArrayList;
Expand All @@ -54,19 +55,43 @@
import org.apache.bookkeeper.mledger.LedgerOffloader;
import org.apache.bookkeeper.mledger.ManagedCursor;
import org.apache.bookkeeper.mledger.ManagedLedgerConfig;
import org.apache.bookkeeper.mledger.ManagedLedgerException;
import org.apache.bookkeeper.mledger.ManagedLedgerFactoryConfig;
import org.apache.bookkeeper.mledger.PositionFactory;
import org.apache.bookkeeper.mledger.proto.MLDataFormats.ManagedLedgerInfo.LedgerInfo;
import org.apache.bookkeeper.mledger.util.MockClock;
import org.apache.bookkeeper.net.BookieId;
import org.apache.bookkeeper.test.MockedBookKeeperTestCase;
import org.apache.pulsar.common.policies.data.OffloadPoliciesImpl;
import org.apache.pulsar.common.policies.data.OffloadedReadPriority;
import org.awaitility.Awaitility;
import org.testng.Assert;
import org.testng.annotations.DataProvider;
import org.testng.annotations.Test;

public class OffloadPrefixReadTest extends MockedBookKeeperTestCase {
@Test
public void testOffloadRead() throws Exception {

private final String offloadTypeReadOnly = "readOnly";

@Override
protected void initManagedLedgerFactoryConfig(ManagedLedgerFactoryConfig config) {
super.initManagedLedgerFactoryConfig(config);
// disable cache.
config.setMaxCacheSize(0);
}

@DataProvider(name = "offloadAndDeleteTypes")
public Object[][] offloadAndDeleteTypes() {
return new Object[][]{
{"normal", true},
{"normal", false},
{offloadTypeReadOnly, true},
{offloadTypeReadOnly, false},
};
}

@Test(dataProvider = "offloadAndDeleteTypes")
public void testOffloadRead(String offloadType, boolean deleteMl) throws Exception {
MockLedgerOffloader offloader = spy(MockLedgerOffloader.class);
ManagedLedgerConfig config = new ManagedLedgerConfig();
config.setMaxEntriesPerLedger(10);
Expand All @@ -89,6 +114,10 @@ public void testOffloadRead() throws Exception {
Assert.assertTrue(ledger.getLedgersInfoAsList().get(1).getOffloadContext().getComplete());
Assert.assertFalse(ledger.getLedgersInfoAsList().get(2).getOffloadContext().getComplete());

if (offloadTypeReadOnly.equals(offloadType)) {
config.setLedgerOffloader(new NonAppendableLedgerOffloader(offloader));
}

UUID firstLedgerUUID = new UUID(ledger.getLedgersInfoAsList().get(0).getOffloadContext().getUidMsb(),
ledger.getLedgersInfoAsList().get(0).getOffloadContext().getUidLsb());
UUID secondLedgerUUID = new UUID(ledger.getLedgersInfoAsList().get(1).getOffloadContext().getUidMsb(),
Expand Down Expand Up @@ -116,13 +145,30 @@ public void testOffloadRead() throws Exception {
verify(offloader, times(2))
.readOffloaded(anyLong(), (UUID) any(), anyMap());

ledger.close();
// Ensure that all the read handles had been closed
assertEquals(offloader.openedReadHandles.get(), 0);
if (!deleteMl) {
ledger.close();
// Ensure that all the read handles had been closed
assertEquals(offloader.openedReadHandles.get(), 0);
} else {
// Verify: the ledger offloaded will be deleted after managed ledger is deleted.
ledger.delete();
Awaitility.await().untilAsserted(() -> {
assertTrue(offloader.offloads.size() <= 1);
assertTrue(ledger.ledgers.size() <= 1);
});
}
}

@Test
public void testBookkeeperFirstOffloadRead() throws Exception {
@DataProvider(name = "offloadTypes")
public Object[][] offloadTypes() {
return new Object[][]{
{"normal"},
{offloadTypeReadOnly},
};
}

@Test(dataProvider = "offloadTypes")
public void testBookkeeperFirstOffloadRead(String offloadType) throws Exception {
MockLedgerOffloader offloader = spy(MockLedgerOffloader.class);
MockClock clock = new MockClock();
offloader.getOffloadPolicies()
Expand Down Expand Up @@ -187,6 +233,10 @@ public void testBookkeeperFirstOffloadRead() throws Exception {
Assert.assertTrue(ledger.getLedgersInfoAsList().get(0).getOffloadContext().getBookkeeperDeleted());
Assert.assertTrue(ledger.getLedgersInfoAsList().get(1).getOffloadContext().getBookkeeperDeleted());

if (offloadTypeReadOnly.equals(offloadType)) {
config.setLedgerOffloader(new NonAppendableLedgerOffloader(offloader));
}

for (Entry e : cursor.readEntries(10)) {
Assert.assertEquals(new String(e.getData()), "entry-" + i++);
}
Expand All @@ -196,6 +246,56 @@ public void testBookkeeperFirstOffloadRead() throws Exception {
.readOffloaded(anyLong(), (UUID) any(), anyMap());
verify(offloader).readOffloaded(anyLong(), eq(secondLedgerUUID), anyMap());

// Verify: the ledger offloaded will be trimmed after if no backlog.
while (cursor.hasMoreEntries()) {
cursor.readEntries(1);
}
config.setRetentionTime(0, TimeUnit.MILLISECONDS);
config.setRetentionSizeInMB(0);
CompletableFuture trimFuture = new CompletableFuture();
ledger.trimConsumedLedgersInBackground(trimFuture);
trimFuture.join();
Awaitility.await().untilAsserted(() -> {
assertTrue(offloader.offloads.size() <= 1);
assertTrue(ledger.ledgers.size() <= 1);
});

// cleanup.
ledger.delete();
}



@Test
public void testSkipOffloadIfReadOnly() throws Exception {
LedgerOffloader ol = new NonAppendableLedgerOffloader(spy(MockLedgerOffloader.class));
ManagedLedgerConfig config = new ManagedLedgerConfig();
config.setMaxEntriesPerLedger(10);
config.setMinimumRolloverTime(0, TimeUnit.SECONDS);
config.setRetentionTime(10, TimeUnit.MINUTES);
config.setRetentionSizeInMB(10);
config.setLedgerOffloader(ol);
ManagedLedgerImpl ledger = (ManagedLedgerImpl) factory.open("my_test_ledger", config);

for (int i = 0; i < 25; i++) {
String content = "entry-" + i;
ledger.addEntry(content.getBytes());
}
assertEquals(ledger.getLedgersInfoAsList().size(), 3);

try {
ledger.offloadPrefix(ledger.getLastConfirmedEntry());
} catch (ManagedLedgerException mle) {
assertTrue(mle.getMessage().contains("does not support offload"));
}

assertEquals(ledger.getLedgersInfoAsList().size(), 3);
Assert.assertFalse(ledger.getLedgersInfoAsList().get(0).getOffloadContext().getComplete());
Assert.assertFalse(ledger.getLedgersInfoAsList().get(1).getOffloadContext().getComplete());
Assert.assertFalse(ledger.getLedgersInfoAsList().get(2).getOffloadContext().getComplete());

// cleanup.
ledger.delete();
}


Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -83,13 +83,17 @@ public final void setUp(Method method) throws Exception {
}

ManagedLedgerFactoryConfig managedLedgerFactoryConfig = new ManagedLedgerFactoryConfig();
// increase default cache eviction interval so that caching could be tested with less flakyness
managedLedgerFactoryConfig.setCacheEvictionIntervalMs(200);
initManagedLedgerFactoryConfig(managedLedgerFactoryConfig);
factory = new ManagedLedgerFactoryImpl(metadataStore, bkc);

setUpTestCase();
}

protected void initManagedLedgerFactoryConfig(ManagedLedgerFactoryConfig config) {
// increase default cache eviction interval so that caching could be tested with less flakyness
config.setCacheEvictionIntervalMs(200);
}

protected void setUpTestCase() throws Exception {

}
Expand Down
Loading
Loading