Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -33,6 +33,7 @@
import com.google.common.base.Preconditions;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.fs.ByteBufferPositionedReadable;
import org.apache.hadoop.fs.ByteBufferReadable;
import org.apache.hadoop.fs.CanSetDropBehind;
import org.apache.hadoop.fs.CanSetReadahead;
Expand Down Expand Up @@ -328,20 +329,40 @@ public int read(long position, byte[] buffer, int offset, int length)
throws IOException {
checkStream();
try {
final int n = ((PositionedReadable) in).read(position, buffer, offset,
final int n = ((PositionedReadable) in).read(position, buffer, offset,
length);
if (n > 0) {
// This operation does not change the current offset of the file
decrypt(position, buffer, offset, n);
}

return n;
} catch (ClassCastException e) {
throw new UnsupportedOperationException("This stream does not support " +
"positioned read.");
}
}


/**
* Positioned readFully using {@link ByteBuffer}s. This method is thread-safe.
*/
// @Override
public void readFully(long position, final ByteBuffer buf)
throws IOException {
checkStream();
if (!(in instanceof ByteBufferPositionedReadable)) {
throw new UnsupportedOperationException(in.getClass().getCanonicalName()
+ " does not support positioned reads with byte buffers.");
}
int bufPos = buf.position();
((ByteBufferPositionedReadable) in).readFully(position, buf);
final int n = buf.position() - bufPos;
if (n > 0) {
// This operation does not change the current offset of the file
decrypt(position, buf, n, bufPos);
}
}

/**
* Decrypt length bytes in buffer starting at offset. Output is also put
* into buffer starting at offset. It is thread-safe.
Expand Down Expand Up @@ -375,7 +396,7 @@ private void decrypt(long position, byte[] buffer, int offset, int length)
returnDecryptor(decryptor);
}
}

/** Positioned read fully. It is thread-safe */
@Override
public void readFully(long position, byte[] buffer, int offset, int length)
Expand Down Expand Up @@ -407,7 +428,7 @@ public void seek(long pos) throws IOException {
checkStream();
try {
/*
* If data of target pos in the underlying stream has already been read
* If data of target pos in the underlying stream has already been read
* and decrypted in outBuffer, we just need to re-position outBuffer.
*/
if (pos <= streamOffset && pos >= (streamOffset - outBuffer.remaining())) {
Expand Down Expand Up @@ -523,7 +544,7 @@ public int read(ByteBuffer buf) throws IOException {
* Output is also buf and same start position.
* buf.position() and buf.limit() should be unchanged after decryption.
*/
private void decrypt(ByteBuffer buf, int n, int start)
private void decrypt(ByteBuffer buf, int n, int start)
throws IOException {
final int pos = buf.position();
final int limit = buf.limit();
Expand All @@ -545,7 +566,52 @@ private void decrypt(ByteBuffer buf, int n, int start)
}
buf.position(pos);
}


private void decrypt(long filePosition, ByteBuffer buf, int length, int start)
throws IOException {
ByteBuffer localInBuffer = null;
ByteBuffer localOutBuffer = null;

// Duplicate the buffer so we don't have to worry about resetting the
// original position and limit at the end of the method
buf = buf.duplicate();

int decryptedBytes = 0;
Decryptor localDecryptor = null;
try {
localInBuffer = getBuffer();
localOutBuffer = getBuffer();
localDecryptor = getDecryptor();
byte[] localIV = initIV.clone();
updateDecryptor(localDecryptor, filePosition, localIV);
byte localPadding = getPadding(filePosition);
// Set proper filePosition for inputdata.
localInBuffer.position(localPadding);

while (decryptedBytes < length) {
buf.position(start + decryptedBytes);
buf.limit(start + decryptedBytes +
Math.min(length - decryptedBytes, localInBuffer.remaining()));
localInBuffer.put(buf);
// Do decryption
try {
decrypt(localDecryptor, localInBuffer, localOutBuffer, localPadding);
buf.position(start + decryptedBytes);
buf.limit(start + length);
decryptedBytes += localOutBuffer.remaining();
buf.put(localOutBuffer);
} finally {
localPadding = afterDecryption(localDecryptor, localInBuffer,
filePosition + length, localIV);
}
}
} finally {
returnBuffer(localInBuffer);
returnBuffer(localOutBuffer);
returnDecryptor(localDecryptor);
}
}

@Override
public int available() throws IOException {
checkStream();
Expand Down Expand Up @@ -605,7 +671,7 @@ public ByteBuffer read(ByteBufferPool bufferPool, int maxLength,
}
return buffer;
} catch (ClassCastException e) {
throw new UnsupportedOperationException("This stream does not support " +
throw new UnsupportedOperationException("This stream does not support " +
"enhanced byte buffer access.");
}
}
Expand Down Expand Up @@ -740,6 +806,7 @@ public boolean hasCapability(String capability) {
case StreamCapabilities.READAHEAD:
case StreamCapabilities.DROPBEHIND:
case StreamCapabilities.UNBUFFER:
case StreamCapabilities.READBYTEBUFFER:
return true;
default:
return false;
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,90 @@
/**
* 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.EOFException;
import java.io.IOException;
import java.nio.ByteBuffer;

import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;

/**
* Implementers of this interface provide a positioned read API that writes to a
* {@link ByteBuffer} rather than a {@code byte[]}.
*
* @see PositionedReadable
* @see ByteBufferReadable
*/
@InterfaceAudience.Public
@InterfaceStability.Evolving
public interface ByteBufferPositionedReadable {
/**
* Reads up to {@code buf.remaining()} bytes into buf from a given position
* in the file and returns the number of bytes read. Callers should use
* {@code buf.limit(...)} to control the size of the desired read and
* {@code buf.position(...)} to control the offset into the buffer the data
* should be written to.
* <p>
* After a successful call, {@code buf.position()} will be advanced by the
* number of bytes read and {@code buf.limit()} will be unchanged.
* <p>
* In the case of an exception, the state of the buffer (the contents of the
* buffer, the {@code buf.position()}, the {@code buf.limit()}, etc.) is
* undefined, and callers should be prepared to recover from this
* eventuality.
* <p>
* Callers should use {@link StreamCapabilities#hasCapability(String)} with
* {@link StreamCapabilities#PREADBYTEBUFFER} to check if the underlying
* stream supports this interface, otherwise they might get a
* {@link UnsupportedOperationException}.
* <p>
* Implementations should treat 0-length requests as legitimate, and must not
* signal an error upon their receipt.
* <p>
* This does not change the current offset of a file, and is thread-safe.
*
* @param position position within file
* @param buf the ByteBuffer to receive the results of the read operation.
* @return the number of bytes read, possibly zero, or -1 if reached
* end-of-stream
* @throws IOException if there is some error performing the read
*/
int read(long position, ByteBuffer buf) throws IOException;

/**
* Reads {@code buf.remaining()} bytes into buf from a given position in
* the file or until the end of the data was reached before the read
* operation completed. Callers should use {@code buf.limit(...)} to
* control the size of the desired read and {@code buf.position(...)} to
* control the offset into the buffer the data should be written to.
* <p>
* This operation provides similar semantics to
* {@link #read(long, ByteBuffer)}, the difference is that this method is
* guaranteed to read data until the {@link ByteBuffer} is full, or until
* the end of the data stream is reached.
*
* @param position position within file
* @param buf the ByteBuffer to receive the results of the read operation.
* @throws IOException if there is some error performing the read
* @throws EOFException the end of the data was reached before
* the read operation completed
* @see #read(long, ByteBuffer)
*/
void readFully(long position, ByteBuffer buf) throws IOException;
}
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@
package org.apache.hadoop.fs;

import java.io.DataInputStream;
import java.io.EOFException;
import java.io.FileDescriptor;
import java.io.FileInputStream;
import java.io.IOException;
Expand All @@ -38,7 +39,8 @@
public class FSDataInputStream extends DataInputStream
implements Seekable, PositionedReadable,
ByteBufferReadable, HasFileDescriptor, CanSetDropBehind, CanSetReadahead,
HasEnhancedByteBufferAccess, CanUnbuffer, StreamCapabilities {
HasEnhancedByteBufferAccess, CanUnbuffer, StreamCapabilities,
ByteBufferPositionedReadable {
/**
* Map ByteBuffers that we have handed out to readers to ByteBufferPool
* objects
Expand All @@ -50,8 +52,8 @@ public class FSDataInputStream extends DataInputStream
public FSDataInputStream(InputStream in) {
super(in);
if( !(in instanceof Seekable) || !(in instanceof PositionedReadable) ) {
throw new IllegalArgumentException(
"In is not an instance of Seekable or PositionedReadable");
throw new IllegalArgumentException(in.getClass().getCanonicalName() +
" is not an instance of Seekable or PositionedReadable");
}
}

Expand Down Expand Up @@ -147,7 +149,8 @@ public int read(ByteBuffer buf) throws IOException {
return ((ByteBufferReadable)in).read(buf);
}

throw new UnsupportedOperationException("Byte-buffer read unsupported by input stream");
throw new UnsupportedOperationException("Byte-buffer read unsupported " +
"by " + in.getClass().getCanonicalName());
}

@Override
Expand All @@ -167,9 +170,8 @@ public void setReadahead(Long readahead)
try {
((CanSetReadahead)in).setReadahead(readahead);
} catch (ClassCastException e) {
throw new UnsupportedOperationException(
"this stream does not support setting the readahead " +
"caching strategy.");
throw new UnsupportedOperationException(in.getClass().getCanonicalName() +
" does not support setting the readahead caching strategy.");
}
}

Expand Down Expand Up @@ -246,4 +248,23 @@ public boolean hasCapability(String capability) {
public String toString() {
return super.toString() + ": " + in;
}

@Override
public int read(long position, ByteBuffer buf) throws IOException {
if (in instanceof ByteBufferPositionedReadable) {
return ((ByteBufferPositionedReadable) in).read(position, buf);
}
throw new UnsupportedOperationException("Byte-buffer pread unsupported " +
"by " + in.getClass().getCanonicalName());
}

@Override
public void readFully(long position, ByteBuffer buf) throws IOException {
if (in instanceof ByteBufferPositionedReadable) {
((ByteBufferPositionedReadable) in).readFully(position, buf);
} else {
throw new UnsupportedOperationException("Byte-buffer pread " +
"unsupported by " + in.getClass().getCanonicalName());
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -59,6 +59,17 @@ public interface StreamCapabilities {
*/
String UNBUFFER = "in:unbuffer";

/**
* Stream read(ByteBuffer) capability implemented by
* {@link ByteBufferReadable#read(java.nio.ByteBuffer)}.
*/
String READBYTEBUFFER = "in:readbytebuffer";
/**
* Stream read(long, ByteBuffer) capability implemented by
* {@link ByteBufferPositionedReadable#read(long, java.nio.ByteBuffer)}.
*/
String PREADBYTEBUFFER = "in:preadbytebuffer";

/**
* Capabilities that a stream can support and be queried for.
*/
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -199,8 +199,5 @@ Java_org_apache_hadoop_security_JniBasedUnixGroupsMapping_getGroupsForUser
if (ginfo) {
hadoop_group_info_free(ginfo);
}
if (jgroupname) {
(*env)->DeleteLocalRef(env, jgroupname);
}
return jgroups;
}
Loading