-
Notifications
You must be signed in to change notification settings - Fork 3.6k
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
Currently the `MessageId` interface hiddens all fields of the `MessageIdData` struct defined in `PulsarApi.proto`. It's usually enough for application users because they don't need to access the fields. But for client developers and developers of other Pulsar ecosystems (e.g. the built-in Kafka connector and the Flink connector in another repo), the `MessageId` interface is too simple and there is no common used abstraction. We can see many code usages like: ```java if (msgId instanceof BatchMessageIdImpl) { // Do type cast and then access fields like ledger id... } else if (msgId instanceof MessageIdImpl) { // Do type cast and then access fields like ledger id... // NOTE: don't put this else if before the previous one because // BatchMessageIdImpl is also a MessageIdImpl } // ... ``` These `MessageId` implementations are used directly. It's a very bad design because any change to the public APIs of these implementations could bring breaking changes. Also, there is a `TopicMessageIdImpl` that each time a `getInnerMessageId()` method must be used to get the underlying `MessageId` object, then do the type assertion and cast again. It makes code unnecessarily complicated. ### Modifications Introduce the `PulsarApiMessageId` interface into the `pulsar-common` module. All `MessageId` implementations so far (except `MultiMessageId`) should extend this interface so we can do the following conversion safely in client code or other modules: ```java long ledgerId = ((PulsarApiMessageId) msgId).getLedgerId(); ``` Regarding the `ack_set` field, use a `BitSet` instead of the `BatchMessageAcker` to record if a message in the batch is acknowledged. Since the `TopicMessageId` is just a proxy of other `MessageId` implementations, it's stored as key or value in the map directly because the `compareTo`/`equal`/`hashCode` methods have the same semantics with the underlying `MessageId`. There is no need to cast the type and call `getInnerMessageId`. Remove all other usages and mark the public methods as deprecated to avoid breaking changes. They could be removed in the next major release. Add a `CustomMessageIdTest` to verify any valid `MessageId` implementation works for `seek` and `acknowledge` APIs.
- Loading branch information
1 parent
01e7eac
commit 4029416
Showing
35 changed files
with
691 additions
and
432 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
125 changes: 125 additions & 0 deletions
125
pulsar-broker/src/test/java/org/apache/pulsar/client/api/CustomMessageIdTest.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,125 @@ | ||
/* | ||
* 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.client.api; | ||
|
||
import static org.testng.Assert.assertEquals; | ||
import static org.testng.Assert.assertNotNull; | ||
import java.util.ArrayList; | ||
import java.util.List; | ||
import java.util.concurrent.TimeUnit; | ||
import lombok.AllArgsConstructor; | ||
import lombok.Cleanup; | ||
import org.testng.annotations.AfterClass; | ||
import org.testng.annotations.BeforeClass; | ||
import org.testng.annotations.Test; | ||
|
||
@Test(groups = "broker-api") | ||
public class CustomMessageIdTest extends ProducerConsumerBase { | ||
|
||
@BeforeClass | ||
@Override | ||
protected void setup() throws Exception { | ||
super.internalSetup(); | ||
super.producerBaseSetup(); | ||
|
||
} | ||
|
||
@AfterClass(alwaysRun = true) | ||
@Override | ||
protected void cleanup() throws Exception { | ||
super.internalCleanup(); | ||
} | ||
|
||
@Test(timeOut = 30000) | ||
public void testSeek() throws Exception { | ||
final String topic = "persistent://my-property/my-ns/test-seek-" + System.currentTimeMillis(); | ||
final var msgIdList = produceMessages(topic); | ||
@Cleanup final var consumer = pulsarClient.newConsumer(Schema.STRING) | ||
.topic(topic) | ||
.subscriptionName("sub") | ||
.subscribe(); | ||
final int ackIndex = msgIdList.size() / 2 + 1; | ||
consumer.seek(msgIdList.get(ackIndex)); | ||
final var msg = consumer.receive(3, TimeUnit.SECONDS); | ||
assertNotNull(msg); | ||
assertEquals(msg.getValue(), "msg-" + (ackIndex + 1)); | ||
} | ||
|
||
@Test(timeOut = 30000) | ||
public void testAck() throws Exception { | ||
final String topic = "persistent://my-property/my-ns/test-ack-" + System.currentTimeMillis(); | ||
@Cleanup final var consumer = pulsarClient.newConsumer(Schema.STRING) | ||
.topic(topic) | ||
.subscriptionName("sub") | ||
.isAckReceiptEnabled(true) | ||
.subscribe(); | ||
produceMessages(topic); | ||
final int ackIndex = 3; | ||
NonBatchedMessageId messageIdToAck = null; | ||
for (int i = 0; i < 10; i++) { | ||
var msg = consumer.receive(); | ||
var msgId = (PulsarApiMessageId) msg.getMessageId(); | ||
if (i == ackIndex) { | ||
messageIdToAck = new NonBatchedMessageId(msgId.getLedgerId(), msgId.getEntryId()); | ||
} | ||
} | ||
assertNotNull(messageIdToAck); | ||
consumer.acknowledgeCumulative(messageIdToAck); | ||
consumer.redeliverUnacknowledgedMessages(); | ||
var msg = consumer.receive(3, TimeUnit.SECONDS); | ||
assertNotNull(msg); | ||
assertEquals(msg.getValue(), "msg-" + (ackIndex + 1)); | ||
} | ||
|
||
private List<NonBatchedMessageId> produceMessages(String topic) throws PulsarClientException { | ||
@Cleanup final var producer = pulsarClient.newProducer(Schema.STRING) | ||
.topic(topic) | ||
.enableBatching(false) | ||
.create(); | ||
final var msgIdList = new ArrayList<NonBatchedMessageId>(); | ||
for (int i = 0; i < 10; i++) { | ||
final var msgId = (PulsarApiMessageId) producer.send("msg-" + i); | ||
msgIdList.add(new NonBatchedMessageId(msgId.getLedgerId(), msgId.getEntryId())); | ||
} | ||
return msgIdList; | ||
} | ||
|
||
@AllArgsConstructor | ||
private static class NonBatchedMessageId implements PulsarApiMessageId { | ||
// For non-batched message id in a single topic, only ledger id and entry id are required | ||
|
||
private final long ledgerId; | ||
private final long entryId; | ||
|
||
@Override | ||
public byte[] toByteArray() { | ||
return new byte[0]; // dummy implementation | ||
} | ||
|
||
@Override | ||
public long getLedgerId() { | ||
return ledgerId; | ||
} | ||
|
||
@Override | ||
public long getEntryId() { | ||
return entryId; | ||
} | ||
} | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -20,6 +20,7 @@ | |
|
||
import java.util.BitSet; | ||
|
||
@Deprecated | ||
public class BatchMessageAcker { | ||
|
||
private BatchMessageAcker() { | ||
|
Oops, something went wrong.