diff --git a/examples/flink/src/main/java/org/apache/flink/streaming/connectors/pulsar/example/PulsarConsumerSourceWordCount.java b/examples/flink/src/main/java/org/apache/flink/streaming/connectors/pulsar/example/PulsarConsumerSourceWordCount.java index 547b635fb913c..942ddc1cdae4d 100644 --- a/examples/flink/src/main/java/org/apache/flink/streaming/connectors/pulsar/example/PulsarConsumerSourceWordCount.java +++ b/examples/flink/src/main/java/org/apache/flink/streaming/connectors/pulsar/example/PulsarConsumerSourceWordCount.java @@ -35,7 +35,6 @@ import org.apache.flink.streaming.api.windowing.time.Time; import org.apache.flink.streaming.connectors.pulsar.FlinkPulsarProducer; import org.apache.flink.streaming.connectors.pulsar.PulsarSourceBuilder; -import org.apache.pulsar.client.api.ProducerConfiguration; /** * Implements a streaming wordcount program on pulsar topics. @@ -99,7 +98,6 @@ public static void main(String[] args) throws Exception { serviceUrl, outputTopic, wordWithCount -> wordWithCount.toString().getBytes(UTF_8), - new ProducerConfiguration(), wordWithCount -> wordWithCount.word )).setParallelism(parallelism); } else { diff --git a/examples/flink/src/main/java/org/apache/flink/streaming/connectors/pulsar/example/PulsarConsumerSourceWordCountToAvroTableSink.java b/examples/flink/src/main/java/org/apache/flink/streaming/connectors/pulsar/example/PulsarConsumerSourceWordCountToAvroTableSink.java index 4563b5669679a..9fdc9a2ffb35b 100644 --- a/examples/flink/src/main/java/org/apache/flink/streaming/connectors/pulsar/example/PulsarConsumerSourceWordCountToAvroTableSink.java +++ b/examples/flink/src/main/java/org/apache/flink/streaming/connectors/pulsar/example/PulsarConsumerSourceWordCountToAvroTableSink.java @@ -35,7 +35,6 @@ import org.apache.flink.table.api.java.StreamTableEnvironment; import org.apache.flink.table.sinks.CsvTableSink; import org.apache.flink.table.sinks.TableSink; -import org.apache.pulsar.client.api.ProducerConfiguration; /** * Implements a streaming wordcount program on pulsar topics. @@ -108,7 +107,7 @@ public static void main(String[] args) throws Exception { table.printSchema(); TableSink sink = null; if (null != outputTopic) { - sink = new PulsarAvroTableSink(serviceUrl, outputTopic, new ProducerConfiguration(), ROUTING_KEY, WordWithCount.class); + sink = new PulsarAvroTableSink(serviceUrl, outputTopic, ROUTING_KEY, WordWithCount.class); } else { // print the results with a csv file sink = new CsvTableSink("./examples/file", "|"); diff --git a/examples/flink/src/main/java/org/apache/flink/streaming/connectors/pulsar/example/PulsarConsumerSourceWordCountToJsonTableSink.java b/examples/flink/src/main/java/org/apache/flink/streaming/connectors/pulsar/example/PulsarConsumerSourceWordCountToJsonTableSink.java index de09146e5ab48..1be9dde72a084 100644 --- a/examples/flink/src/main/java/org/apache/flink/streaming/connectors/pulsar/example/PulsarConsumerSourceWordCountToJsonTableSink.java +++ b/examples/flink/src/main/java/org/apache/flink/streaming/connectors/pulsar/example/PulsarConsumerSourceWordCountToJsonTableSink.java @@ -37,7 +37,6 @@ import org.apache.flink.table.api.java.StreamTableEnvironment; import org.apache.flink.table.sinks.CsvTableSink; import org.apache.flink.table.sinks.TableSink; -import org.apache.pulsar.client.api.ProducerConfiguration; /** * Implements a streaming wordcount program on pulsar topics. @@ -109,7 +108,7 @@ public static void main(String[] args) throws Exception { table.printSchema(); TableSink sink = null; if (null != outputTopic) { - sink = new PulsarJsonTableSink(serviceUrl, outputTopic, new ProducerConfiguration(), ROUTING_KEY); + sink = new PulsarJsonTableSink(serviceUrl, outputTopic, ROUTING_KEY); } else { // print the results with a csv file sink = new CsvTableSink("./examples/file", "|"); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/NamespacesTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/NamespacesTest.java index ffc1ab416cc7b..f32d0b5efd0bc 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/NamespacesTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/NamespacesTest.java @@ -1144,7 +1144,7 @@ public void testDeleteNamespace() throws Exception { final String topicName = "persistent://" + namespace + "/my-topic"; TopicName topic = TopicName.get(topicName); - Producer producer = pulsarClient.createProducer(topicName); + Producer producer = pulsarClient.newProducer().topic(topicName).create(); producer.close(); NamespaceBundle bundle1 = pulsar.getNamespaceService().getBundle(topic); // (2) Delete topic @@ -1171,7 +1171,7 @@ public void testSubscribeRate() throws Exception { admin.topics().createPartitionedTopic(topicName, 2); pulsar.getConfiguration().setAuthorizationEnabled(false); - Consumer consumer = pulsarClient.newConsumer() + Consumer consumer = pulsarClient.newConsumer() .topic(topicName) .subscriptionType(SubscriptionType.Shared) .subscriptionName("subscribe-rate") diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentTopicE2ETest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentTopicE2ETest.java index 71ae98ec8c7b5..2286862281bdc 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentTopicE2ETest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentTopicE2ETest.java @@ -51,19 +51,20 @@ import org.apache.pulsar.client.api.CompressionType; import org.apache.pulsar.client.api.Consumer; import org.apache.pulsar.client.api.Message; -import org.apache.pulsar.client.api.MessageBuilder; import org.apache.pulsar.client.api.MessageId; import org.apache.pulsar.client.api.MessageRoutingMode; import org.apache.pulsar.client.api.Producer; import org.apache.pulsar.client.api.ProducerBuilder; import org.apache.pulsar.client.api.PulsarClient; import org.apache.pulsar.client.api.PulsarClientException; -import org.apache.pulsar.client.api.Schema; import org.apache.pulsar.client.api.PulsarClientException.ProducerBusyException; +import org.apache.pulsar.client.api.Schema; import org.apache.pulsar.client.api.SubscriptionType; +import org.apache.pulsar.client.api.TypedMessageBuilder; import org.apache.pulsar.client.impl.ConsumerImpl; import org.apache.pulsar.client.impl.MessageIdImpl; import org.apache.pulsar.client.impl.ProducerImpl; +import org.apache.pulsar.client.impl.TypedMessageBuilderImpl; import org.apache.pulsar.common.api.proto.PulsarApi.CommandSubscribe.SubType; import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.policies.data.RetentionPolicies; @@ -1168,19 +1169,22 @@ public void testPayloadCorruptionDetection() throws Exception { .create(); Consumer consumer = pulsarClient.newConsumer().topic(topicName).subscriptionName("my-sub").subscribe(); - Message msg1 = MessageBuilder.create().setContent("message-1".getBytes()).build(); - CompletableFuture future1 = producer.sendAsync(msg1); + CompletableFuture future1 = producer.newMessage().value("message-1".getBytes()).sendAsync(); // Stop the broker, and publishes messages. Messages are accumulated in the producer queue and they're checksums // would have already been computed. If we change the message content at that point, it should result in a // checksum validation error stopBroker(); - Message msg2 = MessageBuilder.create().setContent("message-2".getBytes()).build(); - CompletableFuture future2 = producer.sendAsync(msg2); - // Taint msg2 - msg2.getData()[msg2.getData().length - 1] = '3'; // new content would be 'message-3' + byte[] a2 = "message-2".getBytes(); + TypedMessageBuilder msg2 = producer.newMessage().value(a2); + + + CompletableFuture future2 = msg2.sendAsync(); + + // corrupt the message, new content would be 'message-3' + ((TypedMessageBuilderImpl) msg2).getContent().put(a2.length - 1, (byte) '3'); // Restart the broker to have the messages published startBroker(); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorTest.java index 28c6b907211d9..a66179ba6a09f 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorTest.java @@ -51,7 +51,6 @@ import org.apache.pulsar.broker.service.persistent.PersistentReplicator; import org.apache.pulsar.broker.service.persistent.PersistentTopic; import org.apache.pulsar.client.admin.PulsarAdminException.PreconditionFailedException; -import org.apache.pulsar.client.api.MessageBuilder; import org.apache.pulsar.client.api.MessageRoutingMode; import org.apache.pulsar.client.api.Producer; import org.apache.pulsar.client.api.PulsarClient; @@ -418,11 +417,10 @@ public Void call() throws Exception { log.info("--- Starting Consumer --- " + url3); // Produce a message that isn't replicated - producer1.produce(1, MessageBuilder.create().disableReplication()); + producer1.produce(1, producer1.newMessage().disableReplication()); // Produce a message not replicated to r2 - producer1.produce(1, - MessageBuilder.create().setReplicationClusters(Lists.newArrayList("r1", "r3"))); + producer1.produce(1, producer1.newMessage().replicationClusters(Lists.newArrayList("r1", "r3"))); // Produce a default replicated message producer1.produce(1); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorTestBase.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorTestBase.java index 8dd10f4b941f8..98017e1e76d34 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorTestBase.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorTestBase.java @@ -20,6 +20,8 @@ import static org.testng.Assert.assertEquals; +import com.google.common.collect.Sets; + import java.net.URL; import java.util.ArrayList; import java.util.List; @@ -35,12 +37,12 @@ import org.apache.pulsar.client.admin.PulsarAdmin; import org.apache.pulsar.client.api.Consumer; import org.apache.pulsar.client.api.Message; -import org.apache.pulsar.client.api.MessageBuilder; import org.apache.pulsar.client.api.MessageId; import org.apache.pulsar.client.api.MessageRoutingMode; import org.apache.pulsar.client.api.Producer; import org.apache.pulsar.client.api.ProducerBuilder; import org.apache.pulsar.client.api.PulsarClient; +import org.apache.pulsar.client.api.TypedMessageBuilder; import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.policies.data.ClusterData; import org.apache.pulsar.common.policies.data.TenantInfo; @@ -50,9 +52,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import com.google.common.collect.Lists; -import com.google.common.collect.Sets; - public class ReplicatorTestBase { URL url1; URL urlTls1; @@ -309,12 +308,15 @@ void produce(int messages) throws Exception { } - void produce(int messages, MessageBuilder messageBuilder) throws Exception { + TypedMessageBuilder newMessage() { + return producer.newMessage(); + } + + void produce(int messages, TypedMessageBuilder messageBuilder) throws Exception { log.info("Start sending messages"); for (int i = 0; i < messages; i++) { final String m = new String("test-builder-" + i); - messageBuilder.setContent(m.getBytes()); - producer.send(messageBuilder.build()); + messageBuilder.value(m.getBytes()).send(); log.info("Sent message {}", m); } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/v1/V1_ReplicatorTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/v1/V1_ReplicatorTest.java index 92bb64175ad9a..92895816d0a82 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/v1/V1_ReplicatorTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/v1/V1_ReplicatorTest.java @@ -60,7 +60,6 @@ import org.apache.pulsar.broker.service.persistent.PersistentReplicator; import org.apache.pulsar.broker.service.persistent.PersistentTopic; import org.apache.pulsar.client.admin.PulsarAdminException.PreconditionFailedException; -import org.apache.pulsar.client.api.MessageBuilder; import org.apache.pulsar.client.api.Producer; import org.apache.pulsar.client.api.PulsarClient; import org.apache.pulsar.client.api.RawMessage; @@ -418,11 +417,11 @@ public Void call() throws Exception { log.info("--- Starting Consumer --- " + url3); // Produce a message that isn't replicated - producer1.produce(1, MessageBuilder.create().disableReplication()); + producer1.produce(1, producer1.newMessage().disableReplication()); // Produce a message not replicated to r2 producer1.produce(1, - MessageBuilder.create().setReplicationClusters(Lists.newArrayList("r1", "r3"))); + producer1.newMessage().replicationClusters(Lists.newArrayList("r1", "r3"))); // Produce a default replicated message producer1.produce(1); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/v1/V1_ReplicatorTestBase.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/v1/V1_ReplicatorTestBase.java index 00b69a875a776..4026632615188 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/v1/V1_ReplicatorTestBase.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/v1/V1_ReplicatorTestBase.java @@ -38,11 +38,11 @@ import org.apache.pulsar.client.admin.PulsarAdmin; import org.apache.pulsar.client.api.Consumer; import org.apache.pulsar.client.api.Message; -import org.apache.pulsar.client.api.MessageBuilder; import org.apache.pulsar.client.api.MessageId; import org.apache.pulsar.client.api.Producer; import org.apache.pulsar.client.api.ProducerBuilder; import org.apache.pulsar.client.api.PulsarClient; +import org.apache.pulsar.client.api.TypedMessageBuilder; import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.policies.data.ClusterData; import org.apache.pulsar.common.policies.data.TenantInfo; @@ -305,16 +305,20 @@ void produce(int messages) throws Exception { } - void produce(int messages, MessageBuilder messageBuilder) throws Exception { + void produce(int messages, TypedMessageBuilder messageBuilder) throws Exception { log.info("Start sending messages"); for (int i = 0; i < messages; i++) { final String m = new String("test-builder-" + i); - messageBuilder.setContent(m.getBytes()); - producer.send(messageBuilder.build()); + messageBuilder.value(m.getBytes()); + messageBuilder.send(); log.info("Sent message {}", m); } } + TypedMessageBuilder newMessage() { + return producer.newMessage(); + } + void close() throws Exception { client.close(); } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/PartitionedProducerConsumerTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/PartitionedProducerConsumerTest.java index 4263f5e774bef..e49a3d4ab45b6 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/PartitionedProducerConsumerTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/PartitionedProducerConsumerTest.java @@ -35,6 +35,7 @@ import java.util.concurrent.atomic.AtomicReference; import org.apache.pulsar.client.impl.PartitionedProducerImpl; +import org.apache.pulsar.client.impl.TypedMessageBuilderImpl; import org.apache.pulsar.common.naming.TopicName; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -334,9 +335,15 @@ public void testInvalidSequence() throws Exception { Consumer consumer = pulsarClient.newConsumer().topic(topicName.toString()) .subscriptionName("my-subscriber-name").subscribe(); + Producer producer = pulsarClient.newProducer().topic(topicName.toString()) + .enableBatching(false) + .messageRoutingMode(MessageRoutingMode.SinglePartition) + .create(); + try { - Message msg = MessageBuilder.create().setContent("InvalidMessage".getBytes()).build(); - consumer.acknowledge(msg); + TypedMessageBuilderImpl mb = (TypedMessageBuilderImpl) producer.newMessage() + .value("InvalidMessage".getBytes()); + consumer.acknowledge(mb.getMessage()); } catch (PulsarClientException.InvalidMessageException e) { // ok } @@ -357,10 +364,7 @@ public void testInvalidSequence() throws Exception { // ok } - Producer producer = pulsarClient.newProducer().topic(topicName.toString()) - .enableBatching(false) - .messageRoutingMode(MessageRoutingMode.SinglePartition) - .create(); + producer.close(); try { diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/SimpleProducerConsumerTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/SimpleProducerConsumerTest.java index ae70fcead68d4..8392c595d2315 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/SimpleProducerConsumerTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/SimpleProducerConsumerTest.java @@ -29,6 +29,13 @@ import static org.testng.Assert.assertTrue; import static org.testng.Assert.fail; +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; +import com.google.common.collect.Sets; + +import io.netty.buffer.ByteBuf; +import io.netty.buffer.Unpooled; + import java.io.IOException; import java.lang.reflect.Field; import java.lang.reflect.Modifier; @@ -56,15 +63,15 @@ import org.apache.bookkeeper.mledger.impl.EntryCacheImpl; import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl; import org.apache.pulsar.broker.service.persistent.PersistentTopic; -import org.apache.pulsar.client.api.PulsarClientException.InvalidConfigurationException; import org.apache.pulsar.client.impl.ConsumerImpl; -import org.apache.pulsar.common.api.EncryptionContext; -import org.apache.pulsar.common.api.EncryptionContext.EncryptionKey; import org.apache.pulsar.client.impl.MessageCrypto; import org.apache.pulsar.client.impl.MessageIdImpl; import org.apache.pulsar.client.impl.MessageImpl; import org.apache.pulsar.client.impl.TopicMessageImpl; +import org.apache.pulsar.client.impl.TypedMessageBuilderImpl; import org.apache.pulsar.common.api.Commands; +import org.apache.pulsar.common.api.EncryptionContext; +import org.apache.pulsar.common.api.EncryptionContext.EncryptionKey; import org.apache.pulsar.common.api.PulsarDecoder; import org.apache.pulsar.common.api.proto.PulsarApi; import org.apache.pulsar.common.api.proto.PulsarApi.EncryptionKeys; @@ -83,13 +90,6 @@ import org.testng.annotations.DataProvider; import org.testng.annotations.Test; -import com.google.common.collect.Lists; -import com.google.common.collect.Maps; -import com.google.common.collect.Sets; - -import io.netty.buffer.ByteBuf; -import io.netty.buffer.Unpooled; - public class SimpleProducerConsumerTest extends ProducerConsumerBase { private static final Logger log = LoggerFactory.getLogger(SimpleProducerConsumerTest.class); @@ -395,11 +395,15 @@ public void testInvalidSequence() throws Exception { Assert.assertTrue(e instanceof PulsarClientException.AlreadyClosedException); } + Producer producer = pulsarClient.newProducer().topic("persistent://my-property/my-ns/my-topic6") + .create(); + Consumer consumer = pulsarClient.newConsumer().topic("persistent://my-property/my-ns/my-topic6") .subscriptionName("my-subscriber-name").subscribe(); try { - Message msg = MessageBuilder.create().setContent("InvalidMessage".getBytes()).build(); + TypedMessageBuilder builder = producer.newMessage().value("InvalidMessage".getBytes()); + Message msg = ((TypedMessageBuilderImpl) builder).getMessage(); consumer.acknowledge(msg); } catch (PulsarClientException.InvalidMessageException e) { // ok @@ -421,8 +425,6 @@ public void testInvalidSequence() throws Exception { // ok } - Producer producer = pulsarClient.newProducer().topic("persistent://my-property/my-ns/my-topic6") - .create(); producer.close(); try { @@ -622,12 +624,14 @@ public Void call() throws Exception { public void testSendBigMessageSize() throws Exception { log.info("-- Starting {} test --", methodName); + final String topic = "persistent://my-property/my-ns/bigMsg"; + Producer producer = pulsarClient.newProducer().topic(topic).create(); + + // Messages are allowed up to MaxMessageSize - MessageBuilder.create().setContent(new byte[PulsarDecoder.MaxMessageSize]).build(); + producer.newMessage().value(new byte[PulsarDecoder.MaxMessageSize]); try { - final String topic = "persistent://my-property/my-ns/bigMsg"; - Producer producer = pulsarClient.newProducer().topic(topic).create(); producer.send(new byte[PulsarDecoder.MaxMessageSize + 1]); fail("Should have thrown exception"); } catch (PulsarClientException.InvalidMessageException e) { @@ -661,9 +665,7 @@ public void testSendBigMessageSizeButCompressed() throws Exception { .messageRoutingMode(MessageRoutingMode.SinglePartition) .compressionType(CompressionType.LZ4) .create(); - Message message = MessageBuilder.create().setContent(new byte[PulsarDecoder.MaxMessageSize + 1]) - .build(); - producer.send(message); + producer.send(new byte[PulsarDecoder.MaxMessageSize + 1]); producer.close(); // (b) batch-msg @@ -672,9 +674,8 @@ public void testSendBigMessageSizeButCompressed() throws Exception { .messageRoutingMode(MessageRoutingMode.SinglePartition) .compressionType(CompressionType.LZ4) .create(); - message = MessageBuilder.create().setContent(new byte[PulsarDecoder.MaxMessageSize + 1]).build(); try { - producer.send(message); + producer.send(new byte[PulsarDecoder.MaxMessageSize + 1]); fail("Should have thrown exception"); } catch (PulsarClientException.InvalidMessageException e) { // OK @@ -2306,7 +2307,7 @@ public EncryptionKeyInfo getPrivateKey(String keyName, Map keyMe consumer.close(); log.info("-- Exiting {} test --", methodName); } - + @Test(groups = "encryption") public void testRedeliveryOfFailedMessages() throws Exception { log.info("-- Starting {} test --", methodName); @@ -2352,7 +2353,7 @@ public EncryptionKeyInfo getPrivateKey(String keyName, Map keyMe return null; } } - + class InvalidKeyReader implements CryptoKeyReader { EncryptionKeyInfo keyInfo = new EncryptionKeyInfo(); @@ -2366,24 +2367,24 @@ public EncryptionKeyInfo getPrivateKey(String keyName, Map metad return null; } } - + /* * Redelivery functionality guarantees that customer will get a chance to process the message again. * In case of shared subscription eventually every client will get a chance to process the message, till one of them acks it. - * - * For client with Encryption enabled where in cases like a new production rollout or a buggy client configuration, we might have a mismatch of consumers + * + * For client with Encryption enabled where in cases like a new production rollout or a buggy client configuration, we might have a mismatch of consumers * - few which can decrypt, few which can't (due to errors or cryptoReader not configured). - * + * * In that case eventually all messages should be acked as long as there is a single consumer who can decrypt the message. - * + * * Consumer 1 - Can decrypt message * Consumer 2 - Has invalid Reader configured. * Consumer 3 - Has no reader configured. - * + * */ String topicName = "persistent://my-property/my-ns/myrsa-topic1"; - + Producer producer = pulsarClient.newProducer().topic(topicName) .addEncryptionKey(encryptionKeyName).compressionType(CompressionType.LZ4) .cryptoKeyReader(new EncKeyReader()).create(); @@ -2391,47 +2392,47 @@ public EncryptionKeyInfo getPrivateKey(String keyName, Map metad Consumer consumer1 = pulsarClient.newConsumer().topicsPattern(topicName) .subscriptionName("my-subscriber-name").cryptoKeyReader(new EncKeyReader()) .subscriptionType(SubscriptionType.Shared).ackTimeout(1, TimeUnit.SECONDS).subscribe(); - + Consumer consumer2 = pulsarClient.newConsumer().topicsPattern(topicName) .subscriptionName("my-subscriber-name").cryptoKeyReader(new InvalidKeyReader()) .subscriptionType(SubscriptionType.Shared).ackTimeout(1, TimeUnit.SECONDS).subscribe(); Consumer consumer3 = pulsarClient.newConsumer().topicsPattern(topicName) .subscriptionName("my-subscriber-name").subscriptionType(SubscriptionType.Shared).ackTimeout(1, TimeUnit.SECONDS).subscribe(); - + int numberOfMessages = 100; String message = "my-message"; Set messages = new HashSet(); // Since messages are in random order for (int i = 0; i consumer = pulsarClient.newConsumer(Schema.STRING) + .topic("persistent://my-property/use/my-ns/my-topic1") + .subscriptionName("my-subscriber-name") + .subscriptionType(SubscriptionType.Exclusive) + .subscribe(); - ProducerConfiguration producerConf = new ProducerConfiguration(); + ProducerBuilder producerBuilder = pulsarClient.newProducer(Schema.STRING) + .topic("persistent://my-property/use/my-ns/my-topic1"); if (batchMessageDelayMs != 0) { - producerConf.setBatchingEnabled(true); - producerConf.setBatchingMaxPublishDelay(batchMessageDelayMs, TimeUnit.MILLISECONDS); - producerConf.setBatchingMaxMessages(5); + producerBuilder.enableBatching(true) + .batchingMaxPublishDelay(batchMessageDelayMs, TimeUnit.MILLISECONDS) + .batchingMaxMessages(5); } else { - producerConf.setBatchingEnabled(false); + producerBuilder.enableBatching(false); } - Producer producer = pulsarClient.createProducer("persistent://my-property/use/my-ns/my-topic1", producerConf); + Producer producer = producerBuilder.create(); for (int i = 0; i < 10; i++) { - String message = "my-message-" + i; - producer.send(message.getBytes()); + producer.send("my-message-" + i); } - Message msg = null; + Message msg = null; Set messageSet = Sets.newHashSet(); for (int i = 0; i < 10; i++) { msg = consumer.receive(5, TimeUnit.SECONDS); - String receivedMessage = new String(msg.getData()); + String receivedMessage = msg.getValue(); log.debug("Received message: [{}]", receivedMessage); String expectedMessage = "my-message-" + i; testMessageOrderAndDuplicates(messageSet, receivedMessage, expectedMessage); @@ -150,19 +152,19 @@ public void testSyncProducerAndConsumer(int batchMessageDelayMs) throws Exceptio @Test(dataProvider = "batch") public void testAsyncProducerAndAsyncAck(int batchMessageDelayMs) throws Exception { log.info("-- Starting {} test --", methodName); - ConsumerConfiguration conf = new ConsumerConfiguration(); - conf.setSubscriptionType(SubscriptionType.Exclusive); - Consumer consumer = pulsarClient.subscribe("persistent://my-property/use/my-ns/my-topic2", "my-subscriber-name", - conf); - ProducerConfiguration producerConf = new ProducerConfiguration(); - if (batchMessageDelayMs != 0) { - producerConf.setBatchingMaxPublishDelay(batchMessageDelayMs, TimeUnit.MILLISECONDS); - producerConf.setBatchingMaxMessages(5); - producerConf.setBatchingEnabled(true); - } else { - producerConf.setBatchingEnabled(false); - } - Producer producer = pulsarClient.createProducer("persistent://my-property/use/my-ns/my-topic2", producerConf); + Consumer consumer = pulsarClient.newConsumer() + .topic("persistent://my-property/use/my-ns/my-topic2") + .subscriptionName("my-subscriber-name") + .subscriptionType(SubscriptionType.Exclusive) + .subscribe(); + + Producer producer = pulsarClient.newProducer() + .topic("persistent://my-property/use/my-ns/my-topic2") + .batchingMaxMessages(5) + .batchingMaxPublishDelay(batchMessageDelayMs, TimeUnit.MILLISECONDS) + .enableBatching(batchMessageDelayMs != 0) + .create(); + List> futures = Lists.newArrayList(); // Asynchronously produce messages @@ -177,7 +179,7 @@ public void testAsyncProducerAndAsyncAck(int batchMessageDelayMs) throws Excepti future.get(); } - Message msg = null; + Message msg = null; Set messageSet = Sets.newHashSet(); for (int i = 0; i < 10; i++) { msg = consumer.receive(5, TimeUnit.SECONDS); @@ -198,30 +200,29 @@ public void testAsyncProducerAndAsyncAck(int batchMessageDelayMs) throws Excepti @Test(dataProvider = "batch", timeOut = 100000) public void testMessageListener(int batchMessageDelayMs) throws Exception { log.info("-- Starting {} test --", methodName); - ConsumerConfiguration conf = new ConsumerConfiguration(); - conf.setSubscriptionType(SubscriptionType.Exclusive); int numMessages = 100; final CountDownLatch latch = new CountDownLatch(numMessages); - conf.setMessageListener((consumer, msg) -> { - Assert.assertNotNull(msg, "Message cannot be null"); - String receivedMessage = new String(msg.getData()); - log.debug("Received message [{}] in the listener", receivedMessage); - consumer.acknowledgeAsync(msg); - latch.countDown(); - }); - Consumer consumer = pulsarClient.subscribe("persistent://my-property/use/my-ns/my-topic3", "my-subscriber-name", - conf); - ProducerConfiguration producerConf = new ProducerConfiguration(); - if (batchMessageDelayMs != 0) { - producerConf.setBatchingMaxPublishDelay(batchMessageDelayMs, TimeUnit.MILLISECONDS); - producerConf.setBatchingMaxMessages(5); - producerConf.setBatchingEnabled(true); - } else { - producerConf.setBatchingEnabled(false); - } - Producer producer = pulsarClient.createProducer("persistent://my-property/use/my-ns/my-topic3", producerConf); + Consumer consumer = pulsarClient.newConsumer() + .topic("persistent://my-property/use/my-ns/my-topic3") + .subscriptionName("my-subscriber-name") + .subscriptionType(SubscriptionType.Exclusive) + .messageListener((c, msg) -> { + Assert.assertNotNull(msg, "Message cannot be null"); + String receivedMessage = new String(msg.getData()); + log.debug("Received message [{}] in the listener", receivedMessage); + c.acknowledgeAsync(msg); + latch.countDown(); + }) + .subscribe(); + Producer producer = pulsarClient.newProducer() + .topic("persistent://my-property/use/my-ns/my-topic3") + .batchingMaxMessages(5) + .batchingMaxPublishDelay(batchMessageDelayMs, TimeUnit.MILLISECONDS) + .enableBatching(batchMessageDelayMs != 0) + .create(); + List> futures = Lists.newArrayList(); // Asynchronously produce messages @@ -246,19 +247,17 @@ public void testMessageListener(int batchMessageDelayMs) throws Exception { public void testBackoffAndReconnect(int batchMessageDelayMs) throws Exception { log.info("-- Starting {} test --", methodName); // Create consumer and producer - ConsumerConfiguration conf = new ConsumerConfiguration(); - conf.setSubscriptionType(SubscriptionType.Exclusive); - Consumer consumer = pulsarClient.subscribe("persistent://my-property/use/my-ns/my-topic4", "my-subscriber-name", - conf); - ProducerConfiguration producerConf = new ProducerConfiguration(); - if (batchMessageDelayMs != 0) { - producerConf.setBatchingMaxPublishDelay(batchMessageDelayMs, TimeUnit.MILLISECONDS); - producerConf.setBatchingMaxMessages(5); - producerConf.setBatchingEnabled(true); - } else { - producerConf.setBatchingEnabled(false); - } - Producer producer = pulsarClient.createProducer("persistent://my-property/use/my-ns/my-topic4", producerConf); + Consumer consumer = pulsarClient.newConsumer() + .topic("persistent://my-property/use/my-ns/my-topic4") + .subscriptionName("my-subscriber-name") + .subscriptionType(SubscriptionType.Exclusive) + .subscribe(); + Producer producer = pulsarClient.newProducer() + .topic("persistent://my-property/use/my-ns/my-topic4") + .batchingMaxMessages(5) + .batchingMaxPublishDelay(batchMessageDelayMs, TimeUnit.MILLISECONDS) + .enableBatching(batchMessageDelayMs != 0) + .create(); // Produce messages CompletableFuture lastFuture = null; @@ -271,7 +270,7 @@ public void testBackoffAndReconnect(int batchMessageDelayMs) throws Exception { lastFuture.get(); - Message msg = null; + Message msg = null; for (int i = 0; i < 10; i++) { msg = consumer.receive(5, TimeUnit.SECONDS); log.info("Received: [{}]", new String(msg.getData())); @@ -298,21 +297,20 @@ public void testBackoffAndReconnect(int batchMessageDelayMs) throws Exception { public void testSendTimeout(int batchMessageDelayMs) throws Exception { log.info("-- Starting {} test --", methodName); - ConsumerConfiguration consumerConf = new ConsumerConfiguration(); - consumerConf.setSubscriptionType(SubscriptionType.Exclusive); - Consumer consumer = pulsarClient.subscribe("persistent://my-property/use/my-ns/my-topic5", "my-subscriber-name", - consumerConf); - ProducerConfiguration producerConf = new ProducerConfiguration(); - if (batchMessageDelayMs != 0) { - producerConf.setBatchingMaxPublishDelay(2 * batchMessageDelayMs, TimeUnit.MILLISECONDS); - producerConf.setBatchingMaxMessages(5); - producerConf.setBatchingEnabled(true); - } else { - producerConf.setBatchingEnabled(false); - } - producerConf.setSendTimeout(1, TimeUnit.SECONDS); + Consumer consumer = pulsarClient.newConsumer() + .topic("persistent://my-property/use/my-ns/my-topic5") + .subscriptionName("my-subscriber-name") + .subscriptionType(SubscriptionType.Exclusive) + .subscribe(); + + Producer producer = pulsarClient.newProducer() + .topic("persistent://my-property/use/my-ns/my-topic5") + .batchingMaxMessages(5) + .batchingMaxPublishDelay(2 * batchMessageDelayMs, TimeUnit.MILLISECONDS) + .enableBatching(batchMessageDelayMs != 0) + .sendTimeout(1, TimeUnit.SECONDS) + .create(); - Producer producer = pulsarClient.createProducer("persistent://my-property/use/my-ns/my-topic5", producerConf); final String message = "my-message"; // Trigger the send timeout @@ -330,7 +328,7 @@ public void testSendTimeout(int batchMessageDelayMs) throws Exception { startBroker(); // We should not have received any message - Message msg = consumer.receive(3, TimeUnit.SECONDS); + Message msg = consumer.receive(3, TimeUnit.SECONDS); Assert.assertNull(msg); consumer.close(); log.info("-- Exiting {} test --", methodName); @@ -340,31 +338,42 @@ public void testSendTimeout(int batchMessageDelayMs) throws Exception { public void testInvalidSequence() throws Exception { log.info("-- Starting {} test --", methodName); - PulsarClient client1 = PulsarClient.create("http://127.0.0.1:" + BROKER_WEBSERVICE_PORT); + PulsarClient client1 = PulsarClient.builder().serviceUrl("http://127.0.0.1:" + BROKER_WEBSERVICE_PORT).build(); client1.close(); - ConsumerConfiguration consumerConf = new ConsumerConfiguration(); - consumerConf.setSubscriptionType(SubscriptionType.Exclusive); - try { - client1.subscribe("persistent://my-property/use/my-ns/my-topic6", "my-subscriber-name", consumerConf); + client1.newConsumer() + .topic("persistent://my-property/use/my-ns/my-topic6") + .subscriptionName("my-subscriber-name") + .subscriptionType(SubscriptionType.Exclusive) + .subscribe(); Assert.fail("Should fail"); } catch (PulsarClientException e) { Assert.assertTrue(e instanceof PulsarClientException.AlreadyClosedException); } try { - client1.createProducer("persistent://my-property/use/my-ns/my-topic6"); + client1.newProducer().topic("persistent://my-property/use/my-ns/my-topic6").create(); Assert.fail("Should fail"); } catch (PulsarClientException e) { Assert.assertTrue(e instanceof PulsarClientException.AlreadyClosedException); } - Consumer consumer = pulsarClient.subscribe("persistent://my-property/use/my-ns/my-topic6", "my-subscriber-name", - consumerConf); + + + Consumer consumer = pulsarClient.newConsumer() + .topic("persistent://my-property/use/my-ns/my-topic6") + .subscriptionName("my-subscriber-name") + .subscriptionType(SubscriptionType.Exclusive) + .subscribe(); + + Producer producer = pulsarClient.newProducer() + .topic("persistent://my-property/use/my-ns/my-topic6") + .create(); try { - Message msg = MessageBuilder.create().setContent("InvalidMessage".getBytes()).build(); + TypedMessageBuilder builder = producer.newMessage().value("InvalidMessage".getBytes()); + Message msg = ((TypedMessageBuilderImpl) builder).getMessage(); consumer.acknowledge(msg); } catch (PulsarClientException.InvalidMessageException e) { // ok @@ -386,7 +395,7 @@ public void testInvalidSequence() throws Exception { // ok } - Producer producer = pulsarClient.createProducer("persistent://my-property/use/my-ns/my-topic6"); + producer.close(); try { @@ -399,94 +408,84 @@ public void testInvalidSequence() throws Exception { } @Test - public void testSillyUser() { + public void testSillyUser() throws Exception { try { - PulsarClient.create("invalid://url"); + PulsarClient.builder().serviceUrl("invalid://url").build(); Assert.fail("should fail"); } catch (PulsarClientException e) { Assert.assertTrue(e instanceof PulsarClientException.InvalidServiceURL); } - ProducerConfiguration producerConf = new ProducerConfiguration(); - try { - producerConf.setSendTimeout(-1, TimeUnit.SECONDS); + pulsarClient.newProducer().sendTimeout(-1, TimeUnit.SECONDS); Assert.fail("should fail"); } catch (IllegalArgumentException e) { // ok } try { - producerConf.setMaxPendingMessages(0); + pulsarClient.newProducer().maxPendingMessages(0); Assert.fail("should fail"); } catch (IllegalArgumentException e) { // ok } try { - pulsarClient.createProducer("persistent://my-property/use/my-ns/my-topic7", (ProducerConfiguration) null); - Assert.fail("should fail"); - } catch (PulsarClientException e) { - Assert.assertTrue(e instanceof PulsarClientException.InvalidConfigurationException); - } - - try { - pulsarClient.createProducer("invalid://topic", producerConf); + pulsarClient.newProducer().topic("invalid://topic").create(); Assert.fail("should fail"); } catch (PulsarClientException e) { Assert.assertTrue(e instanceof PulsarClientException.InvalidTopicNameException); } - ConsumerConfiguration consumerConf = new ConsumerConfiguration(); - try { - consumerConf.setMessageListener(null); + pulsarClient.newConsumer().messageListener(null); Assert.fail("should fail"); } catch (NullPointerException e) { // ok } try { - consumerConf.setSubscriptionType(null); + pulsarClient.newConsumer().subscriptionType(null); Assert.fail("should fail"); } catch (NullPointerException e) { // ok } try { - consumerConf.setReceiverQueueSize(-1); + pulsarClient.newConsumer().receiverQueueSize(-1); Assert.fail("should fail"); } catch (IllegalArgumentException e) { // ok } try { - pulsarClient.subscribe("persistent://my-property/use/my-ns/my-topic7", "my-subscriber-name", - (ConsumerConfiguration) null); + pulsarClient.newConsumer() + .topic("persistent://my-property/use/my-ns/my-topic7") + .subscriptionName(null) + .subscribe(); Assert.fail("Should fail"); - } catch (PulsarClientException e) { - Assert.assertTrue(e instanceof PulsarClientException.InvalidConfigurationException); - } - - try { - pulsarClient.subscribe("persistent://my-property/use/my-ns/my-topic7", null, consumerConf); - Assert.fail("Should fail"); - } catch (PulsarClientException e) { - Assert.assertTrue(e instanceof PulsarClientException.InvalidConfigurationException); + } catch (IllegalArgumentException e) { + // expected } try { - pulsarClient.subscribe("persistent://my-property/use/my-ns/my-topic7", "", consumerConf); + pulsarClient.newConsumer() + .topic("persistent://my-property/use/my-ns/my-topic7") + .subscriptionName("") + .subscribe(); Assert.fail("Should fail"); - } catch (PulsarClientException e) { - Assert.assertTrue(e instanceof PulsarClientException.InvalidConfigurationException); + } catch (IllegalArgumentException e) { + // Expected } try { - pulsarClient.subscribe("invalid://topic7", "my-subscriber-name", consumerConf); + pulsarClient.newConsumer() + .topic("invalid://topic7") + .subscriptionName(null) + .subscribe(); Assert.fail("Should fail"); - } catch (PulsarClientException e) { - Assert.assertTrue(e instanceof PulsarClientException.InvalidTopicNameException); + } catch (IllegalArgumentException e) { + // Expected } } @@ -498,10 +497,12 @@ public void testConcurrentConsumerReceiveWhileReconnect(int batchMessageDelayMs) final int recvQueueSize = 100; final int numConsumersThreads = 10; - final ConsumerConfiguration conf = new ConsumerConfiguration(); - conf.setReceiverQueueSize(recvQueueSize); String subName = UUID.randomUUID().toString(); - final Consumer consumer = pulsarClient.subscribe("persistent://my-property/use/my-ns/my-topic7", subName, conf); + final Consumer consumer = pulsarClient.newConsumer() + .topic("persistent://my-property/use/my-ns/my-topic7") + .subscriptionName(subName) + .receiverQueueSize(recvQueueSize) + .subscribe(); ExecutorService executor = Executors.newCachedThreadPool(); final CyclicBarrier barrier = new CyclicBarrier(numConsumersThreads + 1); @@ -525,22 +526,19 @@ public Void call() throws Exception { Thread.sleep(2000); // publish 100 messages so that the consumers blocked on receive() will now get the messages - ProducerConfiguration producerConf = new ProducerConfiguration(); - if (batchMessageDelayMs != 0) { - producerConf.setBatchingMaxPublishDelay(batchMessageDelayMs, TimeUnit.MILLISECONDS); - producerConf.setBatchingMaxMessages(5); - producerConf.setBatchingEnabled(true); - } else { - producerConf.setBatchingEnabled(false); - } - Producer producer = pulsarClient.createProducer("persistent://my-property/use/my-ns/my-topic7", producerConf); + Producer producer = pulsarClient.newProducer() + .topic("persistent://my-property/use/my-ns/my-topic7") + .batchingMaxPublishDelay(batchMessageDelayMs, TimeUnit.MILLISECONDS) + .batchingMaxMessages(5) + .enableBatching(batchMessageDelayMs != 0) + .create(); for (int i = 0; i < recvQueueSize; i++) { String message = "my-message-" + i; producer.send(message.getBytes()); } Thread.sleep(500); - ConsumerImpl consumerImpl = (ConsumerImpl) consumer; + ConsumerImpl consumerImpl = (ConsumerImpl) consumer; // The available permits should be 10 and num messages in the queue should be 90 Assert.assertEquals(consumerImpl.getAvailablePermits(), numConsumersThreads); Assert.assertEquals(consumerImpl.numMessagesInQueue(), recvQueueSize - numConsumersThreads); @@ -565,7 +563,7 @@ public Void call() throws Exception { // clear the queue while (true) { - Message msg = consumer.receive(1, TimeUnit.SECONDS); + Message msg = consumer.receive(1, TimeUnit.SECONDS); if (msg == null) { break; } @@ -597,20 +595,22 @@ public Void call() throws Exception { Assert.assertEquals(consumerImpl.getAvailablePermits(), numConsumersThreads); Assert.assertEquals(consumerImpl.numMessagesInQueue(), recvQueueSize - numConsumersThreads); consumer.close(); + + executor.shutdownNow(); } @Test public void testSendBigMessageSize() throws Exception { log.info("-- Starting {} test --", methodName); + final String topic = "persistent://my-property/use/my-ns/bigMsg"; + Producer producer = pulsarClient.newProducer().topic(topic).create(); + // Messages are allowed up to MaxMessageSize - MessageBuilder.create().setContent(new byte[PulsarDecoder.MaxMessageSize]).build(); + producer.newMessage().value(new byte[PulsarDecoder.MaxMessageSize]); try { - final String topic = "persistent://my-property/use/my-ns/bigMsg"; - Producer producer = pulsarClient.createProducer(topic); - Message message = MessageBuilder.create().setContent(new byte[PulsarDecoder.MaxMessageSize + 1]).build(); - producer.send(message); + producer.send(new byte[PulsarDecoder.MaxMessageSize + 1]); fail("Should have thrown exception"); } catch (PulsarClientException.InvalidMessageException e) { // OK @@ -638,22 +638,22 @@ public void testSendBigMessageSizeButCompressed() throws Exception { final String topic = "persistent://my-property/use/my-ns/bigMsg"; // (a) non-batch msg with compression - ProducerConfiguration producerConf = new ProducerConfiguration(); - producerConf.setBatchingEnabled(false); - producerConf.setCompressionType(CompressionType.LZ4); - Producer producer = pulsarClient.createProducer(topic, producerConf); - Message message = MessageBuilder.create().setContent(new byte[PulsarDecoder.MaxMessageSize + 1]).build(); - producer.send(message); + Producer producer = pulsarClient.newProducer() + .topic(topic) + .enableBatching(false) + .compressionType(CompressionType.LZ4) + .create(); + producer.send(new byte[PulsarDecoder.MaxMessageSize + 1]); producer.close(); // (b) batch-msg - producerConf = new ProducerConfiguration(); - producerConf.setBatchingEnabled(true); - producerConf.setCompressionType(CompressionType.LZ4); - producer = pulsarClient.createProducer(topic, producerConf); - message = MessageBuilder.create().setContent(new byte[PulsarDecoder.MaxMessageSize + 1]).build(); + producer = pulsarClient.newProducer() + .topic(topic) + .enableBatching(true) + .compressionType(CompressionType.LZ4) + .create(); try { - producer.send(message); + producer.send(new byte[PulsarDecoder.MaxMessageSize + 1]); fail("Should have thrown exception"); } catch (PulsarClientException.InvalidMessageException e) { // OK @@ -661,13 +661,13 @@ public void testSendBigMessageSizeButCompressed() throws Exception { producer.close(); // (c) non-batch msg without compression - producerConf = new ProducerConfiguration(); - producerConf.setBatchingEnabled(false); - producerConf.setCompressionType(CompressionType.NONE); - producer = pulsarClient.createProducer(topic, producerConf); - message = MessageBuilder.create().setContent(new byte[PulsarDecoder.MaxMessageSize + 1]).build(); + producer = pulsarClient.newProducer() + .topic(topic) + .enableBatching(false) + .compressionType(CompressionType.NONE) + .create(); try { - producer.send(message); + producer.send(new byte[PulsarDecoder.MaxMessageSize + 1]); fail("Should have thrown exception"); } catch (PulsarClientException.InvalidMessageException e) { // OK @@ -675,15 +675,18 @@ public void testSendBigMessageSizeButCompressed() throws Exception { producer.close(); // (d) non-batch msg with compression and try to consume message - producerConf = new ProducerConfiguration(); - producerConf.setBatchingEnabled(false); - producerConf.setCompressionType(CompressionType.LZ4); - producer = pulsarClient.createProducer(topic, producerConf); - Consumer consumer = pulsarClient.subscribe(topic, "sub1"); + producer = pulsarClient.newProducer() + .topic(topic) + .enableBatching(false) + .compressionType(CompressionType.LZ4) + .create(); + Consumer consumer = pulsarClient.newConsumer() + .topic(topic) + .subscriptionName("sub1") + .subscribe(); byte[] content = new byte[PulsarDecoder.MaxMessageSize + 10]; - message = MessageBuilder.create().setContent(content).build(); - producer.send(message); - assertEquals(consumer.receive().getData(), content); + producer.send(content); + assertEquals(consumer.receive().getValue(), content); producer.close(); consumer.close(); @@ -709,25 +712,22 @@ public void testActiveAndInActiveConsumerEntryCacheBehavior() throws Exception { final String topicName = "cache-topic"; final String sub1 = "faster-sub1"; final String sub2 = "slower-sub2"; - ConsumerConfiguration conf = new ConsumerConfiguration(); - conf.setSubscriptionType(SubscriptionType.Shared); - conf.setReceiverQueueSize(receiverSize); - - ProducerConfiguration producerConf = new ProducerConfiguration(); - - if (batchMessageDelayMs != 0) { - producerConf.setBatchingEnabled(true); - producerConf.setBatchingMaxPublishDelay(batchMessageDelayMs, TimeUnit.MILLISECONDS); - producerConf.setBatchingMaxMessages(5); - } else { - producerConf.setBatchingEnabled(false); - } /************ usecase-1: *************/ // 1. Subscriber Faster subscriber - Consumer subscriber1 = pulsarClient.subscribe("persistent://my-property/use/my-ns/" + topicName, sub1, conf); + Consumer subscriber1 = pulsarClient.newConsumer() + .topic("persistent://my-property/use/my-ns/" + topicName) + .subscriptionName(sub1) + .subscriptionType(SubscriptionType.Shared) + .receiverQueueSize(receiverSize) + .subscribe(); final String topic = "persistent://my-property/use/my-ns/" + topicName; - Producer producer = pulsarClient.createProducer(topic, producerConf); + Producer producer = pulsarClient.newProducer() + .topic(topic) + .enableBatching(batchMessageDelayMs != 0) + .batchingMaxPublishDelay(batchMessageDelayMs, TimeUnit.MILLISECONDS) + .batchingMaxMessages(5) + .create(); PersistentTopic topicRef = (PersistentTopic) pulsar.getBrokerService().getTopicReference(topic).get(); ManagedLedgerImpl ledger = (ManagedLedgerImpl) topicRef.getManagedLedger(); @@ -739,7 +739,7 @@ public void testActiveAndInActiveConsumerEntryCacheBehavior() throws Exception { EntryCacheImpl entryCache = spy((EntryCacheImpl) cacheField.get(ledger)); cacheField.set(ledger, entryCache); - Message msg = null; + Messagemsg = null; // 2. Produce messages for (int i = 0; i < 30; i++) { String message = "my-message-" + i; @@ -767,7 +767,11 @@ public void testActiveAndInActiveConsumerEntryCacheBehavior() throws Exception { /************ usecase-2: *************/ // 1.b Subscriber slower-subscriber - Consumer subscriber2 = pulsarClient.subscribe("persistent://my-property/use/my-ns/" + topicName, sub2, conf); + Consumer subscriber2 = pulsarClient.newConsumer() + .topic("persistent://my-property/use/my-ns/" + topicName) + .subscriptionName(sub2) + .subscriptionType(SubscriptionType.Shared) + .subscribe(); // Produce messages final int moreMessages = 10; for (int i = 0; i < receiverSize + moreMessages; i++) { @@ -813,24 +817,27 @@ public void testDeactivatingBacklogConsumer() throws Exception { final String topic = "persistent://my-property/use/my-ns/" + topicName; final String sub1 = "faster-sub1"; final String sub2 = "slower-sub2"; - ConsumerConfiguration conf = new ConsumerConfiguration(); - conf.setSubscriptionType(SubscriptionType.Shared); - conf.setReceiverQueueSize(receiverSize); - - ProducerConfiguration producerConf = new ProducerConfiguration(); - - if (batchMessageDelayMs != 0) { - producerConf.setBatchingEnabled(true); - producerConf.setBatchingMaxPublishDelay(batchMessageDelayMs, TimeUnit.MILLISECONDS); - producerConf.setBatchingMaxMessages(5); - } // 1. Subscriber Faster subscriber: let it consume all messages immediately - Consumer subscriber1 = pulsarClient.subscribe("persistent://my-property/use/my-ns/" + topicName, sub1, conf); + Consumer subscriber1 = pulsarClient.newConsumer() + .topic("persistent://my-property/use/my-ns/" + topicName) + .subscriptionName(sub1) + .subscriptionType(SubscriptionType.Shared) + .receiverQueueSize(receiverSize) + .subscribe(); // 1.b. Subscriber Slow subscriber: - conf.setReceiverQueueSize(receiverSize); - Consumer subscriber2 = pulsarClient.subscribe("persistent://my-property/use/my-ns/" + topicName, sub2, conf); - Producer producer = pulsarClient.createProducer(topic, producerConf); + Consumer subscriber2 = pulsarClient.newConsumer() + .topic("persistent://my-property/use/my-ns/" + topicName) + .subscriptionName(sub2) + .subscriptionType(SubscriptionType.Shared) + .receiverQueueSize(receiverSize) + .subscribe(); + Producer producer = pulsarClient.newProducer() + .topic(topic) + .enableBatching(batchMessageDelayMs != 0) + .batchingMaxPublishDelay(batchMessageDelayMs, TimeUnit.MILLISECONDS) + .batchingMaxMessages(5) + .create(); PersistentTopic topicRef = (PersistentTopic) pulsar.getBrokerService().getTopicReference(topic).get(); ManagedLedgerImpl ledger = (ManagedLedgerImpl) topicRef.getManagedLedger(); @@ -847,7 +854,7 @@ public void testDeactivatingBacklogConsumer() throws Exception { field.set(ledger, maxMessageCacheRetentionTimeMillis); final long maxActiveCursorBacklogEntries = (long) backlogThresholdField.get(ledger); - Message msg = null; + Messagemsg = null; final int totalMsgs = (int) maxActiveCursorBacklogEntries + receiverSize + 1; // 2. Produce messages for (int i = 0; i < totalMsgs; i++) { @@ -895,15 +902,17 @@ public void testAsyncProducerAndConsumer() throws Exception { final int totalMsg = 100; final Set produceMsgs = Sets.newHashSet(); final Set consumeMsgs = Sets.newHashSet(); - final ProducerConfiguration producerConf = new ProducerConfiguration(); - final ConsumerConfiguration conf = new ConsumerConfiguration(); - conf.setSubscriptionType(SubscriptionType.Exclusive); - Consumer consumer = pulsarClient.subscribe("persistent://my-property/use/my-ns/my-topic1", "my-subscriber-name", - conf); + Consumer consumer = pulsarClient.newConsumer() + .topic("persistent://my-property/use/my-ns/my-topic1") + .subscriptionName("my-subscriber-name") + .subscriptionType(SubscriptionType.Exclusive) + .subscribe(); // produce message - Producer producer = pulsarClient.createProducer("persistent://my-property/use/my-ns/my-topic1", producerConf); + Producer producer = pulsarClient.newProducer() + .topic("persistent://my-property/use/my-ns/my-topic1") + .create(); for (int i = 0; i < totalMsg; i++) { String message = "my-message-" + i; producer.send(message.getBytes()); @@ -936,15 +945,17 @@ public void testAsyncProducerAndConsumerWithZeroQueueSize() throws Exception { final int totalMsg = 100; final Set produceMsgs = Sets.newHashSet(); final Set consumeMsgs = Sets.newHashSet(); - final ProducerConfiguration producerConf = new ProducerConfiguration(); - final ConsumerConfiguration conf = new ConsumerConfiguration(); - conf.setSubscriptionType(SubscriptionType.Exclusive); - Consumer consumer = pulsarClient.subscribe("persistent://my-property/use/my-ns/my-topic1", "my-subscriber-name", - conf); + Consumer consumer = pulsarClient.newConsumer() + .topic("persistent://my-property/use/my-ns/my-topic1") + .subscriptionName("my-subscriber-name") + .subscriptionType(SubscriptionType.Exclusive) + .subscribe(); // produce message - Producer producer = pulsarClient.createProducer("persistent://my-property/use/my-ns/my-topic1", producerConf); + Producer producer = pulsarClient.newProducer() + .topic("persistent://my-property/use/my-ns/my-topic1") + .create(); for (int i = 0; i < totalMsg; i++) { String message = "my-message-" + i; producer.send(message.getBytes()); @@ -975,18 +986,19 @@ public void testSendCallBack() throws Exception { log.info("-- Starting {} test --", methodName); final int totalMsg = 100; - final ProducerConfiguration producerConf = new ProducerConfiguration(); - producerConf.setBatchingEnabled(false); - Producer producer = pulsarClient.createProducer("persistent://my-property/use/my-ns/my-topic1", producerConf); + + Producer producer = pulsarClient.newProducer() + .topic("persistent://my-property/use/my-ns/my-topic1") + .enableBatching(false) + .create(); for (int i = 0; i < totalMsg; i++) { final String message = "my-message-" + i; - Message msg = MessageBuilder.create().setContent(message.getBytes()).build(); final AtomicInteger msgLength = new AtomicInteger(); - CompletableFuture future = producer.sendAsync(msg).handle((r, ex) -> { + CompletableFuture future = producer.sendAsync(message.getBytes()).handle((r, ex) -> { if (ex != null) { log.error("Message send failed:", ex); } else { - msgLength.set(msg.getData().length); + msgLength.set(message.length()); } return null; }); @@ -1005,25 +1017,26 @@ public void testSendCallBack() throws Exception { public void testSharedConsumerAckDifferentConsumer() throws Exception { log.info("-- Starting {} test --", methodName); - ConsumerConfiguration conf = new ConsumerConfiguration(); - conf.setReceiverQueueSize(1); - conf.setSubscriptionType(SubscriptionType.Shared); - Consumer consumer1 = pulsarClient.subscribe("persistent://my-property/use/my-ns/my-topic1", - "my-subscriber-name", conf); - Consumer consumer2 = pulsarClient.subscribe("persistent://my-property/use/my-ns/my-topic1", - "my-subscriber-name", conf); - - ProducerConfiguration producerConf = new ProducerConfiguration(); - - Producer producer = pulsarClient.createProducer("persistent://my-property/use/my-ns/my-topic1", producerConf); + ConsumerBuilder cb = pulsarClient.newConsumer() + .topic("persistent://my-property/use/my-ns/my-topic1") + .subscriptionName("my-subscriber-name") + .subscriptionType(SubscriptionType.Shared) + .acknowledgmentGroupTime(0, TimeUnit.SECONDS) + .receiverQueueSize(1); + Consumer consumer1 = cb.subscribe(); + Consumer consumer2 = cb.subscribe(); + + Producer producer = pulsarClient.newProducer() + .topic("persistent://my-property/use/my-ns/my-topic1") + .create(); for (int i = 0; i < 10; i++) { String message = "my-message-" + i; producer.send(message.getBytes()); } - Message msg = null; - Set consumerMsgSet1 = Sets.newHashSet(); - Set consumerMsgSet2 = Sets.newHashSet(); + Messagemsg = null; + Set> consumerMsgSet1 = Sets.newHashSet(); + Set> consumerMsgSet2 = Sets.newHashSet(); for (int i = 0; i < 5; i++) { msg = consumer1.receive(); consumerMsgSet1.add(msg); @@ -1063,10 +1076,10 @@ public void testSharedConsumerAckDifferentConsumer() throws Exception { log.info("-- Exiting {} test --", methodName); } - private void receiveAsync(Consumer consumer, int totalMessage, int currentMessage, CountDownLatch latch, + private void receiveAsync(Consumer consumer, int totalMessage, int currentMessage, CountDownLatch latch, final Set consumeMsg, ExecutorService executor) throws PulsarClientException { if (currentMessage < totalMessage) { - CompletableFuture future = consumer.receiveAsync(); + CompletableFuture> future = consumer.receiveAsync(); future.handle((msg, exception) -> { if (exception == null) { // add message to consumer-queue to verify with produced messages @@ -1111,16 +1124,16 @@ public void testConsumerBlockingWithUnAckedMessages() throws Exception { final int totalProducedMsgs = 600; pulsar.getConfiguration().setMaxUnackedMessagesPerConsumer(unAckedMessagesBufferSize); - ConsumerConfiguration conf = new ConsumerConfiguration(); - conf.setReceiverQueueSize(receiverQueueSize); - conf.setSubscriptionType(SubscriptionType.Shared); - Consumer consumer = pulsarClient.subscribe("persistent://my-property/use/my-ns/unacked-topic", - "subscriber-1", conf); - - ProducerConfiguration producerConf = new ProducerConfiguration(); + Consumer consumer = pulsarClient.newConsumer() + .topic("persistent://my-property/use/my-ns/unacked-topic") + .subscriptionName( "subscriber-1") + .receiverQueueSize(receiverQueueSize) + .subscriptionType(SubscriptionType.Shared) + .subscribe(); - Producer producer = pulsarClient.createProducer("persistent://my-property/use/my-ns/unacked-topic", - producerConf); + Producer producer = pulsarClient.newProducer() + .topic("persistent://my-property/use/my-ns/unacked-topic") + .create(); // (1) Produced Messages for (int i = 0; i < totalProducedMsgs; i++) { @@ -1129,8 +1142,8 @@ public void testConsumerBlockingWithUnAckedMessages() throws Exception { } // (2) try to consume messages: but will be able to consume number of messages = unAckedMessagesBufferSize - Message msg = null; - List messages = Lists.newArrayList(); + Messagemsg = null; + List> messages = Lists.newArrayList(); for (int i = 0; i < totalProducedMsgs; i++) { msg = consumer.receive(1, TimeUnit.SECONDS); if (msg != null) { @@ -1196,16 +1209,16 @@ public void testConsumerBlockingWithUnAckedMessagesMultipleIteration() throws Ex // receiver consumes messages in iteration after acknowledging broker final int totalReceiveIteration = totalProducedMsgs / unAckedMessagesBufferSize; pulsar.getConfiguration().setMaxUnackedMessagesPerConsumer(unAckedMessagesBufferSize); - ConsumerConfiguration conf = new ConsumerConfiguration(); - conf.setReceiverQueueSize(receiverQueueSize); - conf.setSubscriptionType(SubscriptionType.Shared); - Consumer consumer = pulsarClient.subscribe("persistent://my-property/use/my-ns/unacked-topic", - "subscriber-1", conf); - - ProducerConfiguration producerConf = new ProducerConfiguration(); + Consumer consumer = pulsarClient.newConsumer() + .topic("persistent://my-property/use/my-ns/unacked-topic") + .subscriptionName("subscriber-1") + .receiverQueueSize(receiverQueueSize) + .subscriptionType(SubscriptionType.Shared) + .subscribe(); - Producer producer = pulsarClient.createProducer("persistent://my-property/use/my-ns/unacked-topic", - producerConf); + Producer producer = pulsarClient.newProducer() + .topic("persistent://my-property/use/my-ns/unacked-topic") + .create(); // (1) Produced Messages for (int i = 0; i < totalProducedMsgs; i++) { @@ -1217,8 +1230,8 @@ public void testConsumerBlockingWithUnAckedMessagesMultipleIteration() throws Ex // (2) Receive Messages for (int j = 0; j < totalReceiveIteration; j++) { - Message msg = null; - List messages = Lists.newArrayList(); + Messagemsg = null; + List> messages = Lists.newArrayList(); for (int i = 0; i < totalProducedMsgs; i++) { msg = consumer.receive(1, TimeUnit.SECONDS); if (msg != null) { @@ -1273,18 +1286,22 @@ public void testMutlipleSharedConsumerBlockingWithUnAckedMessages() throws Excep int totalReceiveMessages = 0; pulsar.getConfiguration().setMaxUnackedMessagesPerConsumer(maxUnackedMessages); - ConsumerConfiguration conf = new ConsumerConfiguration(); - conf.setReceiverQueueSize(receiverQueueSize); - conf.setSubscriptionType(SubscriptionType.Shared); - Consumer consumer1 = pulsarClient.subscribe("persistent://my-property/use/my-ns/unacked-topic", - "subscriber-1", conf); - Consumer consumer2 = pulsarClient.subscribe("persistent://my-property/use/my-ns/unacked-topic", - "subscriber-1", conf); - - ProducerConfiguration producerConf = new ProducerConfiguration(); - - Producer producer = pulsarClient.createProducer("persistent://my-property/use/my-ns/unacked-topic", - producerConf); + Consumer consumer1 = pulsarClient.newConsumer() + .topic("persistent://my-property/use/my-ns/unacked-topic") + .subscriptionName("subscriber-1") + .receiverQueueSize(receiverQueueSize) + .subscriptionType(SubscriptionType.Shared) + .subscribe(); + Consumer consumer2 = pulsarClient.newConsumer() + .topic("persistent://my-property/use/my-ns/unacked-topic") + .subscriptionName("subscriber-1") + .receiverQueueSize(receiverQueueSize) + .subscriptionType(SubscriptionType.Shared) + .subscribe(); + + Producer producer = pulsarClient.newProducer() + .topic("persistent://my-property/use/my-ns/unacked-topic") + .create(); // (1) Produced Messages for (int i = 0; i < totalProducedMsgs; i++) { @@ -1294,8 +1311,8 @@ public void testMutlipleSharedConsumerBlockingWithUnAckedMessages() throws Excep // (2) Consumer1: consume without ack: // try to consume messages: but will be able to consume number of messages = maxUnackedMessages - Message msg = null; - List messages = Lists.newArrayList(); + Messagemsg = null; + List> messages = Lists.newArrayList(); for (int i = 0; i < totalProducedMsgs; i++) { msg = consumer1.receive(1, TimeUnit.SECONDS); if (msg != null) { @@ -1368,18 +1385,18 @@ public void testShouldNotBlockConsumerIfRedeliverBeforeReceive() throws Exceptio final int receiverQueueSize = 20; final int totalProducedMsgs = 100; - ConsumerConfiguration conf = new ConsumerConfiguration(); - conf.setReceiverQueueSize(receiverQueueSize); - conf.setAckTimeout(1, TimeUnit.SECONDS); - conf.setSubscriptionType(SubscriptionType.Shared); - ConsumerImpl consumer = (ConsumerImpl) pulsarClient - .subscribe("persistent://my-property/use/my-ns/unacked-topic", "subscriber-1", conf); + ConsumerImpl consumer = (ConsumerImpl) pulsarClient.newConsumer() + .topic("persistent://my-property/use/my-ns/unacked-topic") + .subscriptionName("subscriber-1") + .receiverQueueSize(receiverQueueSize) + .ackTimeout(1, TimeUnit.SECONDS) + .subscriptionType(SubscriptionType.Shared) + .subscribe(); - ProducerConfiguration producerConf = new ProducerConfiguration(); - producerConf.setBatchingEnabled(false); - - Producer producer = pulsarClient.createProducer("persistent://my-property/use/my-ns/unacked-topic", - producerConf); + Producer producer = pulsarClient.newProducer() + .topic("persistent://my-property/use/my-ns/unacked-topic") + .enableBatching(false) + .create(); // (1) Produced Messages for (int i = 0; i < totalProducedMsgs; i++) { @@ -1396,7 +1413,7 @@ public void testShouldNotBlockConsumerIfRedeliverBeforeReceive() throws Exceptio assertEquals(consumer.numMessagesInQueue(), receiverQueueSize); for (int i = 0; i < totalProducedMsgs; i++) { - Message msg = consumer.receive(1, TimeUnit.SECONDS); + Messagemsg = consumer.receive(1, TimeUnit.SECONDS); if (msg != null) { consumer.acknowledge(msg); totalReceiveMsg++; @@ -1430,16 +1447,16 @@ public void testUnackBlockRedeliverMessages() throws Exception { final int totalProducedMsgs = 100; pulsar.getConfiguration().setMaxUnackedMessagesPerConsumer(unAckedMessagesBufferSize); - ConsumerConfiguration conf = new ConsumerConfiguration(); - conf.setReceiverQueueSize(receiverQueueSize); - conf.setSubscriptionType(SubscriptionType.Shared); - ConsumerImpl consumer = (ConsumerImpl) pulsarClient - .subscribe("persistent://my-property/use/my-ns/unacked-topic", "subscriber-1", conf); - - ProducerConfiguration producerConf = new ProducerConfiguration(); + ConsumerImpl consumer = (ConsumerImpl) pulsarClient.newConsumer() + .topic("persistent://my-property/use/my-ns/unacked-topic") + .subscriptionName("subscriber-1") + .receiverQueueSize(receiverQueueSize) + .subscriptionType(SubscriptionType.Shared) + .subscribe(); - Producer producer = pulsarClient.createProducer("persistent://my-property/use/my-ns/unacked-topic", - producerConf); + Producer producer = pulsarClient.newProducer() + .topic("persistent://my-property/use/my-ns/unacked-topic") + .create(); // (1) Produced Messages for (int i = 0; i < totalProducedMsgs; i++) { @@ -1448,8 +1465,8 @@ public void testUnackBlockRedeliverMessages() throws Exception { } // (2) try to consume messages: but will be able to consume number of messages = unAckedMessagesBufferSize - Message msg = null; - List messages = Lists.newArrayList(); + Messagemsg = null; + List> messages = Lists.newArrayList(); for (int i = 0; i < totalProducedMsgs; i++) { msg = consumer.receive(1, TimeUnit.SECONDS); if (msg != null) { @@ -1501,24 +1518,19 @@ public void testUnackedBlockAtBatch(int batchMessageDelayMs) throws Exception { int totalReceiveMessages = 0; pulsar.getConfiguration().setMaxUnackedMessagesPerConsumer(maxUnackedMessages); - ConsumerConfiguration conf = new ConsumerConfiguration(); - conf.setReceiverQueueSize(receiverQueueSize); - conf.setSubscriptionType(SubscriptionType.Shared); - Consumer consumer1 = pulsarClient.subscribe("persistent://my-property/use/my-ns/unacked-topic", - "subscriber-1", conf); - - ProducerConfiguration producerConf = new ProducerConfiguration(); - - if (batchMessageDelayMs != 0) { - producerConf.setBatchingEnabled(true); - producerConf.setBatchingMaxPublishDelay(batchMessageDelayMs, TimeUnit.MILLISECONDS); - producerConf.setBatchingMaxMessages(5); - } else { - producerConf.setBatchingEnabled(false); - } - - Producer producer = pulsarClient.createProducer("persistent://my-property/use/my-ns/unacked-topic", - producerConf); + Consumer consumer1 = pulsarClient.newConsumer() + .topic("persistent://my-property/use/my-ns/unacked-topic") + .subscriptionName("subscriber-1") + .subscriptionType(SubscriptionType.Shared) + .receiverQueueSize(receiverQueueSize) + .subscribe(); + + Producer producer = pulsarClient.newProducer() + .topic("persistent://my-property/use/my-ns/unacked-topic") + .enableBatching(batchMessageDelayMs != 0) + .batchingMaxPublishDelay(batchMessageDelayMs, TimeUnit.MILLISECONDS) + .batchingMaxMessages(5) + .create(); List> futures = Lists.newArrayList(); // (1) Produced Messages @@ -1531,8 +1543,8 @@ public void testUnackedBlockAtBatch(int batchMessageDelayMs) throws Exception { // (2) Consumer1: consume without ack: // try to consume messages: but will be able to consume number of messages = maxUnackedMessages - Message msg = null; - List messages = Lists.newArrayList(); + Messagemsg = null; + List> messages = Lists.newArrayList(); for (int i = 0; i < totalProducedMsgs; i++) { msg = consumer1.receive(1, TimeUnit.SECONDS); if (msg != null) { @@ -1598,18 +1610,18 @@ public void testBlockUnackConsumerAckByDifferentConsumer() throws Exception { int totalReceiveMessages = 0; pulsar.getConfiguration().setMaxUnackedMessagesPerConsumer(maxUnackedMessages); - ConsumerConfiguration conf = new ConsumerConfiguration(); - conf.setReceiverQueueSize(receiverQueueSize); - conf.setSubscriptionType(SubscriptionType.Shared); - Consumer consumer1 = pulsarClient.subscribe("persistent://my-property/use/my-ns/unacked-topic", - "subscriber-1", conf); - Consumer consumer2 = pulsarClient.subscribe("persistent://my-property/use/my-ns/unacked-topic", - "subscriber-1", conf); + ConsumerBuilder consumerBuilder = pulsarClient.newConsumer() + .topic("persistent://my-property/use/my-ns/unacked-topic") + .subscriptionName("subscriber-1") + .receiverQueueSize(receiverQueueSize) + .subscriptionType(SubscriptionType.Shared); - ProducerConfiguration producerConf = new ProducerConfiguration(); + Consumer consumer1 = consumerBuilder.subscribe(); + Consumer consumer2 = consumerBuilder.subscribe(); - Producer producer = pulsarClient.createProducer("persistent://my-property/use/my-ns/unacked-topic", - producerConf); + Producer producer = pulsarClient.newProducer() + .topic("persistent://my-property/use/my-ns/unacked-topic") + .create(); // (1) Produced Messages for (int i = 0; i < totalProducedMsgs; i++) { @@ -1619,8 +1631,8 @@ public void testBlockUnackConsumerAckByDifferentConsumer() throws Exception { // (2) Consumer1: consume without ack: // try to consume messages: but will be able to consume number of messages = maxUnackedMessages - Message msg = null; - List messages = Lists.newArrayList(); + Messagemsg = null; + List> messages = Lists.newArrayList(); for (int i = 0; i < totalProducedMsgs; i++) { msg = consumer1.receive(1, TimeUnit.SECONDS); if (msg != null) { @@ -1683,26 +1695,26 @@ public void testEnabledChecksumClient() throws Exception { log.info("-- Starting {} test --", methodName); final int totalMsg = 10; - ConsumerConfiguration conf = new ConsumerConfiguration(); - conf.setSubscriptionType(SubscriptionType.Exclusive); - Consumer consumer = pulsarClient.subscribe("persistent://my-property/use/my-ns/my-topic1", "my-subscriber-name", - conf); - ProducerConfiguration producerConf = new ProducerConfiguration(); + Consumer consumer = pulsarClient.newConsumer() + .topic("persistent://my-property/use/my-ns/my-topic1") + .subscriptionName("my-subscriber-name") + .subscribe(); + final int batchMessageDelayMs = 300; - if (batchMessageDelayMs != 0) { - producerConf.setBatchingEnabled(true); - producerConf.setBatchingMaxPublishDelay(batchMessageDelayMs, TimeUnit.MILLISECONDS); - producerConf.setBatchingMaxMessages(5); - } + Producer producer = pulsarClient.newProducer() + .topic("persistent://my-property/use/my-ns/my-topic1") + .enableBatching(true) + .batchingMaxPublishDelay(batchMessageDelayMs, TimeUnit.MILLISECONDS) + .batchingMaxMessages(5) + .create(); - Producer producer = pulsarClient.createProducer("persistent://my-property/use/my-ns/my-topic1", producerConf); for (int i = 0; i < totalMsg; i++) { String message = "my-message-" + i; producer.send(message.getBytes()); } - Message msg = null; + Messagemsg = null; Set messageSet = Sets.newHashSet(); for (int i = 0; i < totalMsg; i++) { msg = consumer.receive(5, TimeUnit.SECONDS); @@ -1736,16 +1748,16 @@ public void testBlockUnackedConsumerRedeliverySpecificMessagesProduceWithPause() final int totalProducedMsgs = 20; pulsar.getConfiguration().setMaxUnackedMessagesPerConsumer(unAckedMessagesBufferSize); - ConsumerConfiguration conf = new ConsumerConfiguration(); - conf.setReceiverQueueSize(receiverQueueSize); - conf.setSubscriptionType(SubscriptionType.Shared); - ConsumerImpl consumer = (ConsumerImpl) pulsarClient - .subscribe("persistent://my-property/use/my-ns/unacked-topic", "subscriber-1", conf); - - ProducerConfiguration producerConf = new ProducerConfiguration(); + ConsumerImpl consumer = (ConsumerImpl) pulsarClient.newConsumer() + .topic("persistent://my-property/use/my-ns/unacked-topic") + .subscriptionName("subscriber-1") + .subscriptionType(SubscriptionType.Shared) + .receiverQueueSize(receiverQueueSize) + .subscribe(); - Producer producer = pulsarClient.createProducer("persistent://my-property/use/my-ns/unacked-topic", - producerConf); + Producer producer = pulsarClient.newProducer() + .topic("persistent://my-property/use/my-ns/unacked-topic") + .create(); // (1) Produced Messages for (int i = 0; i < totalProducedMsgs; i++) { @@ -1755,8 +1767,8 @@ public void testBlockUnackedConsumerRedeliverySpecificMessagesProduceWithPause() } // (2) try to consume messages: but will be able to consume number of messages = unAckedMessagesBufferSize - Message msg = null; - List messages1 = Lists.newArrayList(); + Messagemsg = null; + List> messages1 = Lists.newArrayList(); for (int i = 0; i < totalProducedMsgs; i++) { msg = consumer.receive(1, TimeUnit.SECONDS); if (msg != null) { @@ -1822,18 +1834,18 @@ public void testBlockUnackedConsumerRedeliverySpecificMessagesCloseConsumerWhile final int totalProducedMsgs = 50; pulsar.getConfiguration().setMaxUnackedMessagesPerConsumer(unAckedMessagesBufferSize); - ConsumerConfiguration conf = new ConsumerConfiguration(); - conf.setReceiverQueueSize(receiverQueueSize); - conf.setSubscriptionType(SubscriptionType.Shared); // Only subscribe consumer - ConsumerImpl consumer = (ConsumerImpl) pulsarClient - .subscribe("persistent://my-property/use/my-ns/unacked-topic", "subscriber-1", conf); + ConsumerImpl consumer = (ConsumerImpl) pulsarClient.newConsumer() + .topic("persistent://my-property/use/my-ns/unacked-topic") + .subscriptionName("subscriber-1") + .receiverQueueSize(receiverQueueSize) + .subscriptionType(SubscriptionType.Shared) + .subscribe(); consumer.close(); - ProducerConfiguration producerConf = new ProducerConfiguration(); - - Producer producer = pulsarClient.createProducer("persistent://my-property/use/my-ns/unacked-topic", - producerConf); + Producer producer = pulsarClient.newProducer() + .topic("persistent://my-property/use/my-ns/unacked-topic") + .create(); // (1) Produced Messages for (int i = 0; i < totalProducedMsgs; i++) { @@ -1843,12 +1855,16 @@ public void testBlockUnackedConsumerRedeliverySpecificMessagesCloseConsumerWhile } // (1.a) start consumer again - consumer = (ConsumerImpl) pulsarClient.subscribe("persistent://my-property/use/my-ns/unacked-topic", - "subscriber-1", conf); + consumer = (ConsumerImpl) pulsarClient.newConsumer() + .topic("persistent://my-property/use/my-ns/unacked-topic") + .receiverQueueSize(receiverQueueSize) + .subscriptionName("subscriber-1") + .subscriptionType(SubscriptionType.Shared) + .subscribe(); // (2) try to consume messages: but will be able to consume number of messages = unAckedMessagesBufferSize - Message msg = null; - List messages1 = Lists.newArrayList(); + Messagemsg = null; + List> messages1 = Lists.newArrayList(); for (int i = 0; i < totalProducedMsgs; i++) { msg = consumer.receive(1, TimeUnit.SECONDS); if (msg != null) { @@ -1897,24 +1913,24 @@ public void testBlockUnackedConsumerRedeliverySpecificMessagesCloseConsumerWhile @Test public void testPriorityConsumer() throws Exception { log.info("-- Starting {} test --", methodName); - ConsumerConfiguration conf1 = new ConsumerConfiguration(); - conf1.setSubscriptionType(SubscriptionType.Shared); - conf1.setPriorityLevel(1); - conf1.setReceiverQueueSize(5); - ConsumerConfiguration conf4 = new ConsumerConfiguration(); - conf4.setSubscriptionType(SubscriptionType.Shared); - conf4.setPriorityLevel(2); - conf4.setReceiverQueueSize(5); - Consumer consumer1 = pulsarClient.subscribe("persistent://my-property/use/my-ns/my-topic2", - "my-subscriber-name", conf1); - Consumer consumer2 = pulsarClient.subscribe("persistent://my-property/use/my-ns/my-topic2", - "my-subscriber-name", conf1); - Consumer consumer3 = pulsarClient.subscribe("persistent://my-property/use/my-ns/my-topic2", - "my-subscriber-name", conf1); - Consumer consumer4 = pulsarClient.subscribe("persistent://my-property/use/my-ns/my-topic2", - "my-subscriber-name", conf4); - ProducerConfiguration producerConf = new ProducerConfiguration(); - Producer producer = pulsarClient.createProducer("persistent://my-property/use/my-ns/my-topic2", producerConf); + + ConsumerBuilder cb1 = pulsarClient.newConsumer() + .topic("persistent://my-property/use/my-ns/my-topic2") + .subscriptionName("my-subscriber-name") + .subscriptionType(SubscriptionType.Shared) + .priorityLevel(1) + .receiverQueueSize(5); + ConsumerBuilder cb4 = cb1.clone() + .priorityLevel(2) + .receiverQueueSize(5); + Consumer consumer1 = cb1.subscribe(); + Consumer consumer2 = cb1.subscribe(); + Consumer consumer3 = cb1.subscribe(); + Consumer consumer4 = cb4.subscribe(); + + Producer producer = pulsarClient.newProducer() + .topic("persistent://my-property/use/my-ns/my-topic2") + .create(); List> futures = Lists.newArrayList(); // Asynchronously produce messages @@ -1968,22 +1984,26 @@ public void testPriorityConsumer() throws Exception { * * @throws Exception */ - @Test(timeOut = 5000) + @Test public void testSharedSamePriorityConsumer() throws Exception { log.info("-- Starting {} test --", methodName); - ConsumerConfiguration conf1 = new ConsumerConfiguration(); - conf1.setSubscriptionType(SubscriptionType.Shared); + final int queueSize = 5; - conf1.setReceiverQueueSize(queueSize); int maxUnAckMsgs = pulsar.getConfiguration().getMaxConcurrentLookupRequest(); pulsar.getConfiguration().setMaxUnackedMessagesPerConsumer(queueSize); - Consumer c1 = pulsarClient.subscribe("persistent://my-property/use/my-ns/my-topic2", "my-subscriber-name", - conf1); - Consumer c2 = pulsarClient.subscribe("persistent://my-property/use/my-ns/my-topic2", "my-subscriber-name", - conf1); - ProducerConfiguration producerConf = new ProducerConfiguration(); - producerConf.setBatchingEnabled(false); - Producer producer = pulsarClient.createProducer("persistent://my-property/use/my-ns/my-topic2", producerConf); + + ConsumerBuilder cb = pulsarClient.newConsumer() + .topic("persistent://my-property/use/my-ns/my-topic2") + .subscriptionName("my-subscriber-name") + .subscriptionType(SubscriptionType.Shared) + .receiverQueueSize(queueSize); + Consumer c1 = cb.subscribe(); + Consumer c2 = cb.subscribe(); + + Producer producer = pulsarClient.newProducer() + .topic("persistent://my-property/use/my-ns/my-topic2") + .enableBatching(false) + .create(); List> futures = Lists.newArrayList(); // Asynchronously produce messages @@ -1999,11 +2019,11 @@ public void testSharedSamePriorityConsumer() throws Exception { future.get(); } - List messages = Lists.newArrayList(); + List> messages = Lists.newArrayList(); // let consumer1 and consumer2 cosume messages up to the queue will be full for (int i = 0; i < totalPublishMessages; i++) { - Message msg = c1.receive(500, TimeUnit.MILLISECONDS); + Messagemsg = c1.receive(500, TimeUnit.MILLISECONDS); if (msg != null) { messages.add(msg); } else { @@ -2011,7 +2031,7 @@ public void testSharedSamePriorityConsumer() throws Exception { } } for (int i = 0; i < totalPublishMessages; i++) { - Message msg = c2.receive(500, TimeUnit.MILLISECONDS); + Messagemsg = c2.receive(500, TimeUnit.MILLISECONDS); if (msg != null) { messages.add(msg); } else { @@ -2022,17 +2042,14 @@ public void testSharedSamePriorityConsumer() throws Exception { Assert.assertEquals(queueSize * 2, messages.size()); // create new consumers with the same priority - Consumer c3 = pulsarClient.subscribe("persistent://my-property/use/my-ns/my-topic2", "my-subscriber-name", - conf1); - Consumer c4 = pulsarClient.subscribe("persistent://my-property/use/my-ns/my-topic2", "my-subscriber-name", - conf1); - Consumer c5 = pulsarClient.subscribe("persistent://my-property/use/my-ns/my-topic2", "my-subscriber-name", - conf1); + Consumer c3 = cb.subscribe(); + Consumer c4 = cb.subscribe(); + Consumer c5 = cb.subscribe(); // c1 and c2 are blocked: so, let c3, c4 and c5 consume rest of the messages for (int i = 0; i < totalPublishMessages; i++) { - Message msg = c4.receive(500, TimeUnit.MILLISECONDS); + Messagemsg = c4.receive(500, TimeUnit.MILLISECONDS); if (msg != null) { messages.add(msg); } else { @@ -2041,7 +2058,7 @@ public void testSharedSamePriorityConsumer() throws Exception { } for (int i = 0; i < totalPublishMessages; i++) { - Message msg = c5.receive(500, TimeUnit.MILLISECONDS); + Messagemsg = c5.receive(500, TimeUnit.MILLISECONDS); if (msg != null) { messages.add(msg); } else { @@ -2050,7 +2067,7 @@ public void testSharedSamePriorityConsumer() throws Exception { } for (int i = 0; i < totalPublishMessages; i++) { - Message msg = c3.receive(500, TimeUnit.MILLISECONDS); + Message msg = c3.receive(500, TimeUnit.MILLISECONDS); if (msg != null) { messages.add(msg); c3.acknowledge(msg); @@ -2079,17 +2096,18 @@ public void testRedeliveryFailOverConsumer() throws Exception { final int receiverQueueSize = 10; - ConsumerConfiguration conf = new ConsumerConfiguration(); - conf.setReceiverQueueSize(receiverQueueSize); - conf.setSubscriptionType(SubscriptionType.Failover); // Only subscribe consumer - ConsumerImpl consumer = (ConsumerImpl) pulsarClient - .subscribe("persistent://my-property/use/my-ns/unacked-topic", "subscriber-1", conf); - - ProducerConfiguration producerConf = new ProducerConfiguration(); - - Producer producer = pulsarClient.createProducer("persistent://my-property/use/my-ns/unacked-topic", - producerConf); + ConsumerImpl consumer = (ConsumerImpl) pulsarClient.newConsumer() + .topic("persistent://my-property/use/my-ns/unacked-topic") + .subscriptionName("subscriber-1") + .receiverQueueSize(receiverQueueSize) + .subscriptionType(SubscriptionType.Failover) + .acknowledgmentGroupTime(0, TimeUnit.SECONDS) + .subscribe(); + + Producer producer = pulsarClient.newProducer() + .topic("persistent://my-property/use/my-ns/unacked-topic") + .create(); // (1) First round to produce-consume messages int consumeMsgInParts = 4; @@ -2099,8 +2117,8 @@ public void testRedeliveryFailOverConsumer() throws Exception { Thread.sleep(10); } // (1.a) consume first consumeMsgInParts msgs and trigger redeliver - Message msg = null; - List messages1 = Lists.newArrayList(); + Message msg = null; + List> messages1 = Lists.newArrayList(); for (int i = 0; i < consumeMsgInParts; i++) { msg = consumer.receive(1, TimeUnit.SECONDS); if (msg != null) { @@ -2161,8 +2179,10 @@ public void testFailReceiveAsyncOnConsumerClose() throws Exception { log.info("-- Starting {} test --", methodName); // (1) simple consumers - Consumer consumer = pulsarClient.subscribe("persistent://my-property/use/my-ns/failAsyncReceive", - "my-subscriber-name", new ConsumerConfiguration()); + Consumer consumer = pulsarClient.newConsumer() + .topic("persistent://my-property/use/my-ns/failAsyncReceive") + .subscriptionName("my-subscriber-name") + .subscribe(); consumer.close(); // receive messages try { @@ -2176,8 +2196,9 @@ public void testFailReceiveAsyncOnConsumerClose() throws Exception { int numPartitions = 4; TopicName topicName = TopicName.get("persistent://my-property/use/my-ns/failAsyncReceive"); admin.topics().createPartitionedTopic(topicName.toString(), numPartitions); - Consumer partitionedConsumer = pulsarClient.subscribe(topicName.toString(), "my-partitioned-subscriber", - new ConsumerConfiguration()); + Consumer partitionedConsumer = pulsarClient.newConsumer().topic(topicName.toString()) + .subscriptionName("my-partitioned-subscriber") + .subscribe(); partitionedConsumer.close(); // receive messages try { @@ -2234,24 +2255,24 @@ public EncryptionKeyInfo getPrivateKey(String keyName, Map keyMe final int totalMsg = 10; Set messageSet = Sets.newHashSet(); - ConsumerConfiguration conf = new ConsumerConfiguration(); - conf.setSubscriptionType(SubscriptionType.Exclusive); - conf.setCryptoKeyReader(new EncKeyReader()); - Consumer consumer = pulsarClient.subscribe("persistent://my-property/use/my-ns/myecdsa-topic1", - "my-subscriber-name", conf); - - ProducerConfiguration producerConf = new ProducerConfiguration(); - producerConf.addEncryptionKey("client-ecdsa.pem"); - producerConf.setCryptoKeyReader(new EncKeyReader()); - - Producer producer = pulsarClient.createProducer("persistent://my-property/use/my-ns/myecdsa-topic1", - producerConf); + Consumer consumer = pulsarClient.newConsumer() + .topic("persistent://my-property/use/my-ns/myecdsa-topic1") + .subscriptionName("my-subscriber-name") + .subscriptionType(SubscriptionType.Exclusive) + .cryptoKeyReader(new EncKeyReader()) + .subscribe(); + + Producer producer = pulsarClient.newProducer() + .topic("persistent://my-property/use/my-ns/myecdsa-topic1") + .addEncryptionKey("client-ecdsa.pem") + .cryptoKeyReader(new EncKeyReader()) + .create(); for (int i = 0; i < totalMsg; i++) { String message = "my-message-" + i; producer.send(message.getBytes()); } - Message msg = null; + Message msg = null; for (int i = 0; i < totalMsg; i++) { msg = consumer.receive(5, TimeUnit.SECONDS); @@ -2310,20 +2331,23 @@ public EncryptionKeyInfo getPrivateKey(String keyName, Map keyMe final int totalMsg = 10; Set messageSet = Sets.newHashSet(); - ConsumerConfiguration conf = new ConsumerConfiguration(); - conf.setSubscriptionType(SubscriptionType.Exclusive); - conf.setCryptoKeyReader(new EncKeyReader()); - Consumer consumer = pulsarClient.subscribe("persistent://my-property/use/my-ns/myrsa-topic1", - "my-subscriber-name", conf); - - ProducerConfiguration producerConf = new ProducerConfiguration(); - producerConf.addEncryptionKey("client-rsa.pem"); - producerConf.setCryptoKeyReader(new EncKeyReader()); - - Producer producer = pulsarClient.createProducer("persistent://my-property/use/my-ns/myrsa-topic1", - producerConf); - Producer producer2 = pulsarClient.createProducer("persistent://my-property/use/my-ns/myrsa-topic1", - producerConf); + Consumer consumer = pulsarClient.newConsumer() + .topic("persistent://my-property/use/my-ns/myrsa-topic1") + .subscriptionName("my-subscriber-name") + .subscriptionType(SubscriptionType.Exclusive) + .cryptoKeyReader(new EncKeyReader()) + .subscribe(); + + Producer producer = pulsarClient.newProducer() + .topic("persistent://my-property/use/my-ns/myrsa-topic1") + .addEncryptionKey("client-rsa.pem") + .cryptoKeyReader(new EncKeyReader()) + .create(); + Producer producer2 = pulsarClient.newProducer() + .topic("persistent://my-property/use/my-ns/myrsa-topic1") + .addEncryptionKey("client-rsa.pem") + .cryptoKeyReader(new EncKeyReader()) + .create(); for (int i = 0; i < totalMsg; i++) { String message = "my-message-" + i; producer.send(message.getBytes()); @@ -2333,7 +2357,7 @@ public EncryptionKeyInfo getPrivateKey(String keyName, Map keyMe producer2.send(message.getBytes()); } - Message msg = null; + Message msg = null; for (int i = 0; i < totalMsg * 2; i++) { msg = consumer.receive(5, TimeUnit.SECONDS); @@ -2387,38 +2411,37 @@ public EncryptionKeyInfo getPrivateKey(String keyName, Map keyMe final int totalMsg = 10; - ProducerConfiguration producerConf = new ProducerConfiguration(); - producerConf.setBatchingEnabled(false); - - Message msg = null; + Message msg = null; Set messageSet = Sets.newHashSet(); - ConsumerConfiguration conf = new ConsumerConfiguration(); - conf.setSubscriptionType(SubscriptionType.Exclusive); - Consumer consumer = pulsarClient.subscribe("persistent://my-property/use/my-ns/myenc-topic1", - "my-subscriber-name", conf); - - // 1. Invalid key name - producerConf.addEncryptionKey("client-non-existant-rsa.pem"); - producerConf.setCryptoKeyReader(new EncKeyReader()); + Consumer consumer = pulsarClient.newConsumer(Schema.STRING) + .topic("persistent://my-property/use/my-ns/myenc-topic1") + .subscriptionName("my-subscriber-name") + .subscriptionType(SubscriptionType.Exclusive) + .subscribe(); try { - pulsarClient.createProducer("persistent://my-property/use/myenc-ns/myenc-topic1", producerConf); + // 1. Invalid key name + pulsarClient.newProducer(Schema.STRING) + .topic("persistent://my-property/use/myenc-ns/myenc-topic1") + .enableBatching(false) + .addEncryptionKey("client-non-existant-rsa.pem") + .cryptoKeyReader(new EncKeyReader()) + .create(); Assert.fail("Producer creation should not suceed if failing to read key"); } catch (Exception e) { // ok } // 2. Producer with valid key name - producerConf = new ProducerConfiguration(); - producerConf.setCryptoKeyReader(new EncKeyReader()); - producerConf.addEncryptionKey("client-rsa.pem"); - producerConf.setBatchingEnabled(false); - Producer producer = pulsarClient.createProducer("persistent://my-property/use/my-ns/myenc-topic1", - producerConf); + Producer producer = pulsarClient.newProducer(Schema.STRING) + .topic("persistent://my-property/use/my-ns/myenc-topic1") + .enableBatching(false) + .addEncryptionKey("client-rsa.pem") + .cryptoKeyReader(new EncKeyReader()) + .create(); for (int i = 0; i < totalMsg; i++) { - String message = "my-message-" + i; - producer.send(message.getBytes()); + producer.send("my-message-" + i); } // 3. KeyReder is not set by consumer @@ -2427,17 +2450,20 @@ public EncryptionKeyInfo getPrivateKey(String keyName, Map keyMe Assert.assertNull(msg, "Receive should have failed with no keyreader"); // 4. Set consumer config to consume even if decryption fails - conf.setCryptoFailureAction(ConsumerCryptoFailureAction.CONSUME); consumer.close(); - consumer = pulsarClient.subscribe("persistent://my-property/use/my-ns/myenc-topic1", "my-subscriber-name", - conf); + consumer = pulsarClient.newConsumer(Schema.STRING) + .topic("persistent://my-property/use/my-ns/myenc-topic1") + .subscriptionName("my-subscriber-name") + .subscriptionType(SubscriptionType.Exclusive) + .cryptoFailureAction(ConsumerCryptoFailureAction.CONSUME) + .subscribe(); int msgNum = 0; try { // Receive should proceed and deliver encrypted message msg = consumer.receive(5, TimeUnit.SECONDS); - String receivedMessage = new String(msg.getData()); - String expectedMessage = "my-message-" + msgNum++; + String receivedMessage = msg.getValue(); + String expectedMessage = "my-message-" + (msgNum++); Assert.assertNotEquals(receivedMessage, expectedMessage, "Received encrypted message " + receivedMessage + " should not match the expected message " + expectedMessage); consumer.acknowledgeCumulative(msg); @@ -2446,16 +2472,18 @@ public EncryptionKeyInfo getPrivateKey(String keyName, Map keyMe } // 5. Set keyreader and failure action - conf.setCryptoFailureAction(ConsumerCryptoFailureAction.FAIL); consumer.close(); - // Set keyreader - conf.setCryptoKeyReader(new EncKeyReader()); - consumer = pulsarClient.subscribe("persistent://my-property/use/my-ns/myenc-topic1", "my-subscriber-name", - conf); + consumer = pulsarClient.newConsumer(Schema.STRING) + .topic("persistent://my-property/use/my-ns/myenc-topic1") + .subscriptionName("my-subscriber-name") + .subscriptionType(SubscriptionType.Exclusive) + .cryptoKeyReader(new EncKeyReader()) + .cryptoFailureAction(ConsumerCryptoFailureAction.FAIL) + .subscribe(); for (int i = msgNum; i < totalMsg - 1; i++) { msg = consumer.receive(5, TimeUnit.SECONDS); - String receivedMessage = new String(msg.getData()); + String receivedMessage = msg.getValue(); log.debug("Received message: [{}]", receivedMessage); String expectedMessage = "my-message-" + i; testMessageOrderAndDuplicates(messageSet, receivedMessage, expectedMessage); @@ -2466,11 +2494,12 @@ public EncryptionKeyInfo getPrivateKey(String keyName, Map keyMe // 6. Set consumer config to discard if decryption fails consumer.close(); - ConsumerConfiguration conf2 = new ConsumerConfiguration(); - conf2.setSubscriptionType(SubscriptionType.Exclusive); - conf2.setCryptoFailureAction(ConsumerCryptoFailureAction.DISCARD); - consumer = pulsarClient.subscribe("persistent://my-property/use/my-ns/myenc-topic1", "my-subscriber-name", - conf2); + consumer = pulsarClient.newConsumer(Schema.STRING) + .topic("persistent://my-property/use/my-ns/myenc-topic1") + .subscriptionName("my-subscriber-name") + .subscriptionType(SubscriptionType.Exclusive) + .cryptoFailureAction(ConsumerCryptoFailureAction.DISCARD) + .subscribe(); // Receive should proceed and discard encrypted messages msg = consumer.receive(5, TimeUnit.SECONDS); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/ConnectionPoolTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/ConnectionPoolTest.java index d8d6f83c259eb..9088e67c42447 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/ConnectionPoolTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/ConnectionPoolTest.java @@ -63,7 +63,9 @@ public void testSingleIpAddress() throws Exception { result.add(InetAddress.getByName("127.0.0.1")); Mockito.when(pool.resolveName("non-existing-dns-name")).thenReturn(CompletableFuture.completedFuture(result)); - client.createProducer("persistent://sample/standalone/ns/my-topic"); + client.newProducer() + .topic("persistent://sample/standalone/ns/my-topic") + .create(); client.close(); } @@ -86,7 +88,7 @@ public void testDoubleIpAddress() throws Exception { Mockito.when(pool.resolveName("non-existing-dns-name")).thenReturn(CompletableFuture.completedFuture(result)); // Create producer should succeed by trying the 2nd IP - client.createProducer("persistent://sample/standalone/ns/my-topic"); + client.newProducer().topic("persistent://sample/standalone/ns/my-topic").create(); client.close(); } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/MessageIdTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/MessageIdTest.java index 187803db435b8..82a830ff9d90d 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/MessageIdTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/MessageIdTest.java @@ -18,19 +18,18 @@ */ package org.apache.pulsar.client.impl; -import static org.mockito.Matchers.any; -import static org.mockito.Matchers.anyLong; import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.doThrow; import static org.mockito.Mockito.spy; -import static org.mockito.Mockito.times; -import static org.mockito.Mockito.verify; import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertFalse; import static org.testng.Assert.assertTrue; import static org.testng.Assert.fail; +import io.netty.buffer.ByteBuf; +import io.netty.buffer.Unpooled; + import java.lang.reflect.Field; import java.util.ArrayList; import java.util.HashSet; @@ -44,11 +43,11 @@ import org.apache.pulsar.client.admin.PulsarAdminException; import org.apache.pulsar.client.api.Consumer; import org.apache.pulsar.client.api.Message; -import org.apache.pulsar.client.api.MessageBuilder; import org.apache.pulsar.client.api.MessageId; import org.apache.pulsar.client.api.MessageRoutingMode; import org.apache.pulsar.client.api.Producer; import org.apache.pulsar.client.api.PulsarClientException; +import org.apache.pulsar.client.api.TypedMessageBuilder; import org.apache.pulsar.client.impl.ProducerImpl.OpSendMsg; import org.apache.pulsar.client.impl.conf.ClientConfigurationData; import org.apache.pulsar.common.api.ByteBufPair; @@ -63,8 +62,6 @@ import org.testng.annotations.BeforeMethod; import org.testng.annotations.Test; -import io.netty.buffer.ByteBuf; - public class MessageIdTest extends BrokerTestBase { private static final Logger log = LoggerFactory.getLogger(MessageIdTest.class); @@ -316,14 +313,15 @@ public void testChecksumVersionComptability() throws Exception { doReturn(producer.brokerChecksumSupportedVersion() - 1).when(mockClientCnx).getRemoteEndpointProtocolVersion(); prod.setClientCnx(mockClientCnx); - Message msg1 = MessageBuilder.create().setContent("message-1".getBytes()).build(); - CompletableFuture future1 = producer.sendAsync(msg1); + CompletableFuture future1 = producer.sendAsync("message-1".getBytes()); - Message msg2 = MessageBuilder.create().setContent("message-2".getBytes()).build(); - CompletableFuture future2 = producer.sendAsync(msg2); + byte[] a2 = "message-2".getBytes(); + TypedMessageBuilder msg2 = producer.newMessage().value(a2); - // corrupt the message - msg2.getData()[msg2.getData().length - 1] = '3'; // new content would be 'message-3' + CompletableFuture future2 = msg2.sendAsync(); + + // corrupt the message, new content would be 'message-3' + ((TypedMessageBuilderImpl) msg2).getContent().put(a2.length - 1, (byte) '3'); prod.setClientCnx(null); @@ -384,15 +382,16 @@ public void testChecksumReconnection() throws Exception { doReturn(producer.brokerChecksumSupportedVersion() - 1).when(mockClientCnx).getRemoteEndpointProtocolVersion(); prod.setClientCnx(mockClientCnx); - Message msg1 = MessageBuilder.create().setContent("message-1".getBytes()).build(); - CompletableFuture future1 = producer.sendAsync(msg1); + CompletableFuture future1 = producer.sendAsync("message-1".getBytes()); + + byte[] a2 = "message-2".getBytes(); + TypedMessageBuilder msg2 = producer.newMessage().value(a2); + + CompletableFuture future2 = msg2.sendAsync(); - Message msg2 = MessageBuilder.create().setContent("message-2".getBytes()).build(); - CompletableFuture future2 = producer.sendAsync(msg2); + // corrupt the message, new content would be 'message-3' + ((TypedMessageBuilderImpl) msg2).getContent().put(a2.length - 1, (byte) '3'); - // corrupt the message - msg2.getData()[msg2.getData().length - 1] = '3'; // new content would be - // 'message-3' // unset mock prod.setClientCnx(null); @@ -456,11 +455,13 @@ public void testCorruptMessageRemove() throws Exception { // enable checksum at producer stopBroker(); - Message msg = MessageBuilder.create().setContent("message-1".getBytes()).build(); - CompletableFuture future = producer.sendAsync(msg); + byte[] a = "message-1".getBytes(); + TypedMessageBuilder msg = producer.newMessage().value(a); - // 3. corrupt the message - msg.getData()[msg.getData().length - 1] = '2'; // new content would be 'message-3' + CompletableFuture future = msg.sendAsync(); + + // corrupt the message, new content would be 'message-2' + ((TypedMessageBuilderImpl) msg).getContent().put(a.length - 1, (byte) '2'); // 4. Restart the broker to have the messages published startBroker(); @@ -480,19 +481,21 @@ public void testCorruptMessageRemove() throws Exception { /** * verify: ProducerImpl.verifyLocalBufferIsNotCorrupted() => validates if message is corrupt */ - MessageImpl msg2 = (MessageImpl) MessageBuilder.create().setContent("message-1".getBytes()) - .build(); - ByteBuf payload = msg2.getDataBuffer(); - Builder metadataBuilder = ((MessageImpl) msg).getMessageBuilder(); + byte[] a2 = "message-2".getBytes(); + + TypedMessageBuilderImpl msg2 = (TypedMessageBuilderImpl) producer.newMessage().value("message-1".getBytes()); + ByteBuf payload = Unpooled.wrappedBuffer(msg2.getContent()); + Builder metadataBuilder = ((TypedMessageBuilderImpl) msg).getMetadataBuilder(); MessageMetadata msgMetadata = metadataBuilder.setProducerName("test").setSequenceId(1).setPublishTime(10L) .build(); ByteBufPair cmd = Commands.newSend(producerId, 1, 1, ChecksumType.Crc32c, msgMetadata, payload); // (a) create OpSendMsg with message-data : "message-1" - OpSendMsg op = OpSendMsg.create(((MessageImpl) msg), cmd, 1, null); + OpSendMsg op = OpSendMsg.create(((MessageImpl) msg2.getMessage()), cmd, 1, null); // a.verify: as message is not corrupt: no need to update checksum assertTrue(producer.verifyLocalBufferIsNotCorrupted(op)); + // (b) corrupt message - msg2.getData()[msg2.getData().length - 1] = '2'; // new content would be 'message-2' + msg2.getContent().put(a2.length - 1, (byte) '2'); // new content would be 'message-2' // b. verify: as message is corrupt: update checksum assertFalse(producer.verifyLocalBufferIsNotCorrupted(op)); @@ -500,9 +503,9 @@ public void testCorruptMessageRemove() throws Exception { // [2] test-recoverChecksumError functionality stopBroker(); - MessageImpl msg1 = (MessageImpl) MessageBuilder.create().setContent("message-1".getBytes()) - .build(); - future = producer.sendAsync(msg1); + + TypedMessageBuilderImpl msg1 = (TypedMessageBuilderImpl) producer.newMessage().value("message-1".getBytes()); + future = msg1.sendAsync(); ClientCnx cnx = spy( new ClientCnx(new ClientConfigurationData(), ((PulsarClientImpl) pulsarClient).eventLoopGroup())); String exc = "broker is already stopped"; diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/compaction/CompactionTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/compaction/CompactionTest.java index 8e7b292d6a228..1928a32b20faf 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/compaction/CompactionTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/compaction/CompactionTest.java @@ -21,6 +21,8 @@ import static org.mockito.Matchers.anyLong; import static org.mockito.Mockito.spy; import static org.mockito.Mockito.when; +import static org.testng.Assert.assertNull; +import static org.testng.Assert.assertTrue; import com.google.common.collect.Sets; import com.google.common.util.concurrent.ThreadFactoryBuilder; @@ -49,7 +51,6 @@ import org.apache.pulsar.client.api.CryptoKeyReader; import org.apache.pulsar.client.api.EncryptionKeyInfo; import org.apache.pulsar.client.api.Message; -import org.apache.pulsar.client.api.MessageBuilder; import org.apache.pulsar.client.api.MessageId; import org.apache.pulsar.client.api.MessageRoutingMode; import org.apache.pulsar.client.api.Producer; @@ -60,9 +61,6 @@ import org.apache.pulsar.common.policies.data.RetentionPolicies; import org.apache.pulsar.common.policies.data.TenantInfo; import org.testng.Assert; - -import static org.testng.Assert.assertNull; -import static org.testng.Assert.assertTrue; import org.testng.annotations.AfterMethod; import org.testng.annotations.BeforeMethod; import org.testng.annotations.Test; @@ -614,41 +612,50 @@ public void testEmptyPayloadDeletes() throws Exception { .create()) { // key0 persists through it all - producerNormal.sendAsync(MessageBuilder.create() - .setKey("key0") - .setContent("my-message-0".getBytes()).build()).get(); + producerNormal.newMessage() + .key("key0") + .value("my-message-0".getBytes()) + .send(); // key1 is added but then deleted - producerNormal.sendAsync(MessageBuilder.create() - .setKey("key1") - .setContent("my-message-1".getBytes()).build()).get(); + producerNormal.newMessage() + .key("key1") + .value("my-message-1".getBytes()) + .send(); - producerNormal.sendAsync(MessageBuilder.create() - .setKey("key1").build()).get(); + producerNormal.newMessage() + .key("key1") + .send(); // key2 is added but deleted in same batch - producerBatch.sendAsync(MessageBuilder.create() - .setKey("key2") - .setContent("my-message-2".getBytes()).build()); - producerBatch.sendAsync(MessageBuilder.create() - .setKey("key3") - .setContent("my-message-3".getBytes()).build()); - producerBatch.sendAsync(MessageBuilder.create() - .setKey("key2").build()).get(); + producerBatch.newMessage() + .key("key2") + .value("my-message-2".getBytes()) + .sendAsync(); + producerBatch.newMessage() + .key("key3") + .value("my-message-3".getBytes()) + .sendAsync(); + producerBatch.newMessage() + .key("key2").send(); // key3 is added in previous batch, deleted in this batch - producerBatch.sendAsync(MessageBuilder.create() - .setKey("key3").build()); - producerBatch.sendAsync(MessageBuilder.create() - .setKey("key4") - .setContent("my-message-3".getBytes()).build()); - producerBatch.sendAsync(MessageBuilder.create() - .setKey("key4").build()).get(); + producerBatch.newMessage() + .key("key3") + .sendAsync(); + producerBatch.newMessage() + .key("key4") + .value("my-message-3".getBytes()) + .sendAsync(); + producerBatch.newMessage() + .key("key4") + .send(); // key4 is added, deleted, then resurrected - producerNormal.sendAsync(MessageBuilder.create() - .setKey("key4") - .setContent("my-message-4".getBytes()).build()).get(); + producerNormal.newMessage() + .key("key4") + .value("my-message-4".getBytes()) + .send(); } // compact the topic @@ -690,41 +697,45 @@ public void testEmptyPayloadDeletesWhenCompressed() throws Exception { .create()) { // key0 persists through it all - producerNormal.sendAsync(MessageBuilder.create() - .setKey("key0") - .setContent("my-message-0".getBytes()).build()).get(); + producerNormal.newMessage() + .key("key0") + .value("my-message-0".getBytes()).send(); // key1 is added but then deleted - producerNormal.sendAsync(MessageBuilder.create() - .setKey("key1") - .setContent("my-message-1".getBytes()).build()).get(); + producerNormal.newMessage() + .key("key1") + .value("my-message-1".getBytes()).send(); - producerNormal.sendAsync(MessageBuilder.create() - .setKey("key1").build()).get(); + producerNormal.newMessage() + .key("key1").send(); // key2 is added but deleted in same batch - producerBatch.sendAsync(MessageBuilder.create() - .setKey("key2") - .setContent("my-message-2".getBytes()).build()); - producerBatch.sendAsync(MessageBuilder.create() - .setKey("key3") - .setContent("my-message-3".getBytes()).build()); - producerBatch.sendAsync(MessageBuilder.create() - .setKey("key2").build()).get(); + producerBatch.newMessage() + .key("key2") + .value("my-message-2".getBytes()).sendAsync(); + producerBatch.newMessage() + .key("key3") + .value("my-message-3".getBytes()).sendAsync(); + producerBatch.newMessage() + .key("key2").send(); // key3 is added in previous batch, deleted in this batch - producerBatch.sendAsync(MessageBuilder.create() - .setKey("key3").build()); - producerBatch.sendAsync(MessageBuilder.create() - .setKey("key4") - .setContent("my-message-3".getBytes()).build()); - producerBatch.sendAsync(MessageBuilder.create() - .setKey("key4").build()).get(); + producerBatch.newMessage() + .key("key3") + .sendAsync(); + producerBatch.newMessage() + .key("key4") + .value("my-message-3".getBytes()) + .sendAsync(); + producerBatch.newMessage() + .key("key4") + .send(); // key4 is added, deleted, then resurrected - producerNormal.sendAsync(MessageBuilder.create() - .setKey("key4") - .setContent("my-message-4".getBytes()).build()).get(); + producerNormal.newMessage() + .key("key4") + .value("my-message-4".getBytes()) + .send(); } // compact the topic @@ -767,9 +778,10 @@ public void testCompactorReadsCompacted() throws Exception { // create the topic on the broker try (Producer producerNormal = pulsarClient.newProducer().topic(topic).create()) { - producerNormal.send(MessageBuilder.create() - .setKey("key0") - .setContent("my-message-0".getBytes()).build()); + producerNormal.newMessage() + .key("key0") + .value("my-message-0".getBytes()) + .send(); } // force ledger roll @@ -777,9 +789,10 @@ public void testCompactorReadsCompacted() throws Exception { // write a message to avoid issue #1517 try (Producer producerNormal = pulsarClient.newProducer().topic(topic).create()) { - producerNormal.send(MessageBuilder.create() - .setKey("key1") - .setContent("my-message-1".getBytes()).build()); + producerNormal.newMessage() + .key("key1") + .value("my-message-1".getBytes()) + .send(); } // verify second ledger created @@ -803,9 +816,10 @@ public void testCompactorReadsCompacted() throws Exception { // write a message to avoid issue #1517 try (Producer producerNormal = pulsarClient.newProducer().topic(topic).create()) { - producerNormal.send(MessageBuilder.create() - .setKey("key2") - .setContent("my-message-2".getBytes()).build()); + producerNormal.newMessage() + .key("key2") + .value("my-message-2".getBytes()) + .send(); } info = pulsar.getManagedLedgerFactory().getManagedLedgerInfo(managedLedgerName); @@ -853,15 +867,18 @@ public void testCompactCompressedNoBatch() throws Exception { try (Producer producer = pulsarClient.newProducer().topic(topic) .compressionType(CompressionType.LZ4).enableBatching(false).create()) { - producer.sendAsync(MessageBuilder.create() - .setKey("key1") - .setContent("my-message-1".getBytes()).build()); - producer.sendAsync(MessageBuilder.create() - .setKey("key2") - .setContent("my-message-2".getBytes()).build()); - producer.sendAsync(MessageBuilder.create() - .setKey("key2") - .setContent("my-message-3".getBytes()).build()).get(); + producer.newMessage() + .key("key1") + .value("my-message-1".getBytes()) + .sendAsync(); + producer.newMessage() + .key("key2") + .value("my-message-2".getBytes()) + .sendAsync(); + producer.newMessage() + .key("key2") + .value("my-message-3".getBytes()) + .send(); } // compact the topic @@ -894,15 +911,18 @@ public void testCompactCompressedBatching() throws Exception { .enableBatching(true) .batchingMaxMessages(3) .batchingMaxPublishDelay(1, TimeUnit.HOURS).create()) { - producer.sendAsync(MessageBuilder.create() - .setKey("key1") - .setContent("my-message-1".getBytes()).build()); - producer.sendAsync(MessageBuilder.create() - .setKey("key2") - .setContent("my-message-2".getBytes()).build()); - producer.sendAsync(MessageBuilder.create() - .setKey("key2") - .setContent("my-message-3".getBytes()).build()).get(); + producer.newMessage() + .key("key1") + .value("my-message-1".getBytes()) + .sendAsync(); + producer.newMessage() + .key("key2") + .value("my-message-2".getBytes()) + .sendAsync(); + producer.newMessage() + .key("key2") + .value("my-message-3".getBytes()) + .send(); } // compact the topic @@ -968,15 +988,18 @@ public void testCompactEncryptedNoBatch() throws Exception { try (Producer producer = pulsarClient.newProducer().topic(topic) .addEncryptionKey("client-ecdsa.pem").cryptoKeyReader(new EncKeyReader()) .enableBatching(false).create()) { - producer.sendAsync(MessageBuilder.create() - .setKey("key1") - .setContent("my-message-1".getBytes()).build()); - producer.sendAsync(MessageBuilder.create() - .setKey("key2") - .setContent("my-message-2".getBytes()).build()); - producer.sendAsync(MessageBuilder.create() - .setKey("key2") - .setContent("my-message-3".getBytes()).build()).get(); + producer.newMessage() + .key("key1") + .value("my-message-1".getBytes()) + .sendAsync(); + producer.newMessage() + .key("key2") + .value("my-message-2".getBytes()) + .sendAsync(); + producer.newMessage() + .key("key2") + .value("my-message-3".getBytes()) + .send(); } // compact the topic @@ -1011,15 +1034,18 @@ public void testCompactEncryptedBatching() throws Exception { .enableBatching(true) .batchingMaxMessages(3) .batchingMaxPublishDelay(1, TimeUnit.HOURS).create()) { - producer.sendAsync(MessageBuilder.create() - .setKey("key1") - .setContent("my-message-1".getBytes()).build()); - producer.sendAsync(MessageBuilder.create() - .setKey("key2") - .setContent("my-message-2".getBytes()).build()); - producer.sendAsync(MessageBuilder.create() - .setKey("key2") - .setContent("my-message-3".getBytes()).build()).get(); + producer.newMessage() + .key("key1") + .value("my-message-1".getBytes()) + .sendAsync(); + producer.newMessage() + .key("key2") + .value("my-message-2".getBytes()) + .sendAsync(); + producer.newMessage() + .key("key2") + .value("my-message-3".getBytes()) + .send(); } // compact the topic @@ -1057,15 +1083,18 @@ public void testCompactEncryptedAndCompressedNoBatch() throws Exception { .addEncryptionKey("client-ecdsa.pem").cryptoKeyReader(new EncKeyReader()) .compressionType(CompressionType.LZ4) .enableBatching(false).create()) { - producer.sendAsync(MessageBuilder.create() - .setKey("key1") - .setContent("my-message-1".getBytes()).build()); - producer.sendAsync(MessageBuilder.create() - .setKey("key2") - .setContent("my-message-2".getBytes()).build()); - producer.sendAsync(MessageBuilder.create() - .setKey("key2") - .setContent("my-message-3".getBytes()).build()).get(); + producer.newMessage() + .key("key1") + .value("my-message-1".getBytes()) + .sendAsync(); + producer.newMessage() + .key("key2") + .value("my-message-2".getBytes()) + .sendAsync(); + producer.newMessage() + .key("key2") + .value("my-message-3".getBytes()) + .send(); } // compact the topic @@ -1101,15 +1130,18 @@ public void testCompactEncryptedAndCompressedBatching() throws Exception { .enableBatching(true) .batchingMaxMessages(3) .batchingMaxPublishDelay(1, TimeUnit.HOURS).create()) { - producer.sendAsync(MessageBuilder.create() - .setKey("key1") - .setContent("my-message-1".getBytes()).build()); - producer.sendAsync(MessageBuilder.create() - .setKey("key2") - .setContent("my-message-2".getBytes()).build()); - producer.sendAsync(MessageBuilder.create() - .setKey("key2") - .setContent("my-message-3".getBytes()).build()).get(); + producer.newMessage() + .key("key1") + .value("my-message-1".getBytes()) + .sendAsync(); + producer.newMessage() + .key("key2") + .value("my-message-2".getBytes()) + .sendAsync(); + producer.newMessage() + .key("key2") + .value("my-message-3".getBytes()) + .send(); } // compact the topic @@ -1158,32 +1190,33 @@ public void testEmptyPayloadDeletesWhenEncrypted() throws Exception { .create()) { // key0 persists through it all - producerNormal.sendAsync(MessageBuilder.create() - .setKey("key0") - .setContent("my-message-0".getBytes()).build()).get(); + producerNormal.newMessage() + .key("key0") + .value("my-message-0".getBytes()).send(); // key1 is added but then deleted - producerNormal.sendAsync(MessageBuilder.create() - .setKey("key1") - .setContent("my-message-1".getBytes()).build()).get(); + producerNormal.newMessage() + .key("key1") + .value("my-message-1".getBytes()).send(); - producerNormal.sendAsync(MessageBuilder.create() - .setKey("key1").build()).get(); + producerNormal.newMessage() + .key("key1") + .send(); // key2 is added but deleted in same batch - producerBatch.sendAsync(MessageBuilder.create() - .setKey("key2") - .setContent("my-message-2".getBytes()).build()); - producerBatch.sendAsync(MessageBuilder.create() - .setKey("key3") - .setContent("my-message-3".getBytes()).build()); - producerBatch.sendAsync(MessageBuilder.create() - .setKey("key2").build()).get(); + producerBatch.newMessage() + .key("key2") + .value("my-message-2".getBytes()).sendAsync(); + producerBatch.newMessage() + .key("key3") + .value("my-message-3".getBytes()).sendAsync(); + producerBatch.newMessage() + .key("key2").send(); // key4 is added, deleted, then resurrected - producerNormal.sendAsync(MessageBuilder.create() - .setKey("key4") - .setContent("my-message-4".getBytes()).build()).get(); + producerNormal.newMessage() + .key("key4") + .value("my-message-4".getBytes()).send(); } // compact the topic @@ -1215,7 +1248,7 @@ public void testEmptyPayloadDeletesWhenEncrypted() throws Exception { Assert.assertEquals(new String(message5.getData()), "my-message-4"); } } - + @Test(timeOut = 20000) public void testCompactionWithLastDeletedKey() throws Exception { String topic = "persistent://my-property/use/my-ns/my-topic1"; @@ -1242,7 +1275,7 @@ public void testCompactionWithLastDeletedKey() throws Exception { assertTrue(expected.remove(m.getKey())); } } - + @Test(timeOut = 20000) public void testEmptyCompactionLedger() throws Exception { String topic = "persistent://my-property/use/my-ns/my-topic1"; diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/compaction/CompactorTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/compaction/CompactorTest.java index 7bd5adc8c9b58..32765ba0e87f7 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/compaction/CompactorTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/compaction/CompactorTest.java @@ -20,6 +20,12 @@ import static org.apache.pulsar.client.impl.RawReaderTest.extractKey; +import com.google.common.collect.Lists; +import com.google.common.collect.Sets; +import com.google.common.util.concurrent.ThreadFactoryBuilder; + +import io.netty.buffer.ByteBuf; + import java.util.ArrayList; import java.util.Enumeration; import java.util.HashMap; @@ -34,7 +40,6 @@ import org.apache.bookkeeper.client.LedgerEntry; import org.apache.bookkeeper.client.LedgerHandle; import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest; -import org.apache.pulsar.client.api.MessageBuilder; import org.apache.pulsar.client.api.MessageRoutingMode; import org.apache.pulsar.client.api.Producer; import org.apache.pulsar.client.api.RawMessage; @@ -47,12 +52,6 @@ import org.testng.annotations.BeforeMethod; import org.testng.annotations.Test; -import com.google.common.collect.Lists; -import com.google.common.collect.Sets; -import com.google.common.util.concurrent.ThreadFactoryBuilder; - -import io.netty.buffer.ByteBuf; - public class CompactorTest extends MockedPulsarServiceBaseTest { private ScheduledExecutorService compactionScheduler; @@ -130,9 +129,10 @@ public void testCompaction() throws Exception { int keyIndex = r.nextInt(maxKeys); String key = "key"+keyIndex; byte[] data = ("my-message-" + key + "-" + j).getBytes(); - producer.send(MessageBuilder.create() - .setKey(key) - .setContent(data).build()); + producer.newMessage() + .key(key) + .value(data) + .send(); expected.put(key, data); } compactAndVerify(topic, expected); @@ -149,23 +149,27 @@ public void testCompactAddCompact() throws Exception { Map expected = new HashMap<>(); - producer.send(MessageBuilder.create() - .setKey("a") - .setContent("A_1".getBytes()).build()); - producer.send(MessageBuilder.create() - .setKey("b") - .setContent("B_1".getBytes()).build()); - producer.send(MessageBuilder.create() - .setKey("a") - .setContent("A_2".getBytes()).build()); + producer.newMessage() + .key("a") + .value("A_1".getBytes()) + .send(); + producer.newMessage() + .key("b") + .value("B_1".getBytes()) + .send(); + producer.newMessage() + .key("a") + .value("A_2".getBytes()) + .send(); expected.put("a", "A_2".getBytes()); expected.put("b", "B_1".getBytes()); compactAndVerify(topic, new HashMap<>(expected)); - producer.send(MessageBuilder.create() - .setKey("b") - .setContent("B_2".getBytes()).build()); + producer.newMessage() + .key("b") + .value("B_2".getBytes()) + .send(); expected.put("b", "B_2".getBytes()); compactAndVerify(topic, expected); @@ -180,18 +184,18 @@ public void testCompactedInOrder() throws Exception { .messageRoutingMode(MessageRoutingMode.SinglePartition) .create(); - producer.send(MessageBuilder.create() - .setKey("c") - .setContent("C_1".getBytes()).build()); - producer.send(MessageBuilder.create() - .setKey("a") - .setContent("A_1".getBytes()).build()); - producer.send(MessageBuilder.create() - .setKey("b") - .setContent("B_1".getBytes()).build()); - producer.send(MessageBuilder.create() - .setKey("a") - .setContent("A_2".getBytes()).build()); + producer.newMessage() + .key("c") + .value("C_1".getBytes()).send(); + producer.newMessage() + .key("a") + .value("A_1".getBytes()).send(); + producer.newMessage() + .key("b") + .value("B_1".getBytes()).send(); + producer.newMessage() + .key("a") + .value("A_2".getBytes()).send(); Map expected = new HashMap<>(); expected.put("a", "A_2".getBytes()); expected.put("b", "B_1".getBytes()); diff --git a/pulsar-client-1x-base/pulsar-client-1x/src/main/java/org/apache/pulsar/client/api/ConsumerConfiguration.java b/pulsar-client-1x-base/pulsar-client-1x/src/main/java/org/apache/pulsar/client/api/ConsumerConfiguration.java index 5ffba38ffceeb..ec7e3972d8d43 100644 --- a/pulsar-client-1x-base/pulsar-client-1x/src/main/java/org/apache/pulsar/client/api/ConsumerConfiguration.java +++ b/pulsar-client-1x-base/pulsar-client-1x/src/main/java/org/apache/pulsar/client/api/ConsumerConfiguration.java @@ -26,7 +26,7 @@ import java.util.concurrent.TimeUnit; import org.apache.pulsar.client.impl.conf.ConsumerConfigurationData; -import org.apache.pulsar.client.api.SubscriptionInitialPosition; +import org.apache.pulsar.client.impl.v1.ConsumerV1Impl; /** * Class specifying the configuration of a consumer. In Exclusive subscription, only a single consumer is allowed to * attach to the subscription. Other consumers will get an error message. In Shared subscription, multiple consumers @@ -48,6 +48,8 @@ public class ConsumerConfiguration implements Serializable { private boolean initializeSubscriptionOnLatest = true; + private MessageListener messageListener; + public ConsumerConfiguration() { // Disable acknowledgment grouping when using v1 API conf.setAcknowledgementsGroupTimeMicros(0); @@ -103,7 +105,7 @@ public ConsumerConfiguration setSubscriptionType(SubscriptionType subscriptionTy * @return the configured {@link MessageListener} for the consumer */ public MessageListener getMessageListener() { - return conf.getMessageListener(); + return messageListener; } /** @@ -117,7 +119,19 @@ public MessageListener getMessageListener() { */ public ConsumerConfiguration setMessageListener(MessageListener messageListener) { checkNotNull(messageListener); - conf.setMessageListener(messageListener); + this.messageListener = messageListener; + conf.setMessageListener(new org.apache.pulsar.shade.client.api.v2.MessageListener() { + + @Override + public void received(org.apache.pulsar.shade.client.api.v2.Consumer consumer, Message msg) { + messageListener.received(new ConsumerV1Impl(consumer), msg); + } + + @Override + public void reachedEndOfTopic(org.apache.pulsar.shade.client.api.v2.Consumer consumer) { + messageListener.reachedEndOfTopic(new ConsumerV1Impl(consumer)); + } + }); return this; } diff --git a/pulsar-client-1x-base/pulsar-client-1x/src/main/java/org/apache/pulsar/client/api/MessageListener.java b/pulsar-client-1x-base/pulsar-client-1x/src/main/java/org/apache/pulsar/client/api/MessageListener.java new file mode 100644 index 0000000000000..42def5dbd28f2 --- /dev/null +++ b/pulsar-client-1x-base/pulsar-client-1x/src/main/java/org/apache/pulsar/client/api/MessageListener.java @@ -0,0 +1,54 @@ +/** + * 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 java.io.Serializable; + +/** + * A listener that will be called in order for every message received. + * + * + */ +public interface MessageListener extends Serializable { + /** + * This method is called whenever a new message is received. + * + * Messages are guaranteed to be delivered in order and from the same thread for a single consumer + * + * This method will only be called once for each message, unless either application or broker crashes. + * + * Application is responsible of handling any exception that could be thrown while processing the message. + * + * @param consumer + * the consumer that received the message + * @param msg + * the message object + */ + void received(Consumer consumer, Message msg); + + /** + * Get the notification when a topic is terminated + * + * @param consumer + * the Consumer object associated with the terminated topic + */ + default void reachedEndOfTopic(Consumer consumer) { + // By default ignore the notification + } +} diff --git a/pulsar-client-1x-base/pulsar-client-1x/src/main/java/org/apache/pulsar/client/api/ReaderConfiguration.java b/pulsar-client-1x-base/pulsar-client-1x/src/main/java/org/apache/pulsar/client/api/ReaderConfiguration.java index 243166cbf8071..2f2cab56bb8c4 100644 --- a/pulsar-client-1x-base/pulsar-client-1x/src/main/java/org/apache/pulsar/client/api/ReaderConfiguration.java +++ b/pulsar-client-1x-base/pulsar-client-1x/src/main/java/org/apache/pulsar/client/api/ReaderConfiguration.java @@ -26,6 +26,7 @@ import java.io.Serializable; import org.apache.pulsar.client.impl.conf.ReaderConfigurationData; +import org.apache.pulsar.client.impl.v1.ReaderV1Impl; /** * @@ -36,11 +37,13 @@ public class ReaderConfiguration implements Serializable { private final ReaderConfigurationData conf = new ReaderConfigurationData<>(); + private ReaderListener readerListener; + /** * @return the configured {@link ReaderListener} for the reader */ public ReaderListener getReaderListener() { - return conf.getReaderListener(); + return readerListener; } /** @@ -54,7 +57,19 @@ public ReaderListener getReaderListener() { */ public ReaderConfiguration setReaderListener(ReaderListener readerListener) { checkNotNull(readerListener); - conf.setReaderListener(readerListener); + this.readerListener = readerListener; + conf.setReaderListener(new org.apache.pulsar.shade.client.api.v2.ReaderListener() { + + @Override + public void received(org.apache.pulsar.shade.client.api.v2.Reader v2Reader, Message msg) { + readerListener.received(new ReaderV1Impl(v2Reader), msg); + } + + @Override + public void reachedEndOfTopic(org.apache.pulsar.shade.client.api.v2.Reader reader) { + readerListener.reachedEndOfTopic(new ReaderV1Impl(reader)); + } + }); return this; } @@ -86,7 +101,7 @@ public ReaderConfiguration setCryptoKeyReader(CryptoKeyReader cryptoKeyReader) { /** * Sets the ConsumerCryptoFailureAction to the value specified - * + * * @param action * The action to take when the decoding fails */ diff --git a/pulsar-client-1x-base/pulsar-client-1x/src/main/java/org/apache/pulsar/client/api/ReaderListener.java b/pulsar-client-1x-base/pulsar-client-1x/src/main/java/org/apache/pulsar/client/api/ReaderListener.java new file mode 100644 index 0000000000000..50677b27dfae3 --- /dev/null +++ b/pulsar-client-1x-base/pulsar-client-1x/src/main/java/org/apache/pulsar/client/api/ReaderListener.java @@ -0,0 +1,52 @@ +/** + * 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 java.io.Serializable; + +/** + * A listener that will be called in order for every message received. + */ +public interface ReaderListener extends Serializable { + /** + * This method is called whenever a new message is received. + * + * Messages are guaranteed to be delivered in order and from the same thread for a single consumer + * + * This method will only be called once for each message, unless either application or broker crashes. + * + * Application is responsible of handling any exception that could be thrown while processing the message. + * + * @param reader + * the Reader object from where the message was received + * @param msg + * the message object + */ + void received(Reader reader, Message msg); + + /** + * Get the notification when a topic is terminated + * + * @param reader + * the Reader object associated with the terminated topic + */ + default void reachedEndOfTopic(Reader reader) { + // By default ignore the notification + } +} diff --git a/pulsar-client-1x-base/pulsar-client-2x-shaded/pom.xml b/pulsar-client-1x-base/pulsar-client-2x-shaded/pom.xml index 6b6b2bc35aaad..21a4b994a1a61 100644 --- a/pulsar-client-1x-base/pulsar-client-2x-shaded/pom.xml +++ b/pulsar-client-1x-base/pulsar-client-2x-shaded/pom.xml @@ -79,6 +79,8 @@ org.apache.pulsar.client.api.Producer org.apache.pulsar.client.api.Consumer org.apache.pulsar.client.api.Reader + org.apache.pulsar.client.api.MessageListener + org.apache.pulsar.client.api.ReaderListener diff --git a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/PulsarAdmin.java b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/PulsarAdmin.java index 0953d22cc2a91..026decac863b2 100644 --- a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/PulsarAdmin.java +++ b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/PulsarAdmin.java @@ -32,11 +32,26 @@ import org.apache.commons.lang3.StringUtils; import org.apache.http.conn.ssl.DefaultHostnameVerifier; import org.apache.http.conn.ssl.NoopHostnameVerifier; -import org.apache.pulsar.client.admin.internal.*; +import org.apache.pulsar.client.admin.internal.BookiesImpl; +import org.apache.pulsar.client.admin.internal.BrokerStatsImpl; +import org.apache.pulsar.client.admin.internal.BrokersImpl; +import org.apache.pulsar.client.admin.internal.ClustersImpl; +import org.apache.pulsar.client.admin.internal.FunctionsImpl; +import org.apache.pulsar.client.admin.internal.JacksonConfigurator; +import org.apache.pulsar.client.admin.internal.LookupImpl; +import org.apache.pulsar.client.admin.internal.NamespacesImpl; +import org.apache.pulsar.client.admin.internal.NonPersistentTopicsImpl; +import org.apache.pulsar.client.admin.internal.PulsarAdminBuilderImpl; +import org.apache.pulsar.client.admin.internal.ResourceQuotasImpl; +import org.apache.pulsar.client.admin.internal.SchemasImpl; +import org.apache.pulsar.client.admin.internal.SinkImpl; +import org.apache.pulsar.client.admin.internal.SourceImpl; +import org.apache.pulsar.client.admin.internal.TenantsImpl; +import org.apache.pulsar.client.admin.internal.TopicsImpl; +import org.apache.pulsar.client.admin.internal.WorkerImpl; import org.apache.pulsar.client.api.Authentication; import org.apache.pulsar.client.api.AuthenticationDataProvider; import org.apache.pulsar.client.api.AuthenticationFactory; -import org.apache.pulsar.client.api.ClientConfiguration; import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.client.impl.auth.AuthenticationDisabled; import org.apache.pulsar.client.impl.conf.ClientConfigurationData; @@ -184,22 +199,6 @@ public PulsarAdmin(String serviceUrl, ClientConfigurationData clientConfigData) this.bookies = new BookiesImpl(root, auth); } - /** - * Construct a new Pulsar Admin client object. - *

- * This client object can be used to perform many subsquent API calls - * - * @param serviceUrl - * the Pulsar service URL (eg. "http://my-broker.example.com:8080") - * @param pulsarConfig - * the ClientConfiguration object to be used to talk with Pulsar - * @deprecated Since 2.0. Use {@link #builder()} to construct a new {@link PulsarAdmin} instance. - */ - @Deprecated - public PulsarAdmin(URL serviceUrl, ClientConfiguration pulsarConfig) throws PulsarClientException { - this(serviceUrl.toString(), pulsarConfig.getConfigurationData()); - } - /** * Construct a new Pulsar Admin client object. *

@@ -213,12 +212,13 @@ public PulsarAdmin(URL serviceUrl, ClientConfiguration pulsarConfig) throws Puls */ @Deprecated public PulsarAdmin(URL serviceUrl, Authentication auth) throws PulsarClientException { - this(serviceUrl, new ClientConfiguration() { - private static final long serialVersionUID = 1L; - { - setAuthentication(auth); - } - }); + this(serviceUrl.toString(), getConfigData(auth)); + } + + private static ClientConfigurationData getConfigData(Authentication auth) { + ClientConfigurationData conf = new ClientConfigurationData(); + conf.setAuthentication(auth); + return conf; } /** @@ -370,7 +370,7 @@ public Sink sink() { public Worker worker() { return worker; } - + /** * @return the broker statics */ diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/api/ClientConfiguration.java b/pulsar-client/src/main/java/org/apache/pulsar/client/api/ClientConfiguration.java deleted file mode 100644 index e5bac914b7865..0000000000000 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/api/ClientConfiguration.java +++ /dev/null @@ -1,390 +0,0 @@ -/** - * 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 com.google.common.base.Preconditions.checkArgument; - -import java.io.Serializable; -import java.util.Map; -import java.util.concurrent.TimeUnit; - -import org.apache.pulsar.client.api.PulsarClientException.UnsupportedAuthenticationException; -import org.apache.pulsar.client.impl.conf.ClientConfigurationData; - -/** - * Class used to specify client side configuration like authentication, etc.. - * - * @deprecated Use {@link PulsarClient#builder()} to construct and configure a new {@link PulsarClient} instance - */ -@Deprecated -public class ClientConfiguration implements Serializable { - - private static final long serialVersionUID = 1L; - - private final ClientConfigurationData confData = new ClientConfigurationData(); - - /** - * @return the authentication provider to be used - */ - public Authentication getAuthentication() { - return confData.getAuthentication(); - } - - /** - * Set the authentication provider to use in the Pulsar client instance. - *

- * Example: - *

- * - *

-     * 
-     * ClientConfiguration confData = new ClientConfiguration();
-     * String authPluginClassName = "org.apache.pulsar.client.impl.auth.MyAuthentication";
-     * String authParamsString = "key1:val1,key2:val2";
-     * Authentication auth = AuthenticationFactory.create(authPluginClassName, authParamsString);
-     * confData.setAuthentication(auth);
-     * PulsarClient client = PulsarClient.create(serviceUrl, confData);
-     * ....
-     * 
-     * 
- * - * @param authentication - */ - public void setAuthentication(Authentication authentication) { - confData.setAuthentication(authentication); - } - - /** - * Set the authentication provider to use in the Pulsar client instance. - *

- * Example: - *

- * - *

-     * 
-     * ClientConfiguration confData = new ClientConfiguration();
-     * String authPluginClassName = "org.apache.pulsar.client.impl.auth.MyAuthentication";
-     * String authParamsString = "key1:val1,key2:val2";
-     * confData.setAuthentication(authPluginClassName, authParamsString);
-     * PulsarClient client = PulsarClient.create(serviceUrl, confData);
-     * ....
-     * 
-     * 
- * - * @param authPluginClassName - * name of the Authentication-Plugin you want to use - * @param authParamsString - * string which represents parameters for the Authentication-Plugin, e.g., "key1:val1,key2:val2" - * @throws UnsupportedAuthenticationException - * failed to instantiate specified Authentication-Plugin - */ - public void setAuthentication(String authPluginClassName, String authParamsString) - throws UnsupportedAuthenticationException { - confData.setAuthentication(AuthenticationFactory.create(authPluginClassName, authParamsString)); - } - - /** - * Set the authentication provider to use in the Pulsar client instance. - *

- * Example: - *

- * - *

-     * 
-     * ClientConfiguration confData = new ClientConfiguration();
-     * String authPluginClassName = "org.apache.pulsar.client.impl.auth.MyAuthentication";
-     * Map authParams = new HashMap();
-     * authParams.put("key1", "val1");
-     * confData.setAuthentication(authPluginClassName, authParams);
-     * PulsarClient client = PulsarClient.create(serviceUrl, confData);
-     * ....
-     * 
-     * 
- * - * @param authPluginClassName - * name of the Authentication-Plugin you want to use - * @param authParams - * map which represents parameters for the Authentication-Plugin - * @throws UnsupportedAuthenticationException - * failed to instantiate specified Authentication-Plugin - */ - public void setAuthentication(String authPluginClassName, Map authParams) - throws UnsupportedAuthenticationException { - confData.setAuthentication(AuthenticationFactory.create(authPluginClassName, authParams)); - } - - /** - * @return the operation timeout in ms - */ - public long getOperationTimeoutMs() { - return confData.getOperationTimeoutMs(); - } - - /** - * Set the operation timeout (default: 30 seconds) - *

- * Producer-create, subscribe and unsubscribe operations will be retried until this interval, after which the - * operation will be marked as failed - * - * @param operationTimeout - * operation timeout - * @param unit - * time unit for {@code operationTimeout} - */ - public void setOperationTimeout(int operationTimeout, TimeUnit unit) { - checkArgument(operationTimeout >= 0); - confData.setOperationTimeoutMs(unit.toMillis(operationTimeout)); - } - - /** - * @return the number of threads to use for handling connections - */ - public int getIoThreads() { - return confData.getNumIoThreads(); - } - - /** - * Set the number of threads to be used for handling connections to brokers (default: 1 thread) - * - * @param numIoThreads - */ - public void setIoThreads(int numIoThreads) { - checkArgument(numIoThreads > 0); - confData.setNumIoThreads(numIoThreads); - } - - /** - * @return the number of threads to use for message listeners - */ - public int getListenerThreads() { - return confData.getNumListenerThreads(); - } - - /** - * Set the number of threads to be used for message listeners (default: 1 thread) - * - * @param numListenerThreads - */ - public void setListenerThreads(int numListenerThreads) { - checkArgument(numListenerThreads > 0); - confData.setNumListenerThreads(numListenerThreads); - } - - /** - * @return the max number of connections per single broker - */ - public int getConnectionsPerBroker() { - return confData.getConnectionsPerBroker(); - } - - /** - * Sets the max number of connection that the client library will open to a single broker. - *

- * By default, the connection pool will use a single connection for all the producers and consumers. Increasing this - * parameter may improve throughput when using many producers over a high latency connection. - *

- * - * @param connectionsPerBroker - * max number of connections per broker (needs to be greater than 0) - */ - public void setConnectionsPerBroker(int connectionsPerBroker) { - checkArgument(connectionsPerBroker > 0, "Connections per broker need to be greater than 0"); - confData.setConnectionsPerBroker(connectionsPerBroker); - } - - /** - * @return whether TCP no-delay should be set on the connections - */ - public boolean isUseTcpNoDelay() { - return confData.isUseTcpNoDelay(); - } - - /** - * Configure whether to use TCP no-delay flag on the connection, to disable Nagle algorithm. - *

- * No-delay features make sure packets are sent out on the network as soon as possible, and it's critical to achieve - * low latency publishes. On the other hand, sending out a huge number of small packets might limit the overall - * throughput, so if latency is not a concern, it's advisable to set the useTcpNoDelay flag to false. - *

- * Default value is true - * - * @param useTcpNoDelay - */ - public void setUseTcpNoDelay(boolean useTcpNoDelay) { - confData.setUseTcpNoDelay(useTcpNoDelay); - } - - /** - * @return whether TLS encryption is used on the connection - */ - public boolean isUseTls() { - return confData.isUseTls(); - } - - /** - * Configure whether to use TLS encryption on the connection (default: false) - * - * @param useTls - */ - public void setUseTls(boolean useTls) { - confData.setUseTls(useTls); - } - - /** - * @return path to the trusted TLS certificate file - */ - public String getTlsTrustCertsFilePath() { - return confData.getTlsTrustCertsFilePath(); - } - - /** - * Set the path to the trusted TLS certificate file - * - * @param tlsTrustCertsFilePath - */ - public void setTlsTrustCertsFilePath(String tlsTrustCertsFilePath) { - confData.setTlsTrustCertsFilePath(tlsTrustCertsFilePath); - } - - /** - * @return whether the Pulsar client accept untrusted TLS certificate from broker - */ - public boolean isTlsAllowInsecureConnection() { - return confData.isTlsAllowInsecureConnection(); - } - - /** - * Configure whether the Pulsar client accept untrusted TLS certificate from broker (default: false) - * - * @param tlsAllowInsecureConnection - */ - public void setTlsAllowInsecureConnection(boolean tlsAllowInsecureConnection) { - confData.setTlsAllowInsecureConnection(tlsAllowInsecureConnection); - } - - /** - * Stats will be activated with positive statsIntervalSeconds - * - * @return the interval between each stat info (default: 60 seconds) - */ - public long getStatsIntervalSeconds() { - return confData.getStatsIntervalSeconds(); - } - - /** - * Set the interval between each stat info (default: 60 seconds) Stats will be activated with positive - * statsIntervalSeconds It should be set to at least 1 second - * - * @param statsIntervalSeconds - * the interval between each stat info - * @param unit - * time unit for {@code statsInterval} - */ - public void setStatsInterval(long statsInterval, TimeUnit unit) { - confData.setStatsIntervalSeconds(unit.toSeconds(statsInterval)); - } - - /** - * Get configured total allowed concurrent lookup-request. - * - * @return - */ - public int getConcurrentLookupRequest() { - return confData.getConcurrentLookupRequest(); - } - - /** - * Number of concurrent lookup-requests allowed on each broker-connection to prevent overload on broker. - * (default: 50000) It should be configured with higher value only in case of it requires to - * produce/subscribe on thousands of topic using created {@link PulsarClient} - * - * @param concurrentLookupRequest - */ - public void setConcurrentLookupRequest(int concurrentLookupRequest) { - confData.setConcurrentLookupRequest(concurrentLookupRequest); - } - - /** - * Get configured max number of reject-request in a time-frame (30 seconds) after which connection will be closed - * - * @return - */ - public int getMaxNumberOfRejectedRequestPerConnection() { - return confData.getMaxNumberOfRejectedRequestPerConnection(); - } - - /** - * Set max number of broker-rejected requests in a certain time-frame (30 seconds) after which current connection - * will be closed and client creates a new connection that give chance to connect a different broker (default: - * 50) - * - * @param maxNumberOfRejectedRequestPerConnection - */ - public void setMaxNumberOfRejectedRequestPerConnection(int maxNumberOfRejectedRequestPerConnection) { - confData.setMaxNumberOfRejectedRequestPerConnection(maxNumberOfRejectedRequestPerConnection); - } - - public boolean isTlsHostnameVerificationEnable() { - return confData.isTlsHostnameVerificationEnable(); - } - - /** - * It allows to validate hostname verification when client connects to broker over tls. It validates incoming x509 - * certificate and matches provided hostname(CN/SAN) with expected broker's host name. It follows RFC 2818, 3.1. - * Server Identity hostname verification. - * - * @see rfc2818 - * - * @param tlsHostnameVerificationEnable - */ - public void setTlsHostnameVerificationEnable(boolean tlsHostnameVerificationEnable) { - confData.setTlsHostnameVerificationEnable(tlsHostnameVerificationEnable); - } - - public ClientConfiguration setServiceUrl(String serviceUrl) { - confData.setServiceUrl(serviceUrl); - return this; - } - - /** - * Set the duration of time to wait for a connection to a broker to be established. If the duration - * passes without a response from the broker, the connection attempt is dropped. - * - * @param duration the duration to wait - * @param unit the time unit in which the duration is defined - */ - public void setConnectionTimeout(int duration, TimeUnit unit) { - confData.setConnectionTimeoutMs((int)unit.toMillis(duration)); - } - - /** - * Get the duration of time for which the client will wait for a connection to a broker to be - * established before giving up. - * - * @return the duration, in milliseconds - */ - public long getConnectionTimeoutMs() { - return confData.getConnectionTimeoutMs(); - } - - public ClientConfigurationData getConfigurationData() { - return confData; - } - -} diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/api/ConsumerConfiguration.java b/pulsar-client/src/main/java/org/apache/pulsar/client/api/ConsumerConfiguration.java deleted file mode 100644 index 4539d6d6e004f..0000000000000 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/api/ConsumerConfiguration.java +++ /dev/null @@ -1,370 +0,0 @@ -/** - * 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 com.google.common.base.Preconditions.checkArgument; -import static com.google.common.base.Preconditions.checkNotNull; -import org.apache.pulsar.client.impl.conf.ConsumerConfigurationData; - -import java.io.Serializable; -import java.util.Map; -import java.util.concurrent.TimeUnit; - - -/** - * Class specifying the configuration of a consumer. In Exclusive subscription, only a single consumer is allowed to - * attach to the subscription. Other consumers will get an error message. In Shared subscription, multiple consumers - * will be able to use the same subscription name and the messages will be dispatched in a round robin fashion. - * - * @deprecated Use {@link PulsarClient#newConsumer} to build and configure a {@link Consumer} instance - */ -@Deprecated -public class ConsumerConfiguration implements Serializable { - - /** - * Resend shouldn't be requested before minAckTimeoutMillis. - */ - static long minAckTimeoutMillis = 1000; - - private static final long serialVersionUID = 1L; - - private final ConsumerConfigurationData conf = new ConsumerConfigurationData<>(); - - private boolean initializeSubscriptionOnLatest = true; - - public ConsumerConfiguration() { - // Disable acknowledgment grouping when using v1 API - conf.setAcknowledgementsGroupTimeMicros(0); - } - - /** - * @return the configured timeout in milliseconds for unacked messages. - */ - public long getAckTimeoutMillis() { - return conf.getAckTimeoutMillis(); - } - - public long getTickDurationMillis() { - return conf.getTickDurationMillis(); - } - - /** - * Set the timeout for unacked messages, truncated to the nearest millisecond. The timeout needs to be greater than - * 10 seconds. - * - * @param ackTimeout - * for unacked messages. - * @param timeUnit - * unit in which the timeout is provided. - * @return {@link ConsumerConfiguration} - */ - public ConsumerConfiguration setAckTimeout(long ackTimeout, TimeUnit timeUnit) { - long ackTimeoutMillis = timeUnit.toMillis(ackTimeout); - checkArgument(ackTimeoutMillis >= minAckTimeoutMillis, - "Ack timeout should be should be greater than " + minAckTimeoutMillis + " ms"); - conf.setAckTimeoutMillis(timeUnit.toMillis(ackTimeout)); - return this; - } - - /** - * @return the configured subscription type - */ - public SubscriptionType getSubscriptionType() { - return conf.getSubscriptionType(); - } - - /** - * Select the subscription type to be used when subscribing to the topic. - *

- * Default is {@link SubscriptionType#Exclusive} - * - * @param subscriptionType - * the subscription type value - */ - public ConsumerConfiguration setSubscriptionType(SubscriptionType subscriptionType) { - checkNotNull(subscriptionType); - conf.setSubscriptionType(subscriptionType); - return this; - } - - /** - * @return the configured {@link MessageListener} for the consumer - */ - public MessageListener getMessageListener() { - return conf.getMessageListener(); - } - - /** - * Sets a {@link MessageListener} for the consumer - *

- * When a {@link MessageListener} is set, application will receive messages through it. Calls to - * {@link Consumer#receive()} will not be allowed. - * - * @param messageListener - * the listener object - */ - public ConsumerConfiguration setMessageListener(MessageListener messageListener) { - checkNotNull(messageListener); - conf.setMessageListener(messageListener); - return this; - } - - /** - * @return this configured {@link ConsumerEventListener} for the consumer. - * @see #setConsumerEventListener(ConsumerEventListener) - * @since 2.0 - */ - public ConsumerEventListener getConsumerEventListener() { - return conf.getConsumerEventListener(); - } - - /** - * Sets a {@link ConsumerEventListener} for the consumer. - * - *

- * The consumer group listener is used for receiving consumer state change in a consumer group for failover - * subscription. Application can then react to the consumer state changes. - * - *

- * This change is experimental. It is subject to changes coming in release 2.0. - * - * @param listener - * the consumer group listener object - * @return consumer configuration - * @since 2.0 - */ - public ConsumerConfiguration setConsumerEventListener(ConsumerEventListener listener) { - checkNotNull(listener); - conf.setConsumerEventListener(listener); - return this; - } - - /** - * @return the configure receiver queue size value - */ - public int getReceiverQueueSize() { - return conf.getReceiverQueueSize(); - } - - /** - * @return the configured max total receiver queue size across partitions - */ - public int getMaxTotalReceiverQueueSizeAcrossPartitions() { - return conf.getMaxTotalReceiverQueueSizeAcrossPartitions(); - } - - /** - * Set the max total receiver queue size across partitons. - *

- * This setting will be used to reduce the receiver queue size for individual partitions - * {@link #setReceiverQueueSize(int)} if the total exceeds this value (default: 50000). - * - * @param maxTotalReceiverQueueSizeAcrossPartitions - */ - public void setMaxTotalReceiverQueueSizeAcrossPartitions(int maxTotalReceiverQueueSizeAcrossPartitions) { - checkArgument(maxTotalReceiverQueueSizeAcrossPartitions >= conf.getReceiverQueueSize()); - conf.setMaxTotalReceiverQueueSizeAcrossPartitions(maxTotalReceiverQueueSizeAcrossPartitions); - } - - /** - * @return the CryptoKeyReader - */ - public CryptoKeyReader getCryptoKeyReader() { - return conf.getCryptoKeyReader(); - } - - /** - * Sets a {@link CryptoKeyReader} - * - * @param cryptoKeyReader - * CryptoKeyReader object - */ - public ConsumerConfiguration setCryptoKeyReader(CryptoKeyReader cryptoKeyReader) { - checkNotNull(cryptoKeyReader); - conf.setCryptoKeyReader(cryptoKeyReader); - return this; - } - - /** - * Sets the ConsumerCryptoFailureAction to the value specified - * - * @param action - * consumer action - */ - public void setCryptoFailureAction(ConsumerCryptoFailureAction action) { - conf.setCryptoFailureAction(action); - } - - /** - * @return The ConsumerCryptoFailureAction - */ - public ConsumerCryptoFailureAction getCryptoFailureAction() { - return conf.getCryptoFailureAction(); - } - - /** - * Sets the size of the consumer receive queue. - *

- * The consumer receive queue controls how many messages can be accumulated by the {@link Consumer} before the - * application calls {@link Consumer#receive()}. Using a higher value could potentially increase the consumer - * throughput at the expense of bigger memory utilization. - *

- *

- * Setting the consumer queue size as zero - *

    - *
  • Decreases the throughput of the consumer, by disabling pre-fetching of messages. This approach improves the - * message distribution on shared subscription, by pushing messages only to the consumers that are ready to process - * them. Neither {@link Consumer#receive(int, TimeUnit)} nor Partitioned Topics can be used if the consumer queue - * size is zero. {@link Consumer#receive()} function call should not be interrupted when the consumer queue size is - * zero.
  • - *
  • Doesn't support Batch-Message: if consumer receives any batch-message then it closes consumer connection with - * broker and {@link Consumer#receive()} call will remain blocked while {@link Consumer#receiveAsync()} receives - * exception in callback. consumer will not be able receive any further message unless batch-message in pipeline - * is removed
  • - *
- *

- * Default value is {@code 1000} messages and should be good for most use cases. - * - * @param receiverQueueSize - * the new receiver queue size value - */ - public ConsumerConfiguration setReceiverQueueSize(int receiverQueueSize) { - checkArgument(receiverQueueSize >= 0, "Receiver queue size cannot be negative"); - conf.setReceiverQueueSize(receiverQueueSize); - return this; - } - - /** - * @return the consumer name - */ - public String getConsumerName() { - return conf.getConsumerName(); - } - - /** - * Set the consumer name. - * - * @param consumerName - */ - public ConsumerConfiguration setConsumerName(String consumerName) { - checkArgument(consumerName != null && !consumerName.equals("")); - conf.setConsumerName(consumerName); - return this; - } - - public int getPriorityLevel() { - return conf.getPriorityLevel(); - } - - /** - * Sets priority level for the shared subscription consumers to which broker gives more priority while dispatching - * messages. Here, broker follows descending priorities. (eg: 0=max-priority, 1, 2,..)
- * In Shared subscription mode, broker will first dispatch messages to max priority-level consumers if they have - * permits, else broker will consider next priority level consumers.
- * If subscription has consumer-A with priorityLevel 0 and Consumer-B with priorityLevel 1 then broker will dispatch - * messages to only consumer-A until it runs out permit and then broker starts dispatching messages to Consumer-B. - * - *
-     * Consumer PriorityLevel Permits
-     * C1       0             2
-     * C2       0             1
-     * C3       0             1
-     * C4       1             2
-     * C5       1             1
-     * Order in which broker dispatches messages to consumers: C1, C2, C3, C1, C4, C5, C4
-     * 
- * - * @param priorityLevel - */ - public void setPriorityLevel(int priorityLevel) { - conf.setPriorityLevel(priorityLevel); - } - - public boolean getReadCompacted() { - return conf.isReadCompacted(); - } - - /** - * If enabled, the consumer will read messages from the compacted topic rather than reading the full message backlog - * of the topic. This means that, if the topic has been compacted, the consumer will only see the latest value for - * each key in the topic, up until the point in the topic message backlog that has been compacted. Beyond that - * point, the messages will be sent as normal. - * - * readCompacted can only be enabled subscriptions to persistent topics, which have a single active consumer (i.e. - * failure or exclusive subscriptions). Attempting to enable it on subscriptions to a non-persistent topics or on a - * shared subscription, will lead to the subscription call throwing a PulsarClientException. - * - * @param readCompacted - * whether to read from the compacted topic - */ - public ConsumerConfiguration setReadCompacted(boolean readCompacted) { - conf.setReadCompacted(readCompacted); - return this; - } - - /** - * Set a name/value property with this consumer. - * - * @param key - * @param value - * @return - */ - public ConsumerConfiguration setProperty(String key, String value) { - checkArgument(key != null); - checkArgument(value != null); - conf.getProperties().put(key, value); - return this; - } - - /** - * Add all the properties in the provided map - * - * @param properties - * @return - */ - public ConsumerConfiguration setProperties(Map properties) { - conf.getProperties().putAll(properties); - return this; - } - - public Map getProperties() { - return conf.getProperties(); - } - - public ConsumerConfigurationData getConfigurationData() { - return conf; - } - - /** - * @param subscriptionInitialPosition the initial position at which to set - * set cursor when subscribing to the topic first time - * Default is {@value InitialPosition.Latest} - */ - public ConsumerConfiguration setSubscriptionInitialPosition(SubscriptionInitialPosition subscriptionInitialPosition) { - conf.setSubscriptionInitialPosition(subscriptionInitialPosition); - return this; - } - - /** - * @return the configured {@link subscriptionInitailPosition} for the consumer - */ - public SubscriptionInitialPosition getSubscriptionInitialPosition(){ - return conf.getSubscriptionInitialPosition(); - } -} diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/api/MessageBuilder.java b/pulsar-client/src/main/java/org/apache/pulsar/client/api/MessageBuilder.java deleted file mode 100644 index 6054dfecd171e..0000000000000 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/api/MessageBuilder.java +++ /dev/null @@ -1,158 +0,0 @@ -/** - * 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 java.nio.ByteBuffer; -import java.util.List; -import java.util.Map; - -import org.apache.pulsar.client.impl.MessageBuilderImpl; - -/** - * Message builder factory. Use this class to create messages to be send to the Pulsar producer - * - * @deprecated since 2.0. Use {@link TypedMessageBuilder} as returned by {@link Producer#newMessage()} to create a new - * message builder. - */ -@Deprecated -public interface MessageBuilder { - /** - * Create a new message builder instance. - *

- * A message builder is suitable for creating one single message - * - * @return a new message builder - */ - static MessageBuilder create(Schema schema) { - return new MessageBuilderImpl<>(schema); - } - - static MessageBuilder create() { - return create(Schema.BYTES); - } - - /** - * Finalize the immutable message - * - * @return a {@link Message} ready to be sent through a {@link Producer} - */ - Message build(); - - /** - * Set a domain object on the message - * - * @param value - * the domain object - */ - MessageBuilder setValue(T value); - - /** - * Set the content of the message - * - * @param data - * array containing the payload - */ - MessageBuilder setContent(byte[] data); - - /** - * Set the content of the message - * - * @param data - * array containing the payload - * @param offset - * offset into the data array - * @param length - * length of the payload starting from the above offset - */ - MessageBuilder setContent(byte[] data, int offset, int length); - - /** - * Set the content of the message - * - * @param buf - * a {@link ByteBuffer} with the payload of the message - */ - MessageBuilder setContent(ByteBuffer buf); - - /** - * Sets a new property on a message. - * - * @param name - * the name of the property - * @param value - * the associated value - */ - MessageBuilder setProperty(String name, String value); - - /** - * Add all the properties in the provided map - */ - MessageBuilder setProperties(Map properties); - - /** - * Sets the key of the message for routing policy - * - * @param key - */ - MessageBuilder setKey(String key); - - /** - * Set the event time for a given message. - * - *

- * Applications can retrieve the event time by calling {@link Message#getEventTime()}. - * - *

- * Note: currently pulsar doesn't support event-time based index. so the subscribers can't seek the messages by - * event time. - * - * @since 1.20.0 - */ - MessageBuilder setEventTime(long timestamp); - - /** - * Specify a custom sequence id for the message being published. - *

- * The sequence id can be used for deduplication purposes and it needs to follow these rules: - *

    - *
  1. sequenceId >= 0 - *
  2. Sequence id for a message needs to be greater than sequence id for earlier messages: - * sequenceId(N+1) > sequenceId(N) - *
  3. It's not necessary for sequence ids to be consecutive. There can be holes between messages. Eg. the - * sequenceId could represent an offset or a cumulative size. - *
- * - * @param sequenceId - * the sequence id to assign to the current message - * @since 1.20.0 - */ - MessageBuilder setSequenceId(long sequenceId); - - /** - * Override the replication clusters for this message. - * - * @param clusters - */ - MessageBuilder setReplicationClusters(List clusters); - - /** - * Disable replication for this message. - */ - MessageBuilder disableReplication(); -} diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/api/Producer.java b/pulsar-client/src/main/java/org/apache/pulsar/client/api/Producer.java index e523484e6e765..36b150bcefaec 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/api/Producer.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/api/Producer.java @@ -111,53 +111,6 @@ public interface Producer extends Closeable { */ TypedMessageBuilder newMessage(); - /** - * Send a message - * - * @param message - * a message - * @return the message id assigned to the published message - * @throws PulsarClientException.TimeoutException - * if the message was not correctly received by the system within the timeout period - * - * @deprecated since 2.0. Use {@link TypedMessageBuilder} as returned by {@link Producer#newMessage()} to create a - * new message builder. - */ - @Deprecated - MessageId send(Message message) throws PulsarClientException; - - /** - * Send a message asynchronously - *

- * When the returned {@link CompletableFuture} is marked as completed successfully, the provided message will - * contain the {@link MessageId} assigned by the broker to the published message. - *

- * Example: - * - *

-     * Message msg = MessageBuilder.create().setContent(myContent).build();
-     * producer.sendAsync(msg).thenRun(v -> {
-     *    System.out.println("Published message: " + msg.getMessageId());
-     * }).exceptionally(e -> {
-     *    // Failed to publish
-     * });
-     * 
- *

- * When the producer queue is full, by default this method will complete the future with an exception - * {@link PulsarClientException.ProducerQueueIsFullError} - *

- * See {@link ProducerBuilder#maxPendingMessages(int)} to configure the producer queue size and - * {@link ProducerBuilder#blockIfQueueFull(boolean)} to change the blocking behavior. - * - * @param message - * a message - * @return a future that can be used to track when the message will have been safely persisted - * @deprecated since 2.0. Use {@link TypedMessageBuilder} as returned by {@link Producer#newMessage()} to create a - * new message builder. - */ - @Deprecated - CompletableFuture sendAsync(Message message); - /** * Get the last sequence id that was published by this producer. *

diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/api/ProducerConfiguration.java b/pulsar-client/src/main/java/org/apache/pulsar/client/api/ProducerConfiguration.java deleted file mode 100644 index 1af077be9cade..0000000000000 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/api/ProducerConfiguration.java +++ /dev/null @@ -1,483 +0,0 @@ -/** - * 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 com.google.common.base.Preconditions.checkArgument; -import static com.google.common.base.Preconditions.checkNotNull; - -import java.io.Serializable; -import java.util.Map; -import java.util.Optional; -import java.util.Set; -import java.util.concurrent.TimeUnit; - -import org.apache.pulsar.client.api.PulsarClientException.ProducerBusyException; -import org.apache.pulsar.client.impl.conf.ProducerConfigurationData; - -import lombok.EqualsAndHashCode; - -/** - * Producer's configuration - * - * @deprecated use {@link PulsarClient#newProducer()} to construct and configure a {@link Producer} instance - */ -@Deprecated -@EqualsAndHashCode -public class ProducerConfiguration implements Serializable { - - private static final long serialVersionUID = 1L; - - private final ProducerConfigurationData conf = new ProducerConfigurationData(); - - @Deprecated - public enum MessageRoutingMode { - SinglePartition, RoundRobinPartition, CustomPartition - } - - @Deprecated - public enum HashingScheme { - JavaStringHash, Murmur3_32Hash - } - - /** - * @return the configured custom producer name or null if no custom name was specified - * @since 1.20.0 - */ - public String getProducerName() { - return conf.getProducerName(); - } - - /** - * Specify a name for the producer - *

- * If not assigned, the system will generate a globally unique name which can be access with - * {@link Producer#getProducerName()}. - *

- * When specifying a name, it is app to the user to ensure that, for a given topic, the producer name is unique - * across all Pulsar's clusters. - *

- * If a producer with the same name is already connected to a particular topic, the - * {@link PulsarClient#createProducer(String)} operation will fail with {@link ProducerBusyException}. - * - * @param producerName - * the custom name to use for the producer - * @since 1.20.0 - */ - public void setProducerName(String producerName) { - conf.setProducerName(producerName); - } - - /** - * @return the message send timeout in ms - */ - public long getSendTimeoutMs() { - return conf.getSendTimeoutMs(); - } - - /** - * Set the send timeout (default: 30 seconds) - *

- * If a message is not acknowledged by the server before the sendTimeout expires, an error will be reported. - * - * @param sendTimeout - * the send timeout - * @param unit - * the time unit of the {@code sendTimeout} - */ - public ProducerConfiguration setSendTimeout(int sendTimeout, TimeUnit unit) { - checkArgument(sendTimeout >= 0); - conf.setSendTimeoutMs(unit.toMillis(sendTimeout)); - return this; - } - - /** - * @return the maximum number of messages allowed in the outstanding messages queue for the producer - */ - public int getMaxPendingMessages() { - return conf.getMaxPendingMessages(); - } - - /** - * Set the max size of the queue holding the messages pending to receive an acknowledgment from the broker. - *

- * When the queue is full, by default, all calls to {@link Producer#send} and {@link Producer#sendAsync} will fail - * unless blockIfQueueFull is set to true. Use {@link #setBlockIfQueueFull} to change the blocking behavior. - * - * @param maxPendingMessages - * @return - */ - public ProducerConfiguration setMaxPendingMessages(int maxPendingMessages) { - checkArgument(maxPendingMessages > 0); - conf.setMaxPendingMessages(maxPendingMessages); - return this; - } - - public HashingScheme getHashingScheme() { - return HashingScheme.valueOf(conf.getHashingScheme().toString()); - } - - public ProducerConfiguration setHashingScheme(HashingScheme hashingScheme) { - conf.setHashingScheme(org.apache.pulsar.client.api.HashingScheme.valueOf(hashingScheme.toString())); - return this; - } - - /** - * - * @return the maximum number of pending messages allowed across all the partitions - */ - public int getMaxPendingMessagesAcrossPartitions() { - return conf.getMaxPendingMessagesAcrossPartitions(); - } - - /** - * Set the number of max pending messages across all the partitions - *

- * This setting will be used to lower the max pending messages for each partition - * ({@link #setMaxPendingMessages(int)}), if the total exceeds the configured value. - * - * @param maxPendingMessagesAcrossPartitions - */ - public void setMaxPendingMessagesAcrossPartitions(int maxPendingMessagesAcrossPartitions) { - checkArgument(maxPendingMessagesAcrossPartitions >= conf.getMaxPendingMessages()); - conf.setMaxPendingMessagesAcrossPartitions(maxPendingMessagesAcrossPartitions); - } - - /** - * - * @return whether the producer will block {@link Producer#send} and {@link Producer#sendAsync} operations when the - * pending queue is full - */ - public boolean getBlockIfQueueFull() { - return conf.isBlockIfQueueFull(); - } - - /** - * Set whether the {@link Producer#send} and {@link Producer#sendAsync} operations should block when the outgoing - * message queue is full. - *

- * Default is false. If set to false, send operations will immediately fail with - * {@link PulsarClientException.ProducerQueueIsFullError} when there is no space left in pending queue. - * - * @param blockIfQueueFull - * whether to block {@link Producer#send} and {@link Producer#sendAsync} operations on queue full - * @return - */ - public ProducerConfiguration setBlockIfQueueFull(boolean blockIfQueueFull) { - conf.setBlockIfQueueFull(blockIfQueueFull); - return this; - } - - /** - * Set the message routing mode for the partitioned producer. - * - * @param messageRouteMode message routing mode. - * @return producer configuration - * @see MessageRoutingMode - */ - public ProducerConfiguration setMessageRoutingMode(MessageRoutingMode messageRouteMode) { - checkNotNull(messageRouteMode); - conf.setMessageRoutingMode( - org.apache.pulsar.client.api.MessageRoutingMode.valueOf(messageRouteMode.toString())); - return this; - } - - /** - * Get the message routing mode for the partitioned producer. - * - * @return message routing mode, default is round-robin routing. - * @see MessageRoutingMode#RoundRobinPartition - */ - public MessageRoutingMode getMessageRoutingMode() { - return MessageRoutingMode.valueOf(conf.getMessageRoutingMode().toString()); - } - - /** - * Set the compression type for the producer. - *

- * By default, message payloads are not compressed. Supported compression types are: - *

    - *
  • CompressionType.LZ4
  • - *
  • CompressionType.ZLIB
  • - *
- * - * @param compressionType - * @return - * - * @since 1.0.28
- * Make sure all the consumer applications have been updated to use this client version, before starting to - * compress messages. - */ - public ProducerConfiguration setCompressionType(CompressionType compressionType) { - conf.setCompressionType(compressionType); - return this; - } - - /** - * @return the configured compression type for this producer - */ - public CompressionType getCompressionType() { - return conf.getCompressionType(); - } - - /** - * Set a custom message routing policy by passing an implementation of MessageRouter - * - * - * @param messageRouter - */ - public ProducerConfiguration setMessageRouter(MessageRouter messageRouter) { - checkNotNull(messageRouter); - setMessageRoutingMode(MessageRoutingMode.CustomPartition); - conf.setCustomMessageRouter(messageRouter); - return this; - } - - /** - * Get the message router set by {@link #setMessageRouter(MessageRouter)}. - * - * @return message router. - * @deprecated since 1.22.0-incubating. numPartitions is already passed as parameter in - * {@link MessageRouter#choosePartition(Message, TopicMetadata)}. - * @see MessageRouter - */ - @Deprecated - public MessageRouter getMessageRouter(int numPartitions) { - return conf.getCustomMessageRouter(); - } - - /** - * Get the message router set by {@link #setMessageRouter(MessageRouter)}. - * - * @return message router set by {@link #setMessageRouter(MessageRouter)}. - */ - public MessageRouter getMessageRouter() { - return conf.getCustomMessageRouter(); - } - - /** - * Return the flag whether automatic message batching is enabled or not. - * - * @return true if batch messages are enabled. otherwise false. - * @since 2.0.0
- * It is enabled by default. - */ - public boolean getBatchingEnabled() { - return conf.isBatchingEnabled(); - } - - /** - * Control whether automatic batching of messages is enabled for the producer. default: false [No batching] - * - * When batching is enabled, multiple calls to Producer.sendAsync can result in a single batch to be sent to the - * broker, leading to better throughput, especially when publishing small messages. If compression is enabled, - * messages will be compressed at the batch level, leading to a much better compression ratio for similar headers or - * contents. - * - * When enabled default batch delay is set to 10 ms and default batch size is 1000 messages - * - * @see ProducerConfiguration#setBatchingMaxPublishDelay(long, TimeUnit) - * @since 1.0.36
- * Make sure all the consumer applications have been updated to use this client version, before starting to - * batch messages. - * - */ - public ProducerConfiguration setBatchingEnabled(boolean batchMessagesEnabled) { - conf.setBatchingEnabled(batchMessagesEnabled); - return this; - } - - /** - * @return the CryptoKeyReader - */ - public CryptoKeyReader getCryptoKeyReader() { - return conf.getCryptoKeyReader(); - } - - /** - * Sets a {@link CryptoKeyReader} - * - * @param cryptoKeyReader - * CryptoKeyReader object - */ - public ProducerConfiguration setCryptoKeyReader(CryptoKeyReader cryptoKeyReader) { - checkNotNull(cryptoKeyReader); - conf.setCryptoKeyReader(cryptoKeyReader); - return this; - } - - /** - * - * @return encryptionKeys - * - */ - public Set getEncryptionKeys() { - return conf.getEncryptionKeys(); - } - - /** - * - * Returns true if encryption keys are added - * - */ - public boolean isEncryptionEnabled() { - return conf.isEncryptionEnabled(); - } - - /** - * Add public encryption key, used by producer to encrypt the data key. - * - * At the time of producer creation, Pulsar client checks if there are keys added to encryptionKeys. If keys are - * found, a callback getKey(String keyName) is invoked against each key to load the values of the key. Application - * should implement this callback to return the key in pkcs8 format. If compression is enabled, message is encrypted - * after compression. If batch messaging is enabled, the batched message is encrypted. - * - */ - public void addEncryptionKey(String key) { - conf.getEncryptionKeys().add(key); - } - - public void removeEncryptionKey(String key) { - conf.getEncryptionKeys().remove(key); - } - - /** - * Sets the ProducerCryptoFailureAction to the value specified - * - * @param action - * The producer action - */ - public void setCryptoFailureAction(ProducerCryptoFailureAction action) { - conf.setCryptoFailureAction(action); - } - - /** - * @return The ProducerCryptoFailureAction - */ - public ProducerCryptoFailureAction getCryptoFailureAction() { - return conf.getCryptoFailureAction(); - } - - /** - * - * @return the batch time period in ms. - * @see ProducerConfiguration#setBatchingMaxPublishDelay(long, TimeUnit) - */ - public long getBatchingMaxPublishDelayMs() { - return TimeUnit.MICROSECONDS.toMillis(conf.getBatchingMaxPublishDelayMicros()); - } - - /** - * Set the time period within which the messages sent will be batched default: 10ms if batch messages are - * enabled. If set to a non zero value, messages will be queued until this time interval or until - * - * @see ProducerConfiguration#batchingMaxMessages threshold is reached; all messages will be published as a single - * batch message. The consumer will be delivered individual messages in the batch in the same order they were - * enqueued - * @since 1.0.36
- * Make sure all the consumer applications have been updated to use this client version, before starting to - * batch messages. - * @param batchDelay - * the batch delay - * @param timeUnit - * the time unit of the {@code batchDelay} - * @return - */ - public ProducerConfiguration setBatchingMaxPublishDelay(long batchDelay, TimeUnit timeUnit) { - long delayInMs = timeUnit.toMillis(batchDelay); - checkArgument(delayInMs >= 1, "configured value for batch delay must be at least 1ms"); - conf.setBatchingMaxPublishDelayMicros(timeUnit.toMicros(batchDelay)); - return this; - } - - /** - * - * @return the maximum number of messages permitted in a batch. - */ - public int getBatchingMaxMessages() { - return conf.getBatchingMaxMessages(); - } - - /** - * Set the maximum number of messages permitted in a batch. default: 1000 If set to a value greater than 1, - * messages will be queued until this threshold is reached or batch interval has elapsed - * - * @see ProducerConfiguration#setBatchingMaxPublishDelay(long, TimeUnit) All messages in batch will be published as - * a single batch message. The consumer will be delivered individual messages in the batch in the same order - * they were enqueued - * @param batchMessagesMaxMessagesPerBatch - * maximum number of messages in a batch - * @return - */ - public ProducerConfiguration setBatchingMaxMessages(int batchMessagesMaxMessagesPerBatch) { - checkArgument(batchMessagesMaxMessagesPerBatch > 0); - conf.setBatchingMaxMessages(batchMessagesMaxMessagesPerBatch); - return this; - } - - public Optional getInitialSequenceId() { - return Optional.ofNullable(conf.getInitialSequenceId()); - } - - /** - * Set the baseline for the sequence ids for messages published by the producer. - *

- * First message will be using (initialSequenceId + 1) as its sequence id and subsequent messages will be assigned - * incremental sequence ids, if not otherwise specified. - * - * @param initialSequenceId - * @return - */ - public ProducerConfiguration setInitialSequenceId(long initialSequenceId) { - conf.setInitialSequenceId(initialSequenceId); - return this; - } - - /** - * Set a name/value property with this producer. - * - * @param key - * @param value - * @return - */ - public ProducerConfiguration setProperty(String key, String value) { - checkArgument(key != null); - checkArgument(value != null); - conf.getProperties().put(key, value); - return this; - } - - /** - * Add all the properties in the provided map - * - * @param properties - * @return - */ - public ProducerConfiguration setProperties(Map properties) { - conf.getProperties().putAll(properties); - return this; - } - - public Map getProperties() { - return conf.getProperties(); - } - - public ProducerConfigurationData getProducerConfigurationData() { - return conf; - } -} diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/api/PulsarClient.java b/pulsar-client/src/main/java/org/apache/pulsar/client/api/PulsarClient.java index 3f2271a5f4f86..1c2efe5e29b86 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/api/PulsarClient.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/api/PulsarClient.java @@ -23,7 +23,6 @@ import java.util.concurrent.CompletableFuture; import org.apache.pulsar.client.impl.ClientBuilderImpl; -import org.apache.pulsar.client.impl.PulsarClientImpl; /** * Class that provides a client interface to Pulsar. @@ -44,38 +43,6 @@ public static ClientBuilder builder() { return new ClientBuilderImpl(); } - /** - * Create a new PulsarClient object using default client configuration - * - * @param serviceUrl - * the url of the Pulsar endpoint to be used - * @return a new pulsar client object - * @throws PulsarClientException.InvalidServiceURL - * if the serviceUrl is invalid - * @deprecated use {@link #builder()} to construct a client instance - */ - @Deprecated - public static PulsarClient create(String serviceUrl) throws PulsarClientException { - return create(serviceUrl, new ClientConfiguration()); - } - - /** - * Create a new PulsarClient object - * - * @param serviceUrl - * the url of the Pulsar endpoint to be used - * @param conf - * the client configuration - * @return a new pulsar client object - * @throws PulsarClientException.InvalidServiceURL - * if the serviceUrl is invalid - * @deprecated use {@link #builder()} to construct a client instance - */ - @Deprecated - public static PulsarClient create(String serviceUrl, ClientConfiguration conf) throws PulsarClientException { - return new PulsarClientImpl(serviceUrl, conf); - } - /** * Create a producer with default for publishing on a specific topic *

@@ -159,184 +126,6 @@ public static PulsarClient create(String serviceUrl, ClientConfiguration conf) t */ ReaderBuilder newReader(Schema schema); - /** - * Create a producer with default {@link ProducerConfiguration} for publishing on a specific topic - * - * @param topic - * The name of the topic where to produce - * @return The producer object - * @throws PulsarClientException.AlreadyClosedException - * if the client was already closed - * @throws PulsarClientException.InvalidTopicNameException - * if the topic name is not valid - * @throws PulsarClientException.AuthenticationException - * if there was an error with the supplied credentials - * @throws PulsarClientException.AuthorizationException - * if the authorization to publish on topic was denied - * @deprecated use {@link #newProducer()} to build a new producer - */ - @Deprecated - Producer createProducer(String topic) throws PulsarClientException; - - /** - * Asynchronously create a producer with default {@link ProducerConfiguration} for publishing on a specific topic - * - * @param topic - * The name of the topic where to produce - * @return Future of the asynchronously created producer object - * @deprecated use {@link #newProducer()} to build a new producer - */ - @Deprecated - CompletableFuture> createProducerAsync(String topic); - - /** - * Create a producer with given {@code ProducerConfiguration} for publishing on a specific topic - * - * @param topic - * The name of the topic where to produce - * @param conf - * The {@code ProducerConfiguration} object - * @return The producer object - * @throws PulsarClientException - * if it was not possible to create the producer - * @throws InterruptedException - * @deprecated use {@link #newProducer()} to build a new producer - */ - @Deprecated - Producer createProducer(String topic, ProducerConfiguration conf) throws PulsarClientException; - - /** - * Asynchronously create a producer with given {@code ProducerConfiguration} for publishing on a specific topic - * - * @param topic - * The name of the topic where to produce - * @param conf - * The {@code ProducerConfiguration} object - * @return Future of the asynchronously created producer object - * @deprecated use {@link #newProducer()} to build a new producer - */ - @Deprecated - CompletableFuture> createProducerAsync(String topic, ProducerConfiguration conf); - - /** - * Subscribe to the given topic and subscription combination with default {@code ConsumerConfiguration} - * - * @param topic - * The name of the topic - * @param subscription - * The name of the subscription - * @return The {@code Consumer} object - * @throws PulsarClientException - * @throws InterruptedException - * - * @deprecated Use {@link #newConsumer()} to build a new consumer - */ - @Deprecated - Consumer subscribe(String topic, String subscription) throws PulsarClientException; - - /** - * Asynchronously subscribe to the given topic and subscription combination using default - * {@code ConsumerConfiguration} - * - * @param topic - * The topic name - * @param subscription - * The subscription name - * @return Future of the {@code Consumer} object - * @deprecated Use {@link #newConsumer()} to build a new consumer - */ - @Deprecated - CompletableFuture> subscribeAsync(String topic, String subscription); - - /** - * Subscribe to the given topic and subscription combination with given {@code ConsumerConfiguration} - * - * @param topic - * The name of the topic - * @param subscription - * The name of the subscription - * @param conf - * The {@code ConsumerConfiguration} object - * @return The {@code Consumer} object - * @throws PulsarClientException - * @deprecated Use {@link #newConsumer()} to build a new consumer - */ - @Deprecated - Consumer subscribe(String topic, String subscription, ConsumerConfiguration conf) throws PulsarClientException; - - /** - * Asynchronously subscribe to the given topic and subscription combination using given - * {@code ConsumerConfiguration} - * - * @param topic - * The name of the topic - * @param subscription - * The name of the subscription - * @param conf - * The {@code ConsumerConfiguration} object - * @return Future of the {@code Consumer} object - * @deprecated Use {@link #newConsumer()} to build a new consumer - */ - @Deprecated - CompletableFuture> subscribeAsync(String topic, String subscription, ConsumerConfiguration conf); - - /** - * Create a topic reader with given {@code ReaderConfiguration} for reading messages from the specified topic. - *

- * The Reader provides a low-level abstraction that allows for manual positioning in the topic, without using a - * subscription. Reader can only work on non-partitioned topics. - *

- * The initial reader positioning is done by specifying a message id. The options are: - *

    - *
  • MessageId.earliest : Start reading from the earliest message available in the topic - *
  • MessageId.latest : Start reading from the end topic, only getting messages published after the - * reader was created - *
  • MessageId : When passing a particular message id, the reader will position itself on that - * specific position. The first message to be read will be the message next to the specified messageId. - *
- * - * @param topic - * The name of the topic where to read - * @param startMessageId - * The message id where the reader will position itself. The first message returned will be the one after - * the specified startMessageId - * @param conf - * The {@code ReaderConfiguration} object - * @return The {@code Reader} object - * @deprecated Use {@link #newReader()} to build a new reader - */ - @Deprecated - Reader createReader(String topic, MessageId startMessageId, ReaderConfiguration conf) throws PulsarClientException; - - /** - * Asynchronously create a topic reader with given {@code ReaderConfiguration} for reading messages from the - * specified topic. - *

- * The Reader provides a low-level abstraction that allows for manual positioning in the topic, without using a - * subscription. Reader can only work on non-partitioned topics. - *

- * The initial reader positioning is done by specifying a message id. The options are: - *

    - *
  • MessageId.earliest : Start reading from the earliest message available in the topic - *
  • MessageId.latest : Start reading from the end topic, only getting messages published after the - * reader was created - *
  • MessageId : When passing a particular message id, the reader will position itself on that - * specific position. The first message to be read will be the message next to the specified messageId. - *
- * - * @param topic - * The name of the topic where to read - * @param startMessageId - * The message id where the reader will position itself. The first message returned will be the one after - * the specified startMessageId - * @param conf - * The {@code ReaderConfiguration} object - * @return Future of the asynchronously created producer object - * @deprecated Use {@link #newReader()} to build a new reader - */ - @Deprecated - CompletableFuture> createReaderAsync(String topic, MessageId startMessageId, ReaderConfiguration conf); - /** * Update the service URL this client is using. * diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/api/ReaderConfiguration.java b/pulsar-client/src/main/java/org/apache/pulsar/client/api/ReaderConfiguration.java deleted file mode 100644 index 243166cbf8071..0000000000000 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/api/ReaderConfiguration.java +++ /dev/null @@ -1,163 +0,0 @@ -/** - * 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 org.apache.commons.lang3.StringUtils; - -import static com.google.common.base.Preconditions.checkArgument; -import static com.google.common.base.Preconditions.checkNotNull; - -import java.io.Serializable; - -import org.apache.pulsar.client.impl.conf.ReaderConfigurationData; - -/** - * - * @deprecated Use {@link PulsarClient#newReader()} to construct and configure a {@link Reader} instance - */ -@Deprecated -public class ReaderConfiguration implements Serializable { - - private final ReaderConfigurationData conf = new ReaderConfigurationData<>(); - - /** - * @return the configured {@link ReaderListener} for the reader - */ - public ReaderListener getReaderListener() { - return conf.getReaderListener(); - } - - /** - * Sets a {@link ReaderListener} for the reader - *

- * When a {@link ReaderListener} is set, application will receive messages through it. Calls to - * {@link Reader#readNext()} will not be allowed. - * - * @param readerListener - * the listener object - */ - public ReaderConfiguration setReaderListener(ReaderListener readerListener) { - checkNotNull(readerListener); - conf.setReaderListener(readerListener); - return this; - } - - /** - * @return the configure receiver queue size value - */ - public int getReceiverQueueSize() { - return conf.getReceiverQueueSize(); - } - - /** - * @return the CryptoKeyReader - */ - public CryptoKeyReader getCryptoKeyReader() { - return conf.getCryptoKeyReader(); - } - - /** - * Sets a {@link CryptoKeyReader} - * - * @param cryptoKeyReader - * CryptoKeyReader object - */ - public ReaderConfiguration setCryptoKeyReader(CryptoKeyReader cryptoKeyReader) { - checkNotNull(cryptoKeyReader); - conf.setCryptoKeyReader(cryptoKeyReader); - return this; - } - - /** - * Sets the ConsumerCryptoFailureAction to the value specified - * - * @param action - * The action to take when the decoding fails - */ - public void setCryptoFailureAction(ConsumerCryptoFailureAction action) { - conf.setCryptoFailureAction(action); - } - - /** - * @return The ConsumerCryptoFailureAction - */ - public ConsumerCryptoFailureAction getCryptoFailureAction() { - return conf.getCryptoFailureAction(); - } - - /** - * Sets the size of the consumer receive queue. - *

- * The consumer receive queue controls how many messages can be accumulated by the {@link Consumer} before the - * application calls {@link Consumer#receive()}. Using a higher value could potentially increase the consumer - * throughput at the expense of bigger memory utilization. - *

- * Default value is {@code 1000} messages and should be good for most use cases. - * - * @param receiverQueueSize - * the new receiver queue size value - */ - public ReaderConfiguration setReceiverQueueSize(int receiverQueueSize) { - checkArgument(receiverQueueSize >= 0, "Receiver queue size cannot be negative"); - conf.setReceiverQueueSize(receiverQueueSize); - return this; - } - - /** - * @return the consumer name - */ - public String getReaderName() { - return conf.getReaderName(); - } - - /** - * Set the consumer name. - * - * @param readerName - */ - public ReaderConfiguration setReaderName(String readerName) { - checkArgument(StringUtils.isNotBlank(readerName)); - conf.setReaderName(readerName); - return this; - } - - /** - * @return the subscription role prefix for subscription auth - */ - public String getSubscriptionRolePrefix() { - return conf.getSubscriptionRolePrefix(); - } - - /** - * Set the subscription role prefix for subscription auth. The default prefix is "reader". - * - * @param subscriptionRolePrefix - */ - public ReaderConfiguration setSubscriptionRolePrefix(String subscriptionRolePrefix) { - checkArgument(StringUtils.isNotBlank(subscriptionRolePrefix)); - conf.setSubscriptionRolePrefix(subscriptionRolePrefix); - return this; - } - - public ReaderConfigurationData getReaderConfigurationData() { - return conf; - } - - private static final long serialVersionUID = 1L; -} diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MessageBuilderImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MessageBuilderImpl.java deleted file mode 100644 index cac174f08735d..0000000000000 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MessageBuilderImpl.java +++ /dev/null @@ -1,125 +0,0 @@ -/** - * 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.impl; - -import static com.google.common.base.Preconditions.checkArgument; - -import com.google.common.base.Preconditions; - -import java.nio.ByteBuffer; -import java.util.List; -import java.util.Map; - -import org.apache.pulsar.client.api.Message; -import org.apache.pulsar.client.api.MessageBuilder; -import org.apache.pulsar.client.api.Schema; -import org.apache.pulsar.common.api.proto.PulsarApi.KeyValue; -import org.apache.pulsar.common.api.proto.PulsarApi.MessageMetadata; - -public class MessageBuilderImpl implements MessageBuilder { - private static final ByteBuffer EMPTY_CONTENT = ByteBuffer.allocate(0); - private final MessageMetadata.Builder msgMetadataBuilder = MessageMetadata.newBuilder(); - private final Schema schema; - private ByteBuffer content = EMPTY_CONTENT; - - public MessageBuilderImpl(Schema schema) { - this.schema = schema; - } - - @Override - public Message build() { - return MessageImpl.create(msgMetadataBuilder, content, schema); - } - - @Override - public MessageBuilder setValue(T value) { - return setContent(schema.encode(value)); - } - - @Override - public MessageBuilder setContent(byte[] data) { - setContent(data, 0, data.length); - return this; - } - - @Override - public MessageBuilder setContent(byte[] data, int offet, int length) { - this.content = ByteBuffer.wrap(data, offet, length); - return this; - } - - @Override - public MessageBuilder setContent(ByteBuffer buf) { - this.content = buf.duplicate(); - return this; - } - - @Override - public MessageBuilder setProperties(Map properties) { - for (Map.Entry entry : properties.entrySet()) { - msgMetadataBuilder - .addProperties(KeyValue.newBuilder().setKey(entry.getKey()).setValue(entry.getValue()).build()); - } - - return this; - } - - @Override - public MessageBuilder setProperty(String name, String value) { - msgMetadataBuilder.addProperties(KeyValue.newBuilder().setKey(name).setValue(value).build()); - return this; - } - - @Override - public MessageBuilder setKey(String key) { - msgMetadataBuilder.setPartitionKey(key); - return this; - } - - @Override - public MessageBuilder setEventTime(long timestamp) { - checkArgument(timestamp > 0, "Invalid timestamp : '%s'", timestamp); - msgMetadataBuilder.setEventTime(timestamp); - return this; - } - - @Override - public MessageBuilder setSequenceId(long sequenceId) { - checkArgument(sequenceId >= 0); - msgMetadataBuilder.setSequenceId(sequenceId); - return this; - } - - @Override - public MessageBuilder setReplicationClusters(List clusters) { - Preconditions.checkNotNull(clusters); - msgMetadataBuilder.clearReplicateTo(); - msgMetadataBuilder.addAllReplicateTo(clusters); - return this; - } - - @Override - public MessageBuilder disableReplication() { - msgMetadataBuilder.clearReplicateTo(); - msgMetadataBuilder.addReplicateTo("__local__"); - return this; - } - - -} diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ProducerBase.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ProducerBase.java index 39e632c97afea..fa046ca1bad6e 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ProducerBase.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ProducerBase.java @@ -61,7 +61,6 @@ public CompletableFuture sendAsync(T message) { } } - @Override public CompletableFuture sendAsync(Message message) { return internalSendAsync(message); } @@ -73,7 +72,6 @@ public TypedMessageBuilder newMessage() { abstract CompletableFuture internalSendAsync(Message message); - @Override public MessageId send(Message message) throws PulsarClientException { try { // enqueue the message to the buffer diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PulsarClientImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PulsarClientImpl.java index de654003acdbc..16c5f9a911478 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PulsarClientImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PulsarClientImpl.java @@ -45,19 +45,14 @@ import java.util.regex.Pattern; import java.util.stream.Collectors; -import org.apache.pulsar.client.api.ClientConfiguration; import org.apache.pulsar.client.api.Consumer; import org.apache.pulsar.client.api.ConsumerBuilder; -import org.apache.pulsar.client.api.ConsumerConfiguration; -import org.apache.pulsar.client.api.MessageId; import org.apache.pulsar.client.api.Producer; import org.apache.pulsar.client.api.ProducerBuilder; -import org.apache.pulsar.client.api.ProducerConfiguration; import org.apache.pulsar.client.api.PulsarClient; import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.client.api.Reader; import org.apache.pulsar.client.api.ReaderBuilder; -import org.apache.pulsar.client.api.ReaderConfiguration; import org.apache.pulsar.client.api.Schema; import org.apache.pulsar.client.api.SubscriptionType; import org.apache.pulsar.client.impl.conf.ClientConfigurationData; @@ -105,23 +100,6 @@ enum State { private final EventLoopGroup eventLoopGroup; - @Deprecated - public PulsarClientImpl(String serviceUrl, ClientConfiguration conf) throws PulsarClientException { - this(conf.setServiceUrl(serviceUrl).getConfigurationData().clone()); - } - - @Deprecated - public PulsarClientImpl(String serviceUrl, ClientConfiguration conf, EventLoopGroup eventLoopGroup) - throws PulsarClientException { - this(conf.setServiceUrl(serviceUrl).getConfigurationData().clone(), eventLoopGroup); - } - - @Deprecated - public PulsarClientImpl(String serviceUrl, ClientConfiguration conf, EventLoopGroup eventLoopGroup, - ConnectionPool cnxPool) throws PulsarClientException { - this(conf.setServiceUrl(serviceUrl).getConfigurationData().clone(), eventLoopGroup, cnxPool); - } - public PulsarClientImpl(ClientConfigurationData conf) throws PulsarClientException { this(conf, getEventLoopGroup(conf)); } @@ -185,62 +163,6 @@ public ReaderBuilder newReader(Schema schema) { return new ReaderBuilderImpl<>(this, schema); } - @Override - public Producer createProducer(String topic) throws PulsarClientException { - try { - ProducerConfigurationData conf = new ProducerConfigurationData(); - conf.setTopicName(topic); - return createProducerAsync(conf).get(); - } catch (ExecutionException e) { - Throwable t = e.getCause(); - if (t instanceof PulsarClientException) { - throw (PulsarClientException) t; - } else { - throw new PulsarClientException(t); - } - } catch (InterruptedException e) { - Thread.currentThread().interrupt(); - throw new PulsarClientException(e); - } - } - - @Override - public Producer createProducer(final String topic, final ProducerConfiguration conf) throws PulsarClientException { - if (conf == null) { - throw new PulsarClientException.InvalidConfigurationException("Invalid null configuration object"); - } - - try { - ProducerConfigurationData confData = conf.getProducerConfigurationData().clone(); - confData.setTopicName(topic); - return createProducerAsync(confData).get(); - } catch (ExecutionException e) { - Throwable t = e.getCause(); - if (t instanceof PulsarClientException) { - throw (PulsarClientException) t; - } else { - throw new PulsarClientException(t); - } - } catch (InterruptedException e) { - Thread.currentThread().interrupt(); - throw new PulsarClientException(e); - } - } - - @Override - public CompletableFuture> createProducerAsync(String topic) { - ProducerConfigurationData conf = new ProducerConfigurationData(); - conf.setTopicName(topic); - return createProducerAsync(conf); - } - - @Override - public CompletableFuture> createProducerAsync(final String topic, final ProducerConfiguration conf) { - ProducerConfigurationData confData = conf.getProducerConfigurationData().clone(); - confData.setTopicName(topic); - return createProducerAsync(confData); - } - public CompletableFuture> createProducerAsync(ProducerConfigurationData conf) { return createProducerAsync(conf, Schema.BYTES, null); } @@ -320,51 +242,6 @@ private CompletableFuture> createProducerAsync(String topic, return producerCreatedFuture; } - @Override - public Consumer subscribe(final String topic, final String subscription) throws PulsarClientException { - return subscribe(topic, subscription, new ConsumerConfiguration()); - } - - @Override - public Consumer subscribe(String topic, String subscription, ConsumerConfiguration conf) - throws PulsarClientException { - try { - return subscribeAsync(topic, subscription, conf).get(); - } catch (ExecutionException e) { - Throwable t = e.getCause(); - if (t instanceof PulsarClientException) { - throw (PulsarClientException) t; - } else { - throw new PulsarClientException(t); - } - } catch (InterruptedException e) { - Thread.currentThread().interrupt(); - throw new PulsarClientException(e); - } - } - - @Override - public CompletableFuture> subscribeAsync(String topic, String subscription) { - ConsumerConfigurationData conf = new ConsumerConfigurationData<>(); - conf.getTopicNames().add(topic); - conf.setSubscriptionName(subscription); - return subscribeAsync(conf); - } - - @Override - public CompletableFuture> subscribeAsync(final String topic, final String subscription, - final ConsumerConfiguration conf) { - if (conf == null) { - return FutureUtil.failedFuture( - new PulsarClientException.InvalidConfigurationException("Invalid null configuration")); - } - - ConsumerConfigurationData confData = conf.getConfigurationData().clone(); - confData.getTopicNames().add(topic); - confData.setSubscriptionName(subscription); - return subscribeAsync(confData); - } - public CompletableFuture> subscribeAsync(ConsumerConfigurationData conf) { return subscribeAsync(conf, Schema.BYTES, null); } @@ -538,33 +415,6 @@ public static List topicsPatternFilter(List original, Pattern to .collect(Collectors.toList()); } - @Override - public Reader createReader(String topic, MessageId startMessageId, ReaderConfiguration conf) - throws PulsarClientException { - try { - return createReaderAsync(topic, startMessageId, conf).get(); - } catch (ExecutionException e) { - Throwable t = e.getCause(); - if (t instanceof PulsarClientException) { - throw (PulsarClientException) t; - } else { - throw new PulsarClientException(t); - } - } catch (InterruptedException e) { - Thread.currentThread().interrupt(); - throw new PulsarClientException(e); - } - } - - @Override - public CompletableFuture> createReaderAsync(String topic, MessageId startMessageId, - ReaderConfiguration conf) { - ReaderConfigurationData confData = conf.getReaderConfigurationData().clone(); - confData.setTopicName(topic); - confData.setStartMessageId(startMessageId); - return createReaderAsync(confData); - } - public CompletableFuture> createReaderAsync(ReaderConfigurationData conf) { return createReaderAsync(conf, Schema.BYTES); } diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/TypedMessageBuilderImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/TypedMessageBuilderImpl.java index b0c116abf4904..b13423ec1f448 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/TypedMessageBuilderImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/TypedMessageBuilderImpl.java @@ -52,12 +52,12 @@ public TypedMessageBuilderImpl(ProducerBase producer, Schema schema) { @Override public MessageId send() throws PulsarClientException { - return producer.send((Message) MessageImpl.create(msgMetadataBuilder, content, schema)); + return producer.send(getMessage()); } @Override public CompletableFuture sendAsync() { - return producer.internalSendAsync((Message) MessageImpl.create(msgMetadataBuilder, content, schema)); + return producer.internalSendAsync(getMessage()); } @Override @@ -130,6 +130,14 @@ public TypedMessageBuilder disableReplication() { return this; } + public MessageMetadata.Builder getMetadataBuilder() { + return msgMetadataBuilder; + } + + public Message getMessage() { + return (Message) MessageImpl.create(msgMetadataBuilder, content, schema); + } + public long getPublishTime() { return msgMetadataBuilder.getPublishTime(); } diff --git a/pulsar-flink/src/main/java/org/apache/flink/streaming/connectors/pulsar/FlinkPulsarProducer.java b/pulsar-flink/src/main/java/org/apache/flink/streaming/connectors/pulsar/FlinkPulsarProducer.java index d515365716908..27701c5dccadf 100644 --- a/pulsar-flink/src/main/java/org/apache/flink/streaming/connectors/pulsar/FlinkPulsarProducer.java +++ b/pulsar-flink/src/main/java/org/apache/flink/streaming/connectors/pulsar/FlinkPulsarProducer.java @@ -18,6 +18,11 @@ */ package org.apache.flink.streaming.connectors.pulsar; +import static org.apache.flink.util.Preconditions.checkArgument; +import static org.apache.flink.util.Preconditions.checkNotNull; + +import java.util.function.Function; + import org.apache.commons.lang3.StringUtils; import org.apache.flink.api.common.functions.RuntimeContext; import org.apache.flink.api.common.serialization.SerializationSchema; @@ -30,20 +35,13 @@ import org.apache.flink.streaming.api.operators.StreamingRuntimeContext; import org.apache.flink.streaming.connectors.pulsar.partitioner.PulsarKeyExtractor; import org.apache.flink.util.SerializableObject; -import org.apache.pulsar.client.api.Message; -import org.apache.pulsar.client.api.MessageBuilder; import org.apache.pulsar.client.api.MessageId; import org.apache.pulsar.client.api.Producer; -import org.apache.pulsar.client.api.ProducerConfiguration; import org.apache.pulsar.client.api.PulsarClient; +import org.apache.pulsar.client.api.TypedMessageBuilder; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.util.function.Function; - -import static org.apache.flink.util.Preconditions.checkArgument; -import static org.apache.flink.util.Preconditions.checkNotNull; - /** * Flink Sink to produce data into a Pulsar topic. */ @@ -58,11 +56,6 @@ public class FlinkPulsarProducer */ protected final String serviceUrl; - /** - * User defined configuration for the producer. - */ - protected final ProducerConfiguration producerConfig; - /** * The name of the default topic this producer is writing data to. */ @@ -94,7 +87,7 @@ public class FlinkPulsarProducer /** * Pulsar Producer instance. */ - protected transient Producer producer; + protected transient Producer producer; /** * The callback than handles error propagation or logging callbacks. @@ -121,14 +114,12 @@ public class FlinkPulsarProducer public FlinkPulsarProducer(String serviceUrl, String defaultTopicName, SerializationSchema serializationSchema, - ProducerConfiguration producerConfig, PulsarKeyExtractor keyExtractor) { checkArgument(StringUtils.isNotBlank(serviceUrl), "Service url cannot be blank"); checkArgument(StringUtils.isNotBlank(defaultTopicName), "TopicName cannot be blank"); this.serviceUrl = serviceUrl; this.defaultTopicName = defaultTopicName; this.schema = checkNotNull(serializationSchema, "Serialization Schema not set"); - this.producerConfig = checkNotNull(producerConfig, "Producer Config is not set"); this.flinkPulsarKeyExtractor = getOrNullKeyExtractor(keyExtractor); ClosureCleaner.ensureSerializable(serializationSchema); } @@ -181,9 +172,9 @@ private static final PulsarKeyExtractor getOrNullKeyExtractor(PulsarKeyEx } } - private Producer createProducer(ProducerConfiguration configuration) throws Exception { - PulsarClient client = PulsarClient.create(serviceUrl); - return client.createProducer(defaultTopicName, configuration); + private Producer createProducer() throws Exception { + PulsarClient client = PulsarClient.builder().serviceUrl(serviceUrl).build(); + return client.newProducer().topic(defaultTopicName).create(); } /** @@ -194,7 +185,7 @@ private Producer createProducer(ProducerConfiguration configuration) throws Exce */ @Override public void open(Configuration parameters) throws Exception { - this.producer = createProducer(producerConfig); + this.producer = createProducer(); RuntimeContext ctx = getRuntimeContext(); @@ -238,24 +229,22 @@ public void invoke(IN value, Context context) throws Exception { byte[] serializedValue = schema.serialize(value); - MessageBuilder msgBuilder = MessageBuilder.create(); + TypedMessageBuilder msgBuilder = producer.newMessage(); if (null != context.timestamp()) { - msgBuilder = msgBuilder.setEventTime(context.timestamp()); + msgBuilder = msgBuilder.eventTime(context.timestamp()); } String msgKey = flinkPulsarKeyExtractor.getKey(value); if (null != msgKey) { - msgBuilder = msgBuilder.setKey(msgKey); + msgBuilder = msgBuilder.key(msgKey); } - Message message = msgBuilder - .setContent(serializedValue) - .build(); if (flushOnCheckpoint) { synchronized (pendingRecordsLock) { pendingRecords++; } } - producer.sendAsync(message) + msgBuilder.value(serializedValue) + .sendAsync() .thenApply(successCallback) .exceptionally(failureCallback); } diff --git a/pulsar-flink/src/main/java/org/apache/flink/streaming/connectors/pulsar/PulsarAvroTableSink.java b/pulsar-flink/src/main/java/org/apache/flink/streaming/connectors/pulsar/PulsarAvroTableSink.java index a7a44121c9fe1..b370345ce1cca 100644 --- a/pulsar-flink/src/main/java/org/apache/flink/streaming/connectors/pulsar/PulsarAvroTableSink.java +++ b/pulsar-flink/src/main/java/org/apache/flink/streaming/connectors/pulsar/PulsarAvroTableSink.java @@ -18,6 +18,12 @@ */ package org.apache.flink.streaming.connectors.pulsar; +import static org.apache.flink.util.Preconditions.checkArgument; +import static org.apache.flink.util.Preconditions.checkNotNull; +import static org.apache.flink.util.Preconditions.checkState; + +import java.util.Arrays; + import org.apache.avro.Schema; import org.apache.avro.specific.SpecificData; import org.apache.avro.specific.SpecificRecord; @@ -30,13 +36,6 @@ import org.apache.flink.table.sinks.AppendStreamTableSink; import org.apache.flink.table.sinks.TableSink; import org.apache.flink.types.Row; -import org.apache.pulsar.client.api.ProducerConfiguration; - -import java.util.Arrays; - -import static org.apache.flink.util.Preconditions.checkArgument; -import static org.apache.flink.util.Preconditions.checkNotNull; -import static org.apache.flink.util.Preconditions.checkState; /** * An append-only table sink to emit a streaming table as a Pulsar stream that serializes data in Avro format. @@ -45,7 +44,6 @@ public class PulsarAvroTableSink implements AppendStreamTableSink { protected final String serviceUrl; protected final String topic; - protected final ProducerConfiguration producerConf; protected final String routingKeyFieldName; protected SerializationSchema serializationSchema; protected String[] fieldNames; @@ -64,12 +62,10 @@ public class PulsarAvroTableSink implements AppendStreamTableSink { public PulsarAvroTableSink( String serviceUrl, String topic, - ProducerConfiguration producerConf, String routingKeyFieldName, Class recordClazz) { this.serviceUrl = checkNotNull(serviceUrl, "Service url not set"); this.topic = checkNotNull(topic, "Topic is null"); - this.producerConf = checkNotNull(producerConf, "Producer configuration not set"); this.routingKeyFieldName = routingKeyFieldName; this.recordClazz = recordClazz; } @@ -83,7 +79,6 @@ protected FlinkPulsarProducer createFlinkPulsarProducer() { serviceUrl, topic, serializationSchema, - producerConf, keyExtractor); } @@ -115,7 +110,7 @@ public TypeInformation[] getFieldTypes() { @Override public TableSink configure(String[] fieldNames, TypeInformation[] fieldTypes) { - PulsarAvroTableSink sink = new PulsarAvroTableSink(serviceUrl, topic, producerConf, routingKeyFieldName, recordClazz); + PulsarAvroTableSink sink = new PulsarAvroTableSink(serviceUrl, topic, routingKeyFieldName, recordClazz); sink.fieldNames = checkNotNull(fieldNames, "Field names are null"); sink.fieldTypes = checkNotNull(fieldTypes, "Field types are null"); diff --git a/pulsar-flink/src/main/java/org/apache/flink/streaming/connectors/pulsar/PulsarJsonTableSink.java b/pulsar-flink/src/main/java/org/apache/flink/streaming/connectors/pulsar/PulsarJsonTableSink.java index d6818fa4a56bc..45c26427c9731 100644 --- a/pulsar-flink/src/main/java/org/apache/flink/streaming/connectors/pulsar/PulsarJsonTableSink.java +++ b/pulsar-flink/src/main/java/org/apache/flink/streaming/connectors/pulsar/PulsarJsonTableSink.java @@ -22,7 +22,6 @@ import org.apache.flink.api.java.typeutils.RowTypeInfo; import org.apache.flink.formats.json.JsonRowSerializationSchema; import org.apache.flink.types.Row; -import org.apache.pulsar.client.api.ProducerConfiguration; /** * Base class for {@link PulsarTableSink} that serializes data in JSON format. @@ -40,9 +39,8 @@ public class PulsarJsonTableSink extends PulsarTableSink { public PulsarJsonTableSink( String serviceUrl, String topic, - ProducerConfiguration producerConf, String routingKeyFieldName) { - super(serviceUrl, topic, producerConf, routingKeyFieldName); + super(serviceUrl, topic, routingKeyFieldName); } @Override @@ -55,7 +53,6 @@ protected PulsarTableSink createSink() { return new PulsarJsonTableSink( serviceUrl, topic, - producerConf, routingKeyFieldName); } } diff --git a/pulsar-flink/src/main/java/org/apache/flink/streaming/connectors/pulsar/PulsarTableSink.java b/pulsar-flink/src/main/java/org/apache/flink/streaming/connectors/pulsar/PulsarTableSink.java index faf420fbc8863..0fc45f7c637c5 100644 --- a/pulsar-flink/src/main/java/org/apache/flink/streaming/connectors/pulsar/PulsarTableSink.java +++ b/pulsar-flink/src/main/java/org/apache/flink/streaming/connectors/pulsar/PulsarTableSink.java @@ -18,6 +18,12 @@ */ package org.apache.flink.streaming.connectors.pulsar; +import static org.apache.flink.util.Preconditions.checkArgument; +import static org.apache.flink.util.Preconditions.checkNotNull; +import static org.apache.flink.util.Preconditions.checkState; + +import java.util.Arrays; + import org.apache.flink.api.common.serialization.SerializationSchema; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.common.typeinfo.Types; @@ -27,13 +33,6 @@ import org.apache.flink.table.sinks.AppendStreamTableSink; import org.apache.flink.table.sinks.TableSink; import org.apache.flink.types.Row; -import org.apache.pulsar.client.api.ProducerConfiguration; - -import java.util.Arrays; - -import static org.apache.flink.util.Preconditions.checkArgument; -import static org.apache.flink.util.Preconditions.checkNotNull; -import static org.apache.flink.util.Preconditions.checkState; /** * An append-only table sink to emit a streaming table as a Pulsar stream. @@ -42,7 +41,6 @@ public abstract class PulsarTableSink implements AppendStreamTableSink { protected final String serviceUrl; protected final String topic; - protected final ProducerConfiguration producerConf; protected SerializationSchema serializationSchema; protected PulsarKeyExtractor keyExtractor; protected String[] fieldNames; @@ -52,11 +50,9 @@ public abstract class PulsarTableSink implements AppendStreamTableSink { public PulsarTableSink( String serviceUrl, String topic, - ProducerConfiguration producerConf, String routingKeyFieldName) { this.serviceUrl = checkNotNull(serviceUrl, "Service url not set"); this.topic = checkNotNull(topic, "Topic is null"); - this.producerConf = checkNotNull(producerConf, "Producer configuration not set"); this.routingKeyFieldName = routingKeyFieldName; } @@ -83,7 +79,6 @@ protected FlinkPulsarProducer createFlinkPulsarProducer() { serviceUrl, topic, serializationSchema, - producerConf, keyExtractor); } diff --git a/pulsar-flink/src/test/java/org/apache/flink/streaming/connectors/pulsar/PulsarAvroTableSinkTest.java b/pulsar-flink/src/test/java/org/apache/flink/streaming/connectors/pulsar/PulsarAvroTableSinkTest.java index ae336ffb198e6..ca7915394fa88 100644 --- a/pulsar-flink/src/test/java/org/apache/flink/streaming/connectors/pulsar/PulsarAvroTableSinkTest.java +++ b/pulsar-flink/src/test/java/org/apache/flink/streaming/connectors/pulsar/PulsarAvroTableSinkTest.java @@ -25,7 +25,6 @@ import org.apache.flink.streaming.connectors.pulsar.partitioner.PulsarKeyExtractor; import org.apache.flink.table.sinks.TableSink; import org.apache.flink.types.Row; -import org.apache.pulsar.client.api.ProducerConfiguration; import org.junit.Assert; import org.junit.Test; import org.mockito.Mockito; @@ -85,7 +84,7 @@ public void testEmitDataStream() throws Exception { private PulsarAvroTableSink spySink() throws Exception { - PulsarAvroTableSink sink = new PulsarAvroTableSink(SERVICE_URL, TOPIC_NAME, new ProducerConfiguration(), ROUTING_KEY,NasaMission.class); + PulsarAvroTableSink sink = new PulsarAvroTableSink(SERVICE_URL, TOPIC_NAME, ROUTING_KEY, NasaMission.class); FlinkPulsarProducer producer = Mockito.mock(FlinkPulsarProducer.class); PowerMockito.whenNew( FlinkPulsarProducer.class @@ -93,7 +92,6 @@ private PulsarAvroTableSink spySink() throws Exception { Mockito.anyString(), Mockito.anyString(), Mockito.any(SerializationSchema.class), - Mockito.any(PowerMockito.class), Mockito.any(PulsarKeyExtractor.class) ).thenReturn(producer); Whitebox.setInternalState(sink, "fieldNames", fieldNames); diff --git a/pulsar-flink/src/test/java/org/apache/flink/streaming/connectors/pulsar/PulsarJsonTableSinkTest.java b/pulsar-flink/src/test/java/org/apache/flink/streaming/connectors/pulsar/PulsarJsonTableSinkTest.java index 746e87eab97eb..49927cef57605 100644 --- a/pulsar-flink/src/test/java/org/apache/flink/streaming/connectors/pulsar/PulsarJsonTableSinkTest.java +++ b/pulsar-flink/src/test/java/org/apache/flink/streaming/connectors/pulsar/PulsarJsonTableSinkTest.java @@ -24,8 +24,6 @@ import org.apache.flink.streaming.connectors.pulsar.partitioner.PulsarKeyExtractor; import org.apache.flink.table.sinks.TableSink; import org.apache.flink.types.Row; - -import org.apache.pulsar.client.api.ProducerConfiguration; import org.junit.Assert; import org.junit.Test; import org.mockito.Mockito; @@ -80,7 +78,7 @@ public void testEmitDataStream() throws Exception { } private PulsarJsonTableSink spySink() throws Exception { - PulsarJsonTableSink sink = new PulsarJsonTableSink(SERVICE_URL, TOPIC_NAME, new ProducerConfiguration(), ROUTING_KEY); + PulsarJsonTableSink sink = new PulsarJsonTableSink(SERVICE_URL, TOPIC_NAME, ROUTING_KEY); FlinkPulsarProducer producer = Mockito.mock(FlinkPulsarProducer.class); PowerMockito.whenNew( FlinkPulsarProducer.class @@ -88,7 +86,6 @@ private PulsarJsonTableSink spySink() throws Exception { Mockito.anyString(), Mockito.anyString(), Mockito.any(SerializationSchema.class), - Mockito.any(PowerMockito.class), Mockito.any(PulsarKeyExtractor.class) ).thenReturn(producer); Whitebox.setInternalState(sink, "fieldNames", fieldNames); diff --git a/pulsar-log4j2-appender/src/main/java/org/apache/pulsar/log4j2/appender/PulsarManager.java b/pulsar-log4j2-appender/src/main/java/org/apache/pulsar/log4j2/appender/PulsarManager.java index c52faa6cd2995..851c78fe0a7ae 100644 --- a/pulsar-log4j2-appender/src/main/java/org/apache/pulsar/log4j2/appender/PulsarManager.java +++ b/pulsar-log4j2-appender/src/main/java/org/apache/pulsar/log4j2/appender/PulsarManager.java @@ -20,32 +20,22 @@ import java.util.Objects; import java.util.concurrent.TimeUnit; -import java.util.function.BiFunction; import java.util.function.Supplier; + import org.apache.logging.log4j.core.LoggerContext; import org.apache.logging.log4j.core.appender.AbstractManager; import org.apache.logging.log4j.core.config.Property; import org.apache.pulsar.client.api.ClientBuilder; -import org.apache.pulsar.client.api.Message; -import org.apache.pulsar.client.api.MessageBuilder; import org.apache.pulsar.client.api.Producer; import org.apache.pulsar.client.api.ProducerBuilder; import org.apache.pulsar.client.api.PulsarClient; import org.apache.pulsar.client.api.PulsarClientException; +import org.apache.pulsar.client.api.TypedMessageBuilder; public class PulsarManager extends AbstractManager { static Supplier PULSAR_CLIENT_BUILDER = () -> PulsarClient.builder(); - static BiFunction> MESSAGE_BUILDER = (key, data) -> { - MessageBuilder messageBuilder = MessageBuilder.create() - .setContent(data); - if (null != key) { - messageBuilder = messageBuilder.setKey(key); - } - return messageBuilder.build(); - }; - private PulsarClient client; private Producer producer; @@ -92,15 +82,22 @@ public void send(final byte[] msg) { newKey = key; } - Message message = MESSAGE_BUILDER.apply(newKey, msg); + + TypedMessageBuilder messageBuilder = producer.newMessage() + .value(msg); + + if (newKey != null) { + messageBuilder.key(newKey); + } + if (syncSend) { try { - producer.send(message); + messageBuilder.send(); } catch (PulsarClientException e) { LOGGER.error("Unable to write to Pulsar in appender [" + getName() + "]", e); } } else { - producer.sendAsync(message) + messageBuilder.sendAsync() .exceptionally(cause -> { LOGGER.error("Unable to write to Pulsar in appender [" + getName() + "]", cause); return null; diff --git a/pulsar-log4j2-appender/src/test/java/org/apache/pulsar/log4j2/appender/PulsarAppenderTest.java b/pulsar-log4j2-appender/src/test/java/org/apache/pulsar/log4j2/appender/PulsarAppenderTest.java index 4431243db205c..49273d6b080aa 100644 --- a/pulsar-log4j2-appender/src/test/java/org/apache/pulsar/log4j2/appender/PulsarAppenderTest.java +++ b/pulsar-log4j2-appender/src/test/java/org/apache/pulsar/log4j2/appender/PulsarAppenderTest.java @@ -25,6 +25,7 @@ import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; +import static org.testng.Assert.assertFalse; import static org.testng.AssertJUnit.assertEquals; import static org.testng.AssertJUnit.assertNotNull; import static org.testng.AssertJUnit.assertNull; @@ -38,8 +39,10 @@ import java.util.Date; import java.util.LinkedList; import java.util.List; +import java.util.Map; import java.util.concurrent.CompletableFuture; import java.util.concurrent.TimeUnit; + import org.apache.logging.log4j.Level; import org.apache.logging.log4j.core.Appender; import org.apache.logging.log4j.core.LogEvent; @@ -52,7 +55,11 @@ import org.apache.pulsar.client.api.Producer; import org.apache.pulsar.client.api.ProducerBuilder; import org.apache.pulsar.client.api.PulsarClient; +import org.apache.pulsar.client.api.PulsarClientException; +import org.apache.pulsar.client.api.Schema; +import org.apache.pulsar.client.api.TypedMessageBuilder; import org.apache.pulsar.client.impl.ClientBuilderImpl; +import org.apache.pulsar.client.impl.TypedMessageBuilderImpl; import org.testng.annotations.BeforeMethod; import org.testng.annotations.Test; @@ -77,6 +84,31 @@ private static Log4jLogEvent createLogEvent() { private LoggerContext ctx; + private class MockedMessageBuilder extends TypedMessageBuilderImpl { + + public MockedMessageBuilder() { + super(null, Schema.BYTES); + } + + @Override + public MessageId send() throws PulsarClientException { + synchronized (history) { + history.add(getMessage()); + } + + return mock(MessageId.class); + } + + @Override + public CompletableFuture sendAsync() { + synchronized (history) { + history.add(getMessage()); + } + + return CompletableFuture.completedFuture(mock(MessageId.class)); + } + } + @BeforeMethod public void setUp() throws Exception { history = new LinkedList<>(); @@ -97,33 +129,9 @@ public void setUp() throws Exception { doReturn(producerBuilder).when(producerBuilder).blockIfQueueFull(anyBoolean()); doReturn(producer).when(producerBuilder).create(); - when(producer.send(any(Message.class))) - .thenAnswer(invocationOnMock -> { - Message msg = invocationOnMock.getArgumentAt(0, Message.class); - synchronized (history) { - history.add(msg); - } - return null; - }); - - when(producer.sendAsync(any(Message.class))) - .thenAnswer(invocationOnMock -> { - Message msg = invocationOnMock.getArgumentAt(0, Message.class); - synchronized (history) { - history.add(msg); - } - CompletableFuture future = new CompletableFuture<>(); - future.complete(mock(MessageId.class)); - return future; - }); + when(producer.newMessage()).then(invocation -> new MockedMessageBuilder()); PulsarManager.PULSAR_CLIENT_BUILDER = () -> clientBuilder; - PulsarManager.MESSAGE_BUILDER = (key, data) -> { - Message msg = mock(Message.class); - when(msg.getKey()).thenReturn(key); - when(msg.getData()).thenReturn(data); - return msg; - }; ctx = Configurator.initialize( "PulsarAppenderTest", @@ -141,7 +149,7 @@ public void testAppendWithLayout() throws Exception { item = history.get(0); } assertNotNull(item); - assertNull(item.getKey()); + assertFalse(item.hasKey()); assertEquals("[" + LOG_MESSAGE + "]", new String(item.getData(), StandardCharsets.UTF_8)); } @@ -156,7 +164,7 @@ public void testAppendWithSerializedLayout() throws Exception { item = history.get(0); } assertNotNull(item); - assertNull(item.getKey()); + assertFalse(item.hasKey()); assertEquals(LOG_MESSAGE, deserializeLogEvent(item.getData()).getMessage().getFormattedMessage()); } @@ -170,7 +178,7 @@ public void testAsyncAppend() throws Exception { item = history.get(0); } assertNotNull(item); - assertNull(item.getKey()); + assertFalse(item.hasKey()); assertEquals(LOG_MESSAGE, new String(item.getData(), StandardCharsets.UTF_8)); } diff --git a/pulsar-spark/pom.xml b/pulsar-spark/pom.xml index 70bc75764cf91..9c0070850b2cf 100644 --- a/pulsar-spark/pom.xml +++ b/pulsar-spark/pom.xml @@ -37,7 +37,7 @@ ${project.groupId} - pulsar-client + pulsar-client-1x ${project.version} diff --git a/pulsar-storm/src/main/java/org/apache/pulsar/storm/PulsarBolt.java b/pulsar-storm/src/main/java/org/apache/pulsar/storm/PulsarBolt.java index bc95e312614a7..5259b5b618b6b 100644 --- a/pulsar-storm/src/main/java/org/apache/pulsar/storm/PulsarBolt.java +++ b/pulsar-storm/src/main/java/org/apache/pulsar/storm/PulsarBolt.java @@ -26,12 +26,11 @@ import java.util.concurrent.ConcurrentMap; import org.apache.pulsar.client.api.ClientBuilder; -import org.apache.pulsar.client.api.ClientConfiguration; -import org.apache.pulsar.client.api.Message; import org.apache.pulsar.client.api.Producer; -import org.apache.pulsar.client.api.ProducerConfiguration; import org.apache.pulsar.client.api.PulsarClientException; +import org.apache.pulsar.client.api.TypedMessageBuilder; import org.apache.pulsar.client.impl.ClientBuilderImpl; +import org.apache.pulsar.client.impl.TypedMessageBuilderImpl; import org.apache.pulsar.client.impl.conf.ClientConfigurationData; import org.apache.pulsar.client.impl.conf.ProducerConfigurationData; import org.apache.storm.metric.api.IMetric; @@ -44,7 +43,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -@SuppressWarnings("deprecation") public class PulsarBolt extends BaseRichBolt implements IMetric { /** * @@ -81,30 +79,6 @@ public PulsarBolt(PulsarBoltConfiguration pulsarBoltConf, ClientBuilder clientBu this.pulsarBoltConf = pulsarBoltConf; } - /** - * @deprecated Use {@link #PulsarBolt(PulsarBoltConfiguration, ClientBuilder)} - */ - @Deprecated - public PulsarBolt(PulsarBoltConfiguration pulsarBoltConf, ClientConfiguration clientConf) { - this(pulsarBoltConf, clientConf, new ProducerConfiguration()); - } - - /** - * @deprecated Use {@link #PulsarBolt(PulsarBoltConfiguration, ClientBuilder)} - */ - @Deprecated - public PulsarBolt(PulsarBoltConfiguration pulsarBoltConf, ClientConfiguration clientConf, - ProducerConfiguration producerConf) { - this.clientConf = clientConf.getConfigurationData().clone(); - this.producerConf = producerConf.getProducerConfigurationData().clone(); - Objects.requireNonNull(pulsarBoltConf.getServiceUrl()); - Objects.requireNonNull(pulsarBoltConf.getTopic()); - Objects.requireNonNull(pulsarBoltConf.getTupleToMessageMapper()); - this.clientConf.setServiceUrl(pulsarBoltConf.getServiceUrl()); - this.producerConf.setTopicName(pulsarBoltConf.getTopic()); - this.pulsarBoltConf = pulsarBoltConf; - } - @SuppressWarnings({ "rawtypes" }) @Override public void prepare(Map conf, TopologyContext context, OutputCollector collector) { @@ -133,15 +107,17 @@ public void execute(Tuple input) { try { if (producer != null) { // a message key can be provided in the mapper - Message msg = pulsarBoltConf.getTupleToMessageMapper().toMessage(input); - if (msg == null) { + TypedMessageBuilder msgBuilder = pulsarBoltConf.getTupleToMessageMapper() + .toMessage(producer.newMessage(), input); + if (msgBuilder == null) { if (LOG.isDebugEnabled()) { LOG.debug("[{}] Cannot send null message, acking the collector", boltId); } collector.ack(input); } else { - final long messageSizeToBeSent = msg.getData().length; - producer.sendAsync(msg).handle((r, ex) -> { + final long messageSizeToBeSent = ((TypedMessageBuilderImpl) msgBuilder).getContent() + .remaining(); + msgBuilder.sendAsync().handle((msgId, ex) -> { synchronized (collector) { if (ex != null) { collector.reportError(ex); @@ -153,7 +129,7 @@ public void execute(Tuple input) { ++messagesSent; messageSizeSent += messageSizeToBeSent; if (LOG.isDebugEnabled()) { - LOG.debug("[{}] Message sent with id {}", boltId, msg.getMessageId()); + LOG.debug("[{}] Message sent with id {}", boltId, msgId); } } } diff --git a/pulsar-storm/src/main/java/org/apache/pulsar/storm/PulsarSpout.java b/pulsar-storm/src/main/java/org/apache/pulsar/storm/PulsarSpout.java index 5df08042331a0..713007be9f9a8 100644 --- a/pulsar-storm/src/main/java/org/apache/pulsar/storm/PulsarSpout.java +++ b/pulsar-storm/src/main/java/org/apache/pulsar/storm/PulsarSpout.java @@ -31,9 +31,7 @@ import java.util.concurrent.TimeUnit; import org.apache.pulsar.client.api.ClientBuilder; -import org.apache.pulsar.client.api.ClientConfiguration; import org.apache.pulsar.client.api.Consumer; -import org.apache.pulsar.client.api.ConsumerConfiguration; import org.apache.pulsar.client.api.Message; import org.apache.pulsar.client.api.MessageId; import org.apache.pulsar.client.api.PulsarClientException; @@ -51,7 +49,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -@SuppressWarnings("deprecation") public class PulsarSpout extends BaseRichSpout implements IMetric { private static final long serialVersionUID = 1L; @@ -94,30 +91,7 @@ public PulsarSpout(PulsarSpoutConfiguration pulsarSpoutConf, ClientBuilder clien this.consumerConf = new ConsumerConfigurationData<>(); this.consumerConf.setTopicNames(Collections.singleton(pulsarSpoutConf.getTopic())); this.consumerConf.setSubscriptionName(pulsarSpoutConf.getSubscriptionName()); - - this.pulsarSpoutConf = pulsarSpoutConf; - this.failedRetriesTimeoutNano = pulsarSpoutConf.getFailedRetriesTimeout(TimeUnit.NANOSECONDS); - this.maxFailedRetries = pulsarSpoutConf.getMaxFailedRetries(); - } - - @Deprecated - public PulsarSpout(PulsarSpoutConfiguration pulsarSpoutConf, ClientConfiguration clientConf) { - this(pulsarSpoutConf, clientConf, new ConsumerConfiguration()); - } - - @Deprecated - public PulsarSpout(PulsarSpoutConfiguration pulsarSpoutConf, ClientConfiguration clientConf, - ConsumerConfiguration consumerConf) { - this.clientConf = clientConf.getConfigurationData().clone(); - this.consumerConf = consumerConf.getConfigurationData().clone(); - Objects.requireNonNull(pulsarSpoutConf.getServiceUrl()); - Objects.requireNonNull(pulsarSpoutConf.getTopic()); - Objects.requireNonNull(pulsarSpoutConf.getSubscriptionName()); - Objects.requireNonNull(pulsarSpoutConf.getMessageToValuesMapper()); - - this.clientConf.setServiceUrl(pulsarSpoutConf.getServiceUrl()); - this.consumerConf.setTopicNames(Collections.singleton(pulsarSpoutConf.getTopic())); - this.consumerConf.setSubscriptionName(pulsarSpoutConf.getSubscriptionName()); + this.consumerConf.setSubscriptionType(pulsarSpoutConf.getSubscriptionType()); this.pulsarSpoutConf = pulsarSpoutConf; this.failedRetriesTimeoutNano = pulsarSpoutConf.getFailedRetriesTimeout(TimeUnit.NANOSECONDS); diff --git a/pulsar-storm/src/main/java/org/apache/pulsar/storm/PulsarSpoutConfiguration.java b/pulsar-storm/src/main/java/org/apache/pulsar/storm/PulsarSpoutConfiguration.java index 7582d74c0e1f4..8bd139bbb5887 100644 --- a/pulsar-storm/src/main/java/org/apache/pulsar/storm/PulsarSpoutConfiguration.java +++ b/pulsar-storm/src/main/java/org/apache/pulsar/storm/PulsarSpoutConfiguration.java @@ -21,6 +21,8 @@ import java.util.Objects; import java.util.concurrent.TimeUnit; +import org.apache.pulsar.client.api.SubscriptionType; + /** * Class used to specify pulsar spout configuration * @@ -42,6 +44,8 @@ public class PulsarSpoutConfiguration extends PulsarStormConfiguration { private int maxFailedRetries = DEFAULT_MAX_FAILED_RETRIES; private boolean sharedConsumerEnabled = false; + private SubscriptionType subscriptionType = SubscriptionType.Shared; + /** * @return the subscription name for the consumer in the spout */ @@ -58,6 +62,14 @@ public void setSubscriptionName(String subscriptionName) { this.subscriptionName = subscriptionName; } + public SubscriptionType getSubscriptionType() { + return subscriptionType; + } + + public void setSubscriptionType(SubscriptionType subscriptionType) { + this.subscriptionType = subscriptionType; + } + /** * @return the mapper to convert pulsar message to a storm tuple */ diff --git a/pulsar-storm/src/main/java/org/apache/pulsar/storm/TupleToMessageMapper.java b/pulsar-storm/src/main/java/org/apache/pulsar/storm/TupleToMessageMapper.java index 703395838b538..452e0ce089dc6 100644 --- a/pulsar-storm/src/main/java/org/apache/pulsar/storm/TupleToMessageMapper.java +++ b/pulsar-storm/src/main/java/org/apache/pulsar/storm/TupleToMessageMapper.java @@ -21,7 +21,7 @@ import java.io.Serializable; import org.apache.pulsar.client.api.Message; - +import org.apache.pulsar.client.api.TypedMessageBuilder; import org.apache.storm.topology.OutputFieldsDeclarer; import org.apache.storm.tuple.Tuple; @@ -32,8 +32,30 @@ public interface TupleToMessageMapper extends Serializable { * * @param tuple * @return + * @deprecated use {@link #toMessage(TypedMessageBuilder, Tuple)} + */ + @Deprecated + default Message toMessage(Tuple tuple) { + return null; + } + + /** + * Set the value on a message builder to prepare the message to be published from the Bolt. + * + * @param tuple + * @return */ - public Message toMessage(Tuple tuple); + default TypedMessageBuilder toMessage(TypedMessageBuilder msgBuilder, Tuple tuple) { + // Default implementation provided for backward compatibility + Message msg = toMessage(tuple); + msgBuilder.value(msg.getData()) + .properties(msg.getProperties()); + if (msg.hasKey()) { + msgBuilder.key(msg.getKey()); + } + return msgBuilder; + } + /** * Declare the output schema for the bolt. diff --git a/pulsar-testclient/src/main/java/org/apache/pulsar/testclient/LoadSimulationClient.java b/pulsar-testclient/src/main/java/org/apache/pulsar/testclient/LoadSimulationClient.java index c8806ce4951eb..e9b25da026474 100644 --- a/pulsar-testclient/src/main/java/org/apache/pulsar/testclient/LoadSimulationClient.java +++ b/pulsar-testclient/src/main/java/org/apache/pulsar/testclient/LoadSimulationClient.java @@ -18,11 +18,17 @@ */ package org.apache.pulsar.testclient; +import com.beust.jcommander.JCommander; +import com.beust.jcommander.Parameter; +import com.beust.jcommander.ParameterException; +import com.google.common.util.concurrent.RateLimiter; + +import io.netty.util.concurrent.DefaultThreadFactory; + import java.io.DataInputStream; import java.io.DataOutputStream; import java.net.ServerSocket; import java.net.Socket; -import java.net.URL; import java.util.Map; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ExecutorService; @@ -35,25 +41,14 @@ import org.apache.pulsar.client.admin.PulsarAdmin; import org.apache.pulsar.client.admin.PulsarAdminException; -import org.apache.pulsar.client.api.ClientConfiguration; import org.apache.pulsar.client.api.Consumer; -import org.apache.pulsar.client.api.ConsumerConfiguration; import org.apache.pulsar.client.api.MessageId; import org.apache.pulsar.client.api.MessageListener; import org.apache.pulsar.client.api.Producer; -import org.apache.pulsar.client.api.ProducerConfiguration; import org.apache.pulsar.client.api.PulsarClient; -import org.apache.pulsar.client.impl.PulsarClientImpl; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import com.beust.jcommander.JCommander; -import com.beust.jcommander.Parameter; -import com.beust.jcommander.ParameterException; -import com.google.common.util.concurrent.RateLimiter; - -import io.netty.util.concurrent.DefaultThreadFactory; - /** * LoadSimulationClient is used to simulate client load by maintaining producers and consumers for topics. Instances of * this class are controlled across a network via LoadSimulationController. @@ -82,9 +77,6 @@ public class LoadSimulationClient { // Pulsar client to create producers and consumers with. private final PulsarClient client; - private final ProducerConfiguration producerConf; - private final ConsumerConfiguration consumerConf; - private final ClientConfiguration clientConf; private final int port; // A TradeUnit is a Consumer and Producer pair. The rate of message @@ -100,17 +92,18 @@ private static class TradeUnit { // message size may be sent/changed while reducing object creation, the // byte[] is wrapped in an AtomicReference. final AtomicReference payload; - final ProducerConfiguration producerConf; final PulsarClient client; final String topic; final Map payloadCache; public TradeUnit(final TradeConfiguration tradeConf, final PulsarClient client, - final ProducerConfiguration producerConf, final ConsumerConfiguration consumerConf, final Map payloadCache) throws Exception { - consumerFuture = client.subscribeAsync(tradeConf.topic, "Subscriber-" + tradeConf.topic, consumerConf); + consumerFuture = client.newConsumer() + .topic(tradeConf.topic) + .subscriptionName("Subscriber-" + tradeConf.topic) + .messageListener(ackListener) + .subscribeAsync(); this.payload = new AtomicReference<>(); - this.producerConf = producerConf; this.payloadCache = payloadCache; this.client = client; topic = tradeConf.topic; @@ -131,10 +124,14 @@ public void change(final TradeConfiguration tradeConf) { // Attempt to create a Producer indefinitely. Useful for ensuring // messages continue to be sent after broker // restarts occur. - private Producer getNewProducer() throws Exception { + private Producer getNewProducer() throws Exception { while (true) { try { - return client.createProducer(topic, producerConf); + return client.newProducer() + .topic(topic) + .sendTimeout(0, TimeUnit.SECONDS) + .create(); + } catch (Exception e) { Thread.sleep(10000); } @@ -146,8 +143,8 @@ private class MutableBoolean { } public void start() throws Exception { - Producer producer = getNewProducer(); - final Consumer consumer = consumerFuture.get(); + Producer producer = getNewProducer(); + final Consumer consumer = consumerFuture.get(); while (!stop.get()) { final MutableBoolean wellnessFlag = new MutableBoolean(); final Function exceptionHandler = e -> { @@ -249,7 +246,7 @@ private void handle(final byte command, final DataInputStream inputStream, final case TRADE_COMMAND: // Create the topic. It is assumed that the topic does not already exist. decodeProducerOptions(tradeConf, inputStream); - final TradeUnit tradeUnit = new TradeUnit(tradeConf, client, producerConf, consumerConf, payloadCache); + final TradeUnit tradeUnit = new TradeUnit(tradeConf, client, payloadCache); topicsToTradeUnits.put(tradeConf.topic, tradeUnit); executor.submit(() -> { try { @@ -305,7 +302,7 @@ private void handle(final byte command, final DataInputStream inputStream, final } // Make listener as lightweight as possible. - private static final MessageListener ackListener = Consumer::acknowledgeAsync; + private static final MessageListener ackListener = Consumer::acknowledgeAsync; /** * Create a LoadSimulationClient with the given JCommander arguments. @@ -317,28 +314,15 @@ public LoadSimulationClient(final MainArguments arguments) throws Exception { payloadCache = new ConcurrentHashMap<>(); topicsToTradeUnits = new ConcurrentHashMap<>(); - clientConf = new ClientConfiguration(); - - clientConf.setConnectionsPerBroker(4); - clientConf.setIoThreads(Runtime.getRuntime().availableProcessors()); - - // Disable stats on the clients to reduce CPU/memory usage. - clientConf.setStatsInterval(0, TimeUnit.SECONDS); - - producerConf = new ProducerConfiguration(); - - // Disable timeout. - producerConf.setSendTimeout(0, TimeUnit.SECONDS); - - producerConf.setMessageRoutingMode(ProducerConfiguration.MessageRoutingMode.RoundRobinPartition); - - // Enable batching. - producerConf.setBatchingMaxPublishDelay(1, TimeUnit.MILLISECONDS); - producerConf.setBatchingEnabled(true); - consumerConf = new ConsumerConfiguration(); - consumerConf.setMessageListener(ackListener); - admin = new PulsarAdmin(new URL(arguments.serviceURL), clientConf); - client = new PulsarClientImpl(arguments.serviceURL, clientConf); + admin = PulsarAdmin.builder() + .serviceHttpUrl(arguments.serviceURL) + .build(); + client = PulsarClient.builder() + .serviceUrl(arguments.serviceURL) + .connectionsPerBroker(4) + .ioThreads(Runtime.getRuntime().availableProcessors()) + .statsInterval(0, TimeUnit.SECONDS) + .build(); port = arguments.port; executor = Executors.newCachedThreadPool(new DefaultThreadFactory("test-client")); } diff --git a/site2/docs/adaptors-storm.md b/site2/docs/adaptors-storm.md index 1f4d7f5960b36..3ae3a2c7b2c47 100644 --- a/site2/docs/adaptors-storm.md +++ b/site2/docs/adaptors-storm.md @@ -27,13 +27,6 @@ The Pulsar Spout allows for the data published on a topic to be consumed by a St The tuples that fail to be processed by the downstream bolts will be re-injected by the spout with an exponential backoff, within a configurable timeout (the default is 60 seconds) or a configurable number of retries, whichever comes first, after which it is acknowledged by the consumer. Here's an example construction of a spout: ```java -// Configure a Pulsar Client -ClientConfiguration clientConf = new ClientConfiguration(); - -// Configure a Pulsar Consumer -ConsumerConfiguration consumerConf = new ConsumerConfiguration(); - -@SuppressWarnings("serial") MessageToValuesMapper messageToValuesMapper = new MessageToValuesMapper() { @Override @@ -56,7 +49,7 @@ spoutConf.setSubscriptionName("my-subscriber-name1"); spoutConf.setMessageToValuesMapper(messageToValuesMapper); // Create a Pulsar Spout -PulsarSpout spout = new PulsarSpout(spoutConf, clientConf, consumerConf); +PulsarSpout spout = new PulsarSpout(spoutConf); ``` ## Pulsar Bolt @@ -66,21 +59,14 @@ The Pulsar bolt allows data in a Storm topology to be published on a topic. It p A partitioned topic can also be used to publish messages on different topics. In the implementation of the `TupleToMessageMapper`, a "key" will need to be provided in the message which will send the messages with the same key to the same topic. Here's an example bolt: ```java -// Configure a Pulsar Client -ClientConfiguration clientConf = new ClientConfiguration(); - -// Configure a Pulsar Producer -ProducerConfiguration producerConf = new ProducerConfiguration(); - -@SuppressWarnings("serial") TupleToMessageMapper tupleToMessageMapper = new TupleToMessageMapper() { @Override - public Message toMessage(Tuple tuple) { + public TypedMessageBuilder toMessage(TypedMessageBuilder msgBuilder, Tuple tuple) { String receivedMessage = tuple.getString(0); // message processing String processedMsg = receivedMessage + "-processed"; - return MessageBuilder.create().setContent(processedMsg.getBytes()).build(); + return msgBuilder.value(processedMsg.getBytes()); } @Override @@ -96,7 +82,7 @@ boltConf.setTopic("persistent://my-property/usw/my-ns/my-topic2"); boltConf.setTupleToMessageMapper(tupleToMessageMapper); // Create a Pulsar Bolt -PulsarBolt bolt = new PulsarBolt(boltConf, clientConf); +PulsarBolt bolt = new PulsarBolt(boltConf); ``` ## Example diff --git a/site2/docs/security-authorization.md b/site2/docs/security-authorization.md index f9bd743df6fc7..2cd168a6c1ccc 100644 --- a/site2/docs/security-authorization.md +++ b/site2/docs/security-authorization.md @@ -80,35 +80,18 @@ You can use [Pulsar Admin Tools](admin-api-permissions.md) for managing permissi ### Pulsar admin authentication ```java -String authPluginClassName = "com.org.MyAuthPluginClass"; -String authParams = "param1:value1"; -boolean useTls = false; -boolean tlsAllowInsecureConnection = false; -String tlsTrustCertsFilePath = null; - -ClientConfiguration config = new ClientConfiguration(); -config.setAuthentication(authPluginClassName, authParams); -config.setUseTls(useTls); -config.setTlsAllowInsecureConnection(tlsAllowInsecureConnection); -config.setTlsTrustCertsFilePath(tlsTrustCertsFilePath); - -PulsarAdmin admin = new PulsarAdmin(url, config); +PulsarAdmin admin = PulsarAdmin.builder() + .serviceHttpUrl("http://broker:8080") + .authentication("com.org.MyAuthPluginClass", "param1:value1") + .build(); ``` To use TLS: ```java -String authPluginClassName = "com.org.MyAuthPluginClass"; -String authParams = "param1:value1"; -boolean useTls = false; -boolean tlsAllowInsecureConnection = false; -String tlsTrustCertsFilePath = null; - -ClientConfiguration config = new ClientConfiguration(); -config.setAuthentication(authPluginClassName, authParams); -config.setUseTls(useTls); -config.setTlsAllowInsecureConnection(tlsAllowInsecureConnection); -config.setTlsTrustCertsFilePath(tlsTrustCertsFilePath); - -PulsarAdmin admin = new PulsarAdmin(url, config); +PulsarAdmin admin = PulsarAdmin.builder() + .serviceHttpUrl("https://broker:8080") + .authentication("com.org.MyAuthPluginClass", "param1:value1") + .tlsTrustCertsFilePath("/path/to/trust/cert") + .build(); ``` diff --git a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/compaction/TestCompaction.java b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/compaction/TestCompaction.java index b9f514ee3211e..52846da18ca17 100644 --- a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/compaction/TestCompaction.java +++ b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/compaction/TestCompaction.java @@ -113,7 +113,7 @@ public void testPublishCompactAndConsumeRest(String serviceUrl) throws Exception pulsarCluster.runAdminCommandOnAnyBroker("namespaces", "set-clusters", "--clusters", pulsarCluster.getClusterName(), namespace); - try (PulsarClient client = PulsarClient.create(serviceUrl)) { + try (PulsarClient client = PulsarClient.builder().serviceUrl(serviceUrl).build()) { client.newConsumer().topic(topic).subscriptionName("sub1").subscribe().close(); try(Producer producer = client.newProducer(Schema.STRING).topic(topic).create()) { diff --git a/tests/pulsar-spark-test/pom.xml b/tests/pulsar-spark-test/pom.xml index fae0bda937cf8..b0d0179af19d5 100644 --- a/tests/pulsar-spark-test/pom.xml +++ b/tests/pulsar-spark-test/pom.xml @@ -39,32 +39,19 @@ org.apache.pulsar pulsar-spark ${project.version} - - - org.apache.pulsar - pulsar-client - - - org.apache.pulsar - pulsar-broker - ${project.version} + org.apache.pulsar.tests + integration + ${project.version} + test-jar - org.apache.pulsar - pulsar-broker - ${project.version} - test-jar - - - - org.apache.pulsar - managed-ledger-original - ${project.version} - test-jar + org.testcontainers + mysql + test @@ -78,4 +65,57 @@ + + + + + org.apache.maven.plugins + maven-surefire-plugin + + + true + + ${project.version} + ${project.build.directory} + + + + + + + + + integrationTests + + + integrationTests + + + + + + org.apache.maven.plugins + maven-surefire-plugin + + + + testRetryCount + 0 + + + listener + org.apache.pulsar.tests.PulsarTestListener,org.apache.pulsar.tests.AnnotationListener + + + -Xmx2G -XX:MaxDirectMemorySize=8G + -Dio.netty.leakDetectionLevel=advanced + + false + 1 + + + + + + diff --git a/tests/pulsar-spark-test/src/test/java/org/apache/pulsar/spark/SparkStreamingPulsarReceiverTest.java b/tests/pulsar-spark-test/src/test/java/org/apache/pulsar/spark/SparkStreamingPulsarReceiverTest.java index b89cd58b0a273..1200bb5dd2166 100644 --- a/tests/pulsar-spark-test/src/test/java/org/apache/pulsar/spark/SparkStreamingPulsarReceiverTest.java +++ b/tests/pulsar-spark-test/src/test/java/org/apache/pulsar/spark/SparkStreamingPulsarReceiverTest.java @@ -18,48 +18,36 @@ */ package org.apache.pulsar.spark; -import static org.mockito.Mockito.spy; import static org.mockito.Mockito.doNothing; - -import org.apache.pulsar.client.api.*; -import org.apache.spark.storage.StorageLevel; -import org.mockito.ArgumentCaptor; - +import static org.mockito.Mockito.spy; import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertNotNull; -import org.testng.annotations.AfterClass; -import org.testng.annotations.BeforeClass; +import org.apache.pulsar.client.api.ClientConfiguration; +import org.apache.pulsar.client.api.Consumer; +import org.apache.pulsar.client.api.ConsumerConfiguration; +import org.apache.pulsar.client.api.Message; +import org.apache.pulsar.client.api.MessageListener; +import org.apache.pulsar.client.api.Producer; +import org.apache.pulsar.client.api.ProducerConfiguration; +import org.apache.pulsar.client.api.PulsarClient; +import org.apache.pulsar.tests.integration.suites.PulsarTestSuite; +import org.apache.spark.storage.StorageLevel; +import org.mockito.ArgumentCaptor; import org.testng.annotations.Test; -import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest; - -public class SparkStreamingPulsarReceiverTest extends MockedPulsarServiceBaseTest { - - private final String URL = "pulsar://127.0.0.1:" + BROKER_PORT + "/"; +public class SparkStreamingPulsarReceiverTest extends PulsarTestSuite { private static final String TOPIC = "persistent://p1/c1/ns1/topic1"; private static final String SUBS = "sub1"; private static final String EXPECTED_MESSAGE = "pulsar-spark test message"; - @BeforeClass - @Override - protected void setup() throws Exception { - internalSetup(); - } - - @AfterClass - @Override - protected void cleanup() throws Exception { - internalCleanup(); - } - - @Test - public void testReceivedMessage() throws Exception { + @Test(dataProvider = "ServiceUrls") + public void testReceivedMessage(String serviceUrl) throws Exception { ClientConfiguration clientConf = new ClientConfiguration(); ConsumerConfiguration consConf = new ConsumerConfiguration(); SparkStreamingPulsarReceiver receiver = spy( - new SparkStreamingPulsarReceiver(clientConf, consConf, URL, TOPIC, SUBS)); + new SparkStreamingPulsarReceiver(clientConf, consConf, serviceUrl, TOPIC, SUBS)); MessageListener msgListener = spy(new MessageListener() { @Override public void received(Consumer consumer, Message msg) { @@ -73,7 +61,7 @@ public void received(Consumer consumer, Message msg) { receiver.onStart(); waitForTransmission(); - PulsarClient pulsarClient = PulsarClient.create(URL, clientConf); + PulsarClient pulsarClient = PulsarClient.create(serviceUrl, clientConf); Producer producer = pulsarClient.createProducer(TOPIC, new ProducerConfiguration()); producer.send(EXPECTED_MESSAGE.getBytes()); waitForTransmission(); @@ -81,27 +69,30 @@ public void received(Consumer consumer, Message msg) { assertEquals(new String(msgCaptor.getValue().getData()), EXPECTED_MESSAGE); } - @Test - public void testDefaultSettingsOfReceiver() { + + @Test(dataProvider = "ServiceUrls") + public void testDefaultSettingsOfReceiver(String serviceUrl) throws Exception { ClientConfiguration clientConf = new ClientConfiguration(); ConsumerConfiguration consConf = new ConsumerConfiguration(); SparkStreamingPulsarReceiver receiver = - new SparkStreamingPulsarReceiver(clientConf, consConf, URL, TOPIC, SUBS); + new SparkStreamingPulsarReceiver(clientConf, consConf, serviceUrl, TOPIC, SUBS); assertEquals(receiver.storageLevel(), StorageLevel.MEMORY_AND_DISK_2()); assertEquals(consConf.getAckTimeoutMillis(), 60_000); assertNotNull(consConf.getMessageListener()); } @Test(expectedExceptions = NullPointerException.class, - expectedExceptionsMessageRegExp = "ClientConfiguration must not be null") - public void testReceiverWhenClientConfigurationIsNull() { - new SparkStreamingPulsarReceiver(null, new ConsumerConfiguration(), URL, TOPIC, SUBS); + expectedExceptionsMessageRegExp = "ClientConfiguration must not be null", + dataProvider = "ServiceUrls") + public void testReceiverWhenClientConfigurationIsNull(String serviceUrl) { + new SparkStreamingPulsarReceiver(null, new ConsumerConfiguration(), serviceUrl, TOPIC, SUBS); } @Test(expectedExceptions = NullPointerException.class, - expectedExceptionsMessageRegExp = "ConsumerConfiguration must not be null") - public void testReceiverWhenConsumerConfigurationIsNull() { - new SparkStreamingPulsarReceiver(new ClientConfiguration(), null, URL, TOPIC, SUBS); + expectedExceptionsMessageRegExp = "ConsumerConfiguration must not be null", + dataProvider = "ServiceUrls") + public void testReceiverWhenConsumerConfigurationIsNull(String serviceUrl) { + new SparkStreamingPulsarReceiver(new ClientConfiguration(), null, serviceUrl, TOPIC, SUBS); } private static void waitForTransmission() { diff --git a/tests/pulsar-storm-test/src/test/java/org/apache/pulsar/storm/PulsarBoltTest.java b/tests/pulsar-storm-test/src/test/java/org/apache/pulsar/storm/PulsarBoltTest.java index 0a1787097287e..50ff491794a84 100644 --- a/tests/pulsar-storm-test/src/test/java/org/apache/pulsar/storm/PulsarBoltTest.java +++ b/tests/pulsar-storm-test/src/test/java/org/apache/pulsar/storm/PulsarBoltTest.java @@ -20,29 +20,28 @@ import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; +import static org.testng.Assert.fail; import java.lang.reflect.Method; import java.util.Map; import java.util.concurrent.TimeUnit; -import org.apache.pulsar.client.api.ClientConfiguration; import org.apache.pulsar.client.api.Consumer; import org.apache.pulsar.client.api.Message; -import org.apache.pulsar.client.api.MessageBuilder; import org.apache.pulsar.client.api.ProducerConsumerBase; +import org.apache.pulsar.client.api.PulsarClient; +import org.apache.pulsar.client.api.TypedMessageBuilder; import org.apache.pulsar.common.policies.data.TopicStats; +import org.apache.storm.task.OutputCollector; +import org.apache.storm.task.TopologyContext; +import org.apache.storm.topology.OutputFieldsDeclarer; +import org.apache.storm.tuple.Tuple; import org.testng.Assert; -import static org.testng.Assert.fail; import org.testng.annotations.AfterMethod; import org.testng.annotations.BeforeMethod; import org.testng.annotations.Test; import org.testng.collections.Maps; -import org.apache.storm.task.OutputCollector; -import org.apache.storm.task.TopologyContext; -import org.apache.storm.topology.OutputFieldsDeclarer; -import org.apache.storm.tuple.Tuple; - public class PulsarBoltTest extends ProducerConsumerBase { private static final int NO_OF_RETRIES = 10; @@ -73,14 +72,14 @@ protected void setup() throws Exception { pulsarBoltConf.setTopic(topic); pulsarBoltConf.setTupleToMessageMapper(tupleToMessageMapper); pulsarBoltConf.setMetricsTimeIntervalInSecs(60); - bolt = new PulsarBolt(pulsarBoltConf, new ClientConfiguration()); + bolt = new PulsarBolt(pulsarBoltConf, PulsarClient.builder()); mockCollector = new MockOutputCollector(); OutputCollector collector = new OutputCollector(mockCollector); TopologyContext context = mock(TopologyContext.class); when(context.getThisComponentId()).thenReturn("test-bolt-" + methodName); when(context.getThisTaskId()).thenReturn(0); bolt.prepare(Maps.newHashMap(), context, collector); - consumer = pulsarClient.subscribe(topic, subscriptionName); + consumer = pulsarClient.newConsumer().topic(topic).subscriptionName(subscriptionName).subscribe(); } @AfterMethod @@ -94,14 +93,14 @@ public void cleanup() throws Exception { static TupleToMessageMapper tupleToMessageMapper = new TupleToMessageMapper() { @Override - public Message toMessage(Tuple tuple) { + public TypedMessageBuilder toMessage(TypedMessageBuilder msgBuilder, Tuple tuple) { if ("message to be dropped".equals(new String(tuple.getBinary(0)))) { return null; } if ("throw exception".equals(new String(tuple.getBinary(0)))) { throw new RuntimeException(); } - return MessageBuilder.create().setContent(tuple.getBinary(0)).build(); + return msgBuilder.value(tuple.getBinary(0)); } @Override @@ -190,7 +189,7 @@ public void testMetrics() throws Exception { public void testSharedProducer() throws Exception { TopicStats topicStats = admin.topics().getStats(topic); Assert.assertEquals(topicStats.publishers.size(), 1); - PulsarBolt otherBolt = new PulsarBolt(pulsarBoltConf, new ClientConfiguration()); + PulsarBolt otherBolt = new PulsarBolt(pulsarBoltConf, PulsarClient.builder()); MockOutputCollector otherMockCollector = new MockOutputCollector(); OutputCollector collector = new OutputCollector(otherMockCollector); TopologyContext context = mock(TopologyContext.class); @@ -210,7 +209,7 @@ public void testSharedProducer() throws Exception { @Test public void testSerializability() throws Exception { // test serializability with no auth - PulsarBolt boltWithNoAuth = new PulsarBolt(pulsarBoltConf, new ClientConfiguration()); + PulsarBolt boltWithNoAuth = new PulsarBolt(pulsarBoltConf, PulsarClient.builder()); TestUtil.testSerializability(boltWithNoAuth); } @@ -221,7 +220,7 @@ public void testFailedProducer() { pulsarBoltConf.setTopic("persistent://invalid"); pulsarBoltConf.setTupleToMessageMapper(tupleToMessageMapper); pulsarBoltConf.setMetricsTimeIntervalInSecs(60); - PulsarBolt bolt = new PulsarBolt(pulsarBoltConf, new ClientConfiguration()); + PulsarBolt bolt = new PulsarBolt(pulsarBoltConf, PulsarClient.builder()); MockOutputCollector mockCollector = new MockOutputCollector(); OutputCollector collector = new OutputCollector(mockCollector); TopologyContext context = mock(TopologyContext.class); diff --git a/tests/pulsar-storm-test/src/test/java/org/apache/pulsar/storm/PulsarSpoutTest.java b/tests/pulsar-storm-test/src/test/java/org/apache/pulsar/storm/PulsarSpoutTest.java index dd7de0f682b98..d316be394e5b1 100644 --- a/tests/pulsar-storm-test/src/test/java/org/apache/pulsar/storm/PulsarSpoutTest.java +++ b/tests/pulsar-storm-test/src/test/java/org/apache/pulsar/storm/PulsarSpoutTest.java @@ -20,33 +20,27 @@ import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; +import static org.testng.Assert.fail; import java.lang.reflect.Method; import java.util.Map; import java.util.concurrent.TimeUnit; -import org.apache.pulsar.storm.MessageToValuesMapper; -import org.apache.pulsar.storm.PulsarSpout; -import org.apache.pulsar.storm.PulsarSpoutConfiguration; -import org.testng.Assert; -import static org.testng.Assert.fail; -import org.testng.annotations.AfterMethod; -import org.testng.annotations.BeforeMethod; -import org.testng.annotations.Test; -import org.testng.collections.Maps; - -import org.apache.pulsar.client.api.ClientConfiguration; -import org.apache.pulsar.client.api.ConsumerConfiguration; import org.apache.pulsar.client.api.Message; import org.apache.pulsar.client.api.Producer; import org.apache.pulsar.client.api.ProducerConsumerBase; +import org.apache.pulsar.client.api.PulsarClient; import org.apache.pulsar.client.api.SubscriptionType; import org.apache.pulsar.common.policies.data.TopicStats; - import org.apache.storm.spout.SpoutOutputCollector; import org.apache.storm.task.TopologyContext; import org.apache.storm.topology.OutputFieldsDeclarer; import org.apache.storm.tuple.Values; +import org.testng.Assert; +import org.testng.annotations.AfterMethod; +import org.testng.annotations.BeforeMethod; +import org.testng.annotations.Test; +import org.testng.collections.Maps; public class PulsarSpoutTest extends ProducerConsumerBase { @@ -55,7 +49,6 @@ public class PulsarSpoutTest extends ProducerConsumerBase { public final String subscriptionName = "my-subscriber-name"; protected PulsarSpoutConfiguration pulsarSpoutConf; - protected ConsumerConfiguration consumerConf; protected PulsarSpout spout; protected MockSpoutOutputCollector mockCollector; protected Producer producer; @@ -81,16 +74,15 @@ protected void setup() throws Exception { pulsarSpoutConf.setMaxFailedRetries(2); pulsarSpoutConf.setSharedConsumerEnabled(true); pulsarSpoutConf.setMetricsTimeIntervalInSecs(60); - consumerConf = new ConsumerConfiguration(); - consumerConf.setSubscriptionType(SubscriptionType.Shared); - spout = new PulsarSpout(pulsarSpoutConf, new ClientConfiguration(), consumerConf); + pulsarSpoutConf.setSubscriptionType(SubscriptionType.Shared); + spout = new PulsarSpout(pulsarSpoutConf, PulsarClient.builder()); mockCollector = new MockSpoutOutputCollector(); SpoutOutputCollector collector = new SpoutOutputCollector(mockCollector); TopologyContext context = mock(TopologyContext.class); when(context.getThisComponentId()).thenReturn("test-spout-" + methodName); when(context.getThisTaskId()).thenReturn(0); spout.open(Maps.newHashMap(), context, collector); - producer = pulsarClient.createProducer(topic); + producer = pulsarClient.newProducer().topic(topic).create(); } @AfterMethod @@ -278,7 +270,7 @@ public void testMetrics() throws Exception { public void testSharedConsumer() throws Exception { TopicStats topicStats = admin.topics().getStats(topic); Assert.assertEquals(topicStats.subscriptions.get(subscriptionName).consumers.size(), 1); - PulsarSpout otherSpout = new PulsarSpout(pulsarSpoutConf, new ClientConfiguration(), consumerConf); + PulsarSpout otherSpout = new PulsarSpout(pulsarSpoutConf, PulsarClient.builder()); MockSpoutOutputCollector otherMockCollector = new MockSpoutOutputCollector(); SpoutOutputCollector collector = new SpoutOutputCollector(otherMockCollector); TopologyContext context = mock(TopologyContext.class); @@ -300,7 +292,7 @@ public void testNoSharedConsumer() throws Exception { TopicStats topicStats = admin.topics().getStats(topic); Assert.assertEquals(topicStats.subscriptions.get(subscriptionName).consumers.size(), 1); pulsarSpoutConf.setSharedConsumerEnabled(false); - PulsarSpout otherSpout = new PulsarSpout(pulsarSpoutConf, new ClientConfiguration(), consumerConf); + PulsarSpout otherSpout = new PulsarSpout(pulsarSpoutConf, PulsarClient.builder()); MockSpoutOutputCollector otherMockCollector = new MockSpoutOutputCollector(); SpoutOutputCollector collector = new SpoutOutputCollector(otherMockCollector); TopologyContext context = mock(TopologyContext.class); @@ -320,7 +312,7 @@ public void testNoSharedConsumer() throws Exception { @Test public void testSerializability() throws Exception { // test serializability with no auth - PulsarSpout spoutWithNoAuth = new PulsarSpout(pulsarSpoutConf, new ClientConfiguration()); + PulsarSpout spoutWithNoAuth = new PulsarSpout(pulsarSpoutConf, PulsarClient.builder()); TestUtil.testSerializability(spoutWithNoAuth); } @@ -335,9 +327,8 @@ public void testFailedConsumer() throws Exception { pulsarSpoutConf.setMaxFailedRetries(2); pulsarSpoutConf.setSharedConsumerEnabled(false); pulsarSpoutConf.setMetricsTimeIntervalInSecs(60); - ConsumerConfiguration consumerConf = new ConsumerConfiguration(); - consumerConf.setSubscriptionType(SubscriptionType.Shared); - PulsarSpout spout = new PulsarSpout(pulsarSpoutConf, new ClientConfiguration(), consumerConf); + pulsarSpoutConf.setSubscriptionType(SubscriptionType.Shared); + PulsarSpout spout = new PulsarSpout(pulsarSpoutConf, PulsarClient.builder()); MockSpoutOutputCollector mockCollector = new MockSpoutOutputCollector(); SpoutOutputCollector collector = new SpoutOutputCollector(mockCollector); TopologyContext context = mock(TopologyContext.class); diff --git a/tests/pulsar-storm-test/src/test/java/org/apache/pulsar/storm/example/StormExample.java b/tests/pulsar-storm-test/src/test/java/org/apache/pulsar/storm/example/StormExample.java index 344a082265da0..a3f85107d6c7a 100644 --- a/tests/pulsar-storm-test/src/test/java/org/apache/pulsar/storm/example/StormExample.java +++ b/tests/pulsar-storm-test/src/test/java/org/apache/pulsar/storm/example/StormExample.java @@ -22,15 +22,18 @@ import java.util.Map; import java.util.concurrent.TimeUnit; +import org.apache.pulsar.client.api.Consumer; +import org.apache.pulsar.client.api.Message; +import org.apache.pulsar.client.api.Producer; +import org.apache.pulsar.client.api.PulsarClient; +import org.apache.pulsar.client.api.PulsarClientException; +import org.apache.pulsar.client.api.TypedMessageBuilder; import org.apache.pulsar.storm.MessageToValuesMapper; import org.apache.pulsar.storm.PulsarBolt; import org.apache.pulsar.storm.PulsarBoltConfiguration; import org.apache.pulsar.storm.PulsarSpout; import org.apache.pulsar.storm.PulsarSpoutConfiguration; import org.apache.pulsar.storm.TupleToMessageMapper; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - import org.apache.storm.Config; import org.apache.storm.LocalCluster; import org.apache.storm.metric.api.IMetricsConsumer; @@ -42,14 +45,8 @@ import org.apache.storm.tuple.Tuple; import org.apache.storm.tuple.Values; import org.apache.storm.utils.Utils; - -import org.apache.pulsar.client.api.ClientConfiguration; -import org.apache.pulsar.client.api.PulsarClient; -import org.apache.pulsar.client.api.PulsarClientException; -import org.apache.pulsar.client.api.Consumer; -import org.apache.pulsar.client.api.Message; -import org.apache.pulsar.client.api.MessageBuilder; -import org.apache.pulsar.client.api.Producer; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; public class StormExample { private static final Logger LOG = LoggerFactory.getLogger(PulsarSpout.class); @@ -74,11 +71,11 @@ public void declareOutputFields(OutputFieldsDeclarer declarer) { static TupleToMessageMapper tupleToMessageMapper = new TupleToMessageMapper() { @Override - public Message toMessage(Tuple tuple) { + public TypedMessageBuilder toMessage(TypedMessageBuilder msgBuilder, Tuple tuple) { String receivedMessage = tuple.getString(0); // message processing String processedMsg = receivedMessage + "-processed"; - return MessageBuilder.create().setContent(processedMsg.getBytes()).build(); + return msgBuilder.value(processedMsg.getBytes()); } @Override @@ -88,8 +85,6 @@ public void declareOutputFields(OutputFieldsDeclarer declarer) { }; public static void main(String[] args) throws PulsarClientException { - - ClientConfiguration clientConf = new ClientConfiguration(); // String authPluginClassName = "org.apache.pulsar.client.impl.auth.MyAuthentication"; // String authParams = "key1:val1,key2:val2"; // clientConf.setAuthentication(authPluginClassName, authParams); @@ -105,14 +100,14 @@ public static void main(String[] args) throws PulsarClientException { spoutConf.setTopic(topic1); spoutConf.setSubscriptionName(subscriptionName1); spoutConf.setMessageToValuesMapper(messageToValuesMapper); - PulsarSpout spout = new PulsarSpout(spoutConf, clientConf); + PulsarSpout spout = new PulsarSpout(spoutConf, PulsarClient.builder()); // create bolt PulsarBoltConfiguration boltConf = new PulsarBoltConfiguration(); boltConf.setServiceUrl(serviceUrl); boltConf.setTopic(topic2); boltConf.setTupleToMessageMapper(tupleToMessageMapper); - PulsarBolt bolt = new PulsarBolt(boltConf, clientConf); + PulsarBolt bolt = new PulsarBolt(boltConf, PulsarClient.builder()); TopologyBuilder builder = new TopologyBuilder(); builder.setSpout("testSpout", spout); @@ -127,18 +122,18 @@ public static void main(String[] args) throws PulsarClientException { cluster.submitTopology("test", conf, builder.createTopology()); Utils.sleep(10000); - PulsarClient pulsarClient = PulsarClient.create(serviceUrl, clientConf); + PulsarClient pulsarClient = PulsarClient.builder().serviceUrl(serviceUrl).build(); // create a consumer on topic2 to receive messages from the bolt when the processing is done - Consumer consumer = pulsarClient.subscribe(topic2, subscriptionName2); + Consumer consumer = pulsarClient.newConsumer().topic(topic2).subscriptionName(subscriptionName2).subscribe(); // create a producer on topic1 to send messages that will be received by the spout - Producer producer = pulsarClient.createProducer(topic1); + Producer producer = pulsarClient.newProducer().topic(topic1).create(); for (int i = 0; i < 10; i++) { String msg = "msg-" + i; producer.send(msg.getBytes()); LOG.info("Message {} sent", msg); } - Message msg = null; + Message msg = null; for (int i = 0; i < 10; i++) { msg = consumer.receive(1, TimeUnit.SECONDS); LOG.info("Message {} received", new String(msg.getData()));