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][test] Fix flaky test RawReaderTest #21008

Closed
wants to merge 13 commits into from
Closed
Show file tree
Hide file tree
Changes from 5 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 @@ -321,7 +321,7 @@ public void testNegativeAcksDeleteFromUnackedTracker() throws Exception {
negativeAcksTracker.close();
}

@Test(timeOut = 10000)
@Test
Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Locally run takes about 8s sometimes, so remove this.

public void testNegativeAcksWithBatchAckEnabled() throws Exception {
cleanup();
conf.setAcknowledgmentAtBatchIndexLevelEnabled(true);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -30,8 +30,11 @@
import java.util.concurrent.Future;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
import java.util.concurrent.atomic.AtomicBoolean;
import lombok.extern.slf4j.Slf4j;
import org.apache.bookkeeper.mledger.ManagedLedger;
import org.apache.commons.lang3.tuple.ImmutableTriple;
import org.apache.pulsar.broker.BrokerTestUtil;
import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest;
import org.apache.pulsar.broker.service.persistent.PersistentTopic;
import org.apache.pulsar.client.api.MessageId;
Expand All @@ -51,6 +54,7 @@
import org.testng.annotations.Test;

@Test(groups = "broker-impl")
@Slf4j
public class RawReaderTest extends MockedPulsarServiceBaseTest {

private static final String subscription = "foobar-sub";
Expand All @@ -62,6 +66,7 @@ public void setup() throws Exception {
"org.apache.pulsar.common.intercept.AppendBrokerTimestampMetadataInterceptor",
"org.apache.pulsar.common.intercept.AppendIndexMetadataInterceptor"
));
conf.setSystemTopicEnabled(false);
conf.setExposingBrokerEntryMetadataToClientEnabled(true);
super.internalSetup();

Expand Down Expand Up @@ -116,7 +121,7 @@ public static String extractKey(RawMessage m) {
@Test
public void testHasMessageAvailableWithoutBatch() throws Exception {
int numKeys = 10;
String topic = "persistent://my-property/my-ns/my-raw-topic";
String topic = "persistent://my-property/my-ns/" + BrokerTestUtil.newUniqueName("reader");
Set<String> keys = publishMessages(topic, numKeys);
RawReader reader = RawReader.create(pulsarClient, topic, subscription).get();
while (true) {
Expand All @@ -133,43 +138,56 @@ public void testHasMessageAvailableWithoutBatch() throws Exception {
}
}
Assert.assertTrue(keys.isEmpty());
reader.closeAsync().get(3, TimeUnit.SECONDS);
}

@Test
public void testHasMessageAvailableWithBatch() throws Exception {
int numKeys = 20;
String topic = "persistent://my-property/my-ns/my-raw-topic";
String topic = "persistent://my-property/my-ns/" + BrokerTestUtil.newUniqueName("reader");
Set<String> keys = publishMessages(topic, numKeys, true);
RawReader reader = RawReader.create(pulsarClient, topic, subscription).get();
int messageCount = 0;
AtomicBoolean error = new AtomicBoolean(false);
List<MessageId> ids = new ArrayList<>();
List<String> keys2 = new ArrayList<>();
while (true) {
boolean hasMsg = reader.hasMessageAvailableAsync().get();
if (hasMsg && (messageCount == numKeys)) {
Assert.fail("HasMessageAvailable shows still has message when there is no message");
error.set(true);
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Why do you change this? If you are using the Atomicboolean, you need to break after the set. Otherwise, this test will be blocked at reader.readNextAsync().get() because it shows there have messages but no message can be read.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I will revert this when find the root cause.

}
if (hasMsg) {
try (RawMessage m = reader.readNextAsync().get()) {
MessageMetadata meta = Commands.parseMessageMetadata(m.getHeadersAndPayload());
messageCount += meta.getNumMessagesInBatch();
RawBatchConverter.extractIdsAndKeysAndSize(m).forEach(batchInfo -> {
String key = batchInfo.getMiddle();
Assert.assertTrue(keys.remove(key));
keys2.add(key);
ids.add(batchInfo.getLeft());
log.info("ids : {}, keys2 : {}", ids, keys2);
if (!error.get()) {
Assert.assertTrue(keys.remove(key));
}
});

}
} else {
break;
}
}
if (error.get()) {
Assert.assertEquals(ids, keys2);
}
Assert.assertEquals(messageCount, numKeys);
Assert.assertTrue(keys.isEmpty());
reader.closeAsync().get(3, TimeUnit.SECONDS);
}

@Test
public void testRawReader() throws Exception {
int numKeys = 10;

String topic = "persistent://my-property/my-ns/my-raw-topic";
String topic = "persistent://my-property/my-ns/" + BrokerTestUtil.newUniqueName("reader");

Set<String> keys = publishMessages(topic, numKeys);

Expand All @@ -185,12 +203,13 @@ public void testRawReader() throws Exception {
}
}
Assert.assertTrue(keys.isEmpty());
reader.closeAsync().get(3, TimeUnit.SECONDS);
}

@Test
public void testSeekToStart() throws Exception {
int numKeys = 10;
String topic = "persistent://my-property/my-ns/my-raw-topic";
String topic = "persistent://my-property/my-ns/" + BrokerTestUtil.newUniqueName("reader");

publishMessages(topic, numKeys);

Expand Down Expand Up @@ -219,12 +238,13 @@ public void testSeekToStart() throws Exception {
}
}
Assert.assertTrue(readKeys.isEmpty());
reader.closeAsync().get(3, TimeUnit.SECONDS);
}

@Test
public void testSeekToMiddle() throws Exception {
int numKeys = 10;
String topic = "persistent://my-property/my-ns/my-raw-topic";
String topic = "persistent://my-property/my-ns/" + BrokerTestUtil.newUniqueName("reader");

publishMessages(topic, numKeys);

Expand Down Expand Up @@ -262,6 +282,7 @@ public void testSeekToMiddle() throws Exception {
}
}
Assert.assertTrue(readKeys.isEmpty());
reader.closeAsync().get(3, TimeUnit.SECONDS);
}

/**
Expand All @@ -270,7 +291,7 @@ public void testSeekToMiddle() throws Exception {
@Test
public void testFlowControl() throws Exception {
int numMessages = RawReaderImpl.DEFAULT_RECEIVER_QUEUE_SIZE * 5;
String topic = "persistent://my-property/my-ns/my-raw-topic";
String topic = "persistent://my-property/my-ns/" + BrokerTestUtil.newUniqueName("reader");

publishMessages(topic, numMessages);

Expand All @@ -296,12 +317,13 @@ public void testFlowControl() throws Exception {
}
Assert.assertEquals(timeouts, 1);
Assert.assertEquals(keys.size(), numMessages);
reader.closeAsync().get(3, TimeUnit.SECONDS);
}

@Test
public void testFlowControlBatch() throws Exception {
int numMessages = RawReaderImpl.DEFAULT_RECEIVER_QUEUE_SIZE * 5;
String topic = "persistent://my-property/my-ns/my-raw-topic";
String topic = "persistent://my-property/my-ns/" + BrokerTestUtil.newUniqueName("reader");

publishMessages(topic, numMessages, true);

Expand All @@ -324,11 +346,12 @@ public void testFlowControlBatch() throws Exception {
}
}
Assert.assertEquals(keys.size(), numMessages);
reader.closeAsync().get(3, TimeUnit.SECONDS);
}

@Test
public void testBatchingExtractKeysAndIds() throws Exception {
String topic = "persistent://my-property/my-ns/my-raw-topic";
String topic = "persistent://my-property/my-ns/" + BrokerTestUtil.newUniqueName("reader");

try (Producer<byte[]> producer = pulsarClient.newProducer().topic(topic)
.maxPendingMessages(3)
Expand Down Expand Up @@ -363,7 +386,7 @@ public void testBatchingExtractKeysAndIds() throws Exception {

@Test
public void testBatchingRebatch() throws Exception {
String topic = "persistent://my-property/my-ns/my-raw-topic";
String topic = "persistent://my-property/my-ns/" + BrokerTestUtil.newUniqueName("reader");

try (Producer<byte[]> producer = pulsarClient.newProducer().topic(topic)
.maxPendingMessages(3)
Expand Down Expand Up @@ -392,7 +415,7 @@ public void testBatchingRebatch() throws Exception {

@Test
public void testBatchingRebatchWithBrokerEntryMetadata() throws Exception {
String topic = "persistent://my-property/my-ns/my-raw-topic";
String topic = "persistent://my-property/my-ns/" + BrokerTestUtil.newUniqueName("reader");

try (Producer<byte[]> producer = pulsarClient.newProducer().topic(topic)
.maxPendingMessages(3)
Expand Down Expand Up @@ -428,7 +451,7 @@ public void testBatchingRebatchWithBrokerEntryMetadata() throws Exception {
public void testAcknowledgeWithProperties() throws Exception {
int numKeys = 10;

String topic = "persistent://my-property/my-ns/my-raw-topic";
String topic = "persistent://my-property/my-ns/" + BrokerTestUtil.newUniqueName("reader");

Set<String> keys = publishMessages(topic, numKeys);

Expand Down Expand Up @@ -459,14 +482,14 @@ public void testAcknowledgeWithProperties() throws Exception {
Assert.assertEquals(
ledger.openCursor(subscription).getProperties().get("foobar"),
Long.valueOf(0xdeadbeefdecaL)));

reader.closeAsync().get(3, TimeUnit.SECONDS);
}

@Test
public void testReadCancellationOnClose() throws Exception {
int numKeys = 10;

String topic = "persistent://my-property/my-ns/my-raw-topic";
String topic = "persistent://my-property/my-ns/" + BrokerTestUtil.newUniqueName("reader");
publishMessages(topic, numKeys/2);

RawReader reader = RawReader.create(pulsarClient, topic, subscription).get();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -2319,12 +2319,16 @@ public CompletableFuture<Boolean> hasMessageAvailableAsync() {
} else {
// read before, use lastDequeueMessage for comparison
if (hasMoreMessages(lastMessageIdInBroker, lastDequeuedMessageId, false)) {
log.info("hasMoreMessages, lastMessageIdInBroker: {}, lastDequeuedMessageId: {}",
lastMessageIdInBroker, lastDequeuedMessageId);
completehasMessageAvailableWithValue(booleanFuture, true);
return booleanFuture;
}

getLastMessageIdAsync().thenAccept(messageId -> {
lastMessageIdInBroker = messageId;
log.info("getLastMessageIdAsync, lastMessageIdInBroker : {}, lastDequeuedMessageId : {}",
lastMessageIdInBroker, lastDequeuedMessageId);
completehasMessageAvailableWithValue(booleanFuture,
hasMoreMessages(lastMessageIdInBroker, lastDequeuedMessageId, false));
}).exceptionally(e -> {
Expand Down
Loading