This is an automated email from the ASF dual-hosted git repository.

sijie pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/pulsar.git


The following commit(s) were added to refs/heads/master by this push:
     new 30f6935  Allow byte[] keys for messages (#1016) (#2612)
30f6935 is described below

commit 30f693521269107cf613a2df59967c871c14b519
Author: Ivan Kelly <iv...@apache.org>
AuthorDate: Thu Sep 27 08:20:23 2018 +0200

    Allow byte[] keys for messages (#1016) (#2612)
    
    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.
---
 .../org/apache/pulsar/client/api/BytesKeyTest.java |  76 ++++++++++++++
 .../java/org/apache/pulsar/client/api/Message.java |  16 ++-
 .../pulsar/client/api/TypedMessageBuilder.java     |   8 ++
 .../org/apache/pulsar/client/impl/MessageImpl.java |  19 ++++
 .../pulsar/client/impl/TopicMessageImpl.java       |  10 ++
 .../client/impl/TypedMessageBuilderImpl.java       |   9 ++
 .../org/apache/pulsar/common/api/Commands.java     |   3 +-
 .../apache/pulsar/common/api/proto/PulsarApi.java  | 114 +++++++++++++++++++++
 pulsar-common/src/main/proto/PulsarApi.proto       |   4 +
 9 files changed, 257 insertions(+), 2 deletions(-)

diff --git 
a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/BytesKeyTest.java 
b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/BytesKeyTest.java
new file mode 100644
index 0000000..3fa8a3a
--- /dev/null
+++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/BytesKeyTest.java
@@ -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);
+    }
+}
diff --git 
a/pulsar-client/src/main/java/org/apache/pulsar/client/api/Message.java 
b/pulsar-client/src/main/java/org/apache/pulsar/client/api/Message.java
index d614962..a33bd2d 100644
--- a/pulsar-client/src/main/java/org/apache/pulsar/client/api/Message.java
+++ b/pulsar-client/src/main/java/org/apache/pulsar/client/api/Message.java
@@ -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.
diff --git 
a/pulsar-client/src/main/java/org/apache/pulsar/client/api/TypedMessageBuilder.java
 
b/pulsar-client/src/main/java/org/apache/pulsar/client/api/TypedMessageBuilder.java
index 72cbbc7..019e50a 100644
--- 
a/pulsar-client/src/main/java/org/apache/pulsar/client/api/TypedMessageBuilder.java
+++ 
b/pulsar-client/src/main/java/org/apache/pulsar/client/api/TypedMessageBuilder.java
@@ -90,6 +90,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
      *
      * @param value
diff --git 
a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MessageImpl.java 
b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MessageImpl.java
index 97e2247..620946a 100644
--- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MessageImpl.java
+++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MessageImpl.java
@@ -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;
 
@@ -29,6 +30,7 @@ import io.netty.util.Recycler.Handle;
 
 import java.io.IOException;
 import java.nio.ByteBuffer;
+import java.util.Base64;
 import java.util.Collections;
 import java.util.List;
 import java.util.Map;
@@ -129,6 +131,7 @@ public class MessageImpl<T> implements Message<T> {
         }
 
         if (singleMessageMetadata.hasPartitionKey()) {
+            
msgMetadataBuilder.setPartitionKeyB64Encoded(singleMessageMetadata.getPartitionKeyB64Encoded());
             
msgMetadataBuilder.setPartitionKey(singleMessageMetadata.getPartitionKey());
         }
 
@@ -290,6 +293,22 @@ public class MessageImpl<T> implements Message<T> {
         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;
     }
diff --git 
a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/TopicMessageImpl.java
 
b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/TopicMessageImpl.java
index 230a022..c7a9887 100644
--- 
a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/TopicMessageImpl.java
+++ 
b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/TopicMessageImpl.java
@@ -119,6 +119,16 @@ public class TopicMessageImpl<T> implements Message<T> {
     }
 
     @Override
+    public boolean hasBase64EncodedKey() {
+        return msg.hasBase64EncodedKey();
+    }
+
+    @Override
+    public byte[] getKeyBytes() {
+        return msg.getKeyBytes();
+    }
+
+    @Override
     public T getValue() {
         return msg.getValue();
     }
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 f30edd6..0b3f0a4 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
@@ -23,6 +23,7 @@ import static 
com.google.common.base.Preconditions.checkArgument;
 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;
@@ -62,6 +63,14 @@ public class TypedMessageBuilderImpl<T> implements 
TypedMessageBuilder<T> {
     @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;
     }
 
diff --git 
a/pulsar-common/src/main/java/org/apache/pulsar/common/api/Commands.java 
b/pulsar-common/src/main/java/org/apache/pulsar/common/api/Commands.java
index 3d62cd3..16e47a0 100644
--- a/pulsar-common/src/main/java/org/apache/pulsar/common/api/Commands.java
+++ b/pulsar-common/src/main/java/org/apache/pulsar/common/api/Commands.java
@@ -1042,7 +1042,8 @@ public class Commands {
         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
diff --git 
a/pulsar-common/src/main/java/org/apache/pulsar/common/api/proto/PulsarApi.java 
b/pulsar-common/src/main/java/org/apache/pulsar/common/api/proto/PulsarApi.java
index 5390853..1ee5f40 100644
--- 
a/pulsar-common/src/main/java/org/apache/pulsar/common/api/proto/PulsarApi.java
+++ 
b/pulsar-common/src/main/java/org/apache/pulsar/common/api/proto/PulsarApi.java
@@ -3039,6 +3039,10 @@ public final class PulsarApi {
     // optional bytes schema_version = 16;
     boolean hasSchemaVersion();
     org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString 
getSchemaVersion();
+    
+    // optional bool partition_key_b64_encoded = 17 [default = false];
+    boolean hasPartitionKeyB64Encoded();
+    boolean getPartitionKeyB64Encoded();
   }
   public static final class MessageMetadata extends
       org.apache.pulsar.shaded.com.google.protobuf.v241.GeneratedMessageLite
@@ -3339,6 +3343,16 @@ public final class PulsarApi {
       return schemaVersion_;
     }
     
+    // optional bool partition_key_b64_encoded = 17 [default = false];
+    public static final int PARTITION_KEY_B64_ENCODED_FIELD_NUMBER = 17;
+    private boolean partitionKeyB64Encoded_;
+    public boolean hasPartitionKeyB64Encoded() {
+      return ((bitField0_ & 0x00001000) == 0x00001000);
+    }
+    public boolean getPartitionKeyB64Encoded() {
+      return partitionKeyB64Encoded_;
+    }
+    
     private void initFields() {
       producerName_ = "";
       sequenceId_ = 0L;
@@ -3355,6 +3369,7 @@ public final class PulsarApi {
       encryptionAlgo_ = "";
       encryptionParam_ = 
org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString.EMPTY;
       schemaVersion_ = 
org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString.EMPTY;
+      partitionKeyB64Encoded_ = false;
     }
     private byte memoizedIsInitialized = -1;
     public final boolean isInitialized() {
@@ -3442,6 +3457,9 @@ public final class PulsarApi {
       if (((bitField0_ & 0x00000800) == 0x00000800)) {
         output.writeBytes(16, schemaVersion_);
       }
+      if (((bitField0_ & 0x00001000) == 0x00001000)) {
+        output.writeBool(17, partitionKeyB64Encoded_);
+      }
     }
     
     private int memoizedSerializedSize = -1;
@@ -3515,6 +3533,10 @@ public final class PulsarApi {
         size += 
org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream
           .computeBytesSize(16, schemaVersion_);
       }
+      if (((bitField0_ & 0x00001000) == 0x00001000)) {
+        size += 
org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream
+          .computeBoolSize(17, partitionKeyB64Encoded_);
+      }
       memoizedSerializedSize = size;
       return size;
     }
@@ -3658,6 +3680,8 @@ public final class PulsarApi {
         bitField0_ = (bitField0_ & ~0x00002000);
         schemaVersion_ = 
org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString.EMPTY;
         bitField0_ = (bitField0_ & ~0x00004000);
+        partitionKeyB64Encoded_ = false;
+        bitField0_ = (bitField0_ & ~0x00008000);
         return this;
       }
       
@@ -3755,6 +3779,10 @@ public final class PulsarApi {
           to_bitField0_ |= 0x00000800;
         }
         result.schemaVersion_ = schemaVersion_;
+        if (((from_bitField0_ & 0x00008000) == 0x00008000)) {
+          to_bitField0_ |= 0x00001000;
+        }
+        result.partitionKeyB64Encoded_ = partitionKeyB64Encoded_;
         result.bitField0_ = to_bitField0_;
         return result;
       }
@@ -3827,6 +3855,9 @@ public final class PulsarApi {
         if (other.hasSchemaVersion()) {
           setSchemaVersion(other.getSchemaVersion());
         }
+        if (other.hasPartitionKeyB64Encoded()) {
+          setPartitionKeyB64Encoded(other.getPartitionKeyB64Encoded());
+        }
         return this;
       }
       
@@ -3961,6 +3992,11 @@ public final class PulsarApi {
               schemaVersion_ = input.readBytes();
               break;
             }
+            case 136: {
+              bitField0_ |= 0x00008000;
+              partitionKeyB64Encoded_ = input.readBool();
+              break;
+            }
           }
         }
       }
@@ -4522,6 +4558,27 @@ public final class PulsarApi {
         return this;
       }
       
+      // optional bool partition_key_b64_encoded = 17 [default = false];
+      private boolean partitionKeyB64Encoded_ ;
+      public boolean hasPartitionKeyB64Encoded() {
+        return ((bitField0_ & 0x00008000) == 0x00008000);
+      }
+      public boolean getPartitionKeyB64Encoded() {
+        return partitionKeyB64Encoded_;
+      }
+      public Builder setPartitionKeyB64Encoded(boolean value) {
+        bitField0_ |= 0x00008000;
+        partitionKeyB64Encoded_ = value;
+        
+        return this;
+      }
+      public Builder clearPartitionKeyB64Encoded() {
+        bitField0_ = (bitField0_ & ~0x00008000);
+        partitionKeyB64Encoded_ = false;
+        
+        return this;
+      }
+      
       // @@protoc_insertion_point(builder_scope:pulsar.proto.MessageMetadata)
     }
     
@@ -4557,6 +4614,10 @@ public final class PulsarApi {
     // optional uint64 event_time = 5 [default = 0];
     boolean hasEventTime();
     long getEventTime();
+    
+    // optional bool partition_key_b64_encoded = 6 [default = false];
+    boolean hasPartitionKeyB64Encoded();
+    boolean getPartitionKeyB64Encoded();
   }
   public static final class SingleMessageMetadata extends
       org.apache.pulsar.shaded.com.google.protobuf.v241.GeneratedMessageLite
@@ -4676,12 +4737,23 @@ public final class PulsarApi {
       return eventTime_;
     }
     
+    // optional bool partition_key_b64_encoded = 6 [default = false];
+    public static final int PARTITION_KEY_B64_ENCODED_FIELD_NUMBER = 6;
+    private boolean partitionKeyB64Encoded_;
+    public boolean hasPartitionKeyB64Encoded() {
+      return ((bitField0_ & 0x00000010) == 0x00000010);
+    }
+    public boolean getPartitionKeyB64Encoded() {
+      return partitionKeyB64Encoded_;
+    }
+    
     private void initFields() {
       properties_ = java.util.Collections.emptyList();
       partitionKey_ = "";
       payloadSize_ = 0;
       compactedOut_ = false;
       eventTime_ = 0L;
+      partitionKeyB64Encoded_ = false;
     }
     private byte memoizedIsInitialized = -1;
     public final boolean isInitialized() {
@@ -4725,6 +4797,9 @@ public final class PulsarApi {
       if (((bitField0_ & 0x00000008) == 0x00000008)) {
         output.writeUInt64(5, eventTime_);
       }
+      if (((bitField0_ & 0x00000010) == 0x00000010)) {
+        output.writeBool(6, partitionKeyB64Encoded_);
+      }
     }
     
     private int memoizedSerializedSize = -1;
@@ -4753,6 +4828,10 @@ public final class PulsarApi {
         size += 
org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream
           .computeUInt64Size(5, eventTime_);
       }
+      if (((bitField0_ & 0x00000010) == 0x00000010)) {
+        size += 
org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream
+          .computeBoolSize(6, partitionKeyB64Encoded_);
+      }
       memoizedSerializedSize = size;
       return size;
     }
@@ -4876,6 +4955,8 @@ public final class PulsarApi {
         bitField0_ = (bitField0_ & ~0x00000008);
         eventTime_ = 0L;
         bitField0_ = (bitField0_ & ~0x00000010);
+        partitionKeyB64Encoded_ = false;
+        bitField0_ = (bitField0_ & ~0x00000020);
         return this;
       }
       
@@ -4930,6 +5011,10 @@ public final class PulsarApi {
           to_bitField0_ |= 0x00000008;
         }
         result.eventTime_ = eventTime_;
+        if (((from_bitField0_ & 0x00000020) == 0x00000020)) {
+          to_bitField0_ |= 0x00000010;
+        }
+        result.partitionKeyB64Encoded_ = partitionKeyB64Encoded_;
         result.bitField0_ = to_bitField0_;
         return result;
       }
@@ -4958,6 +5043,9 @@ public final class PulsarApi {
         if (other.hasEventTime()) {
           setEventTime(other.getEventTime());
         }
+        if (other.hasPartitionKeyB64Encoded()) {
+          setPartitionKeyB64Encoded(other.getPartitionKeyB64Encoded());
+        }
         return this;
       }
       
@@ -5023,6 +5111,11 @@ public final class PulsarApi {
               eventTime_ = input.readUInt64();
               break;
             }
+            case 48: {
+              bitField0_ |= 0x00000020;
+              partitionKeyB64Encoded_ = input.readBool();
+              break;
+            }
           }
         }
       }
@@ -5217,6 +5310,27 @@ public final class PulsarApi {
         return this;
       }
       
+      // optional bool partition_key_b64_encoded = 6 [default = false];
+      private boolean partitionKeyB64Encoded_ ;
+      public boolean hasPartitionKeyB64Encoded() {
+        return ((bitField0_ & 0x00000020) == 0x00000020);
+      }
+      public boolean getPartitionKeyB64Encoded() {
+        return partitionKeyB64Encoded_;
+      }
+      public Builder setPartitionKeyB64Encoded(boolean value) {
+        bitField0_ |= 0x00000020;
+        partitionKeyB64Encoded_ = value;
+        
+        return this;
+      }
+      public Builder clearPartitionKeyB64Encoded() {
+        bitField0_ = (bitField0_ & ~0x00000020);
+        partitionKeyB64Encoded_ = false;
+        
+        return this;
+      }
+      
       // 
@@protoc_insertion_point(builder_scope:pulsar.proto.SingleMessageMetadata)
     }
     
diff --git a/pulsar-common/src/main/proto/PulsarApi.proto 
b/pulsar-common/src/main/proto/PulsarApi.proto
index 8753aaf..aa21798 100644
--- a/pulsar-common/src/main/proto/PulsarApi.proto
+++ b/pulsar-common/src/main/proto/PulsarApi.proto
@@ -98,6 +98,8 @@ message MessageMetadata {
        // Additional parameters required by encryption
        optional bytes encryption_param = 15;
        optional bytes schema_version = 16;
+
+        optional bool partition_key_b64_encoded = 17 [ default = false ];
 }
 
 
@@ -110,6 +112,8 @@ message SingleMessageMetadata {
        // the timestamp that this event occurs. it is typically set by 
applications.
        // if this field is omitted, `publish_time` can be used for the purpose 
of `event_time`.
        optional uint64 event_time = 5 [default = 0];
+
+        optional bool partition_key_b64_encoded = 6 [ default = false ];
 }
 
 enum ServerError {

Reply via email to