hadoop-common-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From d...@apache.org
Subject svn commit: r810451 - in /hadoop/common/trunk: ./ src/java/org/apache/hadoop/io/ src/java/org/apache/hadoop/io/file/tfile/ src/test/core/org/apache/hadoop/io/
Date Wed, 02 Sep 2009 11:07:27 GMT
Author: ddas
Date: Wed Sep  2 11:07:27 2009
New Revision: 810451

URL: http://svn.apache.org/viewvc?rev=810451&view=rev
Log:
HADOOP-6226. Moves BoundedByteArrayOutputStream from the tfile package to the io package and
makes it available to other users (MAPREDUCE-318). Contributed by Jothi Padmanabhan.

Added:
    hadoop/common/trunk/src/java/org/apache/hadoop/io/BoundedByteArrayOutputStream.java
    hadoop/common/trunk/src/test/core/org/apache/hadoop/io/TestBoundedByteArrayOutputStream.java
Removed:
    hadoop/common/trunk/src/java/org/apache/hadoop/io/file/tfile/BoundedByteArrayOutputStream.java
Modified:
    hadoop/common/trunk/CHANGES.txt
    hadoop/common/trunk/src/java/org/apache/hadoop/io/file/tfile/TFile.java

Modified: hadoop/common/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/CHANGES.txt?rev=810451&r1=810450&r2=810451&view=diff
==============================================================================
--- hadoop/common/trunk/CHANGES.txt (original)
+++ hadoop/common/trunk/CHANGES.txt Wed Sep  2 11:07:27 2009
@@ -162,6 +162,10 @@
     HADOOP-6120. Add support for Avro specific and reflect data.
     (sharad via cutting)
 
+    HADOOP-6226. Moves BoundedByteArrayOutputStream from the tfile package to
+    the io package and makes it available to other users (MAPREDUCE-318). 
+    (Jothi Padmanabhan via ddas)
+
   IMPROVEMENTS
 
     HADOOP-4565. Added CombineFileInputFormat to use data locality information

Added: hadoop/common/trunk/src/java/org/apache/hadoop/io/BoundedByteArrayOutputStream.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/src/java/org/apache/hadoop/io/BoundedByteArrayOutputStream.java?rev=810451&view=auto
==============================================================================
--- hadoop/common/trunk/src/java/org/apache/hadoop/io/BoundedByteArrayOutputStream.java (added)
+++ hadoop/common/trunk/src/java/org/apache/hadoop/io/BoundedByteArrayOutputStream.java Wed
Sep  2 11:07:27 2009
@@ -0,0 +1,119 @@
+/**
+ * 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.hadoop.io;
+
+import java.io.EOFException;
+import java.io.IOException;
+import java.io.OutputStream;
+
+/**
+ * A byte array backed output stream with a limit. The limit should be smaller
+ * than the buffer capacity. The object can be reused through <code>reset</code>
+ * API and choose different limits in each round.
+ */
+public class BoundedByteArrayOutputStream extends OutputStream {
+  private final byte[] buffer;
+  private int limit;
+  private int count;
+
+  /**
+   * Create a BoundedByteArrayOutputStream with the specified
+   * capacity
+   * @param capacity The capacity of the underlying byte array
+   */
+  public BoundedByteArrayOutputStream(int capacity) {
+    this(capacity, capacity);
+  }
+
+  /**
+   * Create a BoundedByteArrayOutputStream with the specified
+   * capacity and limit.
+   * @param capacity The capacity of the underlying byte array
+   * @param limit The maximum limit upto which data can be written
+   */
+  public BoundedByteArrayOutputStream(int capacity, int limit) {
+    if ((capacity < limit) || (capacity | limit) < 0) {
+      throw new IllegalArgumentException("Invalid capacity/limit");
+    }
+    this.buffer = new byte[capacity];
+    this.limit = limit;
+    this.count = 0;
+  }
+
+  @Override
+  public void write(int b) throws IOException {
+    if (count >= limit) {
+      throw new EOFException("Reaching the limit of the buffer.");
+    }
+    buffer[count++] = (byte) b;
+  }
+
+  @Override
+  public void write(byte b[], int off, int len) throws IOException {
+    if ((off < 0) || (off > b.length) || (len < 0) || ((off + len) > b.length)
+        || ((off + len) < 0)) {
+      throw new IndexOutOfBoundsException();
+    } else if (len == 0) {
+      return;
+    }
+
+    if (count + len > limit) {
+      throw new EOFException("Reach the limit of the buffer");
+    }
+
+    System.arraycopy(b, off, buffer, count, len);
+    count += len;
+  }
+
+  /**
+   * Reset the limit 
+   * @param newlim New Limit
+   */
+  public void reset(int newlim) {
+    if (newlim > buffer.length) {
+      throw new IndexOutOfBoundsException("Limit exceeds buffer size");
+    }
+    this.limit = newlim;
+    this.count = 0;
+  }
+
+  /** Reset the buffer */
+  public void reset() {
+    this.limit = buffer.length;
+    this.count = 0;
+  }
+
+  /** Return the current limit */
+  public int getLimit() {
+    return limit;
+  }
+
+  /** Returns the underlying buffer.
+   *  Data is only valid to {@link #size()}.
+   */
+  public byte[] getBuffer() {
+    return buffer;
+  }
+
+  /** Returns the length of the valid data 
+   * currently in the buffer.
+   */
+  public int size() {
+    return count;
+  }
+}

Modified: hadoop/common/trunk/src/java/org/apache/hadoop/io/file/tfile/TFile.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/src/java/org/apache/hadoop/io/file/tfile/TFile.java?rev=810451&r1=810450&r2=810451&view=diff
==============================================================================
--- hadoop/common/trunk/src/java/org/apache/hadoop/io/file/tfile/TFile.java (original)
+++ hadoop/common/trunk/src/java/org/apache/hadoop/io/file/tfile/TFile.java Wed Sep  2 11:07:27
2009
@@ -34,6 +34,7 @@
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.io.BoundedByteArrayOutputStream;
 import org.apache.hadoop.io.BytesWritable;
 import org.apache.hadoop.io.DataInputBuffer;
 import org.apache.hadoop.io.DataOutputBuffer;

Added: hadoop/common/trunk/src/test/core/org/apache/hadoop/io/TestBoundedByteArrayOutputStream.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/src/test/core/org/apache/hadoop/io/TestBoundedByteArrayOutputStream.java?rev=810451&view=auto
==============================================================================
--- hadoop/common/trunk/src/test/core/org/apache/hadoop/io/TestBoundedByteArrayOutputStream.java
(added)
+++ hadoop/common/trunk/src/test/core/org/apache/hadoop/io/TestBoundedByteArrayOutputStream.java
Wed Sep  2 11:07:27 2009
@@ -0,0 +1,91 @@
+/**
+ * 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.hadoop.io;
+
+import java.io.IOException;
+
+import junit.framework.TestCase;
+import java.util.Arrays;
+import java.util.Random;
+
+
+/** Unit tests for BoundedByteArrayOutputStream */
+public class TestBoundedByteArrayOutputStream extends TestCase {
+
+  private static final int SIZE = 1024;
+  private static final byte[] INPUT = new byte[SIZE];
+  static {
+      new Random().nextBytes(INPUT);
+  }
+  
+  public void testBoundedStream() throws IOException {
+    
+    BoundedByteArrayOutputStream stream = 
+      new BoundedByteArrayOutputStream(SIZE);
+
+    // Write to the stream, get the data back and check for contents
+    stream.write(INPUT, 0, SIZE);
+    assertTrue("Array Contents Mismatch",
+        Arrays.equals(INPUT, stream.getBuffer()));
+    
+    // Try writing beyond end of buffer. Should throw an exception
+    boolean caughtException = false;
+    
+    try {
+      stream.write(INPUT[0]);
+    } catch (Exception e) {
+      caughtException = true;
+    }
+    
+    assertTrue("Writing beyond limit did not throw an exception",
+        caughtException);
+    
+    //Reset the stream and try, should succeed 
+    stream.reset();
+    assertTrue("Limit did not get reset correctly", 
+        (stream.getLimit() == SIZE));
+    stream.write(INPUT, 0, SIZE);
+    assertTrue("Array Contents Mismatch",
+        Arrays.equals(INPUT, stream.getBuffer()));
+  
+    // Try writing one more byte, should fail
+    caughtException = false;
+    try {
+      stream.write(INPUT[0]);
+    } catch (Exception e) {
+      caughtException = true;
+    }
+  
+    // Reset the stream, but set a lower limit. Writing beyond
+    // the limit should throw an exception
+    stream.reset(SIZE - 1);
+    assertTrue("Limit did not get reset correctly",
+        (stream.getLimit() == SIZE -1));
+    caughtException = false;
+    
+    try {
+      stream.write(INPUT, 0, SIZE);
+    } catch (Exception e) {
+      caughtException = true;
+    }
+    
+    assertTrue("Writing beyond limit did not throw an exception",
+        caughtException);
+  }
+}



Mime
View raw message