You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by gd...@apache.org on 2010/07/21 17:13:59 UTC

svn commit: r966274 - in /cassandra/trunk/src/java/org/apache/cassandra/io/util: DataOutputBuffer.java OutputBuffer.java

Author: gdusbabek
Date: Wed Jul 21 15:13:59 2010
New Revision: 966274

URL: http://svn.apache.org/viewvc?rev=966274&view=rev
Log:
split OutputBuffer from DataOutputBuffer. patch by stuhood, reviewed by gdusbabek. CASSANDRA-1186

Added:
    cassandra/trunk/src/java/org/apache/cassandra/io/util/OutputBuffer.java
Modified:
    cassandra/trunk/src/java/org/apache/cassandra/io/util/DataOutputBuffer.java

Modified: cassandra/trunk/src/java/org/apache/cassandra/io/util/DataOutputBuffer.java
URL: http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/io/util/DataOutputBuffer.java?rev=966274&r1=966273&r2=966274&view=diff
==============================================================================
--- cassandra/trunk/src/java/org/apache/cassandra/io/util/DataOutputBuffer.java (original)
+++ cassandra/trunk/src/java/org/apache/cassandra/io/util/DataOutputBuffer.java Wed Jul 21 15:13:59 2010
@@ -28,51 +28,29 @@ import java.io.IOException;
  * An implementation of the DataOutputStream interface. This class is completely thread
  * unsafe.
  */
-public class DataOutputBuffer extends DataOutputStream
+public final class DataOutputBuffer extends DataOutputStream
 {
-    private static class Buffer extends ByteArrayOutputStream
+    public DataOutputBuffer()
     {
-        public byte[] getData()
-        {
-            return buf;
-        }
-        
-        public int getLength()
-        {
-            return count;
-        }
-        
-        public void reset()
-        {
-            count = 0;
-        }
-        
-        public void write(DataInput in, int len) throws IOException
-        {
-            int newcount = count + len;
-            if (newcount > buf.length)
-            {
-                byte newbuf[] = new byte[Math.max(buf.length << 1, newcount)];
-                System.arraycopy(buf, 0, newbuf, 0, count);
-                buf = newbuf;
-            }
-            in.readFully(buf, count, len);
-            count = newcount;
-        }
+        this(128);
     }
     
-    private Buffer buffer;
-    
-    /** Constructs a new empty buffer. */
-    public DataOutputBuffer()
+    public DataOutputBuffer(int size)
     {
-        this(new Buffer());
+        super(new OutputBuffer(size));
     }
     
-    private DataOutputBuffer(Buffer buffer)
+    private OutputBuffer buffer()
+    {
+        return (OutputBuffer)out;
+    }
+
+    /**
+     * @return The valid contents of the buffer, possibly by copying: only safe for one-time-use buffers.
+     */
+    public byte[] asByteArray()
     {
-        super(buffer);
-        this.buffer = buffer;
+        return buffer().asByteArray();
     }
     
     /**
@@ -81,20 +59,20 @@ public class DataOutputBuffer extends Da
      */
     public byte[] getData()
     {
-        return buffer.getData();
+        return buffer().getData();
     }
     
     /** Returns the length of the valid data currently in the buffer. */
     public int getLength()
     {
-        return buffer.getLength();
+        return buffer().getLength();
     }
     
     /** Resets the buffer to empty. */
     public DataOutputBuffer reset()
     {
         this.written = 0;
-        buffer.reset();
+        buffer().reset();
         return this;
     }
 }

Added: cassandra/trunk/src/java/org/apache/cassandra/io/util/OutputBuffer.java
URL: http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/io/util/OutputBuffer.java?rev=966274&view=auto
==============================================================================
--- cassandra/trunk/src/java/org/apache/cassandra/io/util/OutputBuffer.java (added)
+++ cassandra/trunk/src/java/org/apache/cassandra/io/util/OutputBuffer.java Wed Jul 21 15:13:59 2010
@@ -0,0 +1,75 @@
+/**
+ * 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.cassandra.io.util;
+
+import java.io.ByteArrayOutputStream;
+import java.io.DataInput;
+import java.io.DataOutputStream;
+import java.io.IOException;
+
+/**
+ * Extends ByteArrayOutputStream to minimize copies.
+ */
+public final class OutputBuffer extends ByteArrayOutputStream
+{
+    public OutputBuffer()
+    {
+        this(128);
+    }
+
+    public OutputBuffer(int size)
+    {
+        super(size);
+    }
+
+    public byte[] getData()
+    {
+        return buf;
+    }
+    
+    public int getLength()
+    {
+        return count;
+    }
+    
+    public void write(DataInput in, int len) throws IOException
+    {
+        int newcount = count + len;
+        if (newcount > buf.length)
+        {
+            byte newbuf[] = new byte[Math.max(buf.length << 1, newcount)];
+            System.arraycopy(buf, 0, newbuf, 0, count);
+            buf = newbuf;
+        }
+        in.readFully(buf, count, len);
+        count = newcount;
+    }
+    
+    /**
+     * @return The valid contents of the buffer, possibly by copying: only safe for one-time-use buffers.
+     */
+    public byte[] asByteArray()
+    {
+        if (count == buf.length)
+            // no-copy
+            return buf;
+        // copy
+        return this.toByteArray();
+    }
+}