Issue 205: Avoid copying bytebuf for constructing log record to write

Descriptions of the changes in this PR:

- avoid copying memory for writer facing constructor for log records
- construct a ByteBuf by copying ByteBuffer for reader facing constructors

This closes #205, #206 
diff --git a/distributedlog-protocol/src/main/java/org/apache/distributedlog/LogRecord.java b/distributedlog-protocol/src/main/java/org/apache/distributedlog/LogRecord.java
index 5d666a0..63b694a 100644
--- a/distributedlog-protocol/src/main/java/org/apache/distributedlog/LogRecord.java
+++ b/distributedlog-protocol/src/main/java/org/apache/distributedlog/LogRecord.java
@@ -172,15 +172,13 @@
 
     /**
      * Construct a log record with <i>txid</i> and ByteBuf <i>payload</i>.
+     *
      * @param txid transaction id
      * @param payload playload
      */
     public LogRecord(long txid, ByteBuf payload) {
         this.txid = txid;
-        // Need to make a copy since the passed payload is using a ref-count buffer that we don't know when could
-        // release, since the record is passed to the user. Also, the passed ByteBuf is coming from network and is
-        // backed by a direct buffer which we could not expose as a byte[]
-        this.payload = Unpooled.copiedBuffer(payload);
+        this.payload = payload;
         this.metadata = 0;
     }
 
diff --git a/distributedlog-protocol/src/main/java/org/apache/distributedlog/LogRecordWithDLSN.java b/distributedlog-protocol/src/main/java/org/apache/distributedlog/LogRecordWithDLSN.java
index 096fc81..ac9c3a8 100644
--- a/distributedlog-protocol/src/main/java/org/apache/distributedlog/LogRecordWithDLSN.java
+++ b/distributedlog-protocol/src/main/java/org/apache/distributedlog/LogRecordWithDLSN.java
@@ -19,6 +19,7 @@
 
 import com.google.common.annotations.VisibleForTesting;
 import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
 
 /**
  * Log record with {@link DLSN} and <code>SequenceId</code>.
@@ -75,7 +76,10 @@
 
     @VisibleForTesting
     public LogRecordWithDLSN(DLSN dlsn, long txid, ByteBuf buffer, long startSequenceIdOfCurrentSegment) {
-        super(txid, buffer);
+        // Need to make a copy since the passed payload is using a ref-count buffer that we don't know when could
+        // release, since the record is passed to the user. Also, the passed ByteBuf is coming from network and is
+        // backed by a direct buffer which we could not expose as a byte[]
+        super(txid, Unpooled.copiedBuffer(buffer));
         this.dlsn = dlsn;
         this.startSequenceIdOfCurrentSegment = startSequenceIdOfCurrentSegment;
     }