|
21 | 21 | import static org.apache.bookkeeper.mledger.proto.MLDataFormats.ManagedLedgerInfo.LedgerInfo;
|
22 | 22 |
|
23 | 23 | import java.time.Duration;
|
| 24 | +import java.util.ArrayList; |
24 | 25 | import java.util.List;
|
| 26 | +import java.util.concurrent.CompletableFuture; |
25 | 27 | import java.util.concurrent.TimeUnit;
|
26 | 28 | import lombok.Cleanup;
|
27 | 29 | import org.apache.bookkeeper.mledger.ManagedCursor;
|
|
36 | 38 | import org.apache.pulsar.client.impl.MessageIdImpl;
|
37 | 39 | import org.apache.pulsar.client.impl.MessageImpl;
|
38 | 40 | import org.apache.pulsar.common.api.proto.BrokerEntryMetadata;
|
| 41 | +import org.apache.pulsar.common.util.FutureUtil; |
39 | 42 | import org.assertj.core.util.Sets;
|
40 | 43 | import org.awaitility.Awaitility;
|
41 | 44 | import org.testng.Assert;
|
@@ -211,57 +214,75 @@ public void testBatchMessage() throws Exception {
|
211 | 214 | final String topic = newTopicName();
|
212 | 215 | final String subscription = "my-sub";
|
213 | 216 | final long eventTime= 200;
|
| 217 | + final int msgNum = 2; |
214 | 218 |
|
215 | 219 | @Cleanup
|
216 | 220 | Producer<byte[]> producer = pulsarClient.newProducer()
|
217 | 221 | .topic(topic)
|
| 222 | + // make sure 2 messages in one batch, because if only one message in batch, |
| 223 | + // producer will not send batched messages |
| 224 | + .batchingMaxPublishDelay(Long.MAX_VALUE, TimeUnit.MILLISECONDS) |
| 225 | + .batchingMaxMessages(msgNum) |
| 226 | + .batchingMaxBytes(Integer.MAX_VALUE) |
218 | 227 | .enableBatching(true)
|
219 | 228 | .create();
|
220 | 229 |
|
221 | 230 | long sendTime = System.currentTimeMillis();
|
222 |
| - // send message which is batch message and only contains one message, so do not set the deliverAtTime |
223 |
| - MessageIdImpl messageId = (MessageIdImpl) producer.newMessage() |
| 231 | + // send message which is batch message, so do not set the deliverAtTime |
| 232 | + List<CompletableFuture<MessageId>> messageIdsFuture = new ArrayList<>(msgNum); |
| 233 | + for (int i = 0; i < msgNum; ++i) { |
| 234 | + CompletableFuture<MessageId> messageId = producer.newMessage() |
224 | 235 | .eventTime(eventTime)
|
225 |
| - .value(("hello").getBytes()) |
226 |
| - .send(); |
| 236 | + .value(("hello" + i).getBytes()) |
| 237 | + .sendAsync(); |
| 238 | + messageIdsFuture.add(messageId); |
| 239 | + } |
| 240 | + FutureUtil.waitForAll(messageIdsFuture); |
227 | 241 |
|
228 | 242 | // 1. test for peekMessages
|
229 | 243 | admin.topics().createSubscription(topic, subscription, MessageId.earliest);
|
230 |
| - final List<Message<byte[]>> messages = admin.topics().peekMessages(topic, subscription, 1); |
231 |
| - Assert.assertEquals(messages.size(), 1); |
232 |
| - |
233 |
| - MessageImpl message = (MessageImpl) messages.get(0); |
234 |
| - Assert.assertEquals(message.getData(), ("hello").getBytes()); |
235 |
| - Assert.assertTrue(message.getPublishTime() >= sendTime); |
236 |
| - BrokerEntryMetadata entryMetadata = message.getBrokerEntryMetadata(); |
237 |
| - Assert.assertTrue(entryMetadata.getBrokerTimestamp() >= sendTime); |
238 |
| - Assert.assertEquals(entryMetadata.getIndex(), 0); |
239 |
| - System.out.println(message.getProperties()); |
240 |
| - Assert.assertEquals(Integer.parseInt(message.getProperty(BATCH_HEADER)), 1); |
241 |
| - // make sure BATCH_SIZE_HEADER > 0 |
242 |
| - Assert.assertTrue(Integer.parseInt(message.getProperty(BATCH_SIZE_HEADER)) > 0); |
| 244 | + final List<Message<byte[]>> messages = admin.topics().peekMessages(topic, subscription, msgNum); |
| 245 | + Assert.assertEquals(messages.size(), msgNum); |
| 246 | + |
| 247 | + MessageImpl message; |
| 248 | + BrokerEntryMetadata entryMetadata; |
| 249 | + for (int i = 0; i < msgNum; ++i) { |
| 250 | + message = (MessageImpl) messages.get(i); |
| 251 | + Assert.assertEquals(message.getData(), ("hello" + i).getBytes()); |
| 252 | + Assert.assertTrue(message.getPublishTime() >= sendTime); |
| 253 | + entryMetadata = message.getBrokerEntryMetadata(); |
| 254 | + Assert.assertTrue(entryMetadata.getBrokerTimestamp() >= sendTime); |
| 255 | + Assert.assertEquals(entryMetadata.getIndex(), msgNum - 1); |
| 256 | + System.out.println(message.getProperties()); |
| 257 | + Assert.assertEquals(Integer.parseInt(message.getProperty(BATCH_HEADER)), msgNum); |
| 258 | + // make sure BATCH_SIZE_HEADER > 0 |
| 259 | + Assert.assertTrue(Integer.parseInt(message.getProperty(BATCH_SIZE_HEADER)) > 0); |
| 260 | + } |
243 | 261 |
|
| 262 | + // getMessagesById and examineMessage only return the first messages in the batch |
244 | 263 | // 2. test for getMessagesById
|
| 264 | + MessageIdImpl messageId = (MessageIdImpl) messageIdsFuture.get(0).get(); |
245 | 265 | message = (MessageImpl) admin.topics().getMessageById(topic, messageId.getLedgerId(), messageId.getEntryId());
|
246 |
| - Assert.assertEquals(message.getData(), ("hello").getBytes()); |
| 266 | + // getMessagesById return the first message in the batch |
| 267 | + Assert.assertEquals(message.getData(), ("hello" + 0).getBytes()); |
247 | 268 | Assert.assertTrue(message.getPublishTime() >= sendTime);
|
248 | 269 | entryMetadata = message.getBrokerEntryMetadata();
|
249 | 270 | Assert.assertTrue(entryMetadata.getBrokerTimestamp() >= sendTime);
|
250 |
| - Assert.assertEquals(entryMetadata.getIndex(), 0); |
| 271 | + Assert.assertEquals(entryMetadata.getIndex(), msgNum - 1); |
251 | 272 | System.out.println(message.getProperties());
|
252 |
| - Assert.assertEquals(Integer.parseInt(message.getProperty(BATCH_HEADER)), 1); |
| 273 | + Assert.assertEquals(Integer.parseInt(message.getProperty(BATCH_HEADER)), msgNum); |
253 | 274 | // make sure BATCH_SIZE_HEADER > 0
|
254 | 275 | Assert.assertTrue(Integer.parseInt(message.getProperty(BATCH_SIZE_HEADER)) > 0);
|
255 | 276 |
|
256 | 277 | // 3. test for examineMessage
|
257 | 278 | message = (MessageImpl) admin.topics().examineMessage(topic, "earliest", 1);
|
258 |
| - Assert.assertEquals(message.getData(), ("hello").getBytes()); |
| 279 | + Assert.assertEquals(message.getData(), ("hello" + 0).getBytes()); |
259 | 280 | Assert.assertTrue(message.getPublishTime() >= sendTime);
|
260 | 281 | entryMetadata = message.getBrokerEntryMetadata();
|
261 | 282 | Assert.assertTrue(entryMetadata.getBrokerTimestamp() >= sendTime);
|
262 |
| - Assert.assertEquals(entryMetadata.getIndex(), 0); |
| 283 | + Assert.assertEquals(entryMetadata.getIndex(), msgNum - 1); |
263 | 284 | System.out.println(message.getProperties());
|
264 |
| - Assert.assertEquals(Integer.parseInt(message.getProperty(BATCH_HEADER)), 1); |
| 285 | + Assert.assertEquals(Integer.parseInt(message.getProperty(BATCH_HEADER)), msgNum); |
265 | 286 | // make sure BATCH_SIZE_HEADER > 0
|
266 | 287 | Assert.assertTrue(Integer.parseInt(message.getProperty(BATCH_SIZE_HEADER)) > 0);
|
267 | 288 | }
|
|
0 commit comments