You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pulsar.apache.org by GitBox <gi...@apache.org> on 2018/01/17 16:44:16 UTC

[GitHub] merlimat closed pull request #1039: Serialization and Deserialization for RawMessage

merlimat closed pull request #1039: Serialization and Deserialization for RawMessage
URL: https://github.com/apache/incubator-pulsar/pull/1039
 
 
   

This is a PR merged from a forked repository.
As GitHub hides the original diff on merge, it is displayed below for
the sake of provenance:

As this is a foreign pull request (from a fork), the diff is supplied
below (as it won't show otherwise due to GitHub magic):

diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/client/api/RawMessage.java b/pulsar-broker/src/main/java/org/apache/pulsar/client/api/RawMessage.java
index 01ea641f8..a68204c34 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/client/api/RawMessage.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/client/api/RawMessage.java
@@ -44,6 +44,12 @@
      */
     ByteBuf getHeadersAndPayload();
 
+    /**
+     * Serialize a raw message to a ByteBuf. The caller is responsible for releasing
+     * the returned ByteBuf.
+     */
+    ByteBuf serialize();
+
     @Override
     void close();
 }
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/client/impl/RawMessageImpl.java b/pulsar-broker/src/main/java/org/apache/pulsar/client/impl/RawMessageImpl.java
index 6b81d6745..2134e96bc 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/client/impl/RawMessageImpl.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/client/impl/RawMessageImpl.java
@@ -18,13 +18,25 @@
  */
 package org.apache.pulsar.client.impl;
 
+import java.io.IOException;
+
 import io.netty.buffer.ByteBuf;
+import io.netty.buffer.PooledByteBufAllocator;
+
 import org.apache.pulsar.client.api.MessageId;
 import org.apache.pulsar.client.api.RawMessage;
+import org.apache.pulsar.common.api.DoubleByteBuf;
 import org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData;
 import org.apache.pulsar.client.impl.BatchMessageIdImpl;
+import org.apache.pulsar.common.util.protobuf.ByteBufCodedInputStream;
+import org.apache.pulsar.common.util.protobuf.ByteBufCodedOutputStream;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 public class RawMessageImpl implements RawMessage {
+    private static final Logger log = LoggerFactory.getLogger(RawMessageImpl.class);
+
     private final MessageIdData id;
     private final ByteBuf headersAndPayload;
 
@@ -53,4 +65,49 @@ public ByteBuf getHeadersAndPayload() {
     public void close() {
         headersAndPayload.release();
     }
+
+    @Override
+    public ByteBuf serialize() {
+        // Format: [IdSize][Id][PayloadAndMetadataSize][PayloadAndMetadata]
+        int idSize = id.getSerializedSize();
+        int headerSize = 4 /* IdSize */ + idSize + 4 /* PayloadAndMetadataSize */;
+
+        ByteBuf headers = PooledByteBufAllocator.DEFAULT.buffer(headerSize);
+        headers.writeInt(idSize);
+        try {
+            ByteBufCodedOutputStream outStream = ByteBufCodedOutputStream.get(headers);
+            id.writeTo(outStream);
+            outStream.recycle();
+        } catch (IOException e) {
+            // This is in-memory serialization, should not fail
+            log.error("IO exception serializing to ByteBuf (this shouldn't happen as operation is in-memory)", e);
+            throw new RuntimeException(e);
+        }
+        headers.writeInt(headersAndPayload.readableBytes());
+
+        return DoubleByteBuf.get(headers, headersAndPayload);
+    }
+
+    static public RawMessage deserializeFrom(ByteBuf buffer) {
+        try {
+            int idSize = buffer.readInt();
+
+            int writerIndex = buffer.writerIndex();
+            buffer.writerIndex(buffer.readerIndex() + idSize);
+            ByteBufCodedInputStream stream = ByteBufCodedInputStream.get(buffer);
+            MessageIdData.Builder builder = MessageIdData.newBuilder();
+            MessageIdData id = builder.mergeFrom(stream, null).build();
+            buffer.writerIndex(writerIndex);
+            builder.recycle();
+
+            int payloadAndMetadataSize = buffer.readInt();
+            ByteBuf metadataAndPayload = buffer.slice(buffer.readerIndex(), payloadAndMetadataSize);
+
+            return new RawMessageImpl(id, metadataAndPayload);
+        } catch (IOException e) {
+            // This is in-memory deserialization, should not fail
+            log.error("IO exception deserializing ByteBuf (this shouldn't happen as operation is in-memory)", e);
+            throw new RuntimeException(e);
+        }
+    }
 }
diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/RawMessageSerDeserTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/RawMessageSerDeserTest.java
new file mode 100644
index 000000000..a80f25c3a
--- /dev/null
+++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/RawMessageSerDeserTest.java
@@ -0,0 +1,60 @@
+/**
+ * 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 io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+
+import org.apache.pulsar.client.api.RawMessage;
+import org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.testng.Assert;
+import org.testng.annotations.Test;
+
+public class RawMessageSerDeserTest {
+    static final Logger log = LoggerFactory.getLogger(RawMessageSerDeserTest.class);
+
+    @Test
+    public void testSerializationAndDeserialization() throws Exception {
+        int payload = 0xbeefcafe;
+        ByteBuf headersAndPayload = Unpooled.buffer(4);
+        headersAndPayload.writeInt(payload);
+
+        MessageIdData id = MessageIdData.newBuilder()
+            .setLedgerId(0xf00).setEntryId(0xbaa)
+            .setPartition(10).setBatchIndex(20).build();
+
+        RawMessage m = new RawMessageImpl(id, headersAndPayload);
+        ByteBuf serialized = m.serialize();
+        byte[] bytes = new byte[serialized.readableBytes()];
+        serialized.readBytes(bytes);
+
+        RawMessage m2 = RawMessageImpl.deserializeFrom(Unpooled.wrappedBuffer(bytes));
+
+        Assert.assertEquals(m2.getMessageIdData().getLedgerId(), m.getMessageIdData().getLedgerId());
+        Assert.assertEquals(m2.getMessageIdData().getEntryId(), m.getMessageIdData().getEntryId());
+        Assert.assertEquals(m2.getMessageIdData().getPartition(), m.getMessageIdData().getPartition());
+        Assert.assertEquals(m2.getMessageIdData().getBatchIndex(), m.getMessageIdData().getBatchIndex());
+        Assert.assertEquals(m2.getHeadersAndPayload(), m.getHeadersAndPayload());
+    }
+}


 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services