Skip to content

Commit

Permalink
Allow byte[] keys for messages (apache#1016) (apache#2612)
Browse files Browse the repository at this point in the history
Sometimes it can be useful to send something more complex than a
string as the key of the message. However, early on Pulsar chose to
make String the only way to send a key, and this permeates throughout
the code, so we can't very well change it now.

This patch adds rudamentary byte[] key support. If a user adds a
byte[] key, the byte[] is base64 encoded and stored in the normal key
field. We also send a flag to denote that it is base64 encoded, so the
receiving end knows to decode it correct. There's no schema or
anything attached to this. Any SerDe has to be handled manually by the
client.
  • Loading branch information
ivankelly authored and sijie committed Sep 27, 2018
1 parent e15a606 commit 30f6935
Show file tree
Hide file tree
Showing 9 changed files with 257 additions and 2 deletions.
Original file line number Diff line number Diff line change
@@ -0,0 +1,76 @@
/**
* 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.util.Random;
import java.util.concurrent.TimeUnit;

import org.testng.Assert;
import org.testng.annotations.AfterMethod;
import org.testng.annotations.BeforeMethod;
import org.testng.annotations.Test;

public class BytesKeyTest extends ProducerConsumerBase {

@BeforeMethod
@Override
protected void setup() throws Exception {
super.internalSetup();
super.producerBaseSetup();
}

@AfterMethod
@Override
protected void cleanup() throws Exception {
super.internalCleanup();
}

private void byteKeysTest(boolean batching) throws Exception {
Random r = new Random(0);
Consumer<String> consumer = pulsarClient.newConsumer(Schema.STRING)
.topic("persistent://my-property/my-ns/my-topic1")
.subscriptionName("my-subscriber-name").subscribe();

Producer<String> producer = pulsarClient.newProducer(Schema.STRING)
.enableBatching(batching)
.batchingMaxPublishDelay(Long.MAX_VALUE, TimeUnit.SECONDS)
.batchingMaxMessages(Integer.MAX_VALUE)
.topic("persistent://my-property/my-ns/my-topic1").create();

byte[] byteKey = new byte[1000];
r.nextBytes(byteKey);
producer.newMessage().keyBytes(byteKey).value("TestMessage").sendAsync();
producer.flush();

Message<String> m = consumer.receive();
Assert.assertEquals(m.getValue(), "TestMessage");
Assert.assertEquals(m.getKeyBytes(), byteKey);
Assert.assertTrue(m.hasBase64EncodedKey());
}

@Test
public void testBytesKeyBatch() throws Exception {
byteKeysTest(true);
}

@Test
public void testBytesKeyNoBatch() throws Exception {
byteKeysTest(false);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -131,7 +131,21 @@ public interface Message<T> {
* @return the key of the message
*/
String getKey();


/**
* Check whether the key has been base64 encoded.
*
* @return true if the key is base64 encoded, false otherwise
*/
boolean hasBase64EncodedKey();

/**
* Get bytes in key. If the key has been base64 encoded, it is decoded before being returned.
* Otherwise, if the key is a plain string, this method returns the UTF_8 encoded bytes of the string.
* @return the key in byte[] form
*/
byte[] getKeyBytes();

/**
* {@link EncryptionContext} contains encryption and compression information in it using which application can
* decrypt consumed message with encrypted-payload.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -89,6 +89,14 @@ public interface TypedMessageBuilder<T> extends Serializable {
*/
TypedMessageBuilder<T> key(String key);

/**
* Sets the bytes of the key of the message for routing policy.
* Internally the bytes will be base64 encoded.
*
* @param key routing key for message, in byte array form
*/
TypedMessageBuilder<T> keyBytes(byte[] key);

/**
* Set a domain object on the message
*
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@
package org.apache.pulsar.client.impl;

import static com.google.common.base.Preconditions.checkNotNull;
import static java.nio.charset.StandardCharsets.UTF_8;

import com.google.common.collect.Maps;

Expand All @@ -29,6 +30,7 @@

import java.io.IOException;
import java.nio.ByteBuffer;
import java.util.Base64;
import java.util.Collections;
import java.util.List;
import java.util.Map;
Expand Down Expand Up @@ -129,6 +131,7 @@ static MessageImpl<byte[]> create(MessageMetadata.Builder msgMetadataBuilder, By
}

if (singleMessageMetadata.hasPartitionKey()) {
msgMetadataBuilder.setPartitionKeyB64Encoded(singleMessageMetadata.getPartitionKeyB64Encoded());
msgMetadataBuilder.setPartitionKey(singleMessageMetadata.getPartitionKey());
}

Expand Down Expand Up @@ -290,6 +293,22 @@ public String getKey() {
return msgMetadataBuilder.getPartitionKey();
}

@Override
public boolean hasBase64EncodedKey() {
checkNotNull(msgMetadataBuilder);
return msgMetadataBuilder.getPartitionKeyB64Encoded();
}

@Override
public byte[] getKeyBytes() {
checkNotNull(msgMetadataBuilder);
if (hasBase64EncodedKey()) {
return Base64.getDecoder().decode(getKey());
} else {
return getKey().getBytes(UTF_8);
}
}

public ClientCnx getCnx() {
return cnx;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -118,6 +118,16 @@ public String getKey() {
return msg.getKey();
}

@Override
public boolean hasBase64EncodedKey() {
return msg.hasBase64EncodedKey();
}

@Override
public byte[] getKeyBytes() {
return msg.getKeyBytes();
}

@Override
public T getValue() {
return msg.getValue();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,7 @@
import com.google.common.base.Preconditions;

import java.nio.ByteBuffer;
import java.util.Base64;
import java.util.List;
import java.util.Map;
import java.util.concurrent.CompletableFuture;
Expand Down Expand Up @@ -62,6 +63,14 @@ public CompletableFuture<MessageId> sendAsync() {
@Override
public TypedMessageBuilder<T> key(String key) {
msgMetadataBuilder.setPartitionKey(key);
msgMetadataBuilder.setPartitionKeyB64Encoded(false);
return this;
}

@Override
public TypedMessageBuilder<T> keyBytes(byte[] key) {
msgMetadataBuilder.setPartitionKey(Base64.getEncoder().encodeToString(key));
msgMetadataBuilder.setPartitionKeyB64Encoded(true);
return this;
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -1042,7 +1042,8 @@ public static ByteBuf serializeSingleMessageInBatchWithPayload(PulsarApi.Message
PulsarApi.SingleMessageMetadata.Builder singleMessageMetadataBuilder = PulsarApi.SingleMessageMetadata
.newBuilder();
if (msgBuilder.hasPartitionKey()) {
singleMessageMetadataBuilder = singleMessageMetadataBuilder.setPartitionKey(msgBuilder.getPartitionKey());
singleMessageMetadataBuilder = singleMessageMetadataBuilder.setPartitionKey(msgBuilder.getPartitionKey())
.setPartitionKeyB64Encoded(msgBuilder.getPartitionKeyB64Encoded());
}
if (!msgBuilder.getPropertiesList().isEmpty()) {
singleMessageMetadataBuilder = singleMessageMetadataBuilder
Expand Down
Loading

0 comments on commit 30f6935

Please sign in to comment.