HADOOP-8135. Add ByteBufferReadable interface to FSDataInputStream. Contributed by Henry Robinson.

(svn merge -c 1296556 from trunk)


git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-2@1348208 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Todd Lipcon 2012-06-08 19:52:59 +00:00
parent d1607ba557
commit e3721d582c
3 changed files with 58 additions and 1 deletions

View File

@ -12,6 +12,9 @@ Release 2.0.1-alpha - UNRELEASED
HDFS-3042. Automatic failover support for NameNode HA (todd) HDFS-3042. Automatic failover support for NameNode HA (todd)
(see dedicated section below for breakdown of subtasks) (see dedicated section below for breakdown of subtasks)
HADOOP-8135. Add ByteBufferReadable interface to FSDataInputStream. (Henry
Robinson via atm)
IMPROVEMENTS IMPROVEMENTS
HADOOP-8340. SNAPSHOT build versions should compare as less than their eventual HADOOP-8340. SNAPSHOT build versions should compare as less than their eventual

View File

@ -0,0 +1,45 @@
/**
* 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.fs;
import java.io.IOException;
import java.nio.ByteBuffer;
/**
* Implementers of this interface provide a read API that writes to a
* ByteBuffer, not a byte[].
*/
public interface ByteBufferReadable {
/**
* Reads up to buf.remaining() bytes into buf. Callers should use
* buf.limit(..) to control the size of the desired read.
*
* After the call, buf.position() should be unchanged, and therefore any data
* can be immediately read from buf.
*
* Many implementations will throw {@link UnsupportedOperationException}, so
* callers that are not confident in support for this method from the
* underlying filesystem should be prepared to handle that exception.
*
* @param buf
* the ByteBuffer to receive the results of the read operation
* @return the number of bytes available to read from buf
* @throws IOException if there is some error performing the read
*/
public int read(ByteBuffer buf) throws IOException;
}

View File

@ -18,6 +18,7 @@
package org.apache.hadoop.fs; package org.apache.hadoop.fs;
import java.io.*; import java.io.*;
import java.nio.ByteBuffer;
import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.classification.InterfaceStability;
@ -27,7 +28,7 @@ import org.apache.hadoop.classification.InterfaceStability;
@InterfaceAudience.Public @InterfaceAudience.Public
@InterfaceStability.Stable @InterfaceStability.Stable
public class FSDataInputStream extends DataInputStream public class FSDataInputStream extends DataInputStream
implements Seekable, PositionedReadable, Closeable { implements Seekable, PositionedReadable, Closeable, ByteBufferReadable {
public FSDataInputStream(InputStream in) public FSDataInputStream(InputStream in)
throws IOException { throws IOException {
@ -116,4 +117,12 @@ public class FSDataInputStream extends DataInputStream
public InputStream getWrappedStream() { public InputStream getWrappedStream() {
return in; return in;
} }
public int read(ByteBuffer buf) throws IOException {
if (in instanceof ByteBufferReadable) {
return ((ByteBufferReadable)in).read(buf);
}
throw new UnsupportedOperationException("Byte-buffer read unsupported by input stream");
}
} }