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 shadow topics cannot be consumed when the entry is not cached #23147

Merged
Merged
Show file tree
Hide file tree
Changes from 3 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 @@ -31,6 +31,7 @@
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.atomic.AtomicInteger;
import org.apache.bookkeeper.client.api.BKException;
import org.apache.bookkeeper.client.api.LedgerEntries;
import org.apache.bookkeeper.client.api.LedgerEntry;
import org.apache.bookkeeper.client.api.ReadHandle;
import org.apache.bookkeeper.mledger.AsyncCallbacks;
Expand Down Expand Up @@ -249,7 +250,7 @@ private void asyncReadEntry0(ReadHandle lh, Position position, final ReadEntryCa
manager.mlFactoryMBean.recordCacheHit(cachedEntry.getLength());
callback.readEntryComplete(cachedEntry, ctx);
} else {
lh.readAsync(position.getEntryId(), position.getEntryId()).thenAcceptAsync(
readAsync(lh, position.getEntryId(), position.getEntryId()).thenAcceptAsync(
ledgerEntries -> {
try {
Iterator<LedgerEntry> iterator = ledgerEntries.iterator();
Expand Down Expand Up @@ -429,7 +430,7 @@ public void readEntriesFailed(ManagedLedgerException exception, Object ctx) {
CompletableFuture<List<EntryImpl>> readFromStorage(ReadHandle lh,
long firstEntry, long lastEntry, boolean shouldCacheEntry) {
final int entriesToRead = (int) (lastEntry - firstEntry) + 1;
CompletableFuture<List<EntryImpl>> readResult = lh.readAsync(firstEntry, lastEntry)
CompletableFuture<List<EntryImpl>> readResult = readAsync(lh, firstEntry, lastEntry)
.thenApply(
ledgerEntries -> {
requireNonNull(ml.getName());
Expand Down Expand Up @@ -512,5 +513,22 @@ public void invalidateEntriesBeforeTimestamp(long timestamp) {
manager.entriesRemoved(evictedPair.getRight(), evictedPair.getLeft());
}

private CompletableFuture<LedgerEntries> readAsync(ReadHandle handle, long firstEntry, long lastEntry) {
final var lastConfirmedEntry = ml.getLastConfirmedEntry();
if (lastConfirmedEntry == null) {
return CompletableFuture.failedFuture(new IllegalStateException("LastConfirmedEntry is null when reading "
+ handle.getId()));
}
if (handle.getId() > lastConfirmedEntry.getLedgerId()) {
return CompletableFuture.failedFuture(new IllegalStateException("LastConfirmedEntry is "
+ lastConfirmedEntry + " while trying to read ledger " + handle.getId()));
}
if (handle.getId() == lastConfirmedEntry.getLedgerId() && lastEntry > lastConfirmedEntry.getEntryId()) {
return CompletableFuture.failedFuture(new IllegalStateException("Last ConfirmedEntry is "
+ lastConfirmedEntry + " while trying to read entry " + lastEntry));
}
return handle.readUnconfirmedAsync(firstEntry, lastEntry);
}

private static final Logger log = LoggerFactory.getLogger(RangeEntryCacheImpl.class);
}
Original file line number Diff line number Diff line change
Expand Up @@ -314,7 +314,7 @@ public CompletableFuture<LedgerEntries> readAsync(long firstEntry, long lastEntr

@Override
public CompletableFuture<LedgerEntries> readUnconfirmedAsync(long firstEntry, long lastEntry) {
return unsupported();
return readAsync(firstEntry, lastEntry);
}

@Override
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,109 @@
/*
* 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.pulsar.broker.service.persistent;

import com.google.common.collect.Lists;
import java.util.Optional;
import java.util.Set;
import java.util.concurrent.TimeUnit;
import org.apache.bookkeeper.mledger.impl.ShadowManagedLedgerImpl;
import org.apache.pulsar.broker.PulsarService;
import org.apache.pulsar.broker.ServiceConfiguration;
import org.apache.pulsar.client.admin.PulsarAdmin;
import org.apache.pulsar.client.api.SubscriptionInitialPosition;
import org.apache.pulsar.common.naming.TopicName;
import org.apache.pulsar.common.policies.data.ClusterData;
import org.apache.pulsar.common.policies.data.TenantInfo;
import org.apache.pulsar.common.util.PortManager;
import org.apache.pulsar.zookeeper.LocalBookkeeperEnsemble;
import org.awaitility.Awaitility;
import org.testng.Assert;
import org.testng.annotations.AfterClass;
import org.testng.annotations.BeforeClass;
import org.testng.annotations.Test;

public class ShadowTopicRealBkTest {

private static final String cluster = "test";
private final int zkPort = PortManager.nextLockedFreePort();
private final LocalBookkeeperEnsemble bk = new LocalBookkeeperEnsemble(2, zkPort, PortManager::nextLockedFreePort);
private PulsarService pulsar;
private PulsarAdmin admin;

@BeforeClass
public void setup() throws Exception {
bk.start();
final var config = new ServiceConfiguration();
config.setClusterName(cluster);
config.setAdvertisedAddress("localhost");
config.setBrokerServicePort(Optional.of(0));
config.setWebServicePort(Optional.of(0));
config.setMetadataStoreUrl("zk:localhost:" + zkPort);
pulsar = new PulsarService(config);
pulsar.start();
admin = pulsar.getAdminClient();
admin.clusters().createCluster("test", ClusterData.builder().serviceUrl(pulsar.getWebServiceAddress())
.brokerServiceUrl(pulsar.getBrokerServiceUrl()).build());
admin.tenants().createTenant("public", TenantInfo.builder().allowedClusters(Set.of(cluster)).build());
admin.namespaces().createNamespace("public/default");
}

@AfterClass(alwaysRun = true)
public void cleanup() throws Exception {
if (pulsar != null) {
pulsar.close();
}
bk.stop();
}

@Test
public void testReadFromStorage() throws Exception {
final var sourceTopic = TopicName.get("test-read-from-source").toString();
final var shadowTopic = sourceTopic + "-shadow";

admin.topics().createNonPartitionedTopic(sourceTopic);
admin.topics().createShadowTopic(shadowTopic, sourceTopic);
admin.topics().setShadowTopics(sourceTopic, Lists.newArrayList(shadowTopic));

Awaitility.await().atMost(5, TimeUnit.SECONDS).untilAsserted(()->{
final var sourcePersistentTopic = (PersistentTopic) pulsar.getBrokerService()
.getTopicIfExists(sourceTopic).get().orElseThrow();
final var replicator = (ShadowReplicator) sourcePersistentTopic.getShadowReplicators().get(shadowTopic);
Assert.assertNotNull(replicator);
Assert.assertEquals(String.valueOf(replicator.getState()), "Started");
});

final var client = pulsar.getClient();
// When the message was sent, there is no cursor, so it will read from the cache
final var producer = client.newProducer().topic(sourceTopic).create();
producer.send("message".getBytes());
// 1. Verify RangeEntryCacheImpl#readFromStorage
final var consumer = client.newConsumer().topic(shadowTopic).subscriptionName("sub")
.subscriptionInitialPosition(SubscriptionInitialPosition.Earliest).subscribe();
final var msg = consumer.receive(5, TimeUnit.SECONDS);
Assert.assertNotNull(msg);
Assert.assertEquals(msg.getValue(), "message".getBytes());

// 2. Verify EntryCache#asyncReadEntry
final var shadowManagedLedger = ((PersistentTopic) pulsar.getBrokerService().getTopicIfExists(shadowTopic).get()
.orElseThrow()).getManagedLedger();
Assert.assertTrue(shadowManagedLedger instanceof ShadowManagedLedgerImpl);
shadowManagedLedger.getEarliestMessagePublishTimeInBacklog().get(3, TimeUnit.SECONDS);
}
}
Loading